[ https://issues.apache.org/jira/browse/CASSANDRA-19483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17886746#comment-17886746 ]
Brad Schoening edited comment on CASSANDRA-19483 at 10/3/24 10:27 PM: ---------------------------------------------------------------------- [~dcapwell] [~smiklosovic] This Jira has several different components: 1) Removed the throw of InvalidLegacyPrototolMagic. There was no benefit I can see to printing a stack trace when this occurred. 2) Changed the log level to WARN from ERROR. A client failure to connect doesn't seem like it should be an error level warning. 3) Added the noSpam behavior. Since the handshake happens before credentials are exchanged, it would seem a low risk event from a security perspective. Consider: * 9042 is the published client port address * 0xCA552DFA is the fixed magic number in the open source code Anyone trying to connect at 9042 should know the magic number – there are no secrets here. By using a bad magic number, I could only image a denial of service type attack or a misconfigured service trying to connect inadvertently when it expects something else at the port, like Kafka. I'm not a security expert, but I would think this is pretty deep in the connect stack to try to prevent or notice a DoS attack. If you have changed the port to a not well-known address, then port scanning could try to find what connects at the port before credential exchange. In that scenario, I you might consider enhanced logging. While I and the ops team understand these errors in the logs, our thousands of users do not know what they mean and as you might image, a "Unexpected protocol magic number" messages isn't very obvious to a user whats happing on their database server. Regarding subnet masking to suppress these, if I was on an internal White Hat security team, I would not want infrastructure teams to know what IPs to expect security scans from and behave differently with those subnets. The exception and Error log level changes here are the most useful in my view. I'd be ok deferring the noSpam behavior if we can fix those two issues. was (Author: bschoeni): [~dcapwell] [~smiklosovic] This Jira has several different components: 1) Removed the throw of InvalidLegacyPrototolMagic. There was no benefit I can see to printing a stack trace when this occurred. 2) Changed the log level to WARN from ERROR. A client failure to connect doesn't seem like it should be an error level warning. 3) Added the noSpam behavior. Since the handshake happens before credentials are exchanged, it would seem a low risk event from a security perspective. Consider: * 9042 is the published client port address * 0xCA552DFA is the fixed magic number in the open source code Anyone trying to connect at 9042 should know the magic number – there are no secrets here. By using a bad magic number, I could only image a denial of service type attack or a misconfigured service trying to connect inadvertently when it expects something else at the port, like Kafka. I'm not a security expert, but I would think this is pretty deep in the connect stack to try to prevent or notice a DoS attack. If you have changed the port to a not well-known address, then port scanning could try to find what connects at the port before credential exchange. In that scenario, I you might consider enhanced logging. While myself and the ops team understand these errors in the logs, our thousands of users do not know what they mean and as you might image, a "Unexpected protocol magic number" messages isn't very obvious to a user whats happing on their database server. Regarding subnet masking to suppress these, if I was on an internal White Hat security team, I would not want infrastructure teams to know what IPs to expect security scans from and behave differently with those subnets. The exception and Error log level changes here are the most useful in my view. I'd be ok deferring the noSpam behavior if we can fix those two issues. > 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.1.x, 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