Skip to content

Commit

Permalink
Fix TestGossipingPropertyFileSnitch.test_prefer_local_reconnect_on_li…
Browse files Browse the repository at this point in the history
…sten_address

patch by Jyothsna Konisa; reviewed by Jon Meredith, Yifan Cai for CASSANDRA-17700
  • Loading branch information
jyothsnakonisa authored and yifan-c committed Jul 1, 2022
1 parent 9c411d3 commit 7119cf8
Show file tree
Hide file tree
Showing 7 changed files with 28 additions and 19 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
4.2
* Fix TestGossipingPropertyFileSnitch.test_prefer_local_reconnect_on_listen_address (CASSANDRA-17700)
* Add ByteComparable API (CASSANDRA-6936)
* Add guardrail for maximum replication factor (CASSANDRA-17500)
* Increment CQLSH to version 6.2.0 for release 4.2 (CASSANDRA-17646)
Expand Down
11 changes: 9 additions & 2 deletions src/java/org/apache/cassandra/auth/IInternodeAuthenticator.java
Expand Up @@ -82,11 +82,18 @@ default void setupInternode()
}

/**
* Enum that represents connection type of an internode connection.
* Enum that represents connection type of internode connection.
*
* INBOUND - called after connection established, with certificate available if present.
* OUTBOUND - called after connection established, with certificate available if present.
* OUTBOUND_PRECONNECT - called before initiating a connection, without certificate available.
* The outbound connection will be authenticated with the certificate once a redirected connection is established.
* This is an extra check that can be used to detect misconfiguration before reconnection, or ignored by returning true.
*/
enum InternodeConnectionDirection
{
INBOUND,
OUTBOUND
OUTBOUND,
OUTBOUND_PRECONNECT
}
}
Expand Up @@ -30,6 +30,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.cassandra.auth.IInternodeAuthenticator.InternodeConnectionDirection.OUTBOUND_PRECONNECT;

/**
* Sidekick helper for snitches that want to reconnect from one IP addr for a node to another.
* Typically, this is for situations like EC2 where a node will have a public address and a private address,
Expand Down Expand Up @@ -64,7 +66,8 @@ private void reconnect(InetAddressAndPort publicAddress, VersionedValue localAdd
@VisibleForTesting
static void reconnect(InetAddressAndPort publicAddress, InetAddressAndPort localAddress, IEndpointSnitch snitch, String localDc)
{
if (!new OutboundConnectionSettings(publicAddress, localAddress).withDefaults(ConnectionCategory.MESSAGING).authenticate())
final OutboundConnectionSettings settings = new OutboundConnectionSettings(publicAddress, localAddress).withDefaults(ConnectionCategory.MESSAGING);
if (!settings.authenticator().authenticate(settings.to.getAddress(), settings.to.getPort(), null, OUTBOUND_PRECONNECT))
{
logger.debug("InternodeAuthenticator said don't reconnect to {} on {}", publicAddress, localAddress);
return;
Expand Down
10 changes: 0 additions & 10 deletions src/java/org/apache/cassandra/net/InboundConnectionSettings.java
Expand Up @@ -71,16 +71,6 @@ public InboundConnectionSettings()
this(null, null, null, null, null, null, null, null, null);
}

public boolean authenticate(InetAddressAndPort endpoint)
{
return authenticator.authenticate(endpoint.getAddress(), endpoint.getPort());
}

public boolean authenticate(InetAddress address, int port)
{
return authenticator.authenticate(address, port);
}

public String toString()
{
return format("address: (%s), nic: %s, encryption: %s",
Expand Down
10 changes: 10 additions & 0 deletions src/java/org/apache/cassandra/net/OutboundConnectionInitiator.java
Expand Up @@ -59,10 +59,12 @@
import org.apache.cassandra.security.ISslContextFactory;
import org.apache.cassandra.security.SSLFactory;
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.concurrent.ImmediateFuture;
import org.apache.cassandra.utils.memory.BufferPools;

import static java.util.concurrent.TimeUnit.*;
import static org.apache.cassandra.auth.IInternodeAuthenticator.InternodeConnectionDirection.OUTBOUND;
import static org.apache.cassandra.auth.IInternodeAuthenticator.InternodeConnectionDirection.OUTBOUND_PRECONNECT;
import static org.apache.cassandra.net.InternodeConnectionUtils.DISCARD_HANDLER_NAME;
import static org.apache.cassandra.net.InternodeConnectionUtils.SSL_HANDLER_NAME;
import static org.apache.cassandra.net.InternodeConnectionUtils.certificates;
Expand Down Expand Up @@ -137,6 +139,14 @@ private Future<Result<SuccessType>> initiate(EventLoop eventLoop)
if (logger.isTraceEnabled())
logger.trace("creating outbound bootstrap to {}, requestVersion: {}", settings, requestMessagingVersion);

if (!settings.authenticator.authenticate(settings.to.getAddress(), settings.to.getPort(), null, OUTBOUND_PRECONNECT))
{
// interrupt other connections, so they must attempt to re-authenticate
MessagingService.instance().interruptOutbound(settings.to);
return ImmediateFuture.failure(new IOException("authentication failed to " + settings.connectToId()));
}


// this is a bit ugly, but is the easiest way to ensure that if we timeout we can propagate a suitable error message
// and still guarantee that, if on timing out we raced with success, the successfully created channel is handled
AtomicBoolean timedout = new AtomicBoolean();
Expand Down
Expand Up @@ -25,7 +25,6 @@
import org.apache.cassandra.auth.IInternodeAuthenticator;
import org.apache.cassandra.config.Config;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.EncryptionOptions;
import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.locator.IEndpointSnitch;
Expand Down Expand Up @@ -157,11 +156,6 @@ private OutboundConnectionSettings(IInternodeAuthenticator authenticator,
this.endpointToVersion = endpointToVersion;
}

public boolean authenticate()
{
return authenticator.authenticate(to.getAddress(), to.getPort());
}

public boolean withEncryption()
{
return encryption != null;
Expand Down
Expand Up @@ -352,6 +352,10 @@ public static class CertificateVerifyAuthenticator implements IInternodeAuthenti
@Override
public boolean authenticate(InetAddress remoteAddress, int remotePort, Certificate[] certificates, InternodeConnectionDirection connectionType)
{
if (connectionType == InternodeConnectionDirection.OUTBOUND_PRECONNECT)
{
return true;
}
try
{
// Check if the presented certificates during internode authentication are the ones in the keystores
Expand Down

0 comments on commit 7119cf8

Please sign in to comment.