[
https://issues.apache.org/jira/browse/CASSANDRA-18591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17737239#comment-17737239
]
Jakub Zytka commented on CASSANDRA-18591:
-----------------------------------------
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:
testReadsAfterSimpleColumnDrop[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.testReadsAfterSimpleColumnDrop(DropColumnAndReadsTest.java:251)
[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.testReadsAfterSimpleColumnDrop(DropColumnAndReadsTest.java:167)
[junit-timeout] Suppressed: java.lang.AssertionError: droppedcolumn
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$serializeRowBody$0(UnfilteredSerializer.java:245)
[junit-timeout] at
org.apache.cassandra.utils.btree.BTree.applyValue(BTree.java:1762)
[junit-timeout] at
org.apache.cassandra.utils.btree.BTree.applyLeaf(BTree.java:1770)
[junit-timeout] at
org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1785)
[junit-timeout] at
org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1811)
[junit-timeout] at
org.apache.cassandra.db.rows.BTreeRow.apply(BTreeRow.java:199)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredSerializer.serializeRowBody(UnfilteredSerializer.java:238)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:206)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:138)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:126)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:152)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:101)
[junit-timeout] at
org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:86)
[junit-timeout] at
org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:308)
[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)
[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)
{code}
> assertion failure when reading after writes and column drop are executed in
> parallel
> ------------------------------------------------------------------------------------
>
> Key: CASSANDRA-18591
> URL: https://issues.apache.org/jira/browse/CASSANDRA-18591
> Project: Cassandra
> Issue Type: Bug
> Components: Local/Other
> Reporter: Jakub Zytka
> Assignee: Jakub Zytka
> Priority: Normal
>
> This is another face of the problem reported in CASSANDRA-18589, but this
> problem may happen regardless of whether the dropped column is complex.
>
> When writing data in parallel with dropping a column, the subsequent reads
> may fail with AssertionError: until the affected sstable is compacted.
> ERROR [node1_isolatedExecutor:3] node1 2023-06-12 15:45:28,058 Exception in
> thread Thread[node1_ReadStage-2,10,SharedPool]
> java.lang.RuntimeException: java.lang.AssertionError: Caught an error while
> trying to process the command: SELECT * FROM ks.tab LIMIT 500
> 0 ALLOW FILTERING
> at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:108)
> at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:45)
> at
> org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:430)
> at
> org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
> 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)
> Caused by: java.lang.AssertionError: Caught an error while trying to process
> the command: SELECT * FROM ks.tab LIMIT 5000 ALLOW FILTERING
> at
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:83)
> at
> org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:78)
> at
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:64)
> at
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:50)
> at org.apache.cassandra.net.InboundSink.accept(InboundSink.java:97)
> ... 6 common frames omitted
> Caused by: java.lang.AssertionError: v
> at
> org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$serializeRowBody$0(UnfilteredSerializer.java:255)
> at org.apache.cassandra.utils.btree.BTree.applyValue(BTree.java:1762)
> at org.apache.cassandra.utils.btree.BTree.applyLeaf(BTree.java:1770)
> at org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1785)
> at org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1811)
> at org.apache.cassandra.db.rows.BTreeRow.apply(BTreeRow.java:203)
> at
> org.apache.cassandra.db.rows.UnfilteredSerializer.serializeRowBody(UnfilteredSerializer.java:246)
> at
> org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:212)
> at
> org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:141)
> at
> org.apache.cassandra.db.rows.UnfilteredSerializer.serialize(UnfilteredSerializer.java:129)
> at
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:153)
> at
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:102)
> at
> org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87)
> at
> org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:308)
> 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:338)
> at
> org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:64)
> ... 10 common frames omitted
> It is likely that the intended fix of CASSANDRA-18589 will make this issue go
> away.
> For now, I'm reporting two different tickets to not mix two discussions
> together, should they happen.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]