From ef8b3822a8bed4d60664a93f1616ed0d6f8b88c1 Mon Sep 17 00:00:00 2001 From: Sean Zatz Date: Tue, 21 Oct 2025 16:58:28 -0400 Subject: [PATCH 1/4] add get call for configs --- .../api/indices.get_sample_configuration.json | 38 +++ .../10_basic.yml | 179 +++++++++++++ .../GetSampleConfigurationActionIT.java | 147 ++++++++++ .../GetSampleConfigurationAction.java | 253 ++++++++++++++++++ .../RestGetSampleConfigurationAction.java | 84 ++++++ ...TransportGetSampleConfigurationAction.java | 140 ++++++++++ ...SampleConfigurationActionRequestTests.java | 102 +++++++ ...ampleConfigurationActionResponseTests.java | 55 ++++ 8 files changed, 998 insertions(+) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_sample_configuration.json create mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.get_sample_configuration/10_basic.yml create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionIT.java create mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java create mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java create mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/sampling/TransportGetSampleConfigurationAction.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionRequestTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionResponseTests.java diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_sample_configuration.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_sample_configuration.json new file mode 100644 index 0000000000000..83f032ad5d5f8 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_sample_configuration.json @@ -0,0 +1,38 @@ +{ + "indices.get_sample_configuration": { + "documentation": { + "url": "https://www.elastic.co/docs/api/doc/elasticsearch/operation/operation-indices-get-sample-configuration", + "description": "Get sampling configuration for an index or data stream" + }, + "stability": "experimental", + "visibility": "public", + "headers": { + "accept": [ + "application/json" + ] + }, + "url": { + "paths": [ + { + "path": "/{index}/_sample/config", + "methods": [ + "GET" + ], + "parts": { + "index": { + "type": "string", + "description": "The name of a data stream or index" + } + } + } + ] + }, + "params": { + "master_timeout": { + "type": "time", + "description": "Timeout for connection to master node" + } + } + } +} + diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.get_sample_configuration/10_basic.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.get_sample_configuration/10_basic.yml new file mode 100644 index 0000000000000..490172d78d9b0 --- /dev/null +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.get_sample_configuration/10_basic.yml @@ -0,0 +1,179 @@ +--- +setup: + - requires: + cluster_features: [ "random_sampling" ] + reason: requires feature 'random_sampling' to get random samples + +--- +teardown: + - do: + indices.delete: + index: "*test*" + ignore_unavailable: true + allow_no_indices: true + +--- +"Get sampling configuration after setting it": + - do: + indices.create: + index: test-get-config-index + body: + settings: + number_of_shards: 1 + + - do: + indices.put_sample_configuration: + index: test-get-config-index + body: + rate: 0.5 + max_samples: 100 + max_size: "10mb" + time_to_live: "1h" + + - match: { acknowledged: true } + + - do: + indices.get_sample_configuration: + index: test-get-config-index + human: true + + - match: { configuration.rate: 0.5 } + - match: { configuration.max_samples: 100 } + - match: { configuration.max_size: "10mb" } + - match: { configuration.time_to_live: "1h" } + +--- +"Get sampling configuration with minimal parameters": + - do: + indices.create: + index: test-minimal-config-index + + - do: + indices.put_sample_configuration: + index: test-minimal-config-index + body: + rate: 0.1 + + - match: { acknowledged: true } + + - do: + indices.get_sample_configuration: + index: test-minimal-config-index + human: true + + - match: { configuration.rate: 0.1 } + +--- +"Get sampling configuration with condition": + - do: + indices.create: + index: test-condition-config-index + + - do: + indices.put_sample_configuration: + index: test-condition-config-index + body: + rate: 1.0 + max_samples: 50 + if: "ctx?.field == 'sample_me'" + + - match: { acknowledged: true } + + - do: + indices.get_sample_configuration: + index: test-condition-config-index + human: true + + - match: { configuration.rate: 1.0 } + - match: { configuration.max_samples: 50 } + - match: { configuration.if: "ctx?.field == 'sample_me'" } + +--- +"Get sampling configuration for non-existent index": + - do: + catch: missing + indices.get_sample_configuration: + index: non-existent-index + +--- +"Get sampling configuration for index without configuration": + - do: + indices.create: + index: test-no-config-index + + - do: + indices.get_sample_configuration: + index: test-no-config-index + human: true + + # Should return empty or default configuration + - is_false: test-no-config-index.sampling + +--- +"Get sampling configuration after update": + - do: + indices.create: + index: test-update-config-index + + # Set initial configuration + - do: + indices.put_sample_configuration: + index: test-update-config-index + body: + rate: 0.3 + max_samples: 25 + + - match: { acknowledged: true } + + - do: + indices.get_sample_configuration: + index: test-update-config-index + human: true + + - match: { configuration.rate: 0.3 } + - match: { configuration.max_samples: 25 } + + # Update configuration + - do: + indices.put_sample_configuration: + index: test-update-config-index + body: + rate: 0.8 + max_samples: 75 + max_size: "5mb" + + - match: { acknowledged: true } + + # Verify updated configuration + - do: + indices.get_sample_configuration: + index: test-update-config-index + human: true + + - match: { configuration.rate: 0.8 } + - match: { configuration.max_samples: 75 } + - match: { configuration.max_size: "5mb" } + +--- +"Get sampling configuration with master timeout": + - do: + indices.create: + index: test-timeout-config-index + + - do: + indices.put_sample_configuration: + index: test-timeout-config-index + body: + rate: 0.7 + max_samples: 200 + + - match: { acknowledged: true } + + - do: + indices.get_sample_configuration: + index: test-timeout-config-index + master_timeout: "30s" + human: true + + - match: { configuration.rate: 0.7 } + - match: { configuration.max_samples: 200 } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionIT.java new file mode 100644 index 0000000000000..67132f53e0026 --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionIT.java @@ -0,0 +1,147 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.ingest.SamplingService; +import org.elasticsearch.test.ESIntegTestCase; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class GetSampleConfigurationActionIT extends ESIntegTestCase { + + public void testGetSampleConfiguration() { + assumeTrue("Requires the sampling feature flag to be enabled", SamplingService.RANDOM_SAMPLING_FEATURE_FLAG); + + String indexName = randomIdentifier(); + createIndex(indexName); + + // Create a random sampling configuration + SamplingConfiguration config = randomSamplingConfiguration(); + putSamplingConfiguration(indexName, config); + + // Get and verify the sampling configuration + assertGetConfigurationMatches(indexName, config); + } + + public void testGetSampleConfigurationForNonExistentIndex() { + assumeTrue("Requires the sampling feature flag to be enabled", SamplingService.RANDOM_SAMPLING_FEATURE_FLAG); + + String nonExistentIndex = randomIdentifier(); + + // Try to get configuration for non-existent index + GetSampleConfigurationAction.Request getRequest = new GetSampleConfigurationAction.Request(randomValidTimeValue()); + getRequest.indices(nonExistentIndex); + + // This should fail - cannot get config for non-existent index + expectThrows(Exception.class, () -> client().execute(GetSampleConfigurationAction.INSTANCE, getRequest).actionGet()); + } + + public void testGetSampleConfigurationForIndexWithoutConfiguration() { + assumeTrue("Requires the sampling feature flag to be enabled", SamplingService.RANDOM_SAMPLING_FEATURE_FLAG); + + String indexName = randomIdentifier(); + createIndex(indexName); + + // Get configuration for index without any sampling configuration + GetSampleConfigurationAction.Request getRequest = new GetSampleConfigurationAction.Request(randomValidTimeValue()); + getRequest.indices(indexName); + + GetSampleConfigurationAction.Response response = client().execute(GetSampleConfigurationAction.INSTANCE, getRequest).actionGet(); + + // Verify response returns null configuration + assertThat("Response should not be null", response, notNullValue()); + assertThat("Index name should match", response.getIndex(), equalTo(indexName)); + assertThat("Configuration should be null for index without config", response.getConfiguration(), nullValue()); + } + + public void testGetSampleConfigurationAfterUpdate() { + assumeTrue("Requires the sampling feature flag to be enabled", SamplingService.RANDOM_SAMPLING_FEATURE_FLAG); + + String indexName = randomIdentifier(); + createIndex(indexName); + + // Create initial random configuration + SamplingConfiguration initialConfig = randomSamplingConfiguration(); + putSamplingConfiguration(indexName, initialConfig); + + // Get initial configuration + assertGetConfigurationMatches(indexName, initialConfig); + + // Update with new random configuration + SamplingConfiguration updatedConfig = randomSamplingConfiguration(); + putSamplingConfiguration(indexName, updatedConfig); + + // Get and verify updated configuration + assertGetConfigurationMatches(indexName, updatedConfig); + } + + public void testGetSampleConfigurationPersistsAcrossClusterStateUpdates() { + assumeTrue("Requires the sampling feature flag to be enabled", SamplingService.RANDOM_SAMPLING_FEATURE_FLAG); + + String indexName = randomIdentifier(); + createIndex(indexName); + + // Store random sampling configuration + SamplingConfiguration config = randomSamplingConfiguration(); + putSamplingConfiguration(indexName, config); + + // Get initial configuration + assertGetConfigurationMatches(indexName, config); + + // Trigger cluster state updates by creating additional indices with random names + int numDummyIndices = randomIntBetween(2, 5); + for (int i = 0; i < numDummyIndices; i++) { + createIndex(randomIdentifier()); + } + + // Get configuration again after cluster state changes and verify it persists + assertGetConfigurationMatches(indexName, config); + } + + private SamplingConfiguration randomSamplingConfiguration() { + return new SamplingConfiguration( + randomDoubleBetween(0.1, 1.0, true), + randomBoolean() ? randomIntBetween(1, SamplingConfiguration.MAX_SAMPLES_LIMIT) : null, + randomBoolean() ? ByteSizeValue.ofMb(randomIntBetween(1, 100)) : null, + randomBoolean() ? randomValidTimeValue() : null, + randomBoolean() ? randomAlphaOfLengthBetween(5, 30) : null + ); + } + + private void putSamplingConfiguration(String indexName, SamplingConfiguration config) { + PutSampleConfigurationAction.Request putRequest = new PutSampleConfigurationAction.Request( + config, + randomValidTimeValue(), + randomValidTimeValue() + ); + putRequest.indices(indexName); + client().execute(PutSampleConfigurationAction.INSTANCE, putRequest).actionGet(); + ensureGreen(); + } + + private void assertGetConfigurationMatches(String indexName, SamplingConfiguration expectedConfig) { + GetSampleConfigurationAction.Request getRequest = new GetSampleConfigurationAction.Request(randomValidTimeValue()); + getRequest.indices(indexName); + + GetSampleConfigurationAction.Response response = client().execute(GetSampleConfigurationAction.INSTANCE, getRequest).actionGet(); + + assertThat("Response should not be null", response, notNullValue()); + assertThat("Index name should match", response.getIndex(), equalTo(indexName)); + assertThat("Configuration should match", response.getConfiguration(), equalTo(expectedConfig)); + } + + private TimeValue randomValidTimeValue() { + return TimeValue.timeValueDays(randomIntBetween(10, 20)); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java new file mode 100644 index 0000000000000..614b34eca871f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java @@ -0,0 +1,253 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.local.LocalClusterStateRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ChunkedToXContentObject; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +/** + * Action type for retrieving sampling configuration for a specific index. + *

+ * This action allows clients to get the current sampling configuration + * that has been set on a specific index. This action + * targets a single index and returns its configuration in a structured format. + *

+ * The response format matches: + *
+ * {
+ *  "index": "logs",
+ *  "configuration": {
+ *      "rate": ".0",
+ *      "if": "ctx?.network?.name == 'Guest'"
+ *  }
+ * }
+ *  
+ * + * @see SamplingConfiguration + * @see PutSampleConfigurationAction + */ +public class GetSampleConfigurationAction extends ActionType { + + /** + * Singleton instance of the GetSampleConfigurationAction. + * This provides a shared reference to the action type throughout the application. + */ + public static final GetSampleConfigurationAction INSTANCE = new GetSampleConfigurationAction(); + + /** + * The name identifier for this action type used in the transport layer. + */ + public static final String NAME = "indices:admin/sample/config/get"; + + private GetSampleConfigurationAction() { + super(NAME); + } + + /** + * Request object for getting the sampling configuration of a specific index. + *

+ * This request specifies which index's sampling configuration should be retrieved. + * The index name must be provided and cannot be null or empty. + *

+ */ + public static class Request extends LocalClusterStateRequest implements IndicesRequest.Replaceable { + private String index; + + /** + * Constructs a new request for the specified index. + * + * @param masterNodeTimeout the timeout for master node operations, or null for default + */ + public Request(@Nullable TimeValue masterNodeTimeout) { + super(masterNodeTimeout); + } + + /** + * Constructs a new request by deserializing from a StreamInput. + * + * @param in the stream input to read from + * @throws IOException if an I/O error occurs during deserialization + */ + public Request(StreamInput in) throws IOException { + super(in); + this.index = in.readString(); + } + + /** + * Gets the index name for which to retrieve the sampling configuration. + * + * @return the index name + */ + public String getIndex() { + return index; + } + + @Override + public String[] indices() { + return new String[] { index }; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.STRICT_SINGLE_INDEX_NO_EXPAND_FORBID_CLOSED_ALLOW_SELECTORS; + } + + @Override + public boolean includeDataStreams() { + return true; + } + + @Override + public IndicesRequest indices(String... indices) { + this.index = indices[0]; + return this; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (Strings.isNullOrEmpty(index)) { + validationException = addValidationError("index name is required", validationException); + } + return validationException; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(index, request.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, "executing " + action, parentTaskId, headers); + } + } + + /** + * Response object containing the sampling configuration for a specific index. + *

+ * This response contains the index name and its associated sampling configuration. + * The response is designed to match the expected JSON format with an array containing + * a single object with "index" and "configuration" fields. + *

+ */ + public static class Response extends ActionResponse implements ChunkedToXContentObject { + private final String index; + private final SamplingConfiguration configuration; + + /** + * Constructs a new Response with the given index and configuration. + * + * @param index the index name + * @param configuration the sampling configuration for the index, or null if no configuration exists + */ + public Response(String index, SamplingConfiguration configuration) { + this.index = index; + this.configuration = configuration; + } + + /** + * Constructs a new Response by deserializing from a StreamInput. + * + * @param in the stream input to read from + * @throws IOException if an I/O error occurs during deserialization + */ + public Response(StreamInput in) throws IOException { + this.index = in.readString(); + this.configuration = in.readOptionalWriteable(SamplingConfiguration::new); + } + + /** + * Gets the index name. + * + * @return the index name + */ + public String getIndex() { + return index; + } + + /** + * Gets the sampling configuration for the index. + * + * @return the sampling configuration, or null if no configuration exists for this index + */ + public SamplingConfiguration getConfiguration() { + return configuration; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeOptionalWriteable(configuration); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Response response = (Response) o; + return Objects.equals(index, response.index) && Objects.equals(configuration, response.configuration); + } + + @Override + public int hashCode() { + return Objects.hash(index, configuration); + } + + @Override + public Iterator toXContentChunked(ToXContent.Params params) { + return Collections.singletonList((ToXContent) (builder, p) -> toXContent(builder, params)).iterator(); + } + + private XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field("index", index); + if (configuration != null) { + builder.field("configuration", configuration); + } else { + builder.nullField("configuration"); + } + builder.endObject(); + return builder; + } + + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java new file mode 100644 index 0000000000000..6543570534392 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java @@ -0,0 +1,84 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; +import org.elasticsearch.rest.action.RestCancellableNodeClient; +import org.elasticsearch.rest.action.RestRefCountedChunkedToXContentListener; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.elasticsearch.rest.RestRequest.Method.GET; +import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout; + +/** + * REST action for updating sampling configurations for indices. + *

+ * Handles PUT requests to /{index}/_sample/config endpoint and delegates + * to the PutSampleConfigurationAction transport action. + *

+ * + *

Example usage:

+ * GET /my-index/_sample/config + * returns + * [ + * { + * "index": "logs", + * "configuration: + * { + * "rate": "5%", + * "if": "ctx?.network?.name == 'Guest'" + * } + * } + * ] + */ +@ServerlessScope(Scope.INTERNAL) +public class RestGetSampleConfigurationAction extends BaseRestHandler { + + @Override + public List routes() { + return List.of(new Route(GET, "/{index}/_sample/config")); + } + + @Override + public String getName() { + return "get_sample_configuration_action"; + } + + @Override + public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { + String[] indexNames = request.param("index").split(","); + if (indexNames.length > 1) { + throw new ActionRequestValidationException().addValidationError( + "Can only get sampling configuration for a single index at a time, but found " + + Arrays.stream(indexNames).collect(Collectors.joining(", ", "[", "]")) + ); + } + GetSampleConfigurationAction.Request getRequest = new GetSampleConfigurationAction.Request(getMasterNodeTimeout(request)); + + // Set the target index + getRequest.indices(indexNames); + + return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).execute( + GetSampleConfigurationAction.INSTANCE, + getRequest, + new RestRefCountedChunkedToXContentListener<>(channel) + ); + } + +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/TransportGetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/TransportGetSampleConfigurationAction.java new file mode 100644 index 0000000000000..4d1b0f786c5e0 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/TransportGetSampleConfigurationAction.java @@ -0,0 +1,140 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.local.TransportLocalProjectMetadataAction; +import org.elasticsearch.cluster.ProjectState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.ProjectMetadata; +import org.elasticsearch.cluster.project.ProjectResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.injection.guice.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; + +import java.util.Map; + +/** + * Transport action for retrieving sampling configuration for an index. + *

+ * This action retrieves the sampling configuration from the project metadata's custom sampling metadata. + * If no configuration exists for the specified index, a response with null configuration is returned. + *

+ */ +public class TransportGetSampleConfigurationAction extends TransportLocalProjectMetadataAction< + GetSampleConfigurationAction.Request, + GetSampleConfigurationAction.Response> { + + private static final Logger logger = LogManager.getLogger(TransportGetSampleConfigurationAction.class); + private final IndexNameExpressionResolver indexNameExpressionResolver; + + /** + * Constructs a new TransportGetSampleConfigurationAction. + * + * @param actionFilters the action filters to apply + * @param transportService the transport service + * @param clusterService the cluster service + * @param projectResolver the project resolver + * @param indexNameExpressionResolver the index name expression resolver for resolving index names + */ + @Inject + public TransportGetSampleConfigurationAction( + ActionFilters actionFilters, + TransportService transportService, + ClusterService clusterService, + ProjectResolver projectResolver, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + GetSampleConfigurationAction.NAME, + actionFilters, + transportService.getTaskManager(), + clusterService, + EsExecutors.DIRECT_EXECUTOR_SERVICE, + projectResolver + ); + this.indexNameExpressionResolver = indexNameExpressionResolver; + + } + + /** + * Executes the get sampling configuration operation against the local cluster state. + *

+ * Retrieves the sampling configuration for the specified index from the project metadata. + * Returns null configuration if no sampling metadata exists or if no configuration is set for the index. + *

+ * + * @param task the task executing this operation + * @param request the get sampling configuration request containing the index name + * @param state the current project state + * @param listener the listener to notify with the response or failure + */ + @Override + protected void localClusterStateOperation( + Task task, + GetSampleConfigurationAction.Request request, + ProjectState state, + ActionListener listener + ) { + ProjectMetadata projectMetadata = state.metadata(); + + // throws IndexNotFoundException if any index does not exist or more than one index is resolved + try { + indexNameExpressionResolver.concreteIndexNames(projectMetadata, request); + } catch (IndexNotFoundException e) { + logger.debug("Index not found: [{}]", request.getIndex()); + listener.onFailure(e); + return; + } + if (projectMetadata == null) { + logger.debug("No project metadata found for index [{}]", request.getIndex()); + listener.onResponse(new GetSampleConfigurationAction.Response(request.getIndex(), null)); + return; + } + + SamplingMetadata samplingMetadata = projectMetadata.custom(SamplingMetadata.TYPE); + if (samplingMetadata == null) { + logger.debug("No sampling metadata found for index [{}]", request.getIndex()); + listener.onResponse(new GetSampleConfigurationAction.Response(request.getIndex(), null)); + return; + } + + Map indexToSampleConfigMap = samplingMetadata.getIndexToSamplingConfigMap(); + SamplingConfiguration config = indexToSampleConfigMap.get(request.getIndex()); + if (config == null) { + logger.debug("No sampling configuration found for index [{}]", request.getIndex()); + listener.onResponse(new GetSampleConfigurationAction.Response(request.getIndex(), config)); + return; + } + + logger.debug("Retrieved sampling configuration for index [{}]", request.getIndex()); + listener.onResponse(new GetSampleConfigurationAction.Response(request.getIndex(), config)); + } + + /** + * Checks for cluster blocks that would prevent this operation from executing. + * + * @param request the get sampling configuration request + * @param state the current project state + * @return a cluster block exception if the operation is blocked, null otherwise + */ + @Override + protected ClusterBlockException checkBlock(GetSampleConfigurationAction.Request request, ProjectState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } +} diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionRequestTests.java new file mode 100644 index 0000000000000..1ee766bf90b24 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionRequestTests.java @@ -0,0 +1,102 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class GetSampleConfigurationActionRequestTests extends ESTestCase { + + protected GetSampleConfigurationAction.Request createTestInstance() { + GetSampleConfigurationAction.Request request = new GetSampleConfigurationAction.Request(randomBoundedTimeValue()); + request.indices(randomAlphaOfLength(10)); + return request; + } + + protected GetSampleConfigurationAction.Request mutateInstance(GetSampleConfigurationAction.Request instance) { + // Since Request.equals() only compares the index field (not masterTimeout), + // we must mutate the index to ensure the mutated instance is not equal + String newIndex = randomValueOtherThan(instance.getIndex(), () -> randomAlphaOfLengthBetween(1, 10)); + GetSampleConfigurationAction.Request mutated = new GetSampleConfigurationAction.Request( + randomBoolean() ? instance.masterTimeout() : randomBoundedTimeValue() + ); + mutated.indices(newIndex); + return mutated; + } + + public void testRequestValidation() { + // Valid request + GetSampleConfigurationAction.Request validRequest = new GetSampleConfigurationAction.Request(randomBoundedTimeValue()); + validRequest.indices("test-index"); + assertThat(validRequest.validate(), nullValue()); + + // Invalid request with null index + GetSampleConfigurationAction.Request invalidRequest = new GetSampleConfigurationAction.Request(randomBoundedTimeValue()); + ActionRequestValidationException validation = invalidRequest.validate(); + assertThat(validation, notNullValue()); + assertThat(validation.getMessage(), containsString("index name is required")); + + // Invalid request with empty index + GetSampleConfigurationAction.Request emptyIndexRequest = new GetSampleConfigurationAction.Request(randomBoundedTimeValue()); + emptyIndexRequest.indices(""); + validation = emptyIndexRequest.validate(); + assertThat(validation, notNullValue()); + assertThat(validation.getMessage(), containsString("index name is required")); + } + + public void testEqualsAndHashcode() { + for (int i = 0; i < 10; i++) { + GetSampleConfigurationAction.Request request1 = createTestInstance(); + GetSampleConfigurationAction.Request request2 = new GetSampleConfigurationAction.Request(request1.masterTimeout()); + request2.indices(request1.getIndex()); + + // Test equality + assertThat(request1, equalTo(request2)); + assertThat(request1.hashCode(), equalTo(request2.hashCode())); + + // Test mutation creates non-equal instance + GetSampleConfigurationAction.Request mutated = mutateInstance(request1); + assertThat("Mutation should create non-equal instance", request1, not(equalTo(mutated))); + + // Test reflexivity + assertThat(request1, equalTo(request1)); + + // Test null and different class + assertThat(request1.equals(null), equalTo(false)); + assertThat(request1.equals("not a request"), equalTo(false)); + } + } + + public void testGettersAndSetters() { + String indexName = randomAlphaOfLength(10); + GetSampleConfigurationAction.Request request = new GetSampleConfigurationAction.Request(randomBoundedTimeValue()); + request.indices(indexName); + + assertThat(request.getIndex(), equalTo(indexName)); + assertThat(request.indices().length, equalTo(1)); + assertThat(request.indices()[0], equalTo(indexName)); + } + + public void testIncludeDataStreams() { + GetSampleConfigurationAction.Request request = createTestInstance(); + assertThat(request.includeDataStreams(), equalTo(true)); + } + + private TimeValue randomBoundedTimeValue() { + return TimeValue.timeValueSeconds(randomIntBetween(5, 10)); + } +} diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionResponseTests.java new file mode 100644 index 0000000000000..0455458af07c2 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationActionResponseTests.java @@ -0,0 +1,55 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.action.admin.indices.sampling; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +public class GetSampleConfigurationActionResponseTests extends AbstractWireSerializingTestCase { + + @Override + protected Writeable.Reader instanceReader() { + return GetSampleConfigurationAction.Response::new; + } + + @Override + protected GetSampleConfigurationAction.Response createTestInstance() { + String index = randomAlphaOfLengthBetween(1, 20); + SamplingConfiguration config = randomBoolean() ? createRandomSamplingConfiguration() : null; + return new GetSampleConfigurationAction.Response(index, config); + } + + @Override + protected GetSampleConfigurationAction.Response mutateInstance(GetSampleConfigurationAction.Response instance) { + return switch (randomIntBetween(0, 1)) { + case 0 -> new GetSampleConfigurationAction.Response( + randomValueOtherThan(instance.getIndex(), () -> randomAlphaOfLengthBetween(1, 20)), + instance.getConfiguration() + ); + case 1 -> new GetSampleConfigurationAction.Response( + instance.getIndex(), + randomValueOtherThan(instance.getConfiguration(), () -> randomBoolean() ? createRandomSamplingConfiguration() : null) + ); + default -> throw new IllegalStateException("Invalid mutation case"); + }; + } + + private SamplingConfiguration createRandomSamplingConfiguration() { + return new SamplingConfiguration( + randomDoubleBetween(0.1, 1.0, true), + randomBoolean() ? randomIntBetween(1, 1000) : null, + randomBoolean() ? ByteSizeValue.ofMb(randomIntBetween(1, 100)) : null, + randomBoolean() ? TimeValue.timeValueMinutes(randomIntBetween(1, 60)) : null, + randomBoolean() ? "ctx?.field == 'test'" : null + ); + } +} From 89ffa24d012a8b3a159fa61b76fd2563b8e9cb5d Mon Sep 17 00:00:00 2001 From: Sean Zatz Date: Tue, 21 Oct 2025 17:28:41 -0400 Subject: [PATCH 2/4] wire in handlers --- .../src/main/java/org/elasticsearch/action/ActionModule.java | 5 +++++ .../org/elasticsearch/xpack/security/operator/Constants.java | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index dee8480d344b9..dc26766c8f0ad 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -129,14 +129,17 @@ import org.elasticsearch.action.admin.indices.rollover.TransportRolloverAction; import org.elasticsearch.action.admin.indices.sampling.DeleteSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.GetSampleAction; +import org.elasticsearch.action.admin.indices.sampling.GetSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.GetSampleStatsAction; import org.elasticsearch.action.admin.indices.sampling.PutSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.RestDeleteSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.RestGetSampleAction; +import org.elasticsearch.action.admin.indices.sampling.RestGetSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.RestGetSampleStatsAction; import org.elasticsearch.action.admin.indices.sampling.RestPutSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.TransportDeleteSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.TransportGetSampleAction; +import org.elasticsearch.action.admin.indices.sampling.TransportGetSampleConfigurationAction; import org.elasticsearch.action.admin.indices.sampling.TransportGetSampleStatsAction; import org.elasticsearch.action.admin.indices.sampling.TransportPutSampleConfigurationAction; import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsAction; @@ -831,6 +834,7 @@ public void reg actions.register(PutSampleConfigurationAction.INSTANCE, TransportPutSampleConfigurationAction.class); actions.register(GetSampleStatsAction.INSTANCE, TransportGetSampleStatsAction.class); actions.register(DeleteSampleConfigurationAction.INSTANCE, TransportDeleteSampleConfigurationAction.class); + actions.register(GetSampleConfigurationAction.INSTANCE, TransportGetSampleConfigurationAction.class); } return unmodifiableMap(actions.getRegistry()); @@ -1066,6 +1070,7 @@ public void initRestHandlers(Supplier nodesInCluster, Predicate< registerHandler.accept(new RestPutSampleConfigurationAction()); registerHandler.accept(new RestGetSampleStatsAction()); registerHandler.accept(new RestDeleteSampleConfigurationAction()); + registerHandler.accept(new RestGetSampleConfigurationAction()); } } diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index b4506a62b9947..ed60521f1d00f 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -662,6 +662,7 @@ public class Constants { "indices:admin/sample", "indices:admin/sample/config/update", "indices:admin/sample/stats", - "indices:admin/sample/config/delete" + "indices:admin/sample/config/delete", + "indices:admin/sample/config/get" ).filter(Objects::nonNull).collect(Collectors.toUnmodifiableSet()); } From e971337a02f1283a394c91f273a1e82e77c279c1 Mon Sep 17 00:00:00 2001 From: Sean Zatz Date: Tue, 21 Oct 2025 17:34:23 -0400 Subject: [PATCH 3/4] update comments --- .../sampling/GetSampleConfigurationAction.java | 2 +- .../sampling/RestGetSampleConfigurationAction.java | 12 +++++------- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java index 614b34eca871f..80ff2339458e8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java @@ -47,7 +47,7 @@ * { * "index": "logs", * "configuration": { - * "rate": ".0", + * "rate": ".5", * "if": "ctx?.network?.name == 'Guest'" * } * } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java index 6543570534392..81fa3f4070113 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/RestGetSampleConfigurationAction.java @@ -27,25 +27,23 @@ import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout; /** - * REST action for updating sampling configurations for indices. + * REST action for getting sampling configurations for indices. *

- * Handles PUT requests to /{index}/_sample/config endpoint and delegates - * to the PutSampleConfigurationAction transport action. + * Handles GET requests to /{index}/_sample/config endpoint and delegates + * to the GetSampleConfigurationAction transport action. *

* *

Example usage:

* GET /my-index/_sample/config * returns - * [ * { * "index": "logs", - * "configuration: + * "configuration": * { * "rate": "5%", * "if": "ctx?.network?.name == 'Guest'" * } * } - * ] */ @ServerlessScope(Scope.INTERNAL) public class RestGetSampleConfigurationAction extends BaseRestHandler { @@ -65,7 +63,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC String[] indexNames = request.param("index").split(","); if (indexNames.length > 1) { throw new ActionRequestValidationException().addValidationError( - "Can only get sampling configuration for a single index at a time, but found " + "Provided endpoint can only get sampling configuration for a single index at a time, but found " + Arrays.stream(indexNames).collect(Collectors.joining(", ", "[", "]")) ); } From 68aeacfc242c3e6a59b6424612e5acd9a249b3a1 Mon Sep 17 00:00:00 2001 From: Sean Zatz Date: Wed, 22 Oct 2025 12:12:59 -0400 Subject: [PATCH 4/4] update action name --- .../admin/indices/sampling/GetSampleConfigurationAction.java | 2 +- .../org/elasticsearch/xpack/security/operator/Constants.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java index 80ff2339458e8..9bdd60728ba2d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/sampling/GetSampleConfigurationAction.java @@ -67,7 +67,7 @@ public class GetSampleConfigurationAction extends ActionType