org.elasticsearch.xpack.security.authc.ApiKeyService Maven / Gradle / Ivy
Go to download
Show more of this group Show more artifacts with this name
Show all versions of x-pack-security Show documentation
Show all versions of x-pack-security Show documentation
Elasticsearch Expanded Pack Plugin - Security
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.security.authc;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchSecurityException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.TransportVersion;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.bulk.TransportBulkAction;
import org.elasticsearch.action.get.GetRequest;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.cache.Cache;
import org.elasticsearch.common.cache.CacheBuilder;
import org.elasticsearch.common.cache.RemovalListener;
import org.elasticsearch.common.cache.RemovalNotification.RemovalReason;
import org.elasticsearch.common.hash.MessageDigests;
import org.elasticsearch.common.logging.DeprecationCategory;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.HeaderWarning;
import org.elasticsearch.common.settings.SecureString;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.ListenableFuture;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.xcontent.LoggingDeprecationHandler;
import org.elasticsearch.common.xcontent.ObjectParserHelper;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.core.CharArrays;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.core.Tuple;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.telemetry.metric.MeterRegistry;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.DeprecationHandler;
import org.elasticsearch.xcontent.InstantiatingObjectParser;
import org.elasticsearch.xcontent.ObjectParser;
import org.elasticsearch.xcontent.ParseField;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.XContentFactory;
import org.elasticsearch.xcontent.XContentLocation;
import org.elasticsearch.xcontent.XContentParser;
import org.elasticsearch.xcontent.XContentParserConfiguration;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.XPackSettings;
import org.elasticsearch.xpack.core.security.ScrollHelper;
import org.elasticsearch.xpack.core.security.action.ClearSecurityCacheAction;
import org.elasticsearch.xpack.core.security.action.ClearSecurityCacheRequest;
import org.elasticsearch.xpack.core.security.action.ClearSecurityCacheResponse;
import org.elasticsearch.xpack.core.security.action.apikey.AbstractCreateApiKeyRequest;
import org.elasticsearch.xpack.core.security.action.apikey.ApiKey;
import org.elasticsearch.xpack.core.security.action.apikey.BaseBulkUpdateApiKeyRequest;
import org.elasticsearch.xpack.core.security.action.apikey.BaseUpdateApiKeyRequest;
import org.elasticsearch.xpack.core.security.action.apikey.BulkUpdateApiKeyResponse;
import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyResponse;
import org.elasticsearch.xpack.core.security.action.apikey.InvalidateApiKeyResponse;
import org.elasticsearch.xpack.core.security.authc.Authentication;
import org.elasticsearch.xpack.core.security.authc.AuthenticationField;
import org.elasticsearch.xpack.core.security.authc.AuthenticationResult;
import org.elasticsearch.xpack.core.security.authc.AuthenticationToken;
import org.elasticsearch.xpack.core.security.authc.RealmConfig;
import org.elasticsearch.xpack.core.security.authc.RealmDomain;
import org.elasticsearch.xpack.core.security.authc.support.Hasher;
import org.elasticsearch.xpack.core.security.authz.RoleDescriptor;
import org.elasticsearch.xpack.core.security.authz.privilege.ClusterPrivilegeResolver;
import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore;
import org.elasticsearch.xpack.core.security.authz.store.RoleReference;
import org.elasticsearch.xpack.core.security.support.MetadataUtils;
import org.elasticsearch.xpack.core.security.user.User;
import org.elasticsearch.xpack.security.metric.SecurityCacheMetrics;
import org.elasticsearch.xpack.security.support.CacheInvalidatorRegistry;
import org.elasticsearch.xpack.security.support.FeatureNotEnabledException;
import org.elasticsearch.xpack.security.support.FeatureNotEnabledException.Feature;
import org.elasticsearch.xpack.security.support.LockingAtomicCounter;
import org.elasticsearch.xpack.security.support.SecurityIndexManager;
import java.io.Closeable;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.security.MessageDigest;
import java.time.Clock;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static org.elasticsearch.TransportVersions.ROLE_REMOTE_CLUSTER_PRIVS;
import static org.elasticsearch.core.Strings.format;
import static org.elasticsearch.search.SearchService.DEFAULT_KEEPALIVE_SETTING;
import static org.elasticsearch.transport.RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY;
import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg;
import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg;
import static org.elasticsearch.xpack.core.ClientHelper.SECURITY_ORIGIN;
import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin;
import static org.elasticsearch.xpack.core.security.authz.RoleDescriptor.WORKFLOWS_RESTRICTION_VERSION;
import static org.elasticsearch.xpack.security.Security.SECURITY_CRYPTO_THREAD_POOL_NAME;
import static org.elasticsearch.xpack.security.support.SecurityIndexManager.Availability.PRIMARY_SHARDS;
import static org.elasticsearch.xpack.security.support.SecurityIndexManager.Availability.SEARCH_SHARDS;
import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_MAIN_ALIAS;
public class ApiKeyService implements Closeable {
private static final Logger logger = LogManager.getLogger(ApiKeyService.class);
private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(ApiKeyService.class);
public static final Setting PASSWORD_HASHING_ALGORITHM = XPackSettings.defaultStoredHashAlgorithmSetting(
"xpack.security.authc.api_key.hashing.algorithm",
(s) -> Hasher.PBKDF2.name()
);
public static final Setting DELETE_TIMEOUT = Setting.timeSetting(
"xpack.security.authc.api_key.delete.timeout",
TimeValue.MINUS_ONE,
Property.NodeScope
);
public static final Setting DELETE_INTERVAL = Setting.timeSetting(
"xpack.security.authc.api_key.delete.interval",
TimeValue.timeValueHours(24L),
Property.NodeScope,
Property.Dynamic
);
public static final Setting DELETE_RETENTION_PERIOD = Setting.positiveTimeSetting(
"xpack.security.authc.api_key.delete.retention_period",
TimeValue.timeValueDays(7),
Property.NodeScope,
Property.Dynamic
);
public static final Setting CACHE_HASH_ALGO_SETTING = Setting.simpleString(
"xpack.security.authc.api_key.cache.hash_algo",
"ssha256",
Setting.Property.NodeScope
);
public static final Setting CACHE_TTL_SETTING = Setting.timeSetting(
"xpack.security.authc.api_key.cache.ttl",
TimeValue.timeValueHours(24L),
Property.NodeScope
);
public static final Setting CACHE_MAX_KEYS_SETTING = Setting.intSetting(
"xpack.security.authc.api_key.cache.max_keys",
25000,
Property.NodeScope
);
public static final Setting DOC_CACHE_TTL_SETTING = Setting.timeSetting(
"xpack.security.authc.api_key.doc_cache.ttl",
TimeValue.timeValueMinutes(5),
TimeValue.timeValueMinutes(0),
TimeValue.timeValueMinutes(15),
Property.NodeScope
);
private static final RoleDescriptor.Parser ROLE_DESCRIPTOR_PARSER = RoleDescriptor.parserBuilder().allowRestriction(true).build();
private final Clock clock;
private final Client client;
private final SecurityIndexManager securityIndex;
private final ClusterService clusterService;
private final Hasher hasher;
private final boolean enabled;
private final Settings settings;
private final InactiveApiKeysRemover inactiveApiKeysRemover;
private final Cache> apiKeyAuthCache;
private final Hasher cacheHasher;
private final ThreadPool threadPool;
private final ApiKeyDocCache apiKeyDocCache;
// The API key secret is a Base64 encoded v4 UUID without padding. The UUID is 128 bits, i.e. 16 byte,
// which requires 22 digits of Base64 characters for encoding without padding.
// See also UUIDs.randomBase64UUIDSecureString
private static final int API_KEY_SECRET_LENGTH = 22;
private static final long EVICTION_MONITOR_INTERVAL_SECONDS = 300L; // 5 minutes
private static final long EVICTION_MONITOR_INTERVAL_NANOS = EVICTION_MONITOR_INTERVAL_SECONDS * 1_000_000_000L;
private static final long EVICTION_WARNING_THRESHOLD = 15L * EVICTION_MONITOR_INTERVAL_SECONDS; // 15 eviction per sec = 4500 in 5 min
private final AtomicLong lastEvictionCheckedAt = new AtomicLong(0);
private final LongAdder evictionCounter = new LongAdder();
private final List cacheMetrics;
@SuppressWarnings("this-escape")
public ApiKeyService(
Settings settings,
Clock clock,
Client client,
SecurityIndexManager securityIndex,
ClusterService clusterService,
CacheInvalidatorRegistry cacheInvalidatorRegistry,
ThreadPool threadPool,
MeterRegistry meterRegistry
) {
this.clock = clock;
this.client = client;
this.securityIndex = securityIndex;
this.clusterService = clusterService;
this.enabled = XPackSettings.API_KEY_SERVICE_ENABLED_SETTING.get(settings);
this.hasher = Hasher.resolve(PASSWORD_HASHING_ALGORITHM.get(settings));
this.settings = settings;
this.inactiveApiKeysRemover = new InactiveApiKeysRemover(settings, client, clusterService);
this.threadPool = threadPool;
this.cacheHasher = Hasher.resolve(CACHE_HASH_ALGO_SETTING.get(settings));
final TimeValue ttl = CACHE_TTL_SETTING.get(settings);
final int maximumWeight = CACHE_MAX_KEYS_SETTING.get(settings);
if (ttl.getNanos() > 0) {
this.apiKeyAuthCache = CacheBuilder.>builder()
.setExpireAfterAccess(ttl)
.setMaximumWeight(maximumWeight)
.removalListener(getAuthCacheRemovalListener(maximumWeight))
.build();
final TimeValue doc_ttl = DOC_CACHE_TTL_SETTING.get(settings);
this.apiKeyDocCache = doc_ttl.getNanos() == 0 ? null : new ApiKeyDocCache(doc_ttl, maximumWeight);
cacheInvalidatorRegistry.registerCacheInvalidator("api_key", new CacheInvalidatorRegistry.CacheInvalidator() {
@Override
public void invalidate(Collection keys) {
if (apiKeyDocCache != null) {
apiKeyDocCache.invalidate(keys);
}
keys.forEach(apiKeyAuthCache::invalidate);
}
@Override
public void invalidateAll() {
if (apiKeyDocCache != null) {
apiKeyDocCache.invalidateAll();
}
apiKeyAuthCache.invalidateAll();
}
});
cacheInvalidatorRegistry.registerCacheInvalidator("api_key_doc", new CacheInvalidatorRegistry.CacheInvalidator() {
@Override
public void invalidate(Collection keys) {
if (apiKeyDocCache != null) {
apiKeyDocCache.invalidate(keys);
}
}
@Override
public void invalidateAll() {
if (apiKeyDocCache != null) {
apiKeyDocCache.invalidateAll();
}
}
});
} else {
this.apiKeyAuthCache = null;
this.apiKeyDocCache = null;
}
if (enabled) {
final List cacheMetrics = new ArrayList<>();
if (this.apiKeyAuthCache != null) {
cacheMetrics.addAll(
SecurityCacheMetrics.registerAsyncCacheMetrics(
meterRegistry,
this.apiKeyAuthCache,
SecurityCacheMetrics.CacheType.API_KEY_AUTH_CACHE
)
);
}
if (this.apiKeyDocCache != null) {
cacheMetrics.addAll(
SecurityCacheMetrics.registerAsyncCacheMetrics(
meterRegistry,
this.apiKeyDocCache.docCache,
SecurityCacheMetrics.CacheType.API_KEY_DOCS_CACHE
)
);
cacheMetrics.addAll(
SecurityCacheMetrics.registerAsyncCacheMetrics(
meterRegistry,
this.apiKeyDocCache.roleDescriptorsBytesCache,
SecurityCacheMetrics.CacheType.API_KEY_ROLE_DESCRIPTORS_CACHE
)
);
}
this.cacheMetrics = List.copyOf(cacheMetrics);
} else {
this.cacheMetrics = List.of();
}
}
/**
* Asynchronously creates a new API key based off of the request and authentication
* @param authentication the authentication that this api key should be based off of
* @param request the request to create the api key included any permission restrictions
* @param userRoleDescriptors the user's actual roles that we always enforce
* @param listener the listener that will be used to notify of completion
*/
public void createApiKey(
Authentication authentication,
AbstractCreateApiKeyRequest request,
Set userRoleDescriptors,
ActionListener listener
) {
assert request.getType() != ApiKey.Type.CROSS_CLUSTER || false == authentication.isApiKey()
: "cannot create derived cross-cluster API keys (name=["
+ request.getName()
+ "], type=["
+ request.getType()
+ "], auth=["
+ authentication
+ "])";
assert request.getType() != ApiKey.Type.CROSS_CLUSTER || userRoleDescriptors.isEmpty()
: "owner user role descriptor must be empty for cross-cluster API keys (name=["
+ request.getName()
+ "], type=["
+ request.getType()
+ "], roles=["
+ userRoleDescriptors
+ "])";
ensureEnabled();
if (authentication == null) {
listener.onFailure(new IllegalArgumentException("authentication must be provided"));
} else {
final TransportVersion transportVersion = getMinTransportVersion();
if (transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY)
&& hasRemoteIndices(request.getRoleDescriptors())) {
// Creating API keys with roles which define remote indices privileges is not allowed in a mixed cluster.
listener.onFailure(
new IllegalArgumentException(
"all nodes must have version ["
+ TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY.toReleaseVersion()
+ "] or higher to support remote indices privileges for API keys"
)
);
return;
}
if (transportVersion.before(ROLE_REMOTE_CLUSTER_PRIVS) && hasRemoteCluster(request.getRoleDescriptors())) {
// Creating API keys with roles which define remote cluster privileges is not allowed in a mixed cluster.
listener.onFailure(
new IllegalArgumentException(
"all nodes must have version ["
+ ROLE_REMOTE_CLUSTER_PRIVS
+ "] or higher to support remote cluster privileges for API keys"
)
);
return;
}
if (transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY)
&& request.getType() == ApiKey.Type.CROSS_CLUSTER) {
listener.onFailure(
new IllegalArgumentException(
"all nodes must have version ["
+ TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY.toReleaseVersion()
+ "] or higher to support creating cross cluster API keys"
)
);
return;
}
final IllegalArgumentException workflowsValidationException = validateWorkflowsRestrictionConstraints(
transportVersion,
request.getRoleDescriptors(),
userRoleDescriptors
);
if (workflowsValidationException != null) {
listener.onFailure(workflowsValidationException);
return;
}
final Set userRolesWithoutDescription = removeUserRoleDescriptorDescriptions(userRoleDescriptors);
final Set filteredUserRoleDescriptors = maybeRemoveRemotePrivileges(
userRolesWithoutDescription,
transportVersion,
request.getId()
);
createApiKeyAndIndexIt(authentication, request, filteredUserRoleDescriptors, listener);
}
}
/**
* This method removes description from the given user's (limited-by) role descriptors.
* The description field is not supported for API key role descriptors hence storing limited-by roles with descriptions
* would be inconsistent and require handling backwards compatibility.
* Hence why we have to remove them before create/update of API key roles.
*/
static Set removeUserRoleDescriptorDescriptions(Set userRoleDescriptors) {
return userRoleDescriptors.stream().map(roleDescriptor -> {
if (roleDescriptor.hasDescription()) {
return new RoleDescriptor(
roleDescriptor.getName(),
roleDescriptor.getClusterPrivileges(),
roleDescriptor.getIndicesPrivileges(),
roleDescriptor.getApplicationPrivileges(),
roleDescriptor.getConditionalClusterPrivileges(),
roleDescriptor.getRunAs(),
roleDescriptor.getMetadata(),
roleDescriptor.getTransientMetadata(),
roleDescriptor.getRemoteIndicesPrivileges(),
roleDescriptor.getRemoteClusterPermissions(),
roleDescriptor.getRestriction(),
null
);
}
return roleDescriptor;
}).collect(Collectors.toSet());
}
private TransportVersion getMinTransportVersion() {
return clusterService.state().getMinTransportVersion();
}
private static boolean hasRemoteIndices(Collection roleDescriptors) {
return roleDescriptors != null && roleDescriptors.stream().anyMatch(RoleDescriptor::hasRemoteIndicesPrivileges);
}
private static boolean hasRemoteCluster(Collection roleDescriptors) {
return roleDescriptors != null && roleDescriptors.stream().anyMatch(RoleDescriptor::hasRemoteClusterPermissions);
}
private static IllegalArgumentException validateWorkflowsRestrictionConstraints(
TransportVersion transportVersion,
List requestRoleDescriptors,
Set userRoleDescriptors
) {
if (getNumberOfRoleDescriptorsWithRestriction(userRoleDescriptors) > 0L) {
return new IllegalArgumentException("owner user role descriptors must not include restriction");
}
final long numberOfRoleDescriptorsWithRestriction = getNumberOfRoleDescriptorsWithRestriction(requestRoleDescriptors);
if (numberOfRoleDescriptorsWithRestriction > 0L) {
// creating/updating API keys with restrictions is not allowed in a mixed cluster.
if (transportVersion.before(WORKFLOWS_RESTRICTION_VERSION)) {
return new IllegalArgumentException(
"all nodes must have version ["
+ WORKFLOWS_RESTRICTION_VERSION.toReleaseVersion()
+ "] or higher to support restrictions for API keys"
);
}
// It's only allowed to create/update API keys with a single role descriptor that is restricted.
if (numberOfRoleDescriptorsWithRestriction != 1L) {
return new IllegalArgumentException("more than one role descriptor with restriction is not supported");
}
// Combining roles with and without restriction is not allowed either.
if (numberOfRoleDescriptorsWithRestriction != requestRoleDescriptors.size()) {
return new IllegalArgumentException("combining role descriptors with and without restriction is not supported");
}
}
return null;
}
private static long getNumberOfRoleDescriptorsWithRestriction(Collection roleDescriptors) {
if (roleDescriptors == null || roleDescriptors.isEmpty()) {
return 0L;
}
return roleDescriptors.stream().filter(RoleDescriptor::hasRestriction).count();
}
private void createApiKeyAndIndexIt(
Authentication authentication,
AbstractCreateApiKeyRequest request,
Set userRoleDescriptors,
ActionListener listener
) {
final Instant created = clock.instant();
final Instant expiration = getApiKeyExpiration(created, request.getExpiration());
final SecureString apiKey = UUIDs.randomBase64UUIDSecureString();
assert ApiKey.Type.CROSS_CLUSTER != request.getType() || API_KEY_SECRET_LENGTH == apiKey.length()
: "Invalid API key (name=[" + request.getName() + "], type=[" + request.getType() + "], length=[" + apiKey.length() + "])";
computeHashForApiKey(apiKey, listener.delegateFailure((l, apiKeyHashChars) -> {
try (
XContentBuilder builder = newDocument(
apiKeyHashChars,
request.getName(),
authentication,
userRoleDescriptors,
created,
expiration,
request.getRoleDescriptors(),
request.getType(),
ApiKey.CURRENT_API_KEY_VERSION,
request.getMetadata()
)
) {
final BulkRequestBuilder bulkRequestBuilder = client.prepareBulk();
bulkRequestBuilder.add(
client.prepareIndex(SECURITY_MAIN_ALIAS)
.setSource(builder)
.setId(request.getId())
.setOpType(DocWriteRequest.OpType.CREATE)
.request()
);
bulkRequestBuilder.setRefreshPolicy(request.getRefreshPolicy());
final BulkRequest bulkRequest = bulkRequestBuilder.request();
securityIndex.prepareIndexIfNeededThenExecute(
listener::onFailure,
() -> executeAsyncWithOrigin(
client,
SECURITY_ORIGIN,
TransportBulkAction.TYPE,
bulkRequest,
TransportBulkAction.unwrappingSingleItemBulkResponse(ActionListener.wrap(indexResponse -> {
assert request.getId().equals(indexResponse.getId())
: "Mismatched API key (request=["
+ request.getId()
+ "](name=["
+ request.getName()
+ "]) index=["
+ indexResponse.getId()
+ "])";
assert indexResponse.getResult() == DocWriteResponse.Result.CREATED
: "Index response was [" + indexResponse.getResult() + "]";
final ListenableFuture listenableFuture = new ListenableFuture<>();
listenableFuture.onResponse(new CachedApiKeyHashResult(true, apiKey));
apiKeyAuthCache.put(request.getId(), listenableFuture);
listener.onResponse(new CreateApiKeyResponse(request.getName(), request.getId(), apiKey, expiration));
}, listener::onFailure))
)
);
} catch (IOException e) {
listener.onFailure(e);
} finally {
Arrays.fill(apiKeyHashChars, (char) 0);
}
}));
}
public void updateApiKeys(
final Authentication authentication,
final BaseBulkUpdateApiKeyRequest request,
final Set userRoleDescriptors,
final ActionListener listener
) {
assert request.getType() != ApiKey.Type.CROSS_CLUSTER || userRoleDescriptors.isEmpty()
: "owner user role descriptor must be empty for cross-cluster API keys (ids=["
+ (request.getIds().size() <= 10
? request.getIds()
: (request.getIds().size() + " including " + request.getIds().subList(0, 10)))
+ "], type=["
+ request.getType()
+ "], roles=["
+ userRoleDescriptors
+ "])";
ensureEnabled();
if (authentication == null) {
listener.onFailure(new IllegalArgumentException("authentication must be provided"));
return;
} else if (authentication.isApiKey()) {
listener.onFailure(
new IllegalArgumentException("authentication via API key not supported: only the owner user can update an API key")
);
return;
}
final TransportVersion transportVersion = getMinTransportVersion();
if (transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY) && hasRemoteIndices(request.getRoleDescriptors())) {
// Updating API keys with roles which define remote indices privileges is not allowed in a mixed cluster.
listener.onFailure(
new IllegalArgumentException(
"all nodes must have version ["
+ TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY.toReleaseVersion()
+ "] or higher to support remote indices privileges for API keys"
)
);
return;
}
if (transportVersion.before(ROLE_REMOTE_CLUSTER_PRIVS) && hasRemoteCluster(request.getRoleDescriptors())) {
// Updating API keys with roles which define remote cluster privileges is not allowed in a mixed cluster.
listener.onFailure(
new IllegalArgumentException(
"all nodes must have version ["
+ ROLE_REMOTE_CLUSTER_PRIVS
+ "] or higher to support remote indices privileges for API keys"
)
);
return;
}
final Exception workflowsValidationException = validateWorkflowsRestrictionConstraints(
transportVersion,
request.getRoleDescriptors(),
userRoleDescriptors
);
if (workflowsValidationException != null) {
listener.onFailure(workflowsValidationException);
return;
}
final String[] apiKeyIds = request.getIds().toArray(String[]::new);
final Set userRolesWithoutDescription = removeUserRoleDescriptorDescriptions(userRoleDescriptors);
final Set filteredUserRoleDescriptors = maybeRemoveRemotePrivileges(
userRolesWithoutDescription,
transportVersion,
apiKeyIds
);
if (logger.isDebugEnabled()) {
logger.debug("Updating [{}] API keys", buildDelimitedStringWithLimit(10, apiKeyIds));
}
findVersionedApiKeyDocsForSubject(
authentication,
apiKeyIds,
ActionListener.wrap(
versionedDocs -> updateApiKeys(authentication, request, filteredUserRoleDescriptors, versionedDocs, listener),
ex -> listener.onFailure(traceLog("bulk update", ex))
)
);
}
private void updateApiKeys(
final Authentication authentication,
final BaseBulkUpdateApiKeyRequest request,
final Set userRoleDescriptors,
final Collection targetVersionedDocs,
final ActionListener listener
) {
logger.trace("Found [{}] API keys of [{}] requested for update", targetVersionedDocs.size(), request.getIds().size());
assert targetVersionedDocs.size() <= request.getIds().size()
: "more docs were found for update than were requested. found ["
+ targetVersionedDocs.size()
+ "] requested ["
+ request.getIds().size()
+ "]";
final BulkUpdateApiKeyResponse.Builder responseBuilder = BulkUpdateApiKeyResponse.builder();
final BulkRequestBuilder bulkRequestBuilder = client.prepareBulk();
for (VersionedApiKeyDoc versionedDoc : targetVersionedDocs) {
final String apiKeyId = versionedDoc.id();
try {
validateForUpdate(apiKeyId, request.getType(), authentication, versionedDoc.doc());
final IndexRequest indexRequest = maybeBuildIndexRequest(versionedDoc, authentication, request, userRoleDescriptors);
final boolean isNoop = indexRequest == null;
if (isNoop) {
logger.debug("Detected noop update request for API key [{}]. Skipping index request", apiKeyId);
responseBuilder.noop(apiKeyId);
} else {
bulkRequestBuilder.add(indexRequest);
}
} catch (Exception ex) {
responseBuilder.error(apiKeyId, traceLog("prepare index request for update", ex));
}
}
addErrorsForNotFoundApiKeys(responseBuilder, targetVersionedDocs, request.getIds());
if (bulkRequestBuilder.numberOfActions() == 0) {
logger.trace("No bulk request execution necessary for API key update");
listener.onResponse(responseBuilder.build());
return;
}
logger.trace("Executing bulk request to update [{}] API keys", bulkRequestBuilder.numberOfActions());
bulkRequestBuilder.setRefreshPolicy(defaultCreateDocRefreshPolicy(settings));
securityIndex.prepareIndexIfNeededThenExecute(
ex -> listener.onFailure(traceLog("prepare security index before update", ex)),
() -> executeAsyncWithOrigin(
client.threadPool().getThreadContext(),
SECURITY_ORIGIN,
bulkRequestBuilder.request(),
ActionListener.wrap(
bulkResponse -> buildResponseAndClearCache(bulkResponse, responseBuilder, listener),
ex -> listener.onFailure(traceLog("execute bulk request for update", ex))
),
client::bulk
)
);
}
// package-private for testing
void validateForUpdate(
final String apiKeyId,
final ApiKey.Type expectedType,
final Authentication authentication,
final ApiKeyDoc apiKeyDoc
) {
assert authentication.getEffectiveSubject().getUser().principal().equals(apiKeyDoc.creator.get("principal"))
: "Authenticated user should be owner (authentication=["
+ authentication
+ "], owner=["
+ apiKeyDoc.creator
+ "], id=["
+ apiKeyId
+ "])";
if (apiKeyDoc.invalidated) {
throw new IllegalArgumentException("cannot update invalidated API key [" + apiKeyId + "]");
}
boolean expired = apiKeyDoc.expirationTime != -1 && clock.instant().isAfter(Instant.ofEpochMilli(apiKeyDoc.expirationTime));
if (expired) {
throw new IllegalArgumentException("cannot update expired API key [" + apiKeyId + "]");
}
if (Strings.isNullOrEmpty(apiKeyDoc.name)) {
throw new IllegalArgumentException("cannot update legacy API key [" + apiKeyId + "] without name");
}
if (expectedType != apiKeyDoc.type) {
throw new IllegalArgumentException(
"cannot update API key of type [" + apiKeyDoc.type.value() + "] while expected type is [" + expectedType.value() + "]"
);
}
}
/**
* This method removes remote indices and cluster privileges from the given role descriptors
* when we are in a mixed cluster in which some of the nodes do not support remote indices/clusters.
* Storing these roles would cause parsing issues on old nodes
* (i.e. nodes running with transport version before
* {@link org.elasticsearch.transport.RemoteClusterPortSettings#TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY}).
*/
static Set maybeRemoveRemotePrivileges(
final Set userRoleDescriptors,
final TransportVersion transportVersion,
final String... apiKeyIds
) {
if (transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY)
|| transportVersion.before(ROLE_REMOTE_CLUSTER_PRIVS)) {
final Set affectedRoles = new HashSet<>();
final Set result = userRoleDescriptors.stream().map(roleDescriptor -> {
if (roleDescriptor.hasRemoteIndicesPrivileges() || roleDescriptor.hasRemoteClusterPermissions()) {
affectedRoles.add(roleDescriptor);
return new RoleDescriptor(
roleDescriptor.getName(),
roleDescriptor.getClusterPrivileges(),
roleDescriptor.getIndicesPrivileges(),
roleDescriptor.getApplicationPrivileges(),
roleDescriptor.getConditionalClusterPrivileges(),
roleDescriptor.getRunAs(),
roleDescriptor.getMetadata(),
roleDescriptor.getTransientMetadata(),
roleDescriptor.hasRemoteIndicesPrivileges()
&& transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY)
? null
: roleDescriptor.getRemoteIndicesPrivileges(),
roleDescriptor.hasRemoteClusterPermissions() && transportVersion.before(ROLE_REMOTE_CLUSTER_PRIVS)
? null
: roleDescriptor.getRemoteClusterPermissions(),
roleDescriptor.getRestriction(),
roleDescriptor.getDescription()
);
}
return roleDescriptor;
}).collect(Collectors.toSet());
if (false == affectedRoles.isEmpty()) {
List affectedRolesNames = affectedRoles.stream().map(RoleDescriptor::getName).sorted().collect(Collectors.toList());
if (affectedRoles.stream().anyMatch(RoleDescriptor::hasRemoteIndicesPrivileges)
&& transportVersion.before(TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY)) {
logger.info(
"removed remote indices privileges from role(s) {} for API key(s) [{}]",
affectedRolesNames,
buildDelimitedStringWithLimit(10, apiKeyIds)
);
HeaderWarning.addWarning(
"Removed API key's remote indices privileges from role(s) "
+ affectedRolesNames
+ ". Remote indices are not supported by all nodes in the cluster. "
);
}
if (affectedRoles.stream().anyMatch(RoleDescriptor::hasRemoteClusterPermissions)
&& transportVersion.before(ROLE_REMOTE_CLUSTER_PRIVS)) {
logger.info(
"removed remote cluster privileges from role(s) {} for API key(s) [{}]",
affectedRolesNames,
buildDelimitedStringWithLimit(10, apiKeyIds)
);
HeaderWarning.addWarning(
"Removed API key's remote cluster privileges from role(s) "
+ affectedRolesNames
+ ". Remote cluster privileges are not supported by all nodes in the cluster."
);
}
}
return result;
}
return userRoleDescriptors;
}
/**
* Builds a comma delimited string from the given string values (e.g. value1, value2...).
* The number of values included can be controlled with the {@code limit}. The limit must be a positive number.
* Note: package-private for testing
*/
static String buildDelimitedStringWithLimit(final int limit, final String... values) {
if (limit <= 0) {
throw new IllegalArgumentException("limit must be positive number");
}
if (values == null || values.length <= 0) {
return "";
}
final int total = values.length;
final int omitted = Math.max(0, total - limit);
final int valuesToAppend = Math.min(limit, total);
final int capacityForOmittedInfoText = 5; // The number of additional info strings we append when omitting.
final int capacity = valuesToAppend + (omitted > 0 ? capacityForOmittedInfoText : 0);
final StringBuilder sb = new StringBuilder(capacity);
int counter = 0;
while (counter < valuesToAppend) {
sb.append(values[counter]);
counter += 1;
if (counter < valuesToAppend) {
sb.append(", ");
}
}
if (omitted > 0) {
sb.append("... (").append(total).append(" in total, ").append(omitted).append(" omitted)");
}
return sb.toString();
}
/**
* package-private for testing
*/
static XContentBuilder newDocument(
char[] apiKeyHashChars,
String name,
Authentication authentication,
Set userRoleDescriptors,
Instant created,
Instant expiration,
List keyRoleDescriptors,
ApiKey.Type type,
ApiKey.Version version,
@Nullable Map metadata
) throws IOException {
final XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject()
.field("doc_type", "api_key")
.field("type", type.value())
.field("creation_time", created.toEpochMilli())
.field("expiration_time", expiration == null ? null : expiration.toEpochMilli())
.field("api_key_invalidated", false);
addApiKeyHash(builder, apiKeyHashChars);
addRoleDescriptors(builder, keyRoleDescriptors);
addLimitedByRoleDescriptors(builder, userRoleDescriptors);
builder.field("name", name).field("version", version.version()).field("metadata_flattened", metadata);
addCreator(builder, authentication);
return builder.endObject();
}
// package private for testing
/**
* @return `null` if the update is a noop, i.e., if no changes to `currentApiKeyDoc` are required
*/
@Nullable
static XContentBuilder maybeBuildUpdatedDocument(
final String apiKeyId,
final ApiKeyDoc currentApiKeyDoc,
final ApiKey.Version targetDocVersion,
final Authentication authentication,
final BaseUpdateApiKeyRequest request,
final Set userRoleDescriptors,
final Clock clock
) throws IOException {
assert currentApiKeyDoc.type == request.getType()
: "API Key doc does not match request type (key-id=["
+ apiKeyId
+ "], doc=["
+ currentApiKeyDoc.type
+ "], request=["
+ request.getType()
+ "])";
if (isNoop(apiKeyId, currentApiKeyDoc, targetDocVersion, authentication, request, userRoleDescriptors)) {
return null;
}
final XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject()
.field("doc_type", "api_key")
.field("type", currentApiKeyDoc.type.value())
.field("creation_time", currentApiKeyDoc.creationTime)
.field("api_key_invalidated", false);
if (request.getExpiration() != null) {
builder.field("expiration_time", getApiKeyExpiration(clock.instant(), request.getExpiration()).toEpochMilli());
} else {
builder.field("expiration_time", currentApiKeyDoc.expirationTime == -1 ? null : currentApiKeyDoc.expirationTime);
}
addApiKeyHash(builder, currentApiKeyDoc.hash.toCharArray());
final List keyRoles = request.getRoleDescriptors();
if (keyRoles != null) {
logger.trace(() -> format("Building API key doc with updated role descriptors [%s]", keyRoles));
addRoleDescriptors(builder, keyRoles);
} else {
assert currentApiKeyDoc.roleDescriptorsBytes != null : "Role descriptors for [" + apiKeyId + "] are null";
builder.rawField("role_descriptors", currentApiKeyDoc.roleDescriptorsBytes.streamInput(), XContentType.JSON);
}
addLimitedByRoleDescriptors(builder, userRoleDescriptors);
builder.field("name", currentApiKeyDoc.name).field("version", targetDocVersion.version());
assert currentApiKeyDoc.metadataFlattened == null
|| MetadataUtils.containsReservedMetadata(
XContentHelper.convertToMap(currentApiKeyDoc.metadataFlattened, false, XContentType.JSON).v2()
) == false : "API key doc [" + apiKeyId + "] to be updated contains reserved metadata";
final Map metadata = request.getMetadata();
if (metadata != null) {
logger.trace(() -> format("Building API key doc with updated metadata [%s]", metadata));
builder.field("metadata_flattened", metadata);
} else {
builder.rawField(
"metadata_flattened",
currentApiKeyDoc.metadataFlattened == null
? ApiKeyDoc.NULL_BYTES.streamInput()
: currentApiKeyDoc.metadataFlattened.streamInput(),
XContentType.JSON
);
}
addCreator(builder, authentication);
return builder.endObject();
}
private static boolean isNoop(
final String apiKeyId,
final ApiKeyDoc apiKeyDoc,
final ApiKey.Version targetDocVersion,
final Authentication authentication,
final BaseUpdateApiKeyRequest request,
final Set userRoleDescriptors
) throws IOException {
if (apiKeyDoc.version != targetDocVersion.version()) {
return false;
}
if (request.getExpiration() != null) {
// Since expiration is relative current time, it's not likely that it matches the stored value to the ms, so assume update
return false;
}
final Map currentCreator = apiKeyDoc.creator;
final var user = authentication.getEffectiveSubject().getUser();
final var sourceRealm = authentication.getEffectiveSubject().getRealm();
if (false == (Objects.equals(user.principal(), currentCreator.get("principal"))
&& Objects.equals(user.fullName(), currentCreator.get("full_name"))
&& Objects.equals(user.email(), currentCreator.get("email"))
&& Objects.equals(user.metadata(), currentCreator.get("metadata"))
&& Objects.equals(sourceRealm.getName(), currentCreator.get("realm"))
&& Objects.equals(sourceRealm.getType(), currentCreator.get("realm_type")))) {
return false;
}
if (sourceRealm.getDomain() != null) {
if (currentCreator.get("realm_domain") == null) {
return false;
}
@SuppressWarnings("unchecked")
var m = (Map) currentCreator.get("realm_domain");
final RealmDomain currentRealmDomain;
try (var parser = XContentHelper.mapToXContentParser(XContentParserConfiguration.EMPTY, m)) {
currentRealmDomain = RealmDomain.fromXContent(parser);
}
if (sourceRealm.getDomain().equals(currentRealmDomain) == false) {
return false;
}
} else {
if (currentCreator.get("realm_domain") != null) {
return false;
}
}
final Map newMetadata = request.getMetadata();
if (newMetadata != null) {
if (apiKeyDoc.metadataFlattened == null) {
return false;
}
final Map currentMetadata = XContentHelper.convertToMap(apiKeyDoc.metadataFlattened, false, XContentType.JSON)
.v2();
if (newMetadata.equals(currentMetadata) == false) {
return false;
}
}
final List newRoleDescriptors = request.getRoleDescriptors();
if (newRoleDescriptors != null) {
final List currentRoleDescriptors = parseRoleDescriptorsBytes(apiKeyId, apiKeyDoc.roleDescriptorsBytes, false);
if (false == (newRoleDescriptors.size() == currentRoleDescriptors.size()
&& Set.copyOf(newRoleDescriptors).containsAll(currentRoleDescriptors))) {
return false;
}
if (newRoleDescriptors.size() == currentRoleDescriptors.size()) {
for (int i = 0; i < currentRoleDescriptors.size(); i++) {
// if remote cluster permissions are not equal, then it is not a noop
if (currentRoleDescriptors.get(i)
.getRemoteClusterPermissions()
.equals(newRoleDescriptors.get(i).getRemoteClusterPermissions()) == false) {
return false;
}
}
}
}
assert userRoleDescriptors != null : "API Key [" + apiKeyId + "] has null role descriptors";
final List currentLimitedByRoleDescriptors = parseRoleDescriptorsBytes(
apiKeyId,
apiKeyDoc.limitedByRoleDescriptorsBytes,
// We want the 7.x `LEGACY_SUPERUSER_ROLE_DESCRIPTOR` role descriptor to be returned here to auto-update
// `LEGACY_SUPERUSER_ROLE_DESCRIPTOR` to `ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR`, when we update a 7.x API key.
false
);
return (userRoleDescriptors.size() == currentLimitedByRoleDescriptors.size()
&& userRoleDescriptors.containsAll(currentLimitedByRoleDescriptors));
}
void tryAuthenticate(ThreadContext ctx, ApiKeyCredentials credentials, ActionListener> listener) {
if (false == isEnabled()) {
listener.onResponse(AuthenticationResult.notHandled());
}
assert credentials != null : "api key credentials must not be null";
loadApiKeyAndValidateCredentials(ctx, credentials, ActionListener.wrap(response -> {
credentials.close();
listener.onResponse(response);
}, e -> {
credentials.close();
listener.onFailure(e);
}));
}
void loadApiKeyAndValidateCredentials(
ThreadContext ctx,
ApiKeyCredentials credentials,
ActionListener> listener
) {
final String docId = credentials.getId();
Consumer validator = apiKeyDoc -> validateApiKeyCredentials(
docId,
apiKeyDoc,
credentials,
clock,
listener.delegateResponse((l, e) -> {
if (ExceptionsHelper.unwrapCause(e) instanceof EsRejectedExecutionException) {
l.onResponse(AuthenticationResult.terminate("server is too busy to respond", e));
} else {
l.onFailure(e);
}
})
);
final long invalidationCount;
if (apiKeyDocCache != null) {
ApiKeyDoc existing = apiKeyDocCache.get(docId);
if (existing != null) {
validator.accept(existing);
return;
}
// API key doc not found in cache, take a record of the current invalidation count to prepare for caching
invalidationCount = apiKeyDocCache.getInvalidationCount();
} else {
invalidationCount = -1;
}
final GetRequest getRequest = client.prepareGet(SECURITY_MAIN_ALIAS, docId).setFetchSource(true).request();
executeAsyncWithOrigin(ctx, SECURITY_ORIGIN, getRequest, ActionListener.wrap(response -> {
if (response.isExists()) {
final ApiKeyDoc apiKeyDoc;
try (
XContentParser parser = XContentHelper.createParser(
XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE),
response.getSourceAsBytesRef(),
XContentType.JSON
)
) {
apiKeyDoc = ApiKeyDoc.fromXContent(parser);
}
if (invalidationCount != -1) {
apiKeyDocCache.putIfNoInvalidationSince(docId, apiKeyDoc, invalidationCount);
}
validator.accept(apiKeyDoc);
} else {
if (apiKeyAuthCache != null) {
apiKeyAuthCache.invalidate(docId);
}
listener.onResponse(AuthenticationResult.unsuccessful("unable to find apikey with id " + credentials.getId(), null));
}
}, e -> {
if (ExceptionsHelper.unwrapCause(e) instanceof EsRejectedExecutionException) {
listener.onResponse(AuthenticationResult.terminate("server is too busy to respond", e));
} else {
listener.onResponse(
AuthenticationResult.unsuccessful("apikey authentication for id " + credentials.getId() + " encountered a failure", e)
);
}
}), client::get);
}
public List parseRoleDescriptors(
final String apiKeyId,
final Map roleDescriptorsMap,
RoleReference.ApiKeyRoleType roleType
) {
if (roleDescriptorsMap == null) {
return null;
}
final List roleDescriptors = roleDescriptorsMap.entrySet().stream().map(entry -> {
final String name = entry.getKey();
@SuppressWarnings("unchecked")
final Map rdMap = (Map) entry.getValue();
try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) {
builder.map(rdMap);
try (
XContentParser parser = XContentHelper.createParserNotCompressed(
XContentParserConfiguration.EMPTY.withDeprecationHandler(
new ApiKeyLoggingDeprecationHandler(deprecationLogger, apiKeyId)
),
BytesReference.bytes(builder),
XContentType.JSON
)
) {
return ROLE_DESCRIPTOR_PARSER.parse(name, parser);
}
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}).toList();
return roleType == RoleReference.ApiKeyRoleType.LIMITED_BY
? maybeReplaceSuperuserRoleDescriptor(apiKeyId, roleDescriptors)
: roleDescriptors;
}
public List parseRoleDescriptorsBytes(
final String apiKeyId,
BytesReference bytesReference,
RoleReference.ApiKeyRoleType roleType
) {
return parseRoleDescriptorsBytes(apiKeyId, bytesReference, roleType == RoleReference.ApiKeyRoleType.LIMITED_BY);
}
private static List parseRoleDescriptorsBytes(
final String apiKeyId,
BytesReference bytesReference,
final boolean replaceLegacySuperuserRoleDescriptor
) {
if (bytesReference == null) {
return Collections.emptyList();
}
List roleDescriptors = new ArrayList<>();
try (
XContentParser parser = XContentHelper.createParser(
XContentParserConfiguration.EMPTY.withDeprecationHandler(new ApiKeyLoggingDeprecationHandler(deprecationLogger, apiKeyId)),
bytesReference,
XContentType.JSON
)
) {
parser.nextToken(); // skip outer start object
while (parser.nextToken() != XContentParser.Token.END_OBJECT) {
parser.nextToken(); // role name
String roleName = parser.currentName();
roleDescriptors.add(ROLE_DESCRIPTOR_PARSER.parse(roleName, parser));
}
} catch (IOException e) {
throw new UncheckedIOException(e);
}
return replaceLegacySuperuserRoleDescriptor ? maybeReplaceSuperuserRoleDescriptor(apiKeyId, roleDescriptors) : roleDescriptors;
}
// package private for tests
static final RoleDescriptor LEGACY_SUPERUSER_ROLE_DESCRIPTOR = new RoleDescriptor(
"superuser",
new String[] { "all" },
new RoleDescriptor.IndicesPrivileges[] {
RoleDescriptor.IndicesPrivileges.builder().indices("*").privileges("all").allowRestrictedIndices(true).build() },
new RoleDescriptor.ApplicationResourcePrivileges[] {
RoleDescriptor.ApplicationResourcePrivileges.builder().application("*").privileges("*").resources("*").build() },
null,
new String[] { "*" },
MetadataUtils.DEFAULT_RESERVED_METADATA,
Collections.emptyMap()
);
// This method should only be called to replace the superuser role descriptor for the limited-by roles of an API Key.
// We do not replace assigned roles because they are created explicitly by users.
// Before #82049, it is possible to specify a role descriptor for API keys that is identical to the builtin superuser role
// (including the _reserved metadata field).
private static List maybeReplaceSuperuserRoleDescriptor(String apiKeyId, List roleDescriptors) {
// Scan through all the roles because superuser can be one of the roles that a user has. Unlike building the Role object,
// capturing role descriptors does not preempt for superuser.
return roleDescriptors.stream().map(rd -> {
// Since we are only replacing limited-by roles and all limited-by roles are looked up with role providers,
// it is technically possible to just check the name of superuser and the _reserved metadata field.
// But the gain is not much since role resolving is cached and comparing the whole role descriptor is still safer.
if (rd.equals(LEGACY_SUPERUSER_ROLE_DESCRIPTOR)) {
logger.debug("replacing superuser role for API key [{}]", apiKeyId);
return ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR;
}
return rd;
}).toList();
}
/**
* Validates the ApiKey using the source map
* @param docId the identifier of the document that was retrieved from the security index
* @param apiKeyDoc the partially deserialized API key document
* @param credentials the credentials provided by the user
* @param listener the listener to notify after verification
*/
void validateApiKeyCredentials(
String docId,
ApiKeyDoc apiKeyDoc,
ApiKeyCredentials credentials,
Clock clock,
ActionListener> listener
) {
if ("api_key".equals(apiKeyDoc.docType) == false) {
listener.onResponse(
AuthenticationResult.unsuccessful("document [" + docId + "] is [" + apiKeyDoc.docType + "] not an api key", null)
);
} else if (apiKeyDoc.invalidated == null) {
listener.onResponse(AuthenticationResult.unsuccessful("api key document is missing invalidated field", null));
} else if (apiKeyDoc.invalidated) {
if (apiKeyAuthCache != null) {
apiKeyAuthCache.invalidate(docId);
}
listener.onResponse(AuthenticationResult.unsuccessful("api key [" + credentials.getId() + "] has been invalidated", null));
} else {
if (apiKeyDoc.hash == null) {
throw new IllegalStateException("api key hash is missing");
}
if (apiKeyAuthCache != null) {
final AtomicBoolean valueAlreadyInCache = new AtomicBoolean(true);
final ListenableFuture listenableCacheEntry;
try {
listenableCacheEntry = apiKeyAuthCache.computeIfAbsent(credentials.getId(), k -> {
valueAlreadyInCache.set(false);
return new ListenableFuture<>();
});
} catch (ExecutionException e) {
listener.onFailure(e);
return;
}
if (valueAlreadyInCache.get()) {
listenableCacheEntry.addListener(ActionListener.wrap(result -> {
if (result.success) {
if (result.verify(credentials.getKey())) {
// move on
validateApiKeyTypeAndExpiration(apiKeyDoc, credentials, clock, listener);
} else {
listener.onResponse(
AuthenticationResult.unsuccessful("invalid credentials for API key [" + credentials.getId() + "]", null)
);
}
} else if (result.verify(credentials.getKey())) { // same key, pass the same result
listener.onResponse(
AuthenticationResult.unsuccessful("invalid credentials for API key [" + credentials.getId() + "]", null)
);
} else {
apiKeyAuthCache.invalidate(credentials.getId(), listenableCacheEntry);
validateApiKeyCredentials(docId, apiKeyDoc, credentials, clock, listener);
}
}, listener::onFailure), threadPool.generic(), threadPool.getThreadContext());
} else {
verifyKeyAgainstHash(apiKeyDoc.hash, credentials, ActionListener.wrap(verified -> {
listenableCacheEntry.onResponse(new CachedApiKeyHashResult(verified, credentials.getKey()));
if (verified) {
// move on
validateApiKeyTypeAndExpiration(apiKeyDoc, credentials, clock, listener);
} else {
listener.onResponse(
AuthenticationResult.unsuccessful("invalid credentials for API key [" + credentials.getId() + "]", null)
);
}
}, exception -> {
// Crypto threadpool queue is full, invalidate this cache entry and make sure nothing is going to wait on it
logger.warn(
Strings.format(
"rejecting possibly valid API key authentication because the [%s] threadpool is full",
SECURITY_CRYPTO_THREAD_POOL_NAME
)
);
apiKeyAuthCache.invalidate(credentials.getId(), listenableCacheEntry);
listenableCacheEntry.onFailure(exception);
listener.onFailure(exception);
}));
}
} else {
verifyKeyAgainstHash(apiKeyDoc.hash, credentials, ActionListener.wrap(verified -> {
if (verified) {
// move on
validateApiKeyTypeAndExpiration(apiKeyDoc, credentials, clock, listener);
} else {
listener.onResponse(
AuthenticationResult.unsuccessful("invalid credentials for API key [" + credentials.getId() + "]", null)
);
}
}, listener::onFailure));
}
}
}
// pkg private for testing
CachedApiKeyHashResult getFromCache(String id) {
return apiKeyAuthCache == null ? null : apiKeyAuthCache.get(id).result();
}
// pkg private for testing
Cache> getApiKeyAuthCache() {
return apiKeyAuthCache;
}
// pkg private for testing
Cache getDocCache() {
return apiKeyDocCache == null ? null : apiKeyDocCache.docCache;
}
// pkg private for testing
Cache getRoleDescriptorsBytesCache() {
return apiKeyDocCache == null ? null : apiKeyDocCache.roleDescriptorsBytesCache;
}
// package-private for testing
static void validateApiKeyTypeAndExpiration(
ApiKeyDoc apiKeyDoc,
ApiKeyCredentials credentials,
Clock clock,
ActionListener> listener
) {
if (apiKeyDoc.type != credentials.expectedType) {
listener.onResponse(
AuthenticationResult.terminate(
Strings.format(
"authentication expected API key type of [%s], but API key [%s] has type [%s]",
credentials.expectedType.value(),
credentials.getId(),
apiKeyDoc.type.value()
)
)
);
return;
}
if (apiKeyDoc.expirationTime == -1 || Instant.ofEpochMilli(apiKeyDoc.expirationTime).isAfter(clock.instant())) {
final String principal = Objects.requireNonNull((String) apiKeyDoc.creator.get("principal"));
final String fullName = (String) apiKeyDoc.creator.get("full_name");
final String email = (String) apiKeyDoc.creator.get("email");
@SuppressWarnings("unchecked")
Map metadata = (Map) apiKeyDoc.creator.get("metadata");
final User apiKeyUser = new User(principal, Strings.EMPTY_ARRAY, fullName, email, metadata, true);
final Map authResultMetadata = new HashMap<>();
authResultMetadata.put(AuthenticationField.API_KEY_CREATOR_REALM_NAME, apiKeyDoc.creator.get("realm"));
authResultMetadata.put(AuthenticationField.API_KEY_CREATOR_REALM_TYPE, apiKeyDoc.creator.get("realm_type"));
authResultMetadata.put(AuthenticationField.API_KEY_ROLE_DESCRIPTORS_KEY, apiKeyDoc.roleDescriptorsBytes);
authResultMetadata.put(AuthenticationField.API_KEY_LIMITED_ROLE_DESCRIPTORS_KEY, apiKeyDoc.limitedByRoleDescriptorsBytes);
authResultMetadata.put(AuthenticationField.API_KEY_ID_KEY, credentials.getId());
authResultMetadata.put(AuthenticationField.API_KEY_NAME_KEY, apiKeyDoc.name);
authResultMetadata.put(AuthenticationField.API_KEY_TYPE_KEY, apiKeyDoc.type.value());
if (apiKeyDoc.metadataFlattened != null) {
authResultMetadata.put(AuthenticationField.API_KEY_METADATA_KEY, apiKeyDoc.metadataFlattened);
}
listener.onResponse(AuthenticationResult.success(apiKeyUser, authResultMetadata));
} else {
listener.onResponse(AuthenticationResult.unsuccessful("api key is expired", null));
}
}
ApiKeyCredentials parseCredentialsFromApiKeyString(SecureString apiKeyString) {
if (false == isEnabled()) {
return null;
}
return parseApiKey(apiKeyString, ApiKey.Type.REST);
}
static ApiKeyCredentials getCredentialsFromHeader(final String header, ApiKey.Type expectedType) {
return parseApiKey(Authenticator.extractCredentialFromHeaderValue(header, "ApiKey"), expectedType);
}
public static String withApiKeyPrefix(final String encodedApiKey) {
return "ApiKey " + encodedApiKey;
}
private static ApiKeyCredentials parseApiKey(SecureString apiKeyString, ApiKey.Type expectedType) {
if (apiKeyString != null) {
final byte[] decodedApiKeyCredBytes = Base64.getDecoder().decode(CharArrays.toUtf8Bytes(apiKeyString.getChars()));
char[] apiKeyCredChars = null;
try {
apiKeyCredChars = CharArrays.utf8BytesToChars(decodedApiKeyCredBytes);
int colonIndex = -1;
for (int i = 0; i < apiKeyCredChars.length; i++) {
if (apiKeyCredChars[i] == ':') {
colonIndex = i;
break;
}
}
if (colonIndex < 1) {
throw new IllegalArgumentException("invalid ApiKey value");
}
final int secretStartPos = colonIndex + 1;
if (ApiKey.Type.CROSS_CLUSTER == expectedType && API_KEY_SECRET_LENGTH != apiKeyCredChars.length - secretStartPos) {
throw new IllegalArgumentException("invalid cross-cluster API key value");
}
return new ApiKeyCredentials(
new String(Arrays.copyOfRange(apiKeyCredChars, 0, colonIndex)),
new SecureString(Arrays.copyOfRange(apiKeyCredChars, secretStartPos, apiKeyCredChars.length)),
expectedType
);
} finally {
if (apiKeyCredChars != null) {
Arrays.fill(apiKeyCredChars, (char) 0);
}
}
}
return null;
}
void computeHashForApiKey(SecureString apiKey, ActionListener listener) {
threadPool.executor(SECURITY_CRYPTO_THREAD_POOL_NAME).execute(ActionRunnable.supply(listener, () -> hasher.hash(apiKey)));
}
// Protected instance method so this can be mocked
protected void verifyKeyAgainstHash(String apiKeyHash, ApiKeyCredentials credentials, ActionListener listener) {
threadPool.executor(SECURITY_CRYPTO_THREAD_POOL_NAME).execute(ActionRunnable.supply(listener, () -> {
Hasher hasher = Hasher.resolveFromHash(apiKeyHash.toCharArray());
final char[] apiKeyHashChars = apiKeyHash.toCharArray();
try {
return hasher.verify(credentials.getKey(), apiKeyHashChars);
} finally {
Arrays.fill(apiKeyHashChars, (char) 0);
}
}));
}
private static Instant getApiKeyExpiration(Instant now, @Nullable TimeValue expiration) {
if (expiration != null) {
return now.plusSeconds(expiration.getSeconds());
} else {
return null;
}
}
private boolean isEnabled() {
return enabled;
}
public void ensureEnabled() {
if (enabled == false) {
throw new FeatureNotEnabledException(Feature.API_KEY_SERVICE, "api keys are not enabled");
}
}
public void crossClusterApiKeyUsageStats(ActionListener