[ https://issues.apache.org/jira/browse/CASSANDRA-17116?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17444879#comment-17444879 ]
David Capwell commented on CASSANDRA-17116: ------------------------------------------- cool, by trying to write CompleteMessage on the outbound channel (to avoid the race condition issue) LegacySSTableTest starts to fail with {code} ERROR [Reference-Reaper] 2021-11-16 15:55:07,656 LEAK DETECTED: a reference (org.apache.cassandra.utils.concurrent.Ref$State@637953b7) to class org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier@1459220591:/Users/davidcapwell/src/github/apache/cassandra/trunk/test/data/legacy-sstables/nb/legacy_tables/legacy_nb_simple/nb-1-big was not released b efore the reference was garbage collected ERROR [Reference-Reaper] 2021-11-16 15:55:07,657 Allocate trace org.apache.cassandra.utils.concurrent.Ref$State@637953b7: Thread[main,5,main] at java.lang.Thread.getStackTrace(Thread.java:1559) at org.apache.cassandra.utils.concurrent.Ref$Debug.<init>(Ref.java:248) at org.apache.cassandra.utils.concurrent.Ref$State.<init>(Ref.java:178) at org.apache.cassandra.utils.concurrent.Ref.<init>(Ref.java:100) at org.apache.cassandra.io.sstable.format.SSTableReader.<init>(SSTableReader.java:679) at org.apache.cassandra.io.sstable.format.SSTableReader.<init>(SSTableReader.java:643) at org.apache.cassandra.io.sstable.format.big.BigTableReader.<init>(BigTableReader.java:57) at org.apache.cassandra.io.sstable.format.big.BigFormat$ReaderFactory.open(BigFormat.java:103) at org.apache.cassandra.io.sstable.format.SSTableReaderBuilder$ForRead.build(SSTableReaderBuilder.java:370) at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:510) at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:381) at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:376) at org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:371) at org.apache.cassandra.io.sstable.LegacySSTableTest.streamLegacyTable(LegacySSTableTest.java:425) at org.apache.cassandra.io.sstable.LegacySSTableTest.streamLegacyTables(LegacySSTableTest.java:416) at org.apache.cassandra.io.sstable.LegacySSTableTest.testStreamLegacyCqlTables(LegacySSTableTest.java:301) {code} and {code} DEBUG [main] 2021-11-16 15:55:14,453 Got exception trying to acquire sstables org.apache.cassandra.db.repair.PendingAntiCompaction$SSTableAcquisitionException: Prepare phase failed because it encountered legacy sstables that don't support pending repair, run upgradesstables before starting incremental repairs, repair session (a4751540-4738-11ec-8f0a-09f160ae1e48) at org.apache.cassandra.db.repair.PendingAntiCompaction$AntiCompactionPredicate.apply(PendingAntiCompaction.java:132) at org.apache.cassandra.db.repair.PendingAntiCompaction$AntiCompactionPredicate.apply(PendingAntiCompaction.java:104) at com.google.common.base.Predicate.test(Predicate.java:79) at java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:174) at java.util.Iterator.forEachRemaining(Iterator.java:116) at java.util.Spliterators$IteratorSpliterator.forEachRemaining(Spliterators.java:1801) at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482) at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472) at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566) at org.apache.cassandra.db.repair.PendingAntiCompaction$AcquisitionCallable.acquireTuple(PendingAntiCompaction.java:198) at org.apache.cassandra.db.ColumnFamilyStore.runWithCompactionsDisabled(ColumnFamilyStore.java:2447) at org.apache.cassandra.db.repair.PendingAntiCompaction$AcquisitionCallable.call(PendingAntiCompaction.java:234) at org.apache.cassandra.io.sstable.LegacySSTableTest.testPendingAntiCompactionOldSSTables(LegacySSTableTest.java:373) {code} I don't grasp how this patch breaks it o_O > When zero-copy-streaming sees a channel close this triggers the disk failure > policy > ----------------------------------------------------------------------------------- > > Key: CASSANDRA-17116 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17116 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Streaming > Reporter: David Capwell > Assignee: David Capwell > Priority: Normal > Fix For: 4.x > > > Found in CASSANDRA-17085. > https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7264 > https://app.circleci.com/pipelines/github/dcapwell/cassandra/1069/workflows/26b7b83a-686f-4516-a56a-0709d428d4f2/jobs/7256 > {code} > ERROR [Stream-Deserializer-/127.0.0.1:7000-f2eb1a15] 2021-11-02 21:35:40,983 > DefaultFSErrorHandler.java:104 - Exiting forcefully due to file system > exception on startup, disk failure policy "stop" > org.apache.cassandra.io.FSWriteError: java.nio.channels.ClosedChannelException > at > org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:227) > at > org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.writeComponent(BigTableZeroCopyWriter.java:206) > at > org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:125) > at > org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:84) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:51) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:37) > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:50) > at > org.apache.cassandra.streaming.StreamDeserializingTask.run(StreamDeserializingTask.java:62) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.nio.channels.ClosedChannelException: null > at > org.apache.cassandra.net.AsyncStreamingInputPlus.reBuffer(AsyncStreamingInputPlus.java:136) > at > org.apache.cassandra.net.AsyncStreamingInputPlus.consume(AsyncStreamingInputPlus.java:155) > at > org.apache.cassandra.io.sstable.format.big.BigTableZeroCopyWriter.write(BigTableZeroCopyWriter.java:217) > ... 9 common frames omitted > {code} > When bootstrap fails and streaming is closed, this triggers the disk failure > policy which causes the JVM to halt by default (if this happens outside of > bootstrap, then we stop transports and keep the JVM up). > org.apache.cassandra.streaming.StreamDeserializingTask attempts to handle > this by ignoring this exception, but the call to > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize > Does try/catch and inspects exception; triggering this condition. -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org