Skip to content
Open
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
11 changes: 11 additions & 0 deletions src/java/org/apache/cassandra/auth/AuthConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.util.List;

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

Expand All @@ -38,6 +39,16 @@ public final class AuthConfig

private static boolean initialized;

/**
* Resets the initialized flag, enabling AuthConfig to be reconfigured multiple times within a single
* test case.
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Specifically, see AuthConfigTest in this same PR.

*/
@VisibleForTesting
static void reset()
{
initialized = false;
}

public static void applyAuth()
{
// some tests need this
Expand Down
65 changes: 33 additions & 32 deletions src/java/org/apache/cassandra/auth/CassandraRoleManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
Expand Down Expand Up @@ -85,29 +84,16 @@
import static org.apache.cassandra.utils.ByteBufferUtil.bytes;

/**
* Responsible for the creation, maintenance and deletion of roles
* for the purposes of authentication and authorization.
* Role data is stored internally, using the roles and role_members tables
* in the system_auth keyspace.
* Responsible for the creation, maintenance and deletion of roles for the purposes of authentication and
* authorization. Role data is stored internally, using the roles and role_members tables in the system_auth
* keyspace.
*
* Additionally, if org.apache.cassandra.auth.PasswordAuthenticator is used,
* encrypted passwords are also stored in the system_auth.roles table. This
* coupling between the IAuthenticator and IRoleManager implementations exists
* because setting a role's password via CQL is done with a CREATE ROLE or
* ALTER ROLE statement, the processing of which is handled by IRoleManager.
* As IAuthenticator is concerned only with credentials checking and has no
* means to modify passwords, PasswordAuthenticator depends on
* CassandraRoleManager for those functions.
*
* Alternative IAuthenticator implementations may be used in conjunction with
* CassandraRoleManager, but WITH PASSWORD = 'password' will not be supported
* in CREATE/ALTER ROLE statements.
*
* Such a configuration could be implemented using a custom IRoleManager that
* extends CassandraRoleManager and which includes Option.PASSWORD in the {@code Set<Option>}
* returned from supportedOptions/alterableOptions. Any additional processing
* of the password itself (such as storing it in an alternative location) would
* be added in overridden createRole and alterRole implementations.
* Authenticators (implementations of {@link IAuthenticator}) can specify additional attributes to be stored.
* For example, {@link org.apache.cassandra.auth.PasswordAuthenticator}, stores encrypted passwords in the
* system_auth.roles table. This coupling between the IAuthenticator and IRoleManager implementations exists because
* setting a role's password via CQL is done with a CREATE ROLE or ALTER ROLE statement, the processing of which is
* handled by IRoleManager. Authenticators depend on CassandraRoleManager for those functions because IAuthenticator
* is concerned only with credentials checking and has no means to directly modify passwords.
*/
public class CassandraRoleManager implements IRoleManager, CassandraRoleManagerMBean
{
Expand All @@ -117,8 +103,24 @@ public class CassandraRoleManager implements IRoleManager, CassandraRoleManagerM
public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
public static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";

/**
* Role options which are supported for all authentication mechanisms. IAuthenticator implementations can declare
* additional supported role options via {@link IAuthenticator#getSupportedRoleOptions()}.
*/
@VisibleForTesting
static final Set<Option> DEFAULT_SUPPORTED_ROLE_OPTIONS = Set.of(Option.LOGIN, Option.SUPERUSER);

/**
* User-alterable role options which are supported for all authentication mechanisms. IAuthenticator
* implementations can declare additional alterable role options via
* {@link IAuthenticator#getAlterableRoleOptions()}.
*/
@VisibleForTesting
static final Set<Option> DEFAULT_ALTERABLE_ROLE_OPTIONS = Set.of();

@VisibleForTesting
static final String PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD = "invalid_role_disconnect_task_period";

@VisibleForTesting
static final String PARAM_INVALID_ROLE_DISCONNECT_TASK_MAX_JITTER = "invalid_role_disconnect_task_max_jitter";

Expand Down Expand Up @@ -171,17 +173,16 @@ public CassandraRoleManager()

public CassandraRoleManager(Map<String, String> parameters)
{
Set<Option> allowedOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator
? EnumSet.of(Option.LOGIN, Option.SUPERUSER, Option.PASSWORD, Option.HASHED_PASSWORD, Option.GENERATED_PASSWORD, Option.GENERATED_NAME)
: EnumSet.of(Option.LOGIN, Option.SUPERUSER);
Set<Option> allowedOptions = Stream.concat(DEFAULT_SUPPORTED_ROLE_OPTIONS.stream(), DatabaseDescriptor.getAuthenticator().getSupportedRoleOptions().stream())
.collect(Collectors.toSet());

if (Guardrails.roleNamePolicy.getGenerator() != NoOpGenerator.INSTANCE)
allowedOptions.add(Option.OPTIONS);

supportedOptions = ImmutableSet.copyOf(allowedOptions);
alterableOptions = DatabaseDescriptor.getAuthenticator() instanceof PasswordAuthenticator
? ImmutableSet.of(Option.PASSWORD, Option.HASHED_PASSWORD, Option.GENERATED_PASSWORD)
: ImmutableSet.<Option>of();
supportedOptions = Set.copyOf(allowedOptions);

alterableOptions = Stream.concat(DEFAULT_ALTERABLE_ROLE_OPTIONS.stream(), DatabaseDescriptor.getAuthenticator().getAlterableRoleOptions().stream())
.collect(Collectors.toUnmodifiableSet());

// Inherit parsing and validation from existing config parser
invalidClientDisconnectPeriodMillis = new DurationSpec.LongMillisecondsBound(parameters.getOrDefault(PARAM_INVALID_ROLE_DISCONNECT_TASK_PERIOD, "0h")).toMilliseconds();
Expand Down Expand Up @@ -478,8 +479,8 @@ public boolean isExistingRole(RoleResource role)

public Set<? extends IResource> protectedResources()
{
return ImmutableSet.of(DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES),
DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLE_MEMBERS));
return Set.of(DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES),
DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLE_MEMBERS));
}

public void validateConfiguration() throws ConfigurationException
Expand Down
21 changes: 21 additions & 0 deletions src/java/org/apache/cassandra/auth/IAuthenticator.java
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,27 @@ default boolean supportsEarlyAuthentication()
*/
Set<? extends IResource> protectedResources();

/**
* Set of IRoleManager.Options used by this authenticator and supported by CREATE ROLE and ALTER ROLE statements.
*
* @return A set of IRoleManager.Options that this authenticator requires support for.
*/
default Set<IRoleManager.Option> getSupportedRoleOptions()
{
return Set.of();
}

/**
* Set of IRoleManager.Options used by this authenticator that users are allowed to alter via
* ALTER ROLE statements. Alterable role options must also be supported role options.
*
* @return A set of supported role options that users are allowed to alter.
*/
default Set<IRoleManager.Option> getAlterableRoleOptions()
{
return Set.of();
}

/**
* Validates configuration of IAuthenticator implementation (if configurable).
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,13 +77,16 @@ public class MutualTlsAuthenticator implements IAuthenticator
private static final Logger logger = LoggerFactory.getLogger(MutualTlsAuthenticator.class);
private static final NoSpamLogger nospamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.MINUTES);
private static final String VALIDATOR_CLASS_NAME = "validator_class_name";
private static final String CACHE_NAME = "IdentitiesCache";

private final IdentityCache identityCache = new IdentityCache();
private final MutualTlsCertificateValidator certificateValidator;
private static final Set<AuthenticationMode> AUTHENTICATION_MODES = Collections.singleton(MTLS);
private final MutualTlsCertificateValidityPeriodValidator certificateValidityPeriodValidator;
private final DurationSpec.IntMinutesBound certificateValidityWarnThreshold;

@VisibleForTesting
static final String CACHE_NAME = "IdentitiesCache";

// key for the 'identity' value in AuthenticatedUser metadata map.
public static final String METADATA_IDENTITY_KEY = "identity";

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public NetworkPermissionsCache(INetworkAuthorizer authorizer)
DatabaseDescriptor::getRolesCacheActiveUpdate,
authorizer::authorize,
authorizer.bulkLoader(),
() -> DatabaseDescriptor.getAuthenticator().requireAuthentication());
DatabaseDescriptor::isAuthenticationRequired);

MBeanWrapper.instance.registerMBean(this, MBEAN_NAME_BASE + DEPRECATED_CACHE_NAME);
}
Expand Down
37 changes: 35 additions & 2 deletions src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,13 @@
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.function.Supplier;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -75,6 +75,19 @@ public class PasswordAuthenticator implements IAuthenticator, AuthCache.BulkLoad
public static final String PASSWORD_KEY = "password";
private static final Set<AuthenticationMode> AUTHENTICATION_MODES = Collections.singleton(AuthenticationMode.PASSWORD);

@VisibleForTesting
static final Set<IRoleManager.Option> SUPPORTED_ROLE_OPTIONS =
EnumSet.of(IRoleManager.Option.PASSWORD,
IRoleManager.Option.HASHED_PASSWORD,
IRoleManager.Option.GENERATED_PASSWORD,
IRoleManager.Option.GENERATED_NAME);

@VisibleForTesting
static final Set<IRoleManager.Option> ALTERABLE_ROLE_OPTIONS =
EnumSet.of(IRoleManager.Option.PASSWORD,
IRoleManager.Option.HASHED_PASSWORD,
IRoleManager.Option.GENERATED_PASSWORD);

static final byte NUL = 0;
private SelectStatement authenticateStatement;

Expand All @@ -86,7 +99,26 @@ public PasswordAuthenticator()
AuthCacheService.instance.register(cache);
}

/**
* {@inheritDoc}
*/
@Override
public Set<IRoleManager.Option> getSupportedRoleOptions()
{
return SUPPORTED_ROLE_OPTIONS;
}

/**
* {@inheritDoc}
*/
@Override
public Set<IRoleManager.Option> getAlterableRoleOptions()
{
return ALTERABLE_ROLE_OPTIONS;
}

// No anonymous access.
@Override
public boolean requireAuthentication()
{
return true;
Expand Down Expand Up @@ -206,7 +238,7 @@ ResultMessage.Rows select(SelectStatement statement, QueryOptions options)
public Set<DataResource> protectedResources()
{
// Also protected by CassandraRoleManager, but the duplication doesn't hurt and is more explicit
return ImmutableSet.of(DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES));
return Set.of(DataResource.table(SchemaConstants.AUTH_KEYSPACE_NAME, AuthKeyspace.ROLES));
}

public void validateConfiguration() throws ConfigurationException
Expand Down Expand Up @@ -346,6 +378,7 @@ private CredentialsCache(PasswordAuthenticator authenticator)
// invalidate the key if the sentinel is loaded during a refresh
}

@Override
public void invalidateCredentials(String roleName)
{
invalidate(roleName);
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/auth/Roles.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ public class Roles

private static final Role NO_ROLE = new Role("", false, false, Collections.emptyMap(), Collections.emptySet());

public static final RolesCache cache = new RolesCache(DatabaseDescriptor.getRoleManager(), () -> DatabaseDescriptor.getAuthenticator().requireAuthentication());
public static final RolesCache cache = new RolesCache(DatabaseDescriptor.getRoleManager(), DatabaseDescriptor::isAuthenticationRequired);

/** Use {@link AuthCacheService#initializeAndRegisterCaches} rather than calling this directly */
public static void init()
Expand Down
38 changes: 37 additions & 1 deletion src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -1294,7 +1294,7 @@ public static GuardrailsOptions getGuardrailsConfig()
return guardrails;
}

private static void applyGuardrails()
public static void applyGuardrails()
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not thrilled with this, but I didn't see a cleaner way of initializing or defaulting guardrails config to enable CassandraRoleManager to construct in a unit test environment, without fully initializing DatabaseDescriptor. (If guardrails aren't explicitly initialized, CassandraRoleManager NPEs in the constructor.) Open to other suggestions.

Copy link
Contributor

@smiklosovic smiklosovic Nov 7, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jcshepherd what about calling DatabaseDescriptor.daemonInitialization() ? If you grep its usage, it is widely used in tests for these reasons.

You can also mock static methods like this, similar technique for DD you would be doing here

https://github.com/apache/cassandra/blob/trunk/test/unit/org/apache/cassandra/service/snapshot/SnapshotManagerTest.java#L155-L160

{
try
{
Expand Down Expand Up @@ -1959,16 +1959,52 @@ public static void setCryptoProvider(AbstractCryptoProvider cryptoProvider)
DatabaseDescriptor.cryptoProvider = cryptoProvider;
}

/**
* Returns the authenticator configured for this node.
*/
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the future this will become getDefaultAuthenticator(). Additional methods will be added to enable/disable negotiation and provide access to the node's supported authenticators.

public static IAuthenticator getAuthenticator()
{
return authenticator;
}

/**
* Returns an authenticator configured for this node, if it is of the requested type.
* @param clazz The class of the requested authenticator: e.g. PasswordAuthenticator.class.
* @return An Optional of the configured authenticator, if it is of the requested type; otherwise
* returns an empty Optional.
*/
public static <T extends IAuthenticator> Optional<T> getAuthenticator(Class<T> clazz)
{
return hasAuthenticator(clazz) ? Optional.of(clazz.cast(authenticator)) : Optional.empty();
}

/**
* Sets the authenticator used by this node to authenticate clients.
*/
public static void setAuthenticator(IAuthenticator authenticator)
{
DatabaseDescriptor.authenticator = authenticator;
}

/**
* Indicates if this node uses an authenticator that requires authentication.
*/
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the future, the semantics will change from "'The' authenticator requires authentication" to "Any supported authenticator requires authentication." Existing callers to this method should be unaffected by that change.

public static boolean isAuthenticationRequired()
{
return DatabaseDescriptor.getAuthenticator().requireAuthentication();
}

/**
* Indicates if this node is configured with an authenticator of the specified type.
* @param clazz The class of the authenticator.
* @return True if this node has an authenticator of the specified type, false otherwise.
*/
private static boolean hasAuthenticator(Class<? extends IAuthenticator> clazz)
{
return clazz.isAssignableFrom(authenticator.getClass());
}


public static IAuthorizer getAuthorizer()
{
return authorizer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import java.util.Optional;

import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.auth.PasswordAuthenticator;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.marshal.UTF8Type;
Expand All @@ -31,7 +30,7 @@ final class CredentialsCacheKeysTable extends AbstractMutableVirtualTable
private static final String ROLE = "role";

@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private final Optional<PasswordAuthenticator> passwordAuthenticatorOptional;
private final Optional<PasswordAuthenticator> maybePasswordAuthenticator;

CredentialsCacheKeysTable(String keyspace)
{
Expand All @@ -42,18 +41,14 @@ final class CredentialsCacheKeysTable extends AbstractMutableVirtualTable
.addPartitionKeyColumn(ROLE, UTF8Type.instance)
.build());

IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator();
if (authenticator instanceof PasswordAuthenticator)
this.passwordAuthenticatorOptional = Optional.of((PasswordAuthenticator) authenticator);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think now we have the opportunity to change name of this variable to maybePasswordAuthenticator. If you check the codebase, naming like this is more prevalent and reads better.

We also do not need this. here. There is no ambiguity as in what field we are setting.

else
this.passwordAuthenticatorOptional = Optional.empty();
maybePasswordAuthenticator = DatabaseDescriptor.getAuthenticator(PasswordAuthenticator.class);
}

public DataSet data()
{
SimpleDataSet result = new SimpleDataSet(metadata());

passwordAuthenticatorOptional
maybePasswordAuthenticator
.ifPresent(passwordAuthenticator -> passwordAuthenticator.getCredentialsCache().getAll()
.forEach((roleName, ignored) -> result.row(roleName)));

Expand All @@ -65,14 +60,14 @@ protected void applyPartitionDeletion(ColumnValues partitionKey)
{
String roleName = partitionKey.value(0);

passwordAuthenticatorOptional
maybePasswordAuthenticator
.ifPresent(passwordAuthenticator -> passwordAuthenticator.getCredentialsCache().invalidate(roleName));
}

@Override
public void truncate()
{
passwordAuthenticatorOptional
maybePasswordAuthenticator
.ifPresent(passwordAuthenticator -> passwordAuthenticator.getCredentialsCache().invalidate());
}
}
Loading