[ https://issues.apache.org/jira/browse/CASSANDRA-19483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17884584#comment-17884584 ]
Stefan Miklosovic edited comment on CASSANDRA-19483 at 9/25/24 12:04 PM: ------------------------------------------------------------------------- Okay, so ... problems. I was testing it and that patch failed on this (1). The test connects with simple client to the internode port which fails protocol magic check. It basically simulates what happens when some exception is thrown. Later you see that it waits on this log to be present: {code} "address contained in internode_error_reporting_exclusions" {code} That happens when this is logged (2) {code} boolean reportingExclusion = DatabaseDescriptor.getInternodeErrorReportingExclusions().contains(remoteAddress); if (reportingExclusion) logger.debug("Excluding internode exception for {}; address contained in internode_error_reporting_exclusions", remoteAddress, cause); else logger.error("Failed to properly handshake with peer {}. Closing the channel.", remoteAddress, cause); {code} But because we do not throw anymore, it will never come to this place so nothing is logged, so it fails the test which waits for it. If we do not want to throw again, this test would need to be rewritten in such a way that it will still throw on handshake but not on magic check. On the other hand, I am not completely sure if magic check is tested here specifically or any other exception would do it and magich check was just a simple way to test that. If the latter is the case, we would need to find some other way to throw. However, there is DatabaseDescriptor.getInternodeErrorReportingExclusions() since CASSANDRA-16859 where you can specify in cassandra.yaml via internode_error_reporting_exclusions all subnets from which the reported internode errors would be skipped from reporting, or rather, they would be logged just on "debug" which should not show in your logs upon the scanning you have. Unfortnunately, this property is not documented in cassandra.yaml but it would look like this: {code} internode_error_reporting_exclusions: subnets: - 127.0.0.1 - 127.0.0.0/31 {code} There is possibility to specify "client_error_reporting_exclusions" with the same structure as well. [~bschoeni] I would prefer if you suppress your stuff by this mechanism so we do not need to do anything. Another possibility would be to log it with no-spammer log and rethrow it so this logic is triggered and the test does not fail, but in that case we will log it with debug every time as well and I just do not see that as any win. (1) https://github.com/apache/cassandra/blob/trunk/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java#L65 (2) https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java#L381-L384 was (Author: smiklosovic): Okay, so ... problems. I was testing it and that patch failed on this (1). The test connects with simple client to the internode port which fails protocol magic check. It basically simulates what happens when some exception is thrown. Later you see that it waits on this log to be present: {code} "address contained in internode_error_reporting_exclusions" {code} That happens when this is logged (2) {code} boolean reportingExclusion = DatabaseDescriptor.getInternodeErrorReportingExclusions().contains(remoteAddress); if (reportingExclusion) logger.debug("Excluding internode exception for {}; address contained in internode_error_reporting_exclusions", remoteAddress, cause); else logger.error("Failed to properly handshake with peer {}. Closing the channel.", remoteAddress, cause); {code} But because we do not throw anymore, it will never come to this place so nothing is logged, so it fails the test which waits for it. If we do not want to throw again, this test would need to be rewritten in such a way that it will still throw on handshake but not on magic check. On the other hand, I am not completely sure if magic check is tested here specifically or any other exception would do it and magich check was just a simple way to test that. If the latter is the case, we would need to find some other way to throw. However, there is DatabaseDescriptor.getInternodeErrorReportingExclusions() since CASSANDRA-16859 where you can specify in cassandra.yaml via internode_error_reporting_exclusions all subnets from which the reported internode errors would be skipped from reporting, or rather, they would be logged just on "debug" which should not show in your logs upon the scanning you have. Unfortnunately, this property is not documented in cassandra.yaml but it would look like this: {code} internode_error_reporting_exclusions: subnets: - 127.0.0.1 - 127.0.0.0/31 {code} There is possibility to specify "client_error_reporting_exclusions" with the same structure as well. [~bschoeni] I would prefer if you suppress your stuff by this mechanism so we do not need to do anything. (1) https://github.com/apache/cassandra/blob/trunk/test/distributed/org/apache/cassandra/distributed/test/InternodeErrorExclusionTest.java#L65 (2) https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java#L381-L384 > Catch InvalidLegacyProtocolMagic exceptions > ------------------------------------------- > > Key: CASSANDRA-19483 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19483 > Project: Cassandra > Issue Type: Improvement > Components: Observability/Logging > Reporter: Brad Schoening > Assignee: Ilya Zakharov > Priority: Low > Fix For: 4.0.x, 4.0.14, 5.0.x, 5.x > > Time Spent: 40m > Remaining Estimate: 0h > > Similar to CASSANDRA-18839, we frequently see the exception > InvalidLegacyProtocolMagic with full stack traces flooding our logs. These > are due to Qualys vulnerability scans. > It seem to be a simple check in Message.java and would be better handled by: > a) returning a boolean from validateLegacyProtocolMagic() -> > hasValidLegacyProtocolMagic() instead of creating a custom exception class or > b) adding a catch block in HandshakeProtocol.java and return null as is done > for messagingVersion > {code:java} > static Initiate maybeDecode(ByteBuf buf) throws IOException > { > ... > try (DataInputBuffer in = new DataInputBuffer(nio, false)) > { > validateLegacyProtocolMagic(in.readInt()); // throws > exception > int flags = in.readInt(); > // legacy pre40 messagingVersion flag > if (getBits(flags, 8, 8) < VERSION_40) > return null; > int minMessagingVersion = getBits(flags, 16, 8); > int maxMessagingVersion = getBits(flags, 24, 8); > // 5.0+ does not support pre40 > if (maxMessagingVersion < MessagingService.VERSION_40) > return null; > .... > } > catch (EOFException e) > { > return null; > } > } > {code} > {code:java} > static void validateLegacyProtocolMagic(int magic) throws > InvalidLegacyProtocolMagic > { > if (magic != PROTOCOL_MAGIC) > throw new InvalidLegacyProtocolMagic(magic); > } > {code} > {code:java} > {{2024-03-20 03:47:27,380 [ERROR] [Messaging-EventLoop-3-2] cluster_id=9 > ip_address=10.0.0.1 InboundConnectionInitiator.java:360 - Failed to properly > handshake with peer /10.0.2:33356. Closing the channel.}} > {{io.netty.handler.codec.DecoderException: > org.apache.cassandra.net.Message$InvalidLegacyProtocolMagic: Read 1431520594, > Expected -900387334}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:478)}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.channelInputClosed(ByteToMessageDecoder.java:404)}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.channelInputClosed(ByteToMessageDecoder.java:371)}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:354)}} > {{ at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:262)}} > {{ at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:248)}} > {{ at > io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:241)}} > {{ at > io.netty.channel.DefaultChannelPipeline$HeadContext.channelInactive(DefaultChannelPipeline.java:1405)}} > {{ at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:262)}} > {{ at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:248)}} > {{ at > io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:901)}} > {{ at > io.netty.channel.AbstractChannel$AbstractUnsafe$8.run(AbstractChannel.java:819)}} > {{ at > io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)}} > {{ at > io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)}} > {{ at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:384)}} > {{ at > io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)}} > {{ at > io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)}} > {{ at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)}} > {{ at java.base/java.lang.Thread.run(Thread.java:834)}} > {{Caused by: org.apache.cassandra.net.Message$InvalidLegacyProtocolMagic: > Read 1431520594, Expected -900387334}} > {{ at > org.apache.cassandra.net.Message.validateLegacyProtocolMagic(Message.java:343)}} > {{ at > org.apache.cassandra.net.HandshakeProtocol$Initiate.maybeDecode(HandshakeProtocol.java:167)}} > {{ at > org.apache.cassandra.net.InboundConnectionInitiator$Handler.initiate(InboundConnectionInitiator.java:242)}} > {{ at > org.apache.cassandra.net.InboundConnectionInitiator$Handler.decode(InboundConnectionInitiator.java:235)}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:508)}} > {{ at > io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:447)}} > {{ ... 18 common frames omitted}} > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org