[ https://issues.apache.org/jira/browse/CASSANDRA-17663?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17542255#comment-17542255 ]
Caleb Rackliffe edited comment on CASSANDRA-17663 at 5/26/22 4:47 PM: ---------------------------------------------------------------------- The 4.1 and trunk patches are going to be a bit different, but in the mean time, here's 4.0... [4.0 patch|https://github.com/apache/cassandra/pull/1647] [4.0 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17663-4.0&filter=all] UPDATE: Tests look reasonable. There are a couple existing and unrelated failures. was (Author: maedhroz): The 4.1 and trunk patches are going to be a bit different, but in the mean time, here's 4.0... [4.0 patch|https://github.com/apache/cassandra/pull/1647] [4.0 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17663-4.0&filter=all] UPDATE: Tests look reasonable. There are a couple normal/unrelated failures. > Ensure FileStreamTask cannot compromise shared channel proxy for system table > when interrupted > ---------------------------------------------------------------------------------------------- > > Key: CASSANDRA-17663 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17663 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming > Reporter: Caleb Rackliffe > Assignee: Caleb Rackliffe > Priority: Normal > Fix For: 4.0.x, 4.1-beta > > Time Spent: 0.5h > Remaining Estimate: 0h > > This is very similar to what we fixed in CASSANDRA-17466 , just in a > different part of the repair/streaming machine. > {{NettyStreamingMessageSender}} is responsible for dispatching > {{FileStreamTask}} on an executor it manages to stream files to its peers. > When the sender is closed for any reason, like perhaps a peer blowing up > while deserializing the stream, the executor it manages is shut down w/ > interruption (i.e. {{shutdownNow()}}). This is problematic if we happen to > have not gotten very far along in {{FileStreamTask#run()}}. If we're just > about to call {{getOrCreateChannel()}}, which reads from the {{peers_v2}} > system table, the {{ChannelProxy}} read will throw a > {{ClosedByInterruptedException}} and the proxy will be useless. The twist is > that, since CASSANDRA-15666, this exception has essentially been swallowed by > {{FileStreamTask}}'s exception handling. So we don't see a > {{ClosedByInterruptedException}} in the logs, but we do have things like this > pop up when anything else tries to hit the peers table: > {noformat} > ERROR 2022-05-19T21:49:23,218 [AntiEntropyStage:1] > org.apache.cassandra.service.CassandraDaemon:601 - Exception in thread > Thread[AntiEntropyStage:1,5,main] > java.lang.RuntimeException: FSReadError in > .../data/system/peers_v2-c4325fbb8e5e3bafbd070f9250ed818e/system-peers_v2-nb-101-big-Data.db > 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:433) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) > at java.util.concurrent.FutureTask.run(FutureTask.java:264) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > [netty-all-4.1.58.Final.jar:4.1.58.Final] > at java.lang.Thread.run(Thread.java:834) > Caused by: org.apache.cassandra.io.FSReadError: > java.nio.channels.ClosedChannelException > at > org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143) > at > org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115) > > at > org.apache.cassandra.io.util.BufferManagingRebufferer.rebuffer(BufferManagingRebufferer.java:79) > > at > org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:68) > > at > org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:210) > > at > org.apache.cassandra.io.util.FileHandle.createReader(FileHandle.java:151) > {noformat} > ...and... > {noformat} > ERROR 2022-05-19T22:06:20,175 [CompactionExecutor:12] > org.apache.cassandra.service.CassandraDaemon:601 - Exception in thread > Thread[CompactionExecutor:12,1,main] > org.apache.cassandra.io.FSReadError: java.nio.channels.ClosedChannelException > at > org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143) > at > org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115) > > at > org.apache.cassandra.io.util.BufferManagingRebufferer.rebuffer(BufferManagingRebufferer.java:79) > > at > org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:68) > > at > org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:210) > > at > org.apache.cassandra.io.sstable.format.big.BigTableScanner.seekToCurrentRangeStart(BigTableScanner.java:196) > > at > org.apache.cassandra.io.sstable.format.big.BigTableScanner.access$400(BigTableScanner.java:52) > > at > org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator.computeNext(BigTableScanner.java:305) > > at > org.apache.cassandra.io.sstable.format.big.BigTableScanner$KeyScanningIterator.computeNext(BigTableScanner.java:282) > > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:46) > at > org.apache.cassandra.io.sstable.format.big.BigTableScanner.hasNext(BigTableScanner.java:261) > > {noformat} > ...which obviously get us into trouble w/ the disk failure policy. > The fix proposed here is just to get the peers table read out of the thread > that can be interrupted. Specifically, {{NettyStreamingMessageSender}} > materializes a {{connectTo}} address at stream task creation time. This > seemed a better option than making shutdown non-interrupting, since that > would mean changing how the actual file streaming responds to shutdown. -- This message was sent by Atlassian Jira (v8.20.7#820007) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org