Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ Merged from 4.0:
* Add new concurrent_merkle_tree_requests config property to prevent OOM during multi-range and/or multi-table repairs (CASSANDRA-19336)

4.1
* Optionally skip exception logging on invalid legacy protocol magic exception (CASSANDRA-19483)
* Add plugin support for CQLSH (CASSANDRA-16456)
* Request-Based Native Transport Rate-Limiting (CASSANDRA-16663)
* Add information whether sstables are dropped to SchemaChangeListener (CASSANDRA-17582)
Expand Down
1 change: 1 addition & 0 deletions src/java/org/apache/cassandra/config/Config.java
Original file line number Diff line number Diff line change
Expand Up @@ -643,6 +643,7 @@
}

public volatile int consecutive_message_errors_threshold = 1;
public volatile boolean invalid_legacy_protocol_magic_no_spam_enabled = false;

Check warning on line 646 in src/java/org/apache/cassandra/config/Config.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Rename this field "invalid_legacy_protocol_magic_no_spam_enabled" to match the regular expression '^[a-z][a-zA-Z0-9]*$'.

See more on https://sonarcloud.io/project/issues?id=cassandra-stargazer&issues=AZsNHcxHUzPrVzHGzAMb&open=AZsNHcxHUzPrVzHGzAMb&pullRequest=2170

Check warning on line 646 in src/java/org/apache/cassandra/config/Config.java

View check run for this annotation

SonarQubeCloud / SonarCloud Code Analysis

Make invalid_legacy_protocol_magic_no_spam_enabled a static final constant or non-public and provide accessors if needed.

See more on https://sonarcloud.io/project/issues?id=cassandra-stargazer&issues=AZsNHcxHUzPrVzHGzAMa&open=AZsNHcxHUzPrVzHGzAMa&pullRequest=2170

/**
* Default compaction configuration, used if a table does not specify any.
Expand Down
5 changes: 5 additions & 0 deletions src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -3789,4 +3789,9 @@ public static void setNativeTransportTimeout(long timeout, TimeUnit timeUnit)
{
conf.native_transport_timeout_in_ms = timeUnit.toMillis(timeout);
}

public static boolean getInvalidLegacyProtocolMagicNoSpamEnabled()
{
return conf.invalid_legacy_protocol_magic_no_spam_enabled;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,11 @@
import java.net.SocketAddress;
import java.util.List;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -44,12 +46,14 @@
import io.netty.handler.logging.LoggingHandler;
import io.netty.handler.ssl.SslContext;
import io.netty.handler.ssl.SslHandler;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.EncryptionOptions;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.net.OutboundConnectionSettings.Framing;
import org.apache.cassandra.security.SSLFactory;
import org.apache.cassandra.streaming.async.StreamingInboundHandler;
import org.apache.cassandra.utils.NoSpamLogger;
import org.apache.cassandra.utils.memory.BufferPools;

import static java.lang.Math.max;
Expand All @@ -68,6 +72,8 @@ public class InboundConnectionInitiator
{
private static final Logger logger = LoggerFactory.getLogger(InboundConnectionInitiator.class);

private static final NoSpamLogger noSpam5m = NoSpamLogger.getLogger(logger, 5, TimeUnit.MINUTES);

private static class Initializer extends ChannelInitializer<SocketChannel>
{
private final InboundConnectionSettings settings;
Expand Down Expand Up @@ -412,7 +418,11 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause)

private void exceptionCaught(Channel channel, Throwable cause)
{
logger.error("Failed to properly handshake with peer {}. Closing the channel.", channel.remoteAddress(), cause);
if (cause != null && Throwables.getRootCause(cause) instanceof Message.InvalidLegacyProtocolMagic && DatabaseDescriptor.getInvalidLegacyProtocolMagicNoSpamEnabled())
noSpam5m.warn("Failed to properly handshake with peer {}. Closing the channel. Invalid legacy protocol magic.", ((InetSocketAddress) channel.remoteAddress()).getHostName());
else
logger.error("Failed to properly handshake with peer {}. Closing the channel.", channel.remoteAddress(), cause);

try
{
failHandshake(channel);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* Copyright DataStax, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.cassandra.distributed.test;

import java.time.Duration;

import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;

import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.distributed.Cluster;
import org.apache.cassandra.distributed.api.Feature;
import org.apache.cassandra.transport.SimpleClient;

import static org.assertj.core.api.Assertions.assertThat;

public class InternodeErrorExclusionTest extends TestBaseImpl
{
@BeforeClass
public static void beforeClass2()
{
DatabaseDescriptor.clientInitialization();
}

@Test
public void testNoSpammingInvalidLegacyProtocolMagicException() throws Throwable
{
try (Cluster cluster = Cluster.build(1)
.withConfig(c -> c
.with(Feature.NETWORK)
.set("invalid_legacy_protocol_magic_no_spam_enabled", true))
.start())
{
long logMark = cluster.get(1).logs().mark();
causeException();
causeException();
// we used no spam logger so the second message will not be emitted (the size is still 1).
assertThat(cluster.get(1).logs().watchFor(logMark, Duration.ofSeconds(30), "Failed to properly handshake with peer localhost. Closing the channel. Invalid legacy protocol magic.").getResult()).hasSize(1);
}
}

private void causeException()
{
try (SimpleClient client = SimpleClient.builder("127.0.0.1", 7012).build())
{
client.connect(true);
Assert.fail("Connection should fail");
}
catch (Exception e)
{
// expected
}
}
}
Loading