From 1fbd7e990b14006e2d10bd15f22c7a6791169c17 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 06:19:04 +0100 Subject: [PATCH 01/15] Test get-snapshots API with missing details (#111903) Extends the test added in #111786 to check that the API still works correctly even in the BwC case that the details needed are not in the `RepositoryData` and must be read from the individual `SnapshotInfo` blobs. --- .../snapshots/GetSnapshotsIT.java | 126 ++++++++++++++++++ 1 file changed, 126 insertions(+) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java index 66ddd47d7758d..477fd9737394e 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java @@ -8,8 +8,12 @@ package org.elasticsearch.snapshots; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.admin.cluster.repositories.delete.DeleteRepositoryRequest; +import org.elasticsearch.action.admin.cluster.repositories.delete.TransportDeleteRepositoryAction; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; import org.elasticsearch.action.admin.cluster.repositories.put.TransportPutRepositoryAction; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest; @@ -23,17 +27,30 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.blobstore.fs.FsBlobStore; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Predicates; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryMissingException; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.XContentTestUtils; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xcontent.json.JsonXContent; +import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; @@ -819,6 +836,17 @@ public void testAllFeatures() { } }); + if (randomBoolean()) { + // Sometimes also simulate bwc repository contents where some details are missing from the root blob + safeAwait(l -> { + try (var listeners = new RefCountingListener(l.map(v -> null))) { + for (final var repositoryName : randomSubsetOf(repositories)) { + removeDetailsForRandomSnapshots(repositoryName, listeners.acquire()); + } + } + }); + } + Predicate snapshotInfoPredicate = Predicates.always(); // {repository} path parameter @@ -1000,4 +1028,102 @@ public void testAllFeatures() { assertEquals(0, remaining); } + + /** + * Older versions of Elasticsearch don't record in {@link RepositoryData} all the details needed for the get-snapshots API to pick out + * the right snapshots, so in this case the API must fall back to reading those details from each candidate {@link SnapshotInfo} blob. + * Simulate this situation by manipulating the {@link RepositoryData} blob directly to remove all the optional details from some subset + * of its snapshots. + */ + private static void removeDetailsForRandomSnapshots(String repositoryName, ActionListener listener) { + final Set snapshotsWithoutDetails = ConcurrentCollections.newConcurrentSet(); + final var masterRepositoriesService = internalCluster().getCurrentMasterNodeInstance(RepositoriesService.class); + final var repository = asInstanceOf(FsRepository.class, masterRepositoriesService.repository(repositoryName)); + final var repositoryMetadata = repository.getMetadata(); + final var repositorySettings = repositoryMetadata.settings(); + final var repositoryDataBlobPath = asInstanceOf(FsBlobStore.class, repository.blobStore()).path() + .resolve(BlobStoreRepository.INDEX_FILE_PREFIX + repositoryMetadata.generation()); + + SubscribableListener + + // unregister the repository while we're mucking around with its internals + .newForked( + l -> client().execute( + TransportDeleteRepositoryAction.TYPE, + new DeleteRepositoryRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName), + l + ) + ) + .andThenAccept(ElasticsearchAssertions::assertAcked) + + // rewrite the RepositoryData blob with some details removed + .andThenAccept(ignored -> { + // load the existing RepositoryData JSON blob as raw maps/lists/etc. + final var repositoryDataBytes = Files.readAllBytes(repositoryDataBlobPath); + final var repositoryDataMap = XContentHelper.convertToMap( + JsonXContent.jsonXContent, + repositoryDataBytes, + 0, + repositoryDataBytes.length, + true + ); + + // modify the contents + final var snapshotsList = asInstanceOf(List.class, repositoryDataMap.get("snapshots")); + for (final var snapshotObj : snapshotsList) { + if (randomBoolean()) { + continue; + } + final var snapshotMap = asInstanceOf(Map.class, snapshotObj); + snapshotsWithoutDetails.add( + new SnapshotId( + asInstanceOf(String.class, snapshotMap.get("name")), + asInstanceOf(String.class, snapshotMap.get("uuid")) + ) + ); + + // remove the optional details fields + assertNotNull(snapshotMap.remove("start_time_millis")); + assertNotNull(snapshotMap.remove("end_time_millis")); + assertNotNull(snapshotMap.remove("slm_policy")); + } + + // overwrite the RepositoryData JSON blob with its new contents + final var updatedRepositoryDataBytes = XContentTestUtils.convertToXContent(repositoryDataMap, XContentType.JSON); + try (var outputStream = Files.newOutputStream(repositoryDataBlobPath)) { + BytesRef bytesRef; + final var iterator = updatedRepositoryDataBytes.iterator(); + while ((bytesRef = iterator.next()) != null) { + outputStream.write(bytesRef.bytes, bytesRef.offset, bytesRef.length); + } + } + }) + + // re-register the repository + .andThen( + l -> client().execute( + TransportPutRepositoryAction.TYPE, + new PutRepositoryRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName).type(FsRepository.TYPE) + .settings(repositorySettings), + l + ) + ) + .andThenAccept(ElasticsearchAssertions::assertAcked) + + // verify that the details are indeed now missing + .andThen( + l -> masterRepositoriesService.repository(repositoryName).getRepositoryData(EsExecutors.DIRECT_EXECUTOR_SERVICE, l) + ) + .andThenAccept(repositoryData -> { + for (SnapshotId snapshotId : repositoryData.getSnapshotIds()) { + assertEquals( + repositoryName + "/" + snapshotId.toString() + ": " + repositoryData.getSnapshotDetails(snapshotId), + snapshotsWithoutDetails.contains(snapshotId), + repositoryData.hasMissingDetails(snapshotId) + ); + } + }) + + .addListener(listener); + } } From a406333f8750ec76be11f96881295a9049ecdde4 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 06:20:19 +0100 Subject: [PATCH 02/15] Revert "Add 8.15.0 known issue for memory locking in Windows (#111949)" This reverts commit 1e40fe45d638017c797d81f2bdb294aea6365be1. --- docs/reference/release-notes/8.15.0.asciidoc | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docs/reference/release-notes/8.15.0.asciidoc b/docs/reference/release-notes/8.15.0.asciidoc index 395073683b102..1df0969ecc629 100644 --- a/docs/reference/release-notes/8.15.0.asciidoc +++ b/docs/reference/release-notes/8.15.0.asciidoc @@ -32,11 +32,6 @@ Rollup:: Search:: * Change `skip_unavailable` remote cluster setting default value to true {es-pull}105792[#105792] -[[known-issues-8.15.0]] -[float] -=== Known issues -* Elasticsearch will not start on Windows machines when the recommended [bootstrap.memory_lock: true](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup-configuration-memory.html#bootstrap-memory_lock) setting is configured due to [native access refactoring](https://github.com/elastic/elasticsearch/pull/111866). The workaround for 8.15.0 is to downgrade to the previous version. This issue will be fixed in 8.15.1. - [[bug-8.15.0]] [float] === Bug fixes From fe7448e4e5ca3d41eb2371ea037c60df4a692f4b Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 06:32:18 +0100 Subject: [PATCH 03/15] Introduce `StreamingXContentResponse` (#111933) Similar to `ChunkedZipResponse` (#109820) this utility allows Elasticsearch to send an `XContent`-based response constructed out of a sequence of `ChunkedToXContent` fragments, provided in a streaming and asynchronous fashion. This will enable #93735 to proceed without needing to create a temporary index to hold the intermediate results. --- .../rest/StreamingXContentResponseIT.java | 300 ++++++++++++ .../rest/StreamingXContentResponse.java | 435 ++++++++++++++++++ 2 files changed, 735 insertions(+) create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/rest/StreamingXContentResponseIT.java create mode 100644 server/src/main/java/org/elasticsearch/rest/StreamingXContentResponse.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/rest/StreamingXContentResponseIT.java b/server/src/internalClusterTest/java/org/elasticsearch/rest/StreamingXContentResponseIT.java new file mode 100644 index 0000000000000..ae91caea888db --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/rest/StreamingXContentResponseIT.java @@ -0,0 +1,300 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.rest; + +import org.apache.http.ConnectionClosedException; +import org.apache.http.HttpResponse; +import org.apache.http.nio.ContentDecoder; +import org.apache.http.nio.IOControl; +import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import org.apache.http.protocol.HttpContext; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.support.RefCountingRunnable; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.ThrottledIterator; +import org.elasticsearch.common.xcontent.ChunkedToXContentHelper; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.json.JsonXContent; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.hamcrest.Matchers.hasSize; + +@ESIntegTestCase.ClusterScope(numDataNodes = 1) +public class StreamingXContentResponseIT extends ESIntegTestCase { + + @Override + protected boolean addMockHttpTransport() { + return false; + } + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopyNoNullElements(super.nodePlugins(), RandomXContentResponsePlugin.class); + } + + public static class RandomXContentResponsePlugin extends Plugin implements ActionPlugin { + + public static final String ROUTE = "/_random_xcontent_response"; + + public static final String INFINITE_ROUTE = "/_random_infinite_xcontent_response"; + + public final AtomicReference responseRef = new AtomicReference<>(); + + public record Response(Map fragments, CountDownLatch completedLatch) {} + + @Override + public Collection getRestHandlers( + Settings settings, + NamedWriteableRegistry namedWriteableRegistry, + RestController restController, + ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, + SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster, + Predicate clusterSupportsFeature + ) { + return List.of( + // handler that returns a normal (finite) response + new RestHandler() { + @Override + public List routes() { + return List.of(new Route(RestRequest.Method.GET, ROUTE)); + } + + @Override + public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws IOException { + final var response = new Response(new HashMap<>(), new CountDownLatch(1)); + final var entryCount = between(0, 10000); + for (int i = 0; i < entryCount; i++) { + response.fragments().put(randomIdentifier(), randomIdentifier()); + } + assertTrue(responseRef.compareAndSet(null, response)); + handleStreamingXContentRestRequest( + channel, + client.threadPool(), + response.completedLatch(), + response.fragments().entrySet().iterator() + ); + } + }, + + // handler that just keeps on yielding chunks until aborted + new RestHandler() { + @Override + public List routes() { + return List.of(new Route(RestRequest.Method.GET, INFINITE_ROUTE)); + } + + @Override + public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws IOException { + final var response = new Response(new HashMap<>(), new CountDownLatch(1)); + assertTrue(responseRef.compareAndSet(null, new Response(null, response.completedLatch()))); + handleStreamingXContentRestRequest(channel, client.threadPool(), response.completedLatch(), new Iterator<>() { + + private long id; + + // carry on yielding content even after the channel closes + private final Semaphore trailingContentPermits = new Semaphore(between(0, 20)); + + @Override + public boolean hasNext() { + return request.getHttpChannel().isOpen() || trailingContentPermits.tryAcquire(); + } + + @Override + public Map.Entry next() { + return new Map.Entry<>() { + private final String key = Long.toString(id++); + private final String content = randomIdentifier(); + + @Override + public String getKey() { + return key; + } + + @Override + public String getValue() { + return content; + } + + @Override + public String setValue(String value) { + return fail(null, "must not setValue"); + } + }; + } + }); + } + } + ); + } + + private static void handleStreamingXContentRestRequest( + RestChannel channel, + ThreadPool threadPool, + CountDownLatch completionLatch, + Iterator> fragmentIterator + ) throws IOException { + try (var refs = new RefCountingRunnable(completionLatch::countDown)) { + final var streamingXContentResponse = new StreamingXContentResponse(channel, channel.request(), refs.acquire()); + streamingXContentResponse.writeFragment(p -> ChunkedToXContentHelper.startObject(), refs.acquire()); + final var finalRef = refs.acquire(); + ThrottledIterator.run( + fragmentIterator, + (ref, fragment) -> randomFrom(EsExecutors.DIRECT_EXECUTOR_SERVICE, threadPool.generic()).execute( + ActionRunnable.run(ActionListener.releaseAfter(refs.acquireListener(), ref), () -> { + Thread.yield(); + streamingXContentResponse.writeFragment( + p -> ChunkedToXContentHelper.field(fragment.getKey(), fragment.getValue()), + refs.acquire() + ); + }) + ), + between(1, 10), + () -> {}, + () -> { + try (streamingXContentResponse; finalRef) { + streamingXContentResponse.writeFragment(p -> ChunkedToXContentHelper.endObject(), refs.acquire()); + } + } + ); + } + } + } + + public void testRandomStreamingXContentResponse() throws IOException { + final var request = new Request("GET", RandomXContentResponsePlugin.ROUTE); + final var response = getRestClient().performRequest(request); + final var actualEntries = XContentHelper.convertToMap(JsonXContent.jsonXContent, response.getEntity().getContent(), false); + assertEquals(getExpectedEntries(), actualEntries); + } + + public void testAbort() throws IOException { + final var request = new Request("GET", RandomXContentResponsePlugin.INFINITE_ROUTE); + final var responseStarted = new CountDownLatch(1); + final var bodyConsumed = new CountDownLatch(1); + request.setOptions(RequestOptions.DEFAULT.toBuilder().setHttpAsyncResponseConsumerFactory(() -> new HttpAsyncResponseConsumer<>() { + + final ByteBuffer readBuffer = ByteBuffer.allocate(ByteSizeUnit.KB.toIntBytes(4)); + int bytesToConsume = ByteSizeUnit.MB.toIntBytes(1); + + @Override + public void responseReceived(HttpResponse response) { + responseStarted.countDown(); + } + + @Override + public void consumeContent(ContentDecoder decoder, IOControl ioControl) throws IOException { + readBuffer.clear(); + final var bytesRead = decoder.read(readBuffer); + if (bytesRead > 0) { + bytesToConsume -= bytesRead; + } + + if (bytesToConsume <= 0) { + bodyConsumed.countDown(); + ioControl.shutdown(); + } + } + + @Override + public void responseCompleted(HttpContext context) {} + + @Override + public void failed(Exception ex) {} + + @Override + public Exception getException() { + return null; + } + + @Override + public HttpResponse getResult() { + return null; + } + + @Override + public boolean isDone() { + return false; + } + + @Override + public void close() {} + + @Override + public boolean cancel() { + return false; + } + })); + + try { + try (var restClient = createRestClient(internalCluster().getRandomNodeName())) { + // one-node REST client to avoid retries + expectThrows(ConnectionClosedException.class, () -> restClient.performRequest(request)); + } + safeAwait(responseStarted); + safeAwait(bodyConsumed); + } finally { + assertNull(getExpectedEntries()); // mainly just checking that all refs are released + } + } + + private static Map getExpectedEntries() { + final List> nodeResponses = StreamSupport + // concatenate all the chunks in all the entries + .stream(internalCluster().getInstances(PluginsService.class).spliterator(), false) + .flatMap(p -> p.filterPlugins(RandomXContentResponsePlugin.class)) + .flatMap(p -> { + final var response = p.responseRef.getAndSet(null); + if (response == null) { + return Stream.of(); + } else { + safeAwait(response.completedLatch()); // ensures that all refs have been released + return Stream.of(response.fragments()); + } + }) + .toList(); + assertThat(nodeResponses, hasSize(1)); + return nodeResponses.get(0); + } +} diff --git a/server/src/main/java/org/elasticsearch/rest/StreamingXContentResponse.java b/server/src/main/java/org/elasticsearch/rest/StreamingXContentResponse.java new file mode 100644 index 0000000000000..9f20416ff8b06 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/rest/StreamingXContentResponse.java @@ -0,0 +1,435 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.rest; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.io.stream.BytesStream; +import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; +import org.elasticsearch.common.recycler.Recycler; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.core.Streams; +import org.elasticsearch.transport.Transports; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Queue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A REST response with an XContent body to which the caller can write fragments of content in an asynchronous and streaming fashion. + *

+ * Callers submit individual fragments of content using {@link #writeFragment}. Internally, the output entries are held in a queue. + * If the queue becomes empty then the response transmission is paused until the next entry becomes available. + *

+ * The internal queue is unbounded. It is the caller's responsibility to ensure that the response does not consume an excess of resources + * while it's being sent. + *

+ * The caller must eventually call {@link StreamingXContentResponse#close} to finish the transmission of the response. + */ +public final class StreamingXContentResponse implements Releasable { + + /** + * The underlying stream that collects the raw bytes to be transmitted. Mutable, because we collect the contents of each chunk in a + * distinct stream that is held in this field while that chunk is under construction. + */ + @Nullable // if there's no chunk under construction + private BytesStream targetStream; + + private final XContentBuilder xContentBuilder; + + private final RestChannel restChannel; + private final ToXContent.Params params; + private final Releasable onCompletion; + + /** + * A listener for the next fragment to become available for transmission after a pause. Completed with the newly-created unique active + * {@link AvailableFragmentsResponseBodyPart} within {@link #writeFragment}, and subscribed to via + * {@link AvailableFragmentsResponseBodyPart#getNextPart} when the current {@link AvailableFragmentsResponseBodyPart} + * becomes inactive because of a transmission pause. + */ + @Nullable // if the first fragment hasn't been sent yet + private SubscribableListener nextAvailableFragmentListener; + + /** + * A resource to be released when the transmission of the current fragment is complete. Note that we may complete the transmission of + * multiple fragments at the same time, if they are all processed by one call to {@link AvailableFragmentsResponseBodyPart#encodeChunk} + * and transmitted together. + */ + @Nullable // if not currently sending a fragment + private Releasable currentFragmentReleasable; + + /** + * @param restChannel The {@link RestChannel} on which to send the response. + * @param params The {@link ToXContent.Params} to control the serialization. + * @param onCompletion A resource which is released when the transmission is complete. + */ + public StreamingXContentResponse(RestChannel restChannel, ToXContent.Params params, Releasable onCompletion) throws IOException { + this.restChannel = restChannel; + this.params = params; + this.onCompletion = onCompletion; + this.xContentBuilder = restChannel.newBuilder( + restChannel.request().getXContentType(), + null, + true, + Streams.noCloseStream(new OutputStream() { + @Override + public void write(int b) throws IOException { + assert targetStream != null; + targetStream.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + assert targetStream != null; + targetStream.write(b, off, len); + } + }) + ); + } + + /** + * Close this {@link StreamingXContentResponse}, indicating that there will be no more fragments to send. + */ + @Override + public void close() { + writeFragment(p -> NO_MORE_FRAGMENTS, () -> { + if (isRestResponseFinished.compareAndSet(false, true)) { + queueRefs.decRef(); + } + }); + } + + private Iterator getChunksIterator(StreamingFragment fragment) { + return xContentBuilder.getRestApiVersion() == RestApiVersion.V_7 + ? fragment.fragment().toXContentChunkedV7(params) + : fragment.fragment().toXContentChunked(params); + } + + /** + * Enqueue the given fragment for transmission. + * @param fragment The fragment to send. + * @param releasable A resource which is released when the fragment has been completely processed, i.e. when + *

    + *
  • it has been fully sent, or
  • + *
  • the overall response was cancelled before completion and all resources related to the partial transmission of + * this fragment have been released.
  • + *
+ */ + public void writeFragment(ChunkedToXContent fragment, Releasable releasable) { + if (tryAcquireQueueRef()) { + try { + fragmentQueue.add(new StreamingFragment(fragment, releasable)); + if (queueLength.getAndIncrement() == 0) { + // There is no active AvailableChunksZipResponseBodyPart, but there is now an entry in the queue, so we must create a + // AvailableChunksZipResponseBodyPart to process it (along with any other entries that are concurrently added to the + // queue). It's safe to mutate releasable and continuationListener here because they are only otherwise accessed by an + // active AvailableChunksZipResponseBodyPart (which does not exist) or when all queueRefs have been released (which they + // have not here). + final var nextFragment = fragmentQueue.poll(); + assert nextFragment != null; + final var availableFragments = new AvailableFragmentsResponseBodyPart(getChunksIterator(nextFragment)); + assert currentFragmentReleasable == null; + currentFragmentReleasable = nextFragment.releasable(); + final var currentAvailableFragmentListener = nextAvailableFragmentListener; + nextAvailableFragmentListener = new SubscribableListener<>(); + if (currentAvailableFragmentListener == null) { + // We are not resuming after a pause, this is the first fragment to be sent, so we start the response transmission. + restChannel.sendResponse(RestResponse.chunked(RestStatus.OK, availableFragments, this::restResponseFinished)); + } else { + // We are resuming transmission after a pause, so just carry on sending the response body. + assert currentAvailableFragmentListener.isDone() == false; + currentAvailableFragmentListener.onResponse(availableFragments); + } + } + } finally { + queueRefs.decRef(); + } + } else { + Releasables.closeExpectNoException(releasable); + } + } + + /** + * A fragment which is ready for transmission, to be stored in {@link #fragmentQueue}. + * + * @param fragment The fragment of XContent to send. + * @param releasable A resource to release when this fragment has been fully transmitted, or is no longer required because the + * transmission was cancelled. + */ + private record StreamingFragment(ChunkedToXContent fragment, Releasable releasable) {} + + /** + * Queue of fragments that are ready for transmission. + */ + private final Queue fragmentQueue = new LinkedBlockingQueue<>(); + + /** + * Upper bound on the number of fragments in the queue, atomically modified to ensure there's only one thread processing the queue + * at once. + */ + private final AtomicInteger queueLength = new AtomicInteger(); + + /** + * Ref-counting for access to the queue, to avoid clearing the queue on abort concurrently with a fragment being sent. + */ + private final RefCounted queueRefs = AbstractRefCounted.of(this::drainQueue); + + /** + * Flag to indicate if the request has been aborted, at which point we should stop enqueueing more fragments and promptly clean up the + * ones being sent. It's safe to ignore this, but without it in theory a constant stream of calls to {@link #writeFragment} could + * prevent {@link #drainQueue} from running for arbitrarily long. + */ + private final AtomicBoolean isRestResponseFinished = new AtomicBoolean(); + + private boolean tryAcquireQueueRef() { + return isRestResponseFinished.get() == false && queueRefs.tryIncRef(); + } + + private void restResponseFinished() { + assert Transports.assertTransportThread(); + if (isRestResponseFinished.compareAndSet(false, true)) { + queueRefs.decRef(); + } + } + + private void drainQueue() { + assert isRestResponseFinished.get(); + assert queueRefs.hasReferences() == false; + final var taskCount = queueLength.get() + 2 /* currentFragmentReleasable and onCompletion */ ; + final var releasables = new ArrayList(taskCount); + try { + releasables.add(currentFragmentReleasable); + currentFragmentReleasable = null; + StreamingFragment fragment; + while ((fragment = fragmentQueue.poll()) != null) { + releasables.add(fragment.releasable()); + } + assert fragmentQueue.isEmpty() : fragmentQueue.size(); // no concurrent adds + assert releasables.size() == taskCount - 1 || releasables.size() == taskCount - 2 : taskCount + " vs " + releasables.size(); + } finally { + releasables.add(onCompletion); + Releasables.closeExpectNoException(Releasables.wrap(releasables)); + } + } + + /** + * A {@link ChunkedRestResponseBodyPart} which will yield all currently-available fragments by consuming from {@link #fragmentQueue}. + * There is only ever at most one active instance of this class at any time, in the sense that one such instance becoming inactive + * happens-before the creation of the next instance. One of these parts may send chunks for more than one fragment. + */ + private final class AvailableFragmentsResponseBodyPart implements ChunkedRestResponseBodyPart { + + /** + * An iterator over the chunks of the fragment currently being transmitted. + */ + private Iterator fragmentChunksIterator; + + /** + * True when we have run out of chunks ready for immediate transmission, so the response is paused, but we expect to send more data + * later. + */ + private boolean isResponsePaused; + + /** + * True when we have sent the last chunk of the last fragment, or the response was cancelled. + */ + private boolean isResponseComplete; + + /** + * A listener which is created when there are no more available fragments, so transmission is paused, subscribed to in + * {@link #getNextPart}, and then completed with the next body part (sequence of fragments, i.e. a new (unique) active + * {@link AvailableFragmentsResponseBodyPart}). + */ + private SubscribableListener getNextPartListener; + + /** + * A cache for an empty list to be used to collect the {@code Releasable} instances to be released when the next chunk has been + * fully transmitted. It's a list because a call to {@link #encodeChunk} may yield a chunk that completes several fragments, each of + * which has its own resources to release. We cache this value across chunks because most chunks won't release anything, so we can + * keep the empty list around for later to save on allocations. + */ + private ArrayList nextReleasablesCache = new ArrayList<>(); + + AvailableFragmentsResponseBodyPart(Iterator fragmentChunksIterator) { + this.fragmentChunksIterator = fragmentChunksIterator; + } + + /** + * @return whether this part of the response is complete + */ + @Override + public boolean isPartComplete() { + return isResponsePaused || isResponseComplete; + } + + @Override + public boolean isLastPart() { + return isResponseComplete; + } + + @Override + public void getNextPart(ActionListener listener) { + assert getNextPartListener != null; + getNextPartListener.addListener(listener); + } + + /** + * Transfer {@link #currentFragmentReleasable} into the supplied collection (i.e. add it to {@code releasables} and then clear + * {@link #currentFragmentReleasable}). Called when the last chunk of the current fragment is serialized, so that we + * can start serializing chunks of the next fragment straight away whilst delaying the release of the current fragment's resources + * until the transmission of the chunk that is currently under construction. + */ + private void transferCurrentFragmentReleasable(ArrayList releasables) { + assert queueRefs.hasReferences(); + + if (currentFragmentReleasable == null) { + return; + } + + if (releasables == nextReleasablesCache) { + // adding the first value, so we must line up a new cached value for the next caller + nextReleasablesCache = new ArrayList<>(); + } + + releasables.add(currentFragmentReleasable); + currentFragmentReleasable = null; + } + + @Override + public ReleasableBytesReference encodeChunk(int sizeHint, Recycler recycler) throws IOException { + assert Transports.isTransportThread(Thread.currentThread()); + + final ArrayList releasables = nextReleasablesCache; + assert releasables.isEmpty(); + try { + if (tryAcquireQueueRef()) { + try { + assert queueLength.get() > 0; + // This is the current unique active AvailableFragmentsResponseBodyPart (i.e. queueLength is strictly positive and + // we hold a queueRef), so any concurrent calls to writeFragment() at this point will just add to the queue and + // won't spawn a new AvailableFragmentsResponseBodyPart or mutate any fields. + + final RecyclerBytesStreamOutput chunkStream = new RecyclerBytesStreamOutput(recycler); + assert targetStream == null; + targetStream = chunkStream; + + do { + if (fragmentChunksIterator.hasNext()) { + fragmentChunksIterator.next().toXContent(xContentBuilder, params); + } else { + completeCurrentFragment(releasables); + } + } while (isResponseComplete == false && isResponsePaused == false && chunkStream.size() < sizeHint); + + assert (releasables == nextReleasablesCache) == releasables.isEmpty(); + assert nextReleasablesCache.isEmpty(); + + final Releasable chunkStreamReleasable = () -> Releasables.closeExpectNoException(chunkStream); + final var result = new ReleasableBytesReference( + chunkStream.bytes(), + releasables.isEmpty() + ? chunkStreamReleasable + : Releasables.wrap(Iterators.concat(Iterators.single(chunkStreamReleasable), releasables.iterator())) + ); + targetStream = null; + return result; + } finally { + queueRefs.decRef(); + } + } else { + // request aborted, nothing more to send (queue is being cleared by queueRefs#closeInternal) + isResponseComplete = true; + return new ReleasableBytesReference(BytesArray.EMPTY, () -> {}); + } + } catch (Exception e) { + logger.error("failure encoding chunk", e); + throw e; + } finally { + if (targetStream != null) { + assert false : "failure encoding chunk"; + IOUtils.closeWhileHandlingException(targetStream, Releasables.wrap(releasables)); + targetStream = null; + } + } + } + + private void completeCurrentFragment(ArrayList releasables) throws IOException { + transferCurrentFragmentReleasable(releasables); + final var localNextAvailableFragmentListener = nextAvailableFragmentListener; // read before queue len decr + final var newQueueLength = queueLength.decrementAndGet(); + if (fragmentChunksIterator == NO_MORE_FRAGMENTS) { + // The current fragment is the last-fragment sentinel, so we stop processing the queue completely. Note + // that closing the XContentBuilder here ensures that the response is well-formed - it's up to the + // caller to ensure this, even if errors occur. + xContentBuilder.close(); + isResponseComplete = true; + } else if (newQueueLength == 0) { + // The current fragment is complete, but the next fragment isn't available yet, so we pause + // transmission. This means we are no longer an active AvailableFragmentsResponseBodyPart, so any + // concurrent calls to writeFragment() at this point will now spawn a new + // AvailableFragmentsResponseBodyPart to take our place. + xContentBuilder.flush(); + isResponsePaused = true; + assert getNextPartListener == null; + assert localNextAvailableFragmentListener != null; + // Calling our getNextPart() will eventually yield the next fragment supplied to writeFragment(): + getNextPartListener = localNextAvailableFragmentListener; + } else { + // The current fragment is complete, and the next fragment is already available, so we start sending its + // chunks too. This means we're still the unique active AvailableFragmentsResponseBodyPart. We re-use + // this AvailableFragmentsResponseBodyPart instance rather than creating a new one to avoid unnecessary + // allocations. + + final var nextFragment = fragmentQueue.poll(); + assert nextFragment != null; + currentFragmentReleasable = nextFragment.releasable(); + fragmentChunksIterator = getChunksIterator(nextFragment); + } + } + + @Override + public String getResponseContentTypeString() { + return xContentBuilder.getResponseContentTypeString(); + } + } + + /** + * Sentinel fragment indicating the end of the response. + */ + private static final Iterator NO_MORE_FRAGMENTS = new Iterator<>() { + @Override + public boolean hasNext() { + return false; + } + + @Override + public ToXContent next() { + assert false : "not called"; + return ToXContent.EMPTY; + } + }; +} From 1222496cd0f602d7769e7fcdbc59b84685356d6e Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 06:35:52 +0100 Subject: [PATCH 04/15] Improve reaction to blob store corruptions (#111954) Today there are a couple of assertions that can trip if the contents of a snapshot repostiory are corrupted. It makes sense to assert the integrity of snapshots in most tests, but we must also (a) protect against these corruptions in production and (b) allow some tests to verify the behaviour of the system when the repository is corrupted. This commit introduces a flag to disable certain assertions, converts the relevant assertions into production failures too, and introduces a high-level test to verify that we do detect all relevant corruptions without tripping any other assertions. Extracted from #93735 as this change makes sense in its own right. Relates #52622. --- docs/reference/release-notes/8.15.0.asciidoc | 2 + .../blobstore/BlobStoreCorruptionIT.java | 186 ++++++++++++++++++ .../BlobStoreIndexShardSnapshot.java | 18 +- .../BlobStoreIndexShardSnapshots.java | 9 +- ...ndexShardSnapshotsIntegritySuppressor.java | 27 +++ .../blobstore/RepositoryFileType.java | 60 ++++++ 6 files changed, 300 insertions(+), 2 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreCorruptionIT.java create mode 100644 test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java create mode 100644 test/framework/src/main/java/org/elasticsearch/repositories/blobstore/RepositoryFileType.java diff --git a/docs/reference/release-notes/8.15.0.asciidoc b/docs/reference/release-notes/8.15.0.asciidoc index 1df0969ecc629..80e935e130678 100644 --- a/docs/reference/release-notes/8.15.0.asciidoc +++ b/docs/reference/release-notes/8.15.0.asciidoc @@ -16,6 +16,8 @@ after it is killed up to four times in 24 hours. (issue: {es-issue}110530[#11053 * Pipeline aggregations under `time_series` and `categorize_text` aggregations are never returned (issue: {es-issue}111679[#111679]) +* Elasticsearch will not start on Windows machines when the recommended [bootstrap.memory_lock: true](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup-configuration-memory.html#bootstrap-memory_lock) setting is configured due to [native access refactoring](https://github.com/elastic/elasticsearch/pull/111866). The workaround for 8.15.0 is to downgrade to the previous version. This issue will be fixed in 8.15.1. + [[breaking-8.15.0]] [float] === Breaking changes diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreCorruptionIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreCorruptionIT.java new file mode 100644 index 0000000000000..422696d6b61c6 --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreCorruptionIT.java @@ -0,0 +1,186 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.repositories.blobstore; + +import org.apache.lucene.tests.mockfile.ExtrasFS; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.elasticsearch.action.support.ActionTestUtils; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.Strings; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshotsIntegritySuppressor; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Base64; +import java.util.List; + +public class BlobStoreCorruptionIT extends AbstractSnapshotIntegTestCase { + + private static final Logger logger = LogManager.getLogger(BlobStoreCorruptionIT.class); + + @Before + public void suppressConsistencyCheck() { + disableRepoConsistencyCheck("testing corruption detection involves breaking the repo"); + } + + public void testCorruptionDetection() throws Exception { + final var repositoryName = randomIdentifier(); + final var indexName = randomIdentifier(); + final var snapshotName = randomIdentifier(); + final var repositoryRootPath = randomRepoPath(); + + createRepository(repositoryName, FsRepository.TYPE, repositoryRootPath); + createIndexWithRandomDocs(indexName, between(1, 100)); + flushAndRefresh(indexName); + createSnapshot(repositoryName, snapshotName, List.of(indexName)); + + final var corruptedFile = corruptRandomFile(repositoryRootPath); + final var corruptedFileType = RepositoryFileType.getRepositoryFileType(repositoryRootPath, corruptedFile); + final var corruptionDetectors = new ArrayList, ?>>(); + + // detect corruption by listing the snapshots + if (corruptedFileType == RepositoryFileType.SNAPSHOT_INFO) { + corruptionDetectors.add(exceptionListener -> { + logger.info("--> listing snapshots"); + client().admin() + .cluster() + .prepareGetSnapshots(TEST_REQUEST_TIMEOUT, repositoryName) + .execute(ActionTestUtils.assertNoSuccessListener(exceptionListener::onResponse)); + }); + } + + // detect corruption by taking another snapshot + if (corruptedFileType == RepositoryFileType.SHARD_GENERATION) { + corruptionDetectors.add(exceptionListener -> { + logger.info("--> taking another snapshot"); + client().admin() + .cluster() + .prepareCreateSnapshot(TEST_REQUEST_TIMEOUT, repositoryName, randomIdentifier()) + .setWaitForCompletion(true) + .execute(exceptionListener.map(createSnapshotResponse -> { + assertNotEquals(SnapshotState.SUCCESS, createSnapshotResponse.getSnapshotInfo().state()); + return new ElasticsearchException("create-snapshot failed as expected"); + })); + }); + } + + // detect corruption by restoring the snapshot + switch (corruptedFileType) { + case SNAPSHOT_INFO, GLOBAL_METADATA, INDEX_METADATA -> corruptionDetectors.add(exceptionListener -> { + logger.info("--> restoring snapshot"); + client().admin() + .cluster() + .prepareRestoreSnapshot(TEST_REQUEST_TIMEOUT, repositoryName, snapshotName) + .setRestoreGlobalState(corruptedFileType == RepositoryFileType.GLOBAL_METADATA || randomBoolean()) + .setWaitForCompletion(true) + .execute(ActionTestUtils.assertNoSuccessListener(exceptionListener::onResponse)); + }); + case SHARD_SNAPSHOT_INFO, SHARD_DATA -> corruptionDetectors.add(exceptionListener -> { + logger.info("--> restoring snapshot and checking for failed shards"); + SubscribableListener + // if shard-level data is corrupted then the overall restore succeeds but the shard recoveries fail + .newForked(l -> client().admin().indices().prepareDelete(indexName).execute(l)) + .andThenAccept(ElasticsearchAssertions::assertAcked) + + .andThen( + l -> client().admin() + .cluster() + .prepareRestoreSnapshot(TEST_REQUEST_TIMEOUT, repositoryName, snapshotName) + .setRestoreGlobalState(randomBoolean()) + .setWaitForCompletion(true) + .execute(l) + ) + + .addListener(exceptionListener.map(restoreSnapshotResponse -> { + assertNotEquals(0, restoreSnapshotResponse.getRestoreInfo().failedShards()); + return new ElasticsearchException("post-restore recoveries failed as expected"); + })); + }); + } + + try (var ignored = new BlobStoreIndexShardSnapshotsIntegritySuppressor()) { + final var exception = safeAwait(randomFrom(corruptionDetectors)); + logger.info(Strings.format("--> corrupted [%s] and caught exception", corruptedFile), exception); + } + } + + private static Path corruptRandomFile(Path repositoryRootPath) throws IOException { + final var corruptedFileType = getRandomCorruptibleFileType(); + final var corruptedFile = getRandomFileToCorrupt(repositoryRootPath, corruptedFileType); + if (randomBoolean()) { + logger.info("--> deleting [{}]", corruptedFile); + Files.delete(corruptedFile); + } else { + corruptFileContents(corruptedFile); + } + return corruptedFile; + } + + private static void corruptFileContents(Path fileToCorrupt) throws IOException { + final var oldFileContents = Files.readAllBytes(fileToCorrupt); + logger.info("--> contents of [{}] before corruption: [{}]", fileToCorrupt, Base64.getEncoder().encodeToString(oldFileContents)); + final byte[] newFileContents = new byte[randomBoolean() ? oldFileContents.length : between(0, oldFileContents.length)]; + System.arraycopy(oldFileContents, 0, newFileContents, 0, newFileContents.length); + if (newFileContents.length == oldFileContents.length) { + final var corruptionPosition = between(0, newFileContents.length - 1); + newFileContents[corruptionPosition] = randomValueOtherThan(oldFileContents[corruptionPosition], ESTestCase::randomByte); + logger.info( + "--> updating byte at position [{}] from [{}] to [{}]", + corruptionPosition, + oldFileContents[corruptionPosition], + newFileContents[corruptionPosition] + ); + } else { + logger.info("--> truncating file from length [{}] to length [{}]", oldFileContents.length, newFileContents.length); + } + Files.write(fileToCorrupt, newFileContents); + logger.info("--> contents of [{}] after corruption: [{}]", fileToCorrupt, Base64.getEncoder().encodeToString(newFileContents)); + } + + private static RepositoryFileType getRandomCorruptibleFileType() { + return randomValueOtherThanMany( + // these blob types do not have reliable corruption detection, so we must skip them + t -> t == RepositoryFileType.ROOT_INDEX_N || t == RepositoryFileType.ROOT_INDEX_LATEST, + () -> randomFrom(RepositoryFileType.values()) + ); + } + + private static Path getRandomFileToCorrupt(Path repositoryRootPath, RepositoryFileType corruptedFileType) throws IOException { + final var corruptibleFiles = new ArrayList(); + Files.walkFileTree(repositoryRootPath, new SimpleFileVisitor<>() { + @Override + public FileVisitResult visitFile(Path filePath, BasicFileAttributes attrs) throws IOException { + if (ExtrasFS.isExtra(filePath.getFileName().toString()) == false + && RepositoryFileType.getRepositoryFileType(repositoryRootPath, filePath) == corruptedFileType) { + corruptibleFiles.add(filePath); + } + return super.visitFile(filePath, attrs); + } + }); + return randomFrom(corruptibleFiles); + } + +} diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index 2a8fe96151c11..817ecb4601d59 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentParserUtils; import org.elasticsearch.core.Nullable; +import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentFragment; @@ -318,7 +319,11 @@ public static FileInfo fromXContent(XContentParser parser) throws IOException { } case WRITER_UUID -> { writerUuid = new BytesRef(parser.binaryValue()); - assert writerUuid.length > 0; + assert BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED == false || writerUuid.length > 0; + if (writerUuid.length == 0) { + // we never write UNAVAILABLE_WRITER_UUID, so this must be due to corruption + throw new ElasticsearchParseException("invalid (empty) writer uuid"); + } } default -> XContentParserUtils.throwUnknownField(currentFieldName, parser); } @@ -336,6 +341,12 @@ public static FileInfo fromXContent(XContentParser parser) throws IOException { } else if (checksum == null) { throw new ElasticsearchParseException("missing checksum for name [" + name + "]"); } + try { + // check for corruption before asserting writtenBy is parseable in the StoreFileMetadata constructor + org.apache.lucene.util.Version.parse(writtenBy); + } catch (Exception e) { + throw new ElasticsearchParseException("invalid written_by [" + writtenBy + "]"); + } return new FileInfo(name, new StoreFileMetadata(physicalName, length, checksum, writtenBy, metaHash, writerUuid), partSize); } @@ -566,6 +577,11 @@ public static BlobStoreIndexShardSnapshot fromXContent(XContentParser parser) th } } + // check for corruption before asserting snapshot != null in the BlobStoreIndexShardSnapshot ctor + if (snapshot == null) { + throw new CorruptStateException("snapshot missing"); + } + return new BlobStoreIndexShardSnapshot( snapshot, indexFiles == null ? List.of() : indexFiles, diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index b17545a4cbeb6..30fbbba5ed095 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -264,6 +264,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + static volatile boolean INTEGRITY_ASSERTIONS_ENABLED = true; + public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) throws IOException { XContentParser.Token token = parser.currentToken(); if (token == null) { // New parser @@ -317,7 +319,12 @@ public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) t List fileInfosBuilder = new ArrayList<>(); for (String file : entry.v2()) { FileInfo fileInfo = files.get(file); - assert fileInfo != null; + if (fileInfo == null) { + // could happen in production if the repo contents are corrupted + final var exception = new IllegalStateException("shard index inconsistent at file [" + file + "]"); + assert INTEGRITY_ASSERTIONS_ENABLED == false : exception; + throw exception; + } fileInfosBuilder.add(fileInfo); } snapshots.add(new SnapshotFiles(entry.v1(), Collections.unmodifiableList(fileInfosBuilder), historyUUIDs.get(entry.v1()))); diff --git a/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java b/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java new file mode 100644 index 0000000000000..511116d9b2125 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java @@ -0,0 +1,27 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.snapshots.blobstore; + +import org.elasticsearch.core.Releasable; + +/** + * Test utility class to suppress assertions about the integrity of the contents of a blobstore repository, in order to verify the + * production behaviour on encountering invalid data. + */ +public class BlobStoreIndexShardSnapshotsIntegritySuppressor implements Releasable { + + public BlobStoreIndexShardSnapshotsIntegritySuppressor() { + BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED = false; + } + + @Override + public void close() { + BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED = true; + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/RepositoryFileType.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/RepositoryFileType.java new file mode 100644 index 0000000000000..014cbcd2bcc3a --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/RepositoryFileType.java @@ -0,0 +1,60 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.repositories.blobstore; + +import org.elasticsearch.common.Strings; + +import java.nio.file.Path; +import java.util.regex.Pattern; + +/** + * The types of blobs in a {@link BlobStoreRepository}. + */ +public enum RepositoryFileType { + + ROOT_INDEX_N("index-NUM"), + ROOT_INDEX_LATEST("index.latest"), + SNAPSHOT_INFO("snap-UUID.dat"), + GLOBAL_METADATA("meta-UUID.dat"), + INDEX_METADATA("indices/UUID/meta-SHORTUUID.dat"), + SHARD_GENERATION("indices/UUID/NUM/index-UUID"), + SHARD_SNAPSHOT_INFO("indices/UUID/NUM/snap-UUID.dat"), + SHARD_DATA("indices/UUID/NUM/__UUID"), + // NB no support for legacy names (yet) + ; + + private final Pattern pattern; + + RepositoryFileType(String regex) { + pattern = Pattern.compile( + "^(" + + regex + // decimal numbers + .replace("NUM", "(0|[1-9][0-9]*)") + // 15-byte UUIDS from TimeBasedUUIDGenerator + .replace("SHORTUUID", "[0-9a-zA-Z_-]{20}") + // 16-byte UUIDs from RandomBasedUUIDGenerator + .replace("UUID", "[0-9a-zA-Z_-]{22}") + + ")$" + ); + } + + public static RepositoryFileType getRepositoryFileType(Path repositoryRoot, Path blobPath) { + final var relativePath = repositoryRoot.relativize(blobPath).toString().replace(repositoryRoot.getFileSystem().getSeparator(), "/"); + for (final var repositoryFileType : RepositoryFileType.values()) { + if (repositoryFileType.pattern.matcher(relativePath).matches()) { + return repositoryFileType; + } + } + throw new IllegalArgumentException( + Strings.format("[%s] is not the path of a known blob type within [%s]", relativePath, repositoryRoot) + ); + } + +} From 4e1a84c8831cb73ea11aad1e53b6401e1b89677d Mon Sep 17 00:00:00 2001 From: Gergely Kalapos Date: Mon, 19 Aug 2024 09:24:47 +0200 Subject: [PATCH 05/15] x-pack/plugin/otel: introduce x-pack-otel plugin (#111091) * Add YamlTemplateRegistry and OtelIndexTemplateRegistry with resource YAML files * Fix traces-otel template * Adding first yml tests * Base APMIndexTemplateRegistry on YamlTemplateRegistry * Update OTelPlugin.java * Update APMIndexTemplateRegistry.java * Update YamlIngestPipelineConfig.java * Adding traces tests * Update x-pack/plugin/otel-data/src/main/resources/component-templates/ecs-tsdb@mappings.yaml Co-authored-by: Felix Barnsteiner * Add mapper-version * Fix code-style * Rename `status.status_code` to `status.code` * Update otel@mappings.yaml Revert back to date due to missing support in ES|QL for date_nanos * Move dynamic_templates to metrics@mappings in core * Run gradlew :x-pack:plugin:core:spotlessApply * Update x-pack/plugin/otel-data/src/main/resources/component-templates/metrics-otel@mappings.yaml Co-authored-by: Carson Ip * Update 20_metic_tests.yml Workaround for TSDB timestamp issue: we push a custom template with higher priority and set time_series.start_time. * Update CODEOWNERS Adding obs-ds-intake-services as owner of the new otel-data plugin. Since we had some changes, also updating the owner of apm-data to the same team. * Change dynamic: strict to false * Skip "Reject invalid top level field" test * Update 20_metic_tests.yml * Add boolean as dimension test (skipping it for now) * Add booleans_to_keywords and enable corresponding test * Remove processor.event top level mapping Reason: for metrics and logs we can rely on the name of the datastream. For spans vs. transactions there are other fields we can use. * Remove booleans_to_keywords Because booleans are supported now as dimension on TSDB * Add alias service.language.name -> telemetry.sdk.language * cleanup * Update README.md * Update README.md * Update docs/changelog/111091.yaml * Move traces@settings and traces@mappings to core * Update traces-otel@mappings.yaml * Review feedback * Adapt `match` style in tests * Update docs/changelog/111091.yaml * Apply suggestions from code review Co-authored-by: Vishal Raj * Update x-pack/plugin/otel-data/src/main/resources/component-templates/traces-otel@mappings.yaml Co-authored-by: Carson Ip * Changing trace_flags to long Related discussion: https://github.com/elastic/elasticsearch/pull/111091#discussion_r1706698491 * Remove trace_flags see: https://github.com/elastic/opentelemetry-dev/pull/368#pullrequestreview-2229633970 * Apply suggestions from code review Co-authored-by: Andrew Wilkins * Review feedback * Add store_array_source for span links * Define constant `data_stream.type` in `template.yaml`s * Create package-info.java * Move ecs-tsdb@mappings to index template Add test to verify that @custom template can add dynamic templates with a higher precedence * Update metrics@mappings.json Remove summary_gauge and summary_counter since they are covered by summary_metrics * Move clusterService.getClusterSettings().addSettingsUpdateConsumer to registry * Fix code-style * Update x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_logs.tests.yml Co-authored-by: Felix Barnsteiner * Enable logsdb * Update traces@settings.json No lifecycle needed for OTel at this point --------- Co-authored-by: Felix Barnsteiner Co-authored-by: Carson Ip Co-authored-by: Vishal Raj Co-authored-by: Andrew Wilkins Co-authored-by: Elastic Machine Co-authored-by: Felix Barnsteiner --- .github/CODEOWNERS | 8 +- docs/changelog/111091.yaml | 5 + .../apmdata/APMIndexTemplateRegistry.java | 163 ++----------- .../xpack/apmdata/APMPlugin.java | 1 - .../xpack/apmdata/ResourceUtils.java | 43 ---- .../component-templates/traces@mappings.yaml | 11 - .../src/main/resources/resources.yaml | 1 - .../APMIndexTemplateRegistryTests.java | 3 +- .../xpack/core/ClientHelper.java | 1 + .../xpack/core/XPackSettings.java | 7 + .../xpack/core/template/ResourceUtils.java | 44 ++++ .../template}/YamlIngestPipelineConfig.java | 22 +- .../core/template/YamlTemplateRegistry.java | 219 ++++++++++++++++++ .../src/main/resources/metrics@mappings.json | 44 ++++ .../src/main/resources/traces@mappings.json | 28 +++ .../src/main/resources/traces@settings.json | 18 ++ x-pack/plugin/otel-data/README.md | 33 +++ x-pack/plugin/otel-data/build.gradle | 38 +++ .../oteldata/OTelIndexTemplateRegistry.java | 58 +++++ .../xpack/oteldata/OTelPlugin.java | 79 +++++++ .../xpack/oteldata/package-info.java | 17 ++ .../logs-otel@mappings.yaml | 42 ++++ .../metrics-otel@mappings.yaml | 17 ++ .../component-templates/otel@mappings.yaml | 64 +++++ .../semconv-resource-to-ecs@mappings.yaml | 128 ++++++++++ .../traces-otel@mappings.yaml | 68 ++++++ .../index-templates/logs-otel@template.yaml | 27 +++ .../metrics-otel@template.yaml | 36 +++ .../index-templates/traces-otel@template.yaml | 27 +++ .../src/main/resources/resources.yaml | 15 ++ .../xpack/oteldata/OTelYamlTestSuiteIT.java | 54 +++++ .../resources/rest-api-spec/test/10_otel.yml | 38 +++ .../rest-api-spec/test/20_logs.tests.yml | 22 ++ .../rest-api-spec/test/20_metrics_tests.yml | 149 ++++++++++++ .../rest-api-spec/test/20_traces_tests.yml | 94 ++++++++ .../test/30_non_ecs_alias_tests.yml | 37 +++ .../security/authz/AuthorizationUtils.java | 2 + .../xpack/stack/StackTemplateRegistry.java | 22 +- .../stack/StackTemplateRegistryTests.java | 14 +- 39 files changed, 1480 insertions(+), 219 deletions(-) create mode 100644 docs/changelog/111091.yaml delete mode 100644 x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java delete mode 100644 x-pack/plugin/apm-data/src/main/resources/component-templates/traces@mappings.yaml create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/ResourceUtils.java rename x-pack/plugin/{apm-data/src/main/java/org/elasticsearch/xpack/apmdata => core/src/main/java/org/elasticsearch/xpack/core/template}/YamlIngestPipelineConfig.java (56%) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java create mode 100644 x-pack/plugin/core/template-resources/src/main/resources/traces@mappings.json create mode 100644 x-pack/plugin/core/template-resources/src/main/resources/traces@settings.json create mode 100644 x-pack/plugin/otel-data/README.md create mode 100644 x-pack/plugin/otel-data/build.gradle create mode 100644 x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelIndexTemplateRegistry.java create mode 100644 x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelPlugin.java create mode 100644 x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/package-info.java create mode 100644 x-pack/plugin/otel-data/src/main/resources/component-templates/logs-otel@mappings.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/component-templates/metrics-otel@mappings.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/component-templates/otel@mappings.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/component-templates/semconv-resource-to-ecs@mappings.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/component-templates/traces-otel@mappings.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/index-templates/logs-otel@template.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/index-templates/metrics-otel@template.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/index-templates/traces-otel@template.yaml create mode 100644 x-pack/plugin/otel-data/src/main/resources/resources.yaml create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/java/org/elasticsearch/xpack/oteldata/OTelYamlTestSuiteIT.java create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/10_otel.yml create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_logs.tests.yml create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_metrics_tests.yml create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_traces_tests.yml create mode 100644 x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/30_non_ecs_alias_tests.yml diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 0f7e3073ed022..5b98444c044d2 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -27,8 +27,12 @@ libs/logstash-bridge @elastic/logstash x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/KibanaOwnedReservedRoleDescriptors.java @elastic/kibana-security # APM Data index templates, etc. -x-pack/plugin/apm-data/src/main/resources @elastic/apm-server -x-pack/plugin/apm-data/src/yamlRestTest/resources @elastic/apm-server +x-pack/plugin/apm-data/src/main/resources @elastic/obs-ds-intake-services +x-pack/plugin/apm-data/src/yamlRestTest/resources @elastic/obs-ds-intake-services + +# OTel +x-pack/plugin/otel-data/src/main/resources @elastic/obs-ds-intake-services +x-pack/plugin/otel-data/src/yamlRestTest/resources @elastic/obs-ds-intake-services # Delivery gradle @elastic/es-delivery diff --git a/docs/changelog/111091.yaml b/docs/changelog/111091.yaml new file mode 100644 index 0000000000000..8444681a14a48 --- /dev/null +++ b/docs/changelog/111091.yaml @@ -0,0 +1,5 @@ +pr: 111091 +summary: "X-pack/plugin/otel: introduce x-pack-otel plugin" +area: Data streams +type: feature +issues: [] diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistry.java b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistry.java index 04b0257f4180a..6f5d4e13dc56b 100644 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistry.java +++ b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistry.java @@ -7,53 +7,24 @@ package org.elasticsearch.xpack.apmdata; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.client.internal.Client; -import org.elasticsearch.cluster.ClusterChangedEvent; -import org.elasticsearch.cluster.metadata.ComponentTemplate; -import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.core.Nullable; import org.elasticsearch.features.FeatureService; -import org.elasticsearch.features.NodeFeature; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.NamedXContentRegistry; -import org.elasticsearch.xcontent.XContentParserConfiguration; -import org.elasticsearch.xcontent.yaml.YamlXContent; import org.elasticsearch.xpack.core.ClientHelper; -import org.elasticsearch.xpack.core.template.IndexTemplateRegistry; -import org.elasticsearch.xpack.core.template.IngestPipelineConfig; +import org.elasticsearch.xpack.core.template.YamlTemplateRegistry; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.elasticsearch.xpack.apmdata.ResourceUtils.APM_TEMPLATE_VERSION_VARIABLE; -import static org.elasticsearch.xpack.apmdata.ResourceUtils.loadResource; -import static org.elasticsearch.xpack.apmdata.ResourceUtils.loadVersionedResourceUTF8; +import static org.elasticsearch.xpack.apmdata.APMPlugin.APM_DATA_REGISTRY_ENABLED; /** * Creates all index templates and ingest pipelines that are required for using Elastic APM. */ -public class APMIndexTemplateRegistry extends IndexTemplateRegistry { - private static final Logger logger = LogManager.getLogger(APMIndexTemplateRegistry.class); - // this node feature is a redefinition of {@link DataStreamFeatures#DATA_STREAM_LIFECYCLE} and it's meant to avoid adding a - // dependency to the data-streams module just for this - public static final NodeFeature DATA_STREAM_LIFECYCLE = new NodeFeature("data_stream.lifecycle"); - private final int version; +public class APMIndexTemplateRegistry extends YamlTemplateRegistry { - private final Map componentTemplates; - private final Map composableIndexTemplates; - private final List ingestPipelines; - private final FeatureService featureService; - private volatile boolean enabled; + public static final String APM_TEMPLATE_VERSION_VARIABLE = "xpack.apmdata.template.version"; - @SuppressWarnings("unchecked") public APMIndexTemplateRegistry( Settings nodeSettings, ClusterService clusterService, @@ -62,133 +33,29 @@ public APMIndexTemplateRegistry( NamedXContentRegistry xContentRegistry, FeatureService featureService ) { - super(nodeSettings, clusterService, threadPool, client, xContentRegistry); - - try { - final Map apmResources = XContentHelper.convertToMap( - YamlXContent.yamlXContent, - loadResource("/resources.yaml"), - false - ); - version = (((Number) apmResources.get("version")).intValue()); - final List componentTemplateNames = (List) apmResources.get("component-templates"); - final List indexTemplateNames = (List) apmResources.get("index-templates"); - final List ingestPipelineConfigs = (List) apmResources.get("ingest-pipelines"); - - componentTemplates = componentTemplateNames.stream() - .map(o -> (String) o) - .collect(Collectors.toMap(name -> name, name -> loadComponentTemplate(name, version))); - composableIndexTemplates = indexTemplateNames.stream() - .map(o -> (String) o) - .collect(Collectors.toMap(name -> name, name -> loadIndexTemplate(name, version))); - ingestPipelines = ingestPipelineConfigs.stream().map(o -> (Map>) o).map(map -> { - Map.Entry> pipelineConfig = map.entrySet().iterator().next(); - return loadIngestPipeline(pipelineConfig.getKey(), version, (List) pipelineConfig.getValue().get("dependencies")); - }).collect(Collectors.toList()); - this.featureService = featureService; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - public int getVersion() { - return version; - } - - void setEnabled(boolean enabled) { - logger.info("APM index template registry is {}", enabled ? "enabled" : "disabled"); - this.enabled = enabled; - } - - public boolean isEnabled() { - return enabled; - } - - public void close() { - clusterService.removeListener(this); - } - - @Override - protected String getOrigin() { - return ClientHelper.APM_ORIGIN; - } - - @Override - protected boolean isClusterReady(ClusterChangedEvent event) { - // Ensure current version of the components are installed only after versions that support data stream lifecycle - // due to the use of the feature in all the `@lifecycle` component templates - return featureService.clusterHasFeature(event.state(), DATA_STREAM_LIFECYCLE); + super(nodeSettings, clusterService, threadPool, client, xContentRegistry, featureService); } @Override - protected boolean requiresMasterNode() { - return true; + public String getName() { + return "apm"; } @Override - protected Map getComponentTemplateConfigs() { - if (enabled) { - return componentTemplates; - } else { - return Map.of(); + public void initialize() { + super.initialize(); + if (isEnabled()) { + clusterService.getClusterSettings().addSettingsUpdateConsumer(APM_DATA_REGISTRY_ENABLED, this::setEnabled); } } @Override - protected Map getComposableTemplateConfigs() { - if (enabled) { - return composableIndexTemplates; - } else { - return Map.of(); - } - } - - @Override - protected List getIngestPipelines() { - if (enabled) { - return ingestPipelines; - } else { - return Collections.emptyList(); - } - } - - private static ComponentTemplate loadComponentTemplate(String name, int version) { - try { - final byte[] content = loadVersionedResourceUTF8("/component-templates/" + name + ".yaml", version); - try (var parser = YamlXContent.yamlXContent.createParser(XContentParserConfiguration.EMPTY, content)) { - return ComponentTemplate.parse(parser); - } - } catch (Exception e) { - throw new RuntimeException("failed to load APM Ingest plugin's component template: " + name, e); - } - } - - private static ComposableIndexTemplate loadIndexTemplate(String name, int version) { - try { - final byte[] content = loadVersionedResourceUTF8("/index-templates/" + name + ".yaml", version); - try (var parser = YamlXContent.yamlXContent.createParser(XContentParserConfiguration.EMPTY, content)) { - return ComposableIndexTemplate.parse(parser); - } - } catch (Exception e) { - throw new RuntimeException("failed to load APM Ingest plugin's index template: " + name, e); - } - } - - private static IngestPipelineConfig loadIngestPipeline(String name, int version, @Nullable List dependencies) { - if (dependencies == null) { - dependencies = Collections.emptyList(); - } - return new YamlIngestPipelineConfig( - name, - "/ingest-pipelines/" + name + ".yaml", - version, - APM_TEMPLATE_VERSION_VARIABLE, - dependencies - ); + protected String getVersionProperty() { + return APM_TEMPLATE_VERSION_VARIABLE; } @Override - protected boolean applyRolloverAfterTemplateV2Upgrade() { - return true; + protected String getOrigin() { + return ClientHelper.APM_ORIGIN; } } diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMPlugin.java b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMPlugin.java index 102b0d38461c3..aefb45f6186c1 100644 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMPlugin.java +++ b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/APMPlugin.java @@ -60,7 +60,6 @@ public Collection createComponents(PluginServices services) { if (enabled) { APMIndexTemplateRegistry registryInstance = registry.get(); registryInstance.setEnabled(APM_DATA_REGISTRY_ENABLED.get(settings)); - clusterService.getClusterSettings().addSettingsUpdateConsumer(APM_DATA_REGISTRY_ENABLED, registryInstance::setEnabled); registryInstance.initialize(); } return Collections.emptyList(); diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java deleted file mode 100644 index 1e6a9a9998a82..0000000000000 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.apmdata; - -import org.elasticsearch.common.io.Streams; -import org.elasticsearch.xpack.core.template.TemplateUtils; - -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.Map; - -public class ResourceUtils { - - public static final String APM_TEMPLATE_VERSION_VARIABLE = "xpack.apmdata.template.version"; - - static byte[] loadVersionedResourceUTF8(String name, int version) { - return loadVersionedResourceUTF8(name, version, Map.of()); - } - - static byte[] loadVersionedResourceUTF8(String name, int version, Map variables) { - try { - String content = loadResource(name); - content = TemplateUtils.replaceVariables(content, String.valueOf(version), APM_TEMPLATE_VERSION_VARIABLE, variables); - return content.getBytes(StandardCharsets.UTF_8); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - static String loadResource(String name) throws IOException { - InputStream is = APMIndexTemplateRegistry.class.getResourceAsStream(name); - if (is == null) { - throw new IOException("Resource [" + name + "] not found in classpath."); - } - return Streams.readFully(is).utf8ToString(); - } -} diff --git a/x-pack/plugin/apm-data/src/main/resources/component-templates/traces@mappings.yaml b/x-pack/plugin/apm-data/src/main/resources/component-templates/traces@mappings.yaml deleted file mode 100644 index 51c987df4df60..0000000000000 --- a/x-pack/plugin/apm-data/src/main/resources/component-templates/traces@mappings.yaml +++ /dev/null @@ -1,11 +0,0 @@ ---- -version: ${xpack.apmdata.template.version} -_meta: - description: Default mappings for traces data streams - managed: true -template: - mappings: - properties: - data_stream.type: - type: constant_keyword - value: traces diff --git a/x-pack/plugin/apm-data/src/main/resources/resources.yaml b/x-pack/plugin/apm-data/src/main/resources/resources.yaml index efa6ae694c464..fa38fda679e49 100644 --- a/x-pack/plugin/apm-data/src/main/resources/resources.yaml +++ b/x-pack/plugin/apm-data/src/main/resources/resources.yaml @@ -23,7 +23,6 @@ component-templates: - metrics-apm.service_summary@mappings - metrics-apm.service_transaction@mappings - metrics-apm.transaction@mappings - - traces@mappings - traces-apm@mappings - traces-apm.rum@mappings diff --git a/x-pack/plugin/apm-data/src/test/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistryTests.java b/x-pack/plugin/apm-data/src/test/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistryTests.java index e9f0775836c71..1d6faa0f403d4 100644 --- a/x-pack/plugin/apm-data/src/test/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistryTests.java +++ b/x-pack/plugin/apm-data/src/test/java/org/elasticsearch/xpack/apmdata/APMIndexTemplateRegistryTests.java @@ -75,7 +75,6 @@ public class APMIndexTemplateRegistryTests extends ESTestCase { private APMIndexTemplateRegistry apmIndexTemplateRegistry; private StackTemplateRegistryAccessor stackTemplateRegistryAccessor; - private ClusterService clusterService; private ThreadPool threadPool; private VerifyingClient client; @@ -89,7 +88,7 @@ public void createRegistryAndClient() { threadPool = new TestThreadPool(this.getClass().getName()); client = new VerifyingClient(threadPool); - clusterService = ClusterServiceUtils.createClusterService(threadPool, clusterSettings); + ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, clusterSettings); FeatureService featureService = new FeatureService(List.of(new DataStreamFeatures())); stackTemplateRegistryAccessor = new StackTemplateRegistryAccessor( new StackTemplateRegistry(Settings.EMPTY, clusterService, threadPool, client, NamedXContentRegistry.EMPTY, featureService) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java index d27d7a21ddb73..4e7aa37fe1a0b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ClientHelper.java @@ -194,6 +194,7 @@ private static String maybeRewriteSingleAuthenticationHeaderForVersion( public static final String CONNECTORS_ORIGIN = "connectors"; public static final String INFERENCE_ORIGIN = "inference"; public static final String APM_ORIGIN = "apm"; + public static final String OTEL_ORIGIN = "otel"; private ClientHelper() {} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index d33b2aecdab04..f76b0d2bb6d8d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -93,6 +93,13 @@ public Iterator> settings() { /** Setting for enabling or disabling APM Data. Defaults to true. */ public static final Setting APM_DATA_ENABLED = Setting.boolSetting("xpack.apm_data.enabled", true, Setting.Property.NodeScope); + /** Setting for enabling or disabling OTel Data. Defaults to true. */ + public static final Setting OTEL_DATA_ENABLED = Setting.boolSetting( + "xpack.otel_data.enabled", + true, + Setting.Property.NodeScope + ); + /** Setting for enabling or disabling enterprise search. Defaults to true. */ public static final Setting ENTERPRISE_SEARCH_ENABLED = Setting.boolSetting( "xpack.ent_search.enabled", diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/ResourceUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/ResourceUtils.java new file mode 100644 index 0000000000000..9840535989a7c --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/ResourceUtils.java @@ -0,0 +1,44 @@ +/* + * 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.core.template; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +public class ResourceUtils { + static byte[] loadVersionedResourceUTF8(Class clazz, String name, int version, String versionProperty) { + return loadVersionedResourceUTF8(clazz, name, version, versionProperty, Map.of()); + } + + static byte[] loadVersionedResourceUTF8( + Class clazz, + String name, + int version, + String versionProperty, + Map variables + ) { + try { + String content = loadResource(clazz, name); + content = TemplateUtils.replaceVariables(content, String.valueOf(version), versionProperty, variables); + return content.getBytes(StandardCharsets.UTF_8); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static String loadResource(Class clazz, String name) throws IOException { + InputStream is = clazz.getResourceAsStream(name); + if (is == null) { + throw new IOException("Resource [" + name + "] not found in classpath."); + } + return new String(is.readAllBytes(), java.nio.charset.StandardCharsets.UTF_8); + } + +} diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlIngestPipelineConfig.java similarity index 56% rename from x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java rename to x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlIngestPipelineConfig.java index de1b715dd138d..0cb69b490a73a 100644 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlIngestPipelineConfig.java @@ -5,23 +5,29 @@ * 2.0. */ -package org.elasticsearch.xpack.apmdata; +package org.elasticsearch.xpack.core.template; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.core.template.IngestPipelineConfig; import java.util.List; -import static org.elasticsearch.xpack.apmdata.ResourceUtils.loadVersionedResourceUTF8; +import static org.elasticsearch.xpack.core.template.ResourceUtils.loadVersionedResourceUTF8; -/** - * An APM-plugin-specific implementation that loads ingest pipelines in yaml format from a local resources repository - */ public class YamlIngestPipelineConfig extends IngestPipelineConfig { - public YamlIngestPipelineConfig(String id, String resource, int version, String versionProperty, List dependencies) { + private final Class clazz; + + public YamlIngestPipelineConfig( + String id, + String resource, + int version, + String versionProperty, + List dependencies, + Class clazz + ) { super(id, resource, version, versionProperty, dependencies); + this.clazz = clazz; } @Override @@ -31,6 +37,6 @@ public XContentType getXContentType() { @Override public BytesReference loadConfig() { - return new BytesArray(loadVersionedResourceUTF8("/ingest-pipelines/" + id + ".yaml", version, variables)); + return new BytesArray(loadVersionedResourceUTF8(clazz, "/ingest-pipelines/" + id + ".yaml", version, versionProperty, variables)); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java new file mode 100644 index 0000000000000..7471f722261bf --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java @@ -0,0 +1,219 @@ +/* + * 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.core.template; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.metadata.ComponentTemplate; +import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.features.FeatureService; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.yaml.YamlXContent; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.template.ResourceUtils.loadResource; +import static org.elasticsearch.xpack.core.template.ResourceUtils.loadVersionedResourceUTF8; + +/** + * Creates index templates and ingest pipelines based on YAML files from resources. + */ +public abstract class YamlTemplateRegistry extends IndexTemplateRegistry { + private static final Logger logger = LogManager.getLogger(YamlTemplateRegistry.class); + // this node feature is a redefinition of {@link DataStreamFeatures#DATA_STREAM_LIFECYCLE} and it's meant to avoid adding a + // dependency to the data-streams module just for this + public static final NodeFeature DATA_STREAM_LIFECYCLE = new NodeFeature("data_stream.lifecycle"); + private final int version; + + private final Map componentTemplates; + private final Map composableIndexTemplates; + private final List ingestPipelines; + private final FeatureService featureService; + private volatile boolean enabled; + + @SuppressWarnings("unchecked") + public YamlTemplateRegistry( + Settings nodeSettings, + ClusterService clusterService, + ThreadPool threadPool, + Client client, + NamedXContentRegistry xContentRegistry, + FeatureService featureService + ) { + super(nodeSettings, clusterService, threadPool, client, xContentRegistry); + + try { + final Map resources = XContentHelper.convertToMap( + YamlXContent.yamlXContent, + loadResource(this.getClass(), "/resources.yaml"), + false + ); + version = (((Number) resources.get("version")).intValue()); + + final List componentTemplateNames = (List) resources.get("component-templates"); + final List indexTemplateNames = (List) resources.get("index-templates"); + final List ingestPipelineConfigs = (List) resources.get("ingest-pipelines"); + + componentTemplates = Optional.ofNullable(componentTemplateNames) + .orElse(Collections.emptyList()) + .stream() + .map(o -> (String) o) + .collect(Collectors.toMap(name -> name, name -> loadComponentTemplate(name, version))); + composableIndexTemplates = Optional.ofNullable(indexTemplateNames) + .orElse(Collections.emptyList()) + .stream() + .map(o -> (String) o) + .collect(Collectors.toMap(name -> name, name -> loadIndexTemplate(name, version))); + ingestPipelines = Optional.ofNullable(ingestPipelineConfigs) + .orElse(Collections.emptyList()) + .stream() + .map(o -> (Map>) o) + .map(map -> { + Map.Entry> pipelineConfig = map.entrySet().iterator().next(); + return loadIngestPipeline( + pipelineConfig.getKey(), + version, + (List) pipelineConfig.getValue().get("dependencies") + ); + }) + .collect(Collectors.toList()); + this.featureService = featureService; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public int getVersion() { + return version; + } + + /*** + * + * @return A friendly, human readable name of the index template regisry + */ + public abstract String getName(); + + public void setEnabled(boolean enabled) { + logger.info("{} index template registry is {}", getName(), enabled ? "enabled" : "disabled"); + this.enabled = enabled; + } + + public boolean isEnabled() { + return enabled; + } + + public void close() { + clusterService.removeListener(this); + } + + @Override + protected boolean isClusterReady(ClusterChangedEvent event) { + // Ensure current version of the components are installed only after versions that support data stream lifecycle + // due to the use of the feature in all the `@lifecycle` component templates + return featureService.clusterHasFeature(event.state(), DATA_STREAM_LIFECYCLE); + } + + @Override + protected boolean requiresMasterNode() { + return true; + } + + @Override + public Map getComponentTemplateConfigs() { + if (enabled) { + return componentTemplates; + } else { + return Map.of(); + } + } + + @Override + public Map getComposableTemplateConfigs() { + if (enabled) { + return composableIndexTemplates; + } else { + return Map.of(); + } + } + + @Override + public List getIngestPipelines() { + if (enabled) { + return ingestPipelines; + } else { + return Collections.emptyList(); + } + } + + protected abstract String getVersionProperty(); + + private ComponentTemplate loadComponentTemplate(String name, int version) { + try { + final byte[] content = loadVersionedResourceUTF8( + this.getClass(), + "/component-templates/" + name + ".yaml", + version, + getVersionProperty() + ); + try (var parser = YamlXContent.yamlXContent.createParser(XContentParserConfiguration.EMPTY, content)) { + return ComponentTemplate.parse(parser); + } + } catch (Exception e) { + throw new RuntimeException("failed to load " + getName() + " Ingest plugin's component template: " + name, e); + } + } + + private ComposableIndexTemplate loadIndexTemplate(String name, int version) { + try { + final byte[] content = loadVersionedResourceUTF8( + this.getClass(), + "/index-templates/" + name + ".yaml", + version, + getVersionProperty() + ); + try (var parser = YamlXContent.yamlXContent.createParser(XContentParserConfiguration.EMPTY, content)) { + return ComposableIndexTemplate.parse(parser); + } + } catch (Exception e) { + throw new RuntimeException("failed to load " + getName() + " Ingest plugin's index template: " + name, e); + } + } + + private IngestPipelineConfig loadIngestPipeline(String name, int version, @Nullable List dependencies) { + if (dependencies == null) { + dependencies = Collections.emptyList(); + } + return new YamlIngestPipelineConfig( + name, + "/ingest-pipelines/" + name + ".yaml", + version, + getVersionProperty(), + dependencies, + this.getClass() + ); + } + + @Override + protected boolean applyRolloverAfterTemplateV2Upgrade() { + return true; + } +} diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json index b4aa999697632..9c58322f12d03 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json @@ -43,6 +43,50 @@ "default_metric": "value_count" } } + }, + { + "histogram": { + "mapping": { + "type": "histogram", + "ignore_malformed": true + } + } + }, + { + "counter_long": { + "mapping": { + "type": "long", + "time_series_metric": "counter", + "ignore_malformed": true + } + } + }, + { + "gauge_long": { + "mapping": { + "type": "long", + "time_series_metric": "gauge", + "ignore_malformed": true + } + } + }, + { + "counter_double": { + "mapping": { + "type": "double", + "time_series_metric": "counter", + "ignore_malformed": true + } + } + }, + { + "gauge_double": { + "mapping": { + "type": "double", + "time_series_metric": "gauge", + "ignore_malformed": true + } + } } ], "properties": { diff --git a/x-pack/plugin/core/template-resources/src/main/resources/traces@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/traces@mappings.json new file mode 100644 index 0000000000000..e3990a250f0c2 --- /dev/null +++ b/x-pack/plugin/core/template-resources/src/main/resources/traces@mappings.json @@ -0,0 +1,28 @@ +{ + "template": { + "mappings": { + "date_detection": false, + "properties": { + "@timestamp": { + "type": "date" + }, + "data_stream.type": { + "type": "constant_keyword", + "value": "traces" + }, + "data_stream.dataset": { + "type": "constant_keyword" + }, + "data_stream.namespace": { + "type": "constant_keyword" + } + } + } + }, + "_meta": { + "description": "default mappings for the traces index template installed by x-pack", + "managed": true + }, + "version": ${xpack.stack.template.version}, + "deprecated": ${xpack.stack.template.deprecated} +} diff --git a/x-pack/plugin/core/template-resources/src/main/resources/traces@settings.json b/x-pack/plugin/core/template-resources/src/main/resources/traces@settings.json new file mode 100644 index 0000000000000..3f4fdba6f4f46 --- /dev/null +++ b/x-pack/plugin/core/template-resources/src/main/resources/traces@settings.json @@ -0,0 +1,18 @@ +{ + "template": { + "settings": { + "index": { + "codec": "best_compression", + "mapping": { + "ignore_malformed": true + } + } + } + }, + "_meta": { + "description": "default settings for the traces index template installed by x-pack", + "managed": true + }, + "version": ${xpack.stack.template.version}, + "deprecated": ${xpack.stack.template.deprecated} +} diff --git a/x-pack/plugin/otel-data/README.md b/x-pack/plugin/otel-data/README.md new file mode 100644 index 0000000000000..7cab6bfa453d8 --- /dev/null +++ b/x-pack/plugin/otel-data/README.md @@ -0,0 +1,33 @@ +## OpenTelemetry Ingest plugin + +The OpenTelemetry Ingest plugin installs index templates and component templates for OpenTelemetry data. + +All resources are defined as YAML under [src/main/resources](src/main/resources). + +The OpenTelemetry index templates rely on mappings from `x-pack-core`. +See [x-pack/plugin/core/src/main/resources](../core/src/main/resources). + +## Adding/Removing/Updating a resource + +All resources are defined as YAML under [src/main/resources](src/main/resources). + +For a resource to be known to the plugin it must be added to +[src/main/resources/resources.yaml](src/main/resources/resources.yaml) in the +appropriate section. + +Any update to resources included by this package also requires a bump to the +`version` property included in the resources file. + +## Testing + +## Integration testing + +The index templates and ingest pipeline functionality is tested using YAML REST tests. +These can be run with: + +``` +./gradlew :x-pack:plugin:otel-data:yamlRestTest +``` + +Refer to the [rest-api-spec documentation](../../../rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/README.asciidoc) +for information about writing YAML REST tests. diff --git a/x-pack/plugin/otel-data/build.gradle b/x-pack/plugin/otel-data/build.gradle new file mode 100644 index 0000000000000..f56efe21acccc --- /dev/null +++ b/x-pack/plugin/otel-data/build.gradle @@ -0,0 +1,38 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ +apply plugin: 'elasticsearch.internal-es-plugin' +apply plugin: 'elasticsearch.internal-yaml-rest-test' +apply plugin: 'elasticsearch.internal-cluster-test' + +esplugin { + name 'x-pack-otel-data' + description 'The OTEL plugin defines OTEL data streams and ingest pipelines.' + classname 'org.elasticsearch.xpack.oteldata.OTelPlugin' + extendedPlugins = ['x-pack-core'] +} + +dependencies { + compileOnly project(path: xpackModule('core')) + testImplementation project(path: ':x-pack:plugin:stack') + testImplementation(testArtifact(project(xpackModule('core')))) + testImplementation project(':modules:data-streams') + clusterModules project(':modules:data-streams') + clusterModules project(':modules:ingest-common') + clusterModules project(':modules:ingest-geoip') + clusterModules project(':modules:ingest-user-agent') + clusterModules project(':modules:lang-mustache') + clusterModules project(':modules:mapper-extras') + clusterModules project(xpackModule('analytics')) + clusterModules project(xpackModule('ilm')) + clusterModules project(xpackModule('mapper-aggregate-metric')) + clusterModules project(xpackModule('mapper-constant-keyword')) + clusterModules project(xpackModule('mapper-counted-keyword')) + clusterModules project(xpackModule('stack')) + clusterModules project(xpackModule('wildcard')) + clusterModules project(xpackModule('mapper-version')) +} diff --git a/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelIndexTemplateRegistry.java b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelIndexTemplateRegistry.java new file mode 100644 index 0000000000000..435530542c857 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelIndexTemplateRegistry.java @@ -0,0 +1,58 @@ +/* + * 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.oteldata; + +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.features.FeatureService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.template.YamlTemplateRegistry; + +import static org.elasticsearch.xpack.oteldata.OTelPlugin.OTEL_DATA_REGISTRY_ENABLED; + +public class OTelIndexTemplateRegistry extends YamlTemplateRegistry { + + public static final String OTEL_TEMPLATE_VERSION_VARIABLE = "xpack.oteldata.template.version"; + + public OTelIndexTemplateRegistry( + Settings nodeSettings, + ClusterService clusterService, + ThreadPool threadPool, + Client client, + NamedXContentRegistry xContentRegistry, + FeatureService featureService + ) { + super(nodeSettings, clusterService, threadPool, client, xContentRegistry, featureService); + } + + @Override + public void initialize() { + super.initialize(); + if (isEnabled()) { + clusterService.getClusterSettings().addSettingsUpdateConsumer(OTEL_DATA_REGISTRY_ENABLED, this::setEnabled); + } + } + + @Override + protected String getOrigin() { + return ClientHelper.OTEL_ORIGIN; + } + + @Override + public String getName() { + return "OpenTelemetry"; + } + + @Override + protected String getVersionProperty() { + return OTEL_TEMPLATE_VERSION_VARIABLE; + } +} diff --git a/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelPlugin.java b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelPlugin.java new file mode 100644 index 0000000000000..cece2b5373631 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/OTelPlugin.java @@ -0,0 +1,79 @@ +/* + * 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.oteldata; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xpack.core.XPackSettings; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +public class OTelPlugin extends Plugin implements ActionPlugin { + private static final Logger logger = LogManager.getLogger(OTelPlugin.class); + + final SetOnce registry = new SetOnce<>(); + + private final boolean enabled; + + // OTEL_DATA_REGISTRY_ENABLED controls enabling the index template registry. + // + // This setting will be ignored if the plugin is disabled. + static final Setting OTEL_DATA_REGISTRY_ENABLED = Setting.boolSetting( + "xpack.otel_data.registry.enabled", + // OTel-data is under development, and we start with opt-in first. + // Furthermore, this could help with staged rollout in serverless + false, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public OTelPlugin(Settings settings) { + this.enabled = XPackSettings.OTEL_DATA_ENABLED.get(settings); + } + + @Override + public Collection createComponents(PluginServices services) { + logger.info("OTel ingest plugin is {}", enabled ? "enabled" : "disabled"); + Settings settings = services.environment().settings(); + ClusterService clusterService = services.clusterService(); + registry.set( + new OTelIndexTemplateRegistry( + settings, + clusterService, + services.threadPool(), + services.client(), + services.xContentRegistry(), + services.featureService() + ) + ); + if (enabled) { + OTelIndexTemplateRegistry registryInstance = registry.get(); + registryInstance.setEnabled(OTEL_DATA_REGISTRY_ENABLED.get(settings)); + registryInstance.initialize(); + } + return Collections.emptyList(); + } + + @Override + public void close() { + registry.get().close(); + } + + @Override + public List> getSettings() { + return List.of(OTEL_DATA_REGISTRY_ENABLED); + } +} diff --git a/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/package-info.java b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/package-info.java new file mode 100644 index 0000000000000..98c6c9a3999c4 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/java/org/elasticsearch/xpack/oteldata/package-info.java @@ -0,0 +1,17 @@ +/* + * 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. + */ + +/** + * This package contains index templates for OpenTelemetry data. It covers traces (spans), metrics, and logs. + * The plugin is expected to be used in combination with the Elasticsearch exporter defined as the exporter + * within an OpenTelemetry collector with the mapping mode `otel`. + * For more information about the Elasticsearch exporter + * @see + * https://github.com/open-telemetry/opentelemetry-collector-contrib. + * + */ +package org.elasticsearch.xpack.oteldata; diff --git a/x-pack/plugin/otel-data/src/main/resources/component-templates/logs-otel@mappings.yaml b/x-pack/plugin/otel-data/src/main/resources/component-templates/logs-otel@mappings.yaml new file mode 100644 index 0000000000000..a0971f45ccf4f --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/component-templates/logs-otel@mappings.yaml @@ -0,0 +1,42 @@ +--- +version: ${xpack.oteldata.template.version} +_meta: + description: Default mappings for OpenTelemetry logs index template installed by x-pack + managed: true +template: + settings: + index: + mode: logsdb + sort: + field: [ "resource.attributes.host.name" ] + mappings: + properties: + data_stream.type: + type: constant_keyword + value: logs + observed_timestamp: + type: date_nanos + severity_number: + type: byte + severity_text: + type: keyword + log.level: + type: alias + path: severity_text + body_text: + type: match_only_text + message: + type: alias + path: body_text + body_structured: + type: flattened + trace_id: + type: keyword + trace.id: + type: alias + path: trace_id + span_id: + type: keyword + span.id: + type: alias + path: span_id diff --git a/x-pack/plugin/otel-data/src/main/resources/component-templates/metrics-otel@mappings.yaml b/x-pack/plugin/otel-data/src/main/resources/component-templates/metrics-otel@mappings.yaml new file mode 100644 index 0000000000000..b7a17dba973f8 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/component-templates/metrics-otel@mappings.yaml @@ -0,0 +1,17 @@ +version: ${xpack.oteldata.template.version} +_meta: + description: Default mappings for the OpenTelemetry metrics index template installed by x-pack + managed: true +template: + mappings: + properties: + start_timestamp: + type: date_nanos + metrics: + type: passthrough + dynamic: true + priority: 1 + unit: + type: keyword + time_series_dimension: true + ignore_above: 1024 diff --git a/x-pack/plugin/otel-data/src/main/resources/component-templates/otel@mappings.yaml b/x-pack/plugin/otel-data/src/main/resources/component-templates/otel@mappings.yaml new file mode 100644 index 0000000000000..fad85661203d6 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/component-templates/otel@mappings.yaml @@ -0,0 +1,64 @@ +--- +version: ${xpack.oteldata.template.version} +_meta: + description: Default mappings for all OpenTelemetry data streams + managed: true +template: + mappings: + date_detection: false + dynamic: false + properties: + "@timestamp": +#Ultimeately we aim to use date_nanos. Waiting for https://github.com/elastic/elasticsearch/issues/109352 + type: date + data_stream.type: + type: constant_keyword + data_stream.dataset: + type: constant_keyword + data_stream.namespace: + type: constant_keyword + attributes: + type: passthrough + dynamic: true + priority: 10 + time_series_dimension: true + dropped_attributes_count: + type: long + scope: + properties: + name: + type: keyword + ignore_above: 1024 + version: + type: version + schema_url: + type: keyword + ignore_above: 1024 + dropped_attributes_count: + type: long + attributes: + type: passthrough + dynamic: true + priority: 20 + time_series_dimension: true + resource: + properties: + schema_url: + type: keyword + ignore_above: 1024 + dropped_attributes_count: + type: long + attributes: + type: passthrough + dynamic: true + priority: 30 + time_series_dimension: true + dynamic_templates: + - complex_attributes: + path_match: + - resource.attributes.* + - scope.attributes.* + - attributes.* + match_mapping_type: object + mapping: + type: flattened diff --git a/x-pack/plugin/otel-data/src/main/resources/component-templates/semconv-resource-to-ecs@mappings.yaml b/x-pack/plugin/otel-data/src/main/resources/component-templates/semconv-resource-to-ecs@mappings.yaml new file mode 100644 index 0000000000000..711f72ae95220 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/component-templates/semconv-resource-to-ecs@mappings.yaml @@ -0,0 +1,128 @@ +--- +version: ${xpack.oteldata.template.version} +_meta: + description: Aliases from OpenTelemetry SemConv fields to ECS (and some non-ECS) fields + managed: true +template: + mappings: + properties: + resource: + properties: + attributes: + type: passthrough + dynamic: true + priority: 30 + time_series_dimension: true + properties: + host.name: + type: keyword + ignore_above: 1024 + telemetry.sdk.language: + type: keyword + ignore_above: 1024 + service.instance.id: + type: keyword + ignore_above: 1024 + deployment.environment: + type: keyword + ignore_above: 1024 + cloud.platform: + type: keyword + ignore_above: 1024 + container.image.tags: + type: keyword + ignore_above: 1024 + host.arch: + type: keyword + ignore_above: 1024 + process.executable.path: + type: keyword + ignore_above: 1024 + process.runtime.name: + type: keyword + ignore_above: 1024 + process.runtime.version: + type: keyword + ignore_above: 1024 + os.name: + type: keyword + ignore_above: 1024 + os.type: + type: keyword + ignore_above: 1024 + os.description: + type: keyword + ignore_above: 1024 + os.version: + type: keyword + ignore_above: 1024 + k8s.deployment.name: + type: keyword + ignore_above: 1024 + k8s.namespace.name: + type: keyword + ignore_above: 1024 + k8s.node.name: + type: keyword + ignore_above: 1024 + k8s.pod.name: + type: keyword + ignore_above: 1024 + k8s.pod.uid: + type: keyword + ignore_above: 1024 + service.node.name: + type: alias + path: resource.attributes.service.instance.id + service.environment: + type: alias + path: resource.attributes.deployment.environment + cloud.service.name: + type: alias + path: resource.attributes.cloud.platform + container.image.tag: + type: alias + path: resource.attributes.container.image.tags + host.architecture: + type: alias + path: resource.attributes.host.arch + process.executable: + type: alias + path: resource.attributes.process.executable.path + service.runtime.name: + type: alias + path: resource.attributes.process.runtime.name + service.runtime.version: + type: alias + path: resource.attributes.process.runtime.version + host.os.name: + type: alias + path: resource.attributes.os.name + host.os.platform: + type: alias + path: resource.attributes.os.type + host.os.full: + type: alias + path: resource.attributes.os.description + host.os.version: + type: alias + path: resource.attributes.os.version + kubernetes.deployment.name: + type: alias + path: resource.attributes.k8s.deployment.name + kubernetes.namespace: + type: alias + path: resource.attributes.k8s.namespace.name + kubernetes.node.name: + type: alias + path: resource.attributes.k8s.node.name + kubernetes.pod.name: + type: alias + path: resource.attributes.k8s.pod.name + kubernetes.pod.uid: + type: alias + path: resource.attributes.k8s.pod.uid +# Below are non-ECS fields that may be used by Kibana. + service.language.name: + type: alias + path: resource.attributes.telemetry.sdk.language diff --git a/x-pack/plugin/otel-data/src/main/resources/component-templates/traces-otel@mappings.yaml b/x-pack/plugin/otel-data/src/main/resources/component-templates/traces-otel@mappings.yaml new file mode 100644 index 0000000000000..a4c62efeed7a4 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/component-templates/traces-otel@mappings.yaml @@ -0,0 +1,68 @@ +--- +version: ${xpack.oteldata.template.version} +_meta: + description: Default mappings for OpenTelemetry traces + managed: true +template: + settings: + index: + mode: logsdb + sort: + field: [ "resource.attributes.host.name" ] + mappings: + _source: + mode: synthetic + properties: + trace_id: + type: keyword + trace.id: + type: alias + path: trace_id + span_id: + type: keyword + span.id: + type: alias + path: span_id + trace_state: + type: keyword + parent_span_id: + type: keyword + parent.id: + type: alias + path: parent_span_id + name: + type: keyword + span.name: + type: alias + path: name + kind: + type: keyword + duration: + type: long + meta: + unit: nanos + dropped_events_count: + type: long + links: + store_array_source: true + properties: + trace_id: + type: keyword + span_id: + type: keyword + trace_state: + type: keyword + attributes: + type: object + subobjects: false + dynamic: true + dropped_attributes_count: + type: long + dropped_links_count: + type: long + status: + properties: + message: + type: keyword + code: + type: keyword diff --git a/x-pack/plugin/otel-data/src/main/resources/index-templates/logs-otel@template.yaml b/x-pack/plugin/otel-data/src/main/resources/index-templates/logs-otel@template.yaml new file mode 100644 index 0000000000000..6772ec5bc65d4 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/index-templates/logs-otel@template.yaml @@ -0,0 +1,27 @@ +--- +version: ${xpack.oteldata.template.version} +index_patterns: ["logs-*.otel-*"] +priority: 120 +data_stream: {} +allow_auto_create: true +_meta: + description: default OpenTelemetry logs template installed by x-pack + managed: true +composed_of: + - logs@mappings + - logs@settings + - otel@mappings + - logs-otel@mappings + - semconv-resource-to-ecs@mappings + - logs@custom + - logs-otel@custom + - ecs@mappings +ignore_missing_component_templates: + - logs@custom + - logs-otel@custom +template: + mappings: + properties: + data_stream.type: + type: constant_keyword + value: logs diff --git a/x-pack/plugin/otel-data/src/main/resources/index-templates/metrics-otel@template.yaml b/x-pack/plugin/otel-data/src/main/resources/index-templates/metrics-otel@template.yaml new file mode 100644 index 0000000000000..89ff28249aabb --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/index-templates/metrics-otel@template.yaml @@ -0,0 +1,36 @@ +--- +version: ${xpack.oteldata.template.version} +index_patterns: ["metrics-*.otel-*"] +priority: 120 +data_stream: {} +allow_auto_create: true +_meta: + description: default OpenTelemetry metrics template installed by x-pack + managed: true +composed_of: + - metrics@mappings + - metrics@tsdb-settings + - otel@mappings + - metrics-otel@mappings + - semconv-resource-to-ecs@mappings + - metrics@custom + - metrics-otel@custom +ignore_missing_component_templates: + - metrics@custom + - metrics-otel@custom +template: + settings: + index: + mode: time_series + mappings: + properties: + data_stream.type: + type: constant_keyword + value: metrics + dynamic_templates: + - all_strings_to_keywords: + mapping: + ignore_above: 1024 + type: keyword + match_mapping_type: string + diff --git a/x-pack/plugin/otel-data/src/main/resources/index-templates/traces-otel@template.yaml b/x-pack/plugin/otel-data/src/main/resources/index-templates/traces-otel@template.yaml new file mode 100644 index 0000000000000..370b9351c16f5 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/index-templates/traces-otel@template.yaml @@ -0,0 +1,27 @@ +--- +version: ${xpack.oteldata.template.version} +index_patterns: ["traces-*.otel-*"] +priority: 120 +data_stream: {} +allow_auto_create: true +_meta: + description: default OpenTelemetry traces template installed by x-pack + managed: true +composed_of: + - traces@mappings + - traces@settings + - otel@mappings + - traces-otel@mappings + - semconv-resource-to-ecs@mappings + - traces@custom + - traces-otel@custom + - ecs@mappings +ignore_missing_component_templates: + - traces@custom + - traces-otel@custom +template: + mappings: + properties: + data_stream.type: + type: constant_keyword + value: traces diff --git a/x-pack/plugin/otel-data/src/main/resources/resources.yaml b/x-pack/plugin/otel-data/src/main/resources/resources.yaml new file mode 100644 index 0000000000000..8e0a7606cbd05 --- /dev/null +++ b/x-pack/plugin/otel-data/src/main/resources/resources.yaml @@ -0,0 +1,15 @@ +# "version" holds the version of the templates and ingest pipelines installed +# by xpack-plugin otel-data. This must be increased whenever an existing template is +# changed, in order for it to be updated on Elasticsearch upgrade. +version: 1 + +component-templates: + - otel@mappings + - logs-otel@mappings + - semconv-resource-to-ecs@mappings + - metrics-otel@mappings + - traces-otel@mappings +index-templates: + - logs-otel@template + - metrics-otel@template + - traces-otel@template diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/java/org/elasticsearch/xpack/oteldata/OTelYamlTestSuiteIT.java b/x-pack/plugin/otel-data/src/yamlRestTest/java/org/elasticsearch/xpack/oteldata/OTelYamlTestSuiteIT.java new file mode 100644 index 0000000000000..4a5f7d03b12a2 --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/java/org/elasticsearch/xpack/oteldata/OTelYamlTestSuiteIT.java @@ -0,0 +1,54 @@ +/* + * 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.oteldata; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; +import org.junit.ClassRule; + +public class OTelYamlTestSuiteIT extends ESClientYamlSuiteTestCase { + + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .module("constant-keyword") + .module("counted-keyword") + .module("data-streams") + .module("ingest-common") + .module("ingest-geoip") + .module("ingest-user-agent") + .module("lang-mustache") + .module("mapper-extras") + .module("wildcard") + .module("x-pack-analytics") + .module("x-pack-otel-data") + .module("x-pack-aggregate-metric") + .module("x-pack-ilm") + .module("x-pack-stack") + .module("mapper-version") + .setting("ingest.geoip.downloader.enabled", "false") + .setting("xpack.otel_data.registry.enabled", "true") + .build(); + + public OTelYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws Exception { + return ESClientYamlSuiteTestCase.createParameters(); + } + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } +} diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/10_otel.yml b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/10_otel.yml new file mode 100644 index 0000000000000..72b7a127dcd02 --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/10_otel.yml @@ -0,0 +1,38 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid + +--- +"Test traces-otel* template installation": + - skip: + reason: contains is a newly added assertion + features: contains + - do: + indices.get_index_template: + name: traces-otel* + - length: {index_templates: 1} + - contains: {index_templates: {name: traces-otel@template}} + +--- +"Test metrics-otel* template installation": + - skip: + reason: contains is a newly added assertion + features: contains + - do: + indices.get_index_template: + name: metrics-otel* + - length: {index_templates: 1} + - contains: {index_templates: {name: metrics-otel@template}} + +--- +"Test logs-otel* template installation": + - skip: + reason: contains is a newly added assertion + features: contains + - do: + indices.get_index_template: + name: logs-otel* + - length: {index_templates: 1} + - contains: {index_templates: {name: logs-otel@template}} diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_logs.tests.yml b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_logs.tests.yml new file mode 100644 index 0000000000000..d87c2a80deab8 --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_logs.tests.yml @@ -0,0 +1,22 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid +--- +"Default data_stream.type must be logs": + - do: + bulk: + index: logs-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default"}, "attributes": { "foo": "bar"}, "body_text":"Error: Unable to connect to the database.","severity_text":"ERROR","severity_number":3,"trace_id":"abc123xyz456def789ghi012jkl345"}' + - is_false: errors + - do: + search: + index: logs-generic.otel-default + body: + fields: ["data_stream.type"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.data_stream\.type: ["logs"] } diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_metrics_tests.yml b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_metrics_tests.yml new file mode 100644 index 0000000000000..a6591d6c32210 --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_metrics_tests.yml @@ -0,0 +1,149 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid + - do: + cluster.put_component_template: + name: metrics-otel@custom + body: + template: + settings: + index: + routing_path: [unit, attributes.*, resource.attributes.*] + mode: time_series + time_series: + start_time: 2024-07-01T13:03:08.138Z +--- +"Test push service overview metric": + - do: + indices.get_index_template: + name: metrics-otel@template + - length: {index_templates: 1} + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default","type":"metrics"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"},"dropped_attributes_count":0},"attributes":{"processor.event":"metric"}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["service.name", "telemetry.sdk.language", "telemetry.sdk.name" ] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.service\.name: [ "OtelSample" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.language: [ "dotnet" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.name: [ "opentelemetry" ] } +--- +"Query resource attributes as top level": + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default","type":"metrics"},"processor":{"event":"metric"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"},"dropped_attributes_count":0}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["service.name", "telemetry.sdk.language", "telemetry.sdk.name"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.service\.name: [ "OtelSample" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.language: [ "dotnet" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.name: [ "opentelemetry" ] } +--- +"Query attributes as top level": + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default","type":"metrics"},"attributes":{"processor.event":"metric", "foo": "bar"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"},"dropped_attributes_count":0}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["foo"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.foo\: ["bar"] } +--- +"Boolean as dimension": + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default","type":"metrics"},"attributes":{"processor.event":"metric","transaction.root":false},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["transaction.root"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.transaction\.root: [false] } +--- +"Default data_stream.type must be metrics": + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default"},"attributes":{"processor.event":"metric","transaction.root":false},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["data_stream.type"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.data_stream\.type: ["metrics"] } + +--- +"Custom dynamic template": + - do: + cluster.put_component_template: + name: metrics-otel@custom + body: + template: + settings: + index: + routing_path: [unit, attributes.*, resource.attributes.*] + mode: time_series + time_series: + start_time: 2024-07-01T13:03:08.138Z + mappings: + dynamic_templates: + - ip_fields: + mapping: + type: ip + match_mapping_type: string + path_match: "*.ip" + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default"},"attributes":{"host.ip":"127.0.0.1","foo":"bar"}}' + - is_false: errors + - do: + indices.get_data_stream: + name: metrics-generic.otel-default + - set: { data_streams.0.indices.0.index_name: idx0name } + + - do: + indices.get_mapping: + index: $idx0name + expand_wildcards: hidden + - match: { .$idx0name.mappings.properties.attributes.properties.host\.ip.type: 'ip' } + - match: { .$idx0name.mappings.properties.attributes.properties.foo.type: "keyword" } diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_traces_tests.yml b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_traces_tests.yml new file mode 100644 index 0000000000000..abdb8d49d774c --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/20_traces_tests.yml @@ -0,0 +1,94 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid +--- +"Test pushing simple trace": + - do: + bulk: + index: traces-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-02-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","type":"traces","namespace":"default"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}},"name":"foo","trace_id":"7bba9f33312b3dbb8b2c2c62bb7abe2d","span_id":"086e83747d0e381e","kind":"SERVER","status":{"code":"2xx"}}' + - is_false: errors + - do: + search: + index: traces-generic.otel-default + - length: { hits.hits: 1 } + +--- +"Query resource attributes as top level": + - do: + bulk: + index: traces-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-02-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","type":"traces","namespace":"default"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}},"name":"foo","trace_id":"7bba9f33312b3dbb8b2c2c62bb7abe2d","span_id":"086e83747d0e381e","kind":"SERVER","status":{"code":"2xx"}}' + - is_false: errors + - do: + search: + index: traces-generic.otel-default + body: + fields: ["service.name", "telemetry.sdk.language", "telemetry.sdk.name" ] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.service\.name: [ "OtelSample" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.language: [ "dotnet" ] } + - match: { hits.hits.0.fields.telemetry\.sdk\.name: [ "opentelemetry" ] } +--- +"Query attributes as top level": + - do: + bulk: + index: traces-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-02-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","type":"traces","namespace":"default"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}},"attributes":{"db.type":"mssql","db.name":"foo","db.operation":"SELECT","db.statement":"SELECT * FROM wuser_table"},"name":"foo","trace_id":"7bba9f33312b3dbb8b2c2c62bb7abe2d","span_id":"086e83747d0e381e","kind":"SERVER","status":{"code":"2xx"}}' + - is_false: errors + - do: + search: + index: traces-generic.otel-default + body: + fields: ["db.type", "db.name", "db.operation", "db.statement"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.db\.type: [ "mssql" ] } + - match: { hits.hits.0.fields.db\.operation: [ "SELECT" ] } + - match: { hits.hits.0.fields.db\.statement: [ "SELECT * FROM wuser_table" ] } +--- +"Span links test": + - do: + bulk: + index: traces-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-02-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","type":"traces","namespace":"default"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}},"attributes":{"db.type":"mssql","db.name":"foo","db.operation":"SELECT","db.statement":"SELECT * FROM wuser_table"},"links":[{"trace_id":"4aaa9f33312b3dbb8b2c2c62bb7abe1a1","span_id":"086e83747d0e381e","attributes":{"foo":"bar"}},{"trace_id":"4aaa9f33312b3dbb8b2c2c62bb7abe1a1","span_id":"b3b7d1f1f1b4e1e1"}],"name":"foo","trace_id":"7bba9f33312b3dbb8b2c2c62bb7abe2d","span_id":"086e83747d0e381e","kind":"SERVER","status":{"code":"2xx"}}' + - is_false: errors + - do: + search: + index: traces-generic.otel-default + - length: { hits.hits.0._source.links: 2 } + - match: { hits.hits.0._source.links.0.trace_id: "4aaa9f33312b3dbb8b2c2c62bb7abe1a1" } + - match: { hits.hits.0._source.links.0.span_id: "086e83747d0e381e" } + - match: { hits.hits.0._source.links.0.attributes.foo: "bar" } + - match: { hits.hits.0._source.links.1.trace_id: "4aaa9f33312b3dbb8b2c2c62bb7abe1a1" } + - match: { hits.hits.0._source.links.1.span_id: "b3b7d1f1f1b4e1e1" } +--- +"Default data_stream.type must be traces": + - do: + bulk: + index: traces-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-02-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","type":"traces","namespace":"default"},"resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"}},"name":"foo","trace_id":"7bba9f33312b3dbb8b2c2c62bb7abe2d","span_id":"086e83747d0e381e","kind":"SERVER","status":{"code":"2xx"}}' + - is_false: errors + - do: + search: + index: traces-generic.otel-default + body: + fields: ["data_stream.type"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.data_stream\.type: ["traces"] } diff --git a/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/30_non_ecs_alias_tests.yml b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/30_non_ecs_alias_tests.yml new file mode 100644 index 0000000000000..d80c52c756b54 --- /dev/null +++ b/x-pack/plugin/otel-data/src/yamlRestTest/resources/rest-api-spec/test/30_non_ecs_alias_tests.yml @@ -0,0 +1,37 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid + - do: + cluster.put_component_template: + name: metrics-otel@custom + body: + template: + settings: + index: + routing_path: [unit, attributes.*, resource.attributes.*] + mode: time_series + time_series: + start_time: 2024-07-01T13:03:08.138Z +--- +"Test alias from service.language.name non-ecs field to telemetry.sdk.language": + - do: + indices.get_index_template: + name: metrics-otel@template + - length: {index_templates: 1} + - do: + bulk: + index: metrics-generic.otel-default + refresh: true + body: + - create: {} + - '{"@timestamp":"2024-07-18T14:48:33.467654000Z","data_stream":{"dataset":"generic.otel","namespace":"default","type":"metrics"}, "attributes": {"processor.event":"metric"}, "resource":{"attributes":{"service.name":"OtelSample","telemetry.sdk.language":"dotnet","telemetry.sdk.name":"opentelemetry"},"dropped_attributes_count":0}}' + - is_false: errors + - do: + search: + index: metrics-generic.otel-default + body: + fields: ["service.language.name"] + - length: { hits.hits: 1 } + - match: { hits.hits.0.fields.service\.language\.name: [ "dotnet" ] } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java index 194440722545a..4173f3db45409 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/AuthorizationUtils.java @@ -39,6 +39,7 @@ import static org.elasticsearch.xpack.core.ClientHelper.LOGSTASH_MANAGEMENT_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.MONITORING_ORIGIN; +import static org.elasticsearch.xpack.core.ClientHelper.OTEL_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.PROFILING_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.ROLLUP_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.SEARCHABLE_SNAPSHOTS_ORIGIN; @@ -151,6 +152,7 @@ public static void switchUserBasedOnActionOriginAndExecute( case INGEST_ORIGIN: case PROFILING_ORIGIN: case APM_ORIGIN: + case OTEL_ORIGIN: case STACK_ORIGIN: case SEARCHABLE_SNAPSHOTS_ORIGIN: case LOGSTASH_MANAGEMENT_ORIGIN: diff --git a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java index 6a9936f4f27d3..9e847455d2c86 100644 --- a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java +++ b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java @@ -48,7 +48,7 @@ public class StackTemplateRegistry extends IndexTemplateRegistry { // The stack template registry version. This number must be incremented when we make changes // to built-in templates. - public static final int REGISTRY_VERSION = 12; + public static final int REGISTRY_VERSION = 13; public static final String TEMPLATE_VERSION_VARIABLE = "xpack.stack.template.version"; public static final Setting STACK_TEMPLATES_ENABLED = Setting.boolSetting( @@ -106,6 +106,12 @@ public class StackTemplateRegistry extends IndexTemplateRegistry { public static final String METRICS_ILM_POLICY_NAME = "metrics@lifecycle"; public static final String METRICS_INDEX_TEMPLATE_NAME = "metrics"; + ////////////////////////////////////////////////////////// + // Base traces components + ////////////////////////////////////////////////////////// + public static final String TRACES_MAPPINGS_COMPONENT_TEMPLATE_NAME = "traces@mappings"; + public static final String TRACES_SETTINGS_COMPONENT_TEMPLATE_NAME = "traces@settings"; + ////////////////////////////////////////////////////////// // Synthetics components (for matching synthetics-*-* indices) ////////////////////////////////////////////////////////// @@ -192,6 +198,20 @@ private Map loadComponentTemplateConfigs(boolean logs TEMPLATE_VERSION_VARIABLE, ADDITIONAL_TEMPLATE_VARIABLES ), + new IndexTemplateConfig( + TRACES_SETTINGS_COMPONENT_TEMPLATE_NAME, + "/traces@settings.json", + REGISTRY_VERSION, + TEMPLATE_VERSION_VARIABLE, + ADDITIONAL_TEMPLATE_VARIABLES + ), + new IndexTemplateConfig( + TRACES_MAPPINGS_COMPONENT_TEMPLATE_NAME, + "/traces@mappings.json", + REGISTRY_VERSION, + TEMPLATE_VERSION_VARIABLE, + ADDITIONAL_TEMPLATE_VARIABLES + ), new IndexTemplateConfig( SYNTHETICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/synthetics@mappings.json", diff --git a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java b/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java index abb2d5765b128..25ff3b5311fa2 100644 --- a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java +++ b/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java @@ -430,10 +430,12 @@ public void testSameOrHigherVersionTemplateNotUpgraded() { versions.put(StackTemplateRegistry.SYNTHETICS_SETTINGS_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION); versions.put(StackTemplateRegistry.SYNTHETICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION); versions.put(StackTemplateRegistry.KIBANA_REPORTING_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION); + versions.put(StackTemplateRegistry.TRACES_MAPPINGS_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION); + versions.put(StackTemplateRegistry.TRACES_SETTINGS_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION); ClusterChangedEvent sameVersionEvent = createClusterChangedEvent(versions, nodes); client.setVerifier((action, request, listener) -> { - if (action instanceof PutComponentTemplateAction) { - fail("template should not have been re-installed"); + if (request instanceof PutComponentTemplateAction.Request put) { + fail("template should not have been re-installed: " + put.name()); return null; } else if (action == ILMActions.PUT) { // Ignore this, it's verified in another test @@ -489,6 +491,14 @@ public void testSameOrHigherVersionTemplateNotUpgraded() { StackTemplateRegistry.KIBANA_REPORTING_COMPONENT_TEMPLATE_NAME, StackTemplateRegistry.REGISTRY_VERSION + randomIntBetween(1, 1000) ); + versions.put( + StackTemplateRegistry.TRACES_MAPPINGS_COMPONENT_TEMPLATE_NAME, + StackTemplateRegistry.REGISTRY_VERSION + randomIntBetween(1, 1000) + ); + versions.put( + StackTemplateRegistry.TRACES_SETTINGS_COMPONENT_TEMPLATE_NAME, + StackTemplateRegistry.REGISTRY_VERSION + randomIntBetween(1, 1000) + ); ClusterChangedEvent higherVersionEvent = createClusterChangedEvent(versions, nodes); registry.clusterChanged(higherVersionEvent); } From 69f454370ac13663742128d39fb001e0ea1991c6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 08:26:16 +0100 Subject: [PATCH 06/15] Fix known issue docs for #111866 (#111956) The `known-issue-8.15.0` anchor appears twice which breaks the docs build. Also the existing message suggests incorrectly that `bootstrap.memory_lock: true` is recommended. --- docs/reference/release-notes/8.15.0.asciidoc | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/reference/release-notes/8.15.0.asciidoc b/docs/reference/release-notes/8.15.0.asciidoc index 80e935e130678..e2314381a4b06 100644 --- a/docs/reference/release-notes/8.15.0.asciidoc +++ b/docs/reference/release-notes/8.15.0.asciidoc @@ -16,7 +16,11 @@ after it is killed up to four times in 24 hours. (issue: {es-issue}110530[#11053 * Pipeline aggregations under `time_series` and `categorize_text` aggregations are never returned (issue: {es-issue}111679[#111679]) -* Elasticsearch will not start on Windows machines when the recommended [bootstrap.memory_lock: true](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup-configuration-memory.html#bootstrap-memory_lock) setting is configured due to [native access refactoring](https://github.com/elastic/elasticsearch/pull/111866). The workaround for 8.15.0 is to downgrade to the previous version. This issue will be fixed in 8.15.1. +* Elasticsearch will not start on Windows machines if +[`bootstrap.memory_lock` is set to `true`](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup-configuration-memory.html#bootstrap-memory_lock). +Either downgrade to an earlier version, upgrade to 8.15.1, or else follow the +recommendation in the manual to entirely disable swap instead of using the +memory lock feature (issue: {es-issue}111847[#111847]) [[breaking-8.15.0]] [float] From 24b852b3d64239882ab898b9e241a9b69a084109 Mon Sep 17 00:00:00 2001 From: Mary Gouseti Date: Mon, 19 Aug 2024 11:28:01 +0300 Subject: [PATCH 07/15] Clean the last traces from global retention in templates (#111669) --- .../org/elasticsearch/TransportVersions.java | 2 + .../get/GetComponentTemplateAction.java | 53 ++++++++++------- .../get/GetComposableIndexTemplateAction.java | 57 ++++++++++++------- .../TransportGetComponentTemplateAction.java | 11 +--- ...sportGetComposableIndexTemplateAction.java | 11 +--- .../post/SimulateIndexTemplateResponse.java | 31 ++++------ .../TransportSimulateIndexTemplateAction.java | 15 ++--- .../post/TransportSimulateTemplateAction.java | 13 +---- .../rest/action/cat/RestTemplatesAction.java | 2 +- .../GetComponentTemplateResponseTests.java | 19 ++----- ...tComposableIndexTemplateResponseTests.java | 36 ++++++++++-- 11 files changed, 132 insertions(+), 118 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 1995c430472ba..fd3a3d8672966 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -190,6 +190,8 @@ static TransportVersion def(int id) { public static final TransportVersion ML_INFERENCE_EIS_INTEGRATION_ADDED = def(8_720_00_0); public static final TransportVersion INGEST_PIPELINE_EXCEPTION_ADDED = def(8_721_00_0); public static final TransportVersion ZDT_NANOS_SUPPORT = def(8_722_00_0); + public static final TransportVersion REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES = def(8_723_00_0); + /* * STOP! READ THIS FIRST! No, really, * ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _ diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateAction.java index da588cbadc0d8..f0552cc3226f5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateAction.java @@ -121,8 +121,6 @@ public static class Response extends ActionResponse implements ToXContentObject private final Map componentTemplates; @Nullable private final RolloverConfiguration rolloverConfiguration; - @Nullable - private final DataStreamGlobalRetention globalRetention; public Response(StreamInput in) throws IOException { super(in); @@ -132,29 +130,39 @@ public Response(StreamInput in) throws IOException { } else { rolloverConfiguration = null; } - if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) { - globalRetention = in.readOptionalWriteable(DataStreamGlobalRetention::read); - } else { - globalRetention = null; + if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && in.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + in.readOptionalWriteable(DataStreamGlobalRetention::read); } } - public Response(Map componentTemplates, RolloverConfiguration rolloverConfiguration) { - this(componentTemplates, rolloverConfiguration, null); - } - + /** + * Please use {@link GetComponentTemplateAction.Response#Response(Map)} + */ + @Deprecated public Response(Map componentTemplates, @Nullable DataStreamGlobalRetention globalRetention) { - this(componentTemplates, null, globalRetention); + this(componentTemplates, (RolloverConfiguration) null); } + /** + * Please use {@link GetComponentTemplateAction.Response#Response(Map, RolloverConfiguration)} + */ + @Deprecated public Response( Map componentTemplates, @Nullable RolloverConfiguration rolloverConfiguration, - @Nullable DataStreamGlobalRetention globalRetention + @Nullable DataStreamGlobalRetention ignored ) { + this(componentTemplates, rolloverConfiguration); + } + + public Response(Map componentTemplates) { + this(componentTemplates, (RolloverConfiguration) null); + } + + public Response(Map componentTemplates, @Nullable RolloverConfiguration rolloverConfiguration) { this.componentTemplates = componentTemplates; this.rolloverConfiguration = rolloverConfiguration; - this.globalRetention = globalRetention; } public Map getComponentTemplates() { @@ -165,8 +173,14 @@ public RolloverConfiguration getRolloverConfiguration() { return rolloverConfiguration; } + /** + * @return null + * @deprecated The global retention is not used anymore in the component template response + */ + @Deprecated + @Nullable public DataStreamGlobalRetention getGlobalRetention() { - return globalRetention; + return null; } @Override @@ -175,8 +189,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) { out.writeOptionalWriteable(rolloverConfiguration); } - if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) { - out.writeOptionalWriteable(globalRetention); + if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && out.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + out.writeOptionalWriteable(null); } } @@ -186,13 +201,12 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; Response that = (Response) o; return Objects.equals(componentTemplates, that.componentTemplates) - && Objects.equals(rolloverConfiguration, that.rolloverConfiguration) - && Objects.equals(globalRetention, that.globalRetention); + && Objects.equals(rolloverConfiguration, that.rolloverConfiguration); } @Override public int hashCode() { - return Objects.hash(componentTemplates, rolloverConfiguration, globalRetention); + return Objects.hash(componentTemplates, rolloverConfiguration); } @Override @@ -212,5 +226,4 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } } - } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateAction.java index e40977a382ba1..ba07c87e753e6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateAction.java @@ -122,8 +122,6 @@ public static class Response extends ActionResponse implements ToXContentObject private final Map indexTemplates; @Nullable private final RolloverConfiguration rolloverConfiguration; - @Nullable - private final DataStreamGlobalRetention globalRetention; public Response(StreamInput in) throws IOException { super(in); @@ -133,37 +131,57 @@ public Response(StreamInput in) throws IOException { } else { rolloverConfiguration = null; } - if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) { - globalRetention = in.readOptionalWriteable(DataStreamGlobalRetention::read); - } else { - globalRetention = null; + if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && in.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + in.readOptionalWriteable(DataStreamGlobalRetention::read); } } + /** + * Please use {@link GetComposableIndexTemplateAction.Response#Response(Map)} + */ public Response(Map indexTemplates, @Nullable DataStreamGlobalRetention globalRetention) { - this(indexTemplates, null, globalRetention); - } - - public Response(Map indexTemplates) { - this(indexTemplates, null, null); + this(indexTemplates, (RolloverConfiguration) null); } + /** + * Please use {@link GetComposableIndexTemplateAction.Response#Response(Map, RolloverConfiguration)} + */ + @Deprecated public Response( Map indexTemplates, @Nullable RolloverConfiguration rolloverConfiguration, @Nullable DataStreamGlobalRetention globalRetention ) { + this(indexTemplates, rolloverConfiguration); + } + + public Response(Map indexTemplates) { + this(indexTemplates, (RolloverConfiguration) null); + } + + public Response(Map indexTemplates, @Nullable RolloverConfiguration rolloverConfiguration) { this.indexTemplates = indexTemplates; this.rolloverConfiguration = rolloverConfiguration; - this.globalRetention = globalRetention; } public Map indexTemplates() { return indexTemplates; } + /** + * @return null + * @deprecated global retention is not used in composable templates anymore + */ + @Deprecated + @Nullable public DataStreamGlobalRetention getGlobalRetention() { - return globalRetention; + return null; + } + + @Nullable + public RolloverConfiguration getRolloverConfiguration() { + return rolloverConfiguration; } @Override @@ -172,8 +190,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) { out.writeOptionalWriteable(rolloverConfiguration); } - if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) { - out.writeOptionalWriteable(globalRetention); + if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && out.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + out.writeOptionalWriteable(null); } } @@ -182,14 +201,12 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; GetComposableIndexTemplateAction.Response that = (GetComposableIndexTemplateAction.Response) o; - return Objects.equals(indexTemplates, that.indexTemplates) - && Objects.equals(rolloverConfiguration, that.rolloverConfiguration) - && Objects.equals(globalRetention, that.globalRetention); + return Objects.equals(indexTemplates, that.indexTemplates) && Objects.equals(rolloverConfiguration, that.rolloverConfiguration); } @Override public int hashCode() { - return Objects.hash(indexTemplates, rolloverConfiguration, globalRetention); + return Objects.hash(indexTemplates, rolloverConfiguration); } @Override @@ -207,7 +224,5 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); return builder; } - } - } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java index 1739b279014ee..fcc053b8181fa 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java @@ -16,7 +16,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComponentTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionProvider; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -36,7 +35,6 @@ public class TransportGetComponentTemplateAction extends TransportMasterNodeRead GetComponentTemplateAction.Response> { private final ClusterSettings clusterSettings; - private final DataStreamGlobalRetentionProvider globalRetentionResolver; @Inject public TransportGetComponentTemplateAction( @@ -44,8 +42,7 @@ public TransportGetComponentTemplateAction( ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, - DataStreamGlobalRetentionProvider globalRetentionResolver + IndexNameExpressionResolver indexNameExpressionResolver ) { super( GetComponentTemplateAction.NAME, @@ -59,7 +56,6 @@ public TransportGetComponentTemplateAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); clusterSettings = clusterService.getClusterSettings(); - this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -100,12 +96,11 @@ protected void masterOperation( listener.onResponse( new GetComponentTemplateAction.Response( results, - clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - globalRetentionResolver.provide() + clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) ) ); } else { - listener.onResponse(new GetComponentTemplateAction.Response(results, globalRetentionResolver.provide())); + listener.onResponse(new GetComponentTemplateAction.Response(results)); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java index 6ccaad593a448..e2ce172a1bf0b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java @@ -16,7 +16,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionProvider; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -36,7 +35,6 @@ public class TransportGetComposableIndexTemplateAction extends TransportMasterNo GetComposableIndexTemplateAction.Response> { private final ClusterSettings clusterSettings; - private final DataStreamGlobalRetentionProvider globalRetentionResolver; @Inject public TransportGetComposableIndexTemplateAction( @@ -44,8 +42,7 @@ public TransportGetComposableIndexTemplateAction( ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver, - DataStreamGlobalRetentionProvider globalRetentionResolver + IndexNameExpressionResolver indexNameExpressionResolver ) { super( GetComposableIndexTemplateAction.NAME, @@ -59,7 +56,6 @@ public TransportGetComposableIndexTemplateAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); clusterSettings = clusterService.getClusterSettings(); - this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -98,12 +94,11 @@ protected void masterOperation( listener.onResponse( new GetComposableIndexTemplateAction.Response( results, - clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - globalRetentionResolver.provide() + clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) ) ); } else { - listener.onResponse(new GetComposableIndexTemplateAction.Response(results, globalRetentionResolver.provide())); + listener.onResponse(new GetComposableIndexTemplateAction.Response(results)); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/SimulateIndexTemplateResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/SimulateIndexTemplateResponse.java index a2fe2e5056c4d..a27defd2c655c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/SimulateIndexTemplateResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/SimulateIndexTemplateResponse.java @@ -46,27 +46,19 @@ public class SimulateIndexTemplateResponse extends ActionResponse implements ToX @Nullable private final RolloverConfiguration rolloverConfiguration; - @Nullable - private final DataStreamGlobalRetention globalRetention; - public SimulateIndexTemplateResponse( - @Nullable Template resolvedTemplate, - @Nullable Map> overlappingTemplates, - DataStreamGlobalRetention globalRetention - ) { - this(resolvedTemplate, overlappingTemplates, null, globalRetention); + public SimulateIndexTemplateResponse(@Nullable Template resolvedTemplate, @Nullable Map> overlappingTemplates) { + this(resolvedTemplate, overlappingTemplates, null); } public SimulateIndexTemplateResponse( @Nullable Template resolvedTemplate, @Nullable Map> overlappingTemplates, - @Nullable RolloverConfiguration rolloverConfiguration, - @Nullable DataStreamGlobalRetention globalRetention + @Nullable RolloverConfiguration rolloverConfiguration ) { this.resolvedTemplate = resolvedTemplate; this.overlappingTemplates = overlappingTemplates; this.rolloverConfiguration = rolloverConfiguration; - this.globalRetention = globalRetention; } public RolloverConfiguration getRolloverConfiguration() { @@ -89,9 +81,10 @@ public SimulateIndexTemplateResponse(StreamInput in) throws IOException { rolloverConfiguration = in.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X) ? in.readOptionalWriteable(RolloverConfiguration::new) : null; - globalRetention = in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) - ? in.readOptionalWriteable(DataStreamGlobalRetention::read) - : null; + if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && in.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + in.readOptionalWriteable(DataStreamGlobalRetention::read); + } } @Override @@ -110,8 +103,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) { out.writeOptionalWriteable(rolloverConfiguration); } - if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0)) { - out.writeOptionalWriteable(globalRetention); + if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_14_0) + && out.getTransportVersion().before(TransportVersions.REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES)) { + out.writeOptionalWriteable(null); } } @@ -147,13 +141,12 @@ public boolean equals(Object o) { SimulateIndexTemplateResponse that = (SimulateIndexTemplateResponse) o; return Objects.equals(resolvedTemplate, that.resolvedTemplate) && Objects.deepEquals(overlappingTemplates, that.overlappingTemplates) - && Objects.equals(rolloverConfiguration, that.rolloverConfiguration) - && Objects.equals(globalRetention, that.globalRetention); + && Objects.equals(rolloverConfiguration, that.rolloverConfiguration); } @Override public int hashCode() { - return Objects.hash(resolvedTemplate, overlappingTemplates, rolloverConfiguration, globalRetention); + return Objects.hash(resolvedTemplate, overlappingTemplates, rolloverConfiguration); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java index 911648d06faa8..6fcaad47e0d72 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java @@ -16,8 +16,6 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionProvider; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -74,7 +72,6 @@ public class TransportSimulateIndexTemplateAction extends TransportMasterNodeRea private final Set indexSettingProviders; private final ClusterSettings clusterSettings; private final boolean isDslOnlyMode; - private final DataStreamGlobalRetentionProvider globalRetentionResolver; @Inject public TransportSimulateIndexTemplateAction( @@ -87,8 +84,7 @@ public TransportSimulateIndexTemplateAction( NamedXContentRegistry xContentRegistry, IndicesService indicesService, SystemIndices systemIndices, - IndexSettingProviders indexSettingProviders, - DataStreamGlobalRetentionProvider globalRetentionResolver + IndexSettingProviders indexSettingProviders ) { super( SimulateIndexTemplateAction.NAME, @@ -108,7 +104,6 @@ public TransportSimulateIndexTemplateAction( this.indexSettingProviders = indexSettingProviders.getIndexSettingProviders(); this.clusterSettings = clusterService.getClusterSettings(); this.isDslOnlyMode = isDataStreamsLifecycleOnlyMode(clusterService.getSettings()); - this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -118,7 +113,6 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - final DataStreamGlobalRetention globalRetention = globalRetentionResolver.provide(); final ClusterState stateWithTemplate; if (request.getIndexTemplateRequest() != null) { // we'll "locally" add the template defined by the user in the cluster state (as if it existed in the system) @@ -144,7 +138,7 @@ protected void masterOperation( String matchingTemplate = findV2Template(stateWithTemplate.metadata(), request.getIndexName(), false); if (matchingTemplate == null) { - listener.onResponse(new SimulateIndexTemplateResponse(null, null, null)); + listener.onResponse(new SimulateIndexTemplateResponse(null, null)); return; } @@ -172,12 +166,11 @@ protected void masterOperation( new SimulateIndexTemplateResponse( template, overlapping, - clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - globalRetention + clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) ) ); } else { - listener.onResponse(new SimulateIndexTemplateResponse(template, overlapping, globalRetention)); + listener.onResponse(new SimulateIndexTemplateResponse(template, overlapping)); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java index 511efe072960d..ead00dc858a47 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java @@ -15,8 +15,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionProvider; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService; @@ -60,7 +58,6 @@ public class TransportSimulateTemplateAction extends TransportMasterNodeReadActi private final Set indexSettingProviders; private final ClusterSettings clusterSettings; private final boolean isDslOnlyMode; - private final DataStreamGlobalRetentionProvider globalRetentionResolver; @Inject public TransportSimulateTemplateAction( @@ -73,8 +70,7 @@ public TransportSimulateTemplateAction( NamedXContentRegistry xContentRegistry, IndicesService indicesService, SystemIndices systemIndices, - IndexSettingProviders indexSettingProviders, - DataStreamGlobalRetentionProvider globalRetentionResolver + IndexSettingProviders indexSettingProviders ) { super( SimulateTemplateAction.NAME, @@ -94,7 +90,6 @@ public TransportSimulateTemplateAction( this.indexSettingProviders = indexSettingProviders.getIndexSettingProviders(); this.clusterSettings = clusterService.getClusterSettings(); this.isDslOnlyMode = isDataStreamsLifecycleOnlyMode(clusterService.getSettings()); - this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -104,7 +99,6 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - final DataStreamGlobalRetention globalRetention = globalRetentionResolver.provide(); String uuid = UUIDs.randomBase64UUID().toLowerCase(Locale.ROOT); final String temporaryIndexName = "simulate_template_index_" + uuid; final ClusterState stateWithTemplate; @@ -182,12 +176,11 @@ protected void masterOperation( new SimulateIndexTemplateResponse( template, overlapping, - clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - globalRetention + clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) ) ); } else { - listener.onResponse(new SimulateIndexTemplateResponse(template, overlapping, globalRetention)); + listener.onResponse(new SimulateIndexTemplateResponse(template, overlapping)); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestTemplatesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestTemplatesAction.java index 849e2d68cb2dc..876edad49a7dc 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestTemplatesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestTemplatesAction.java @@ -76,7 +76,7 @@ protected RestChannelConsumer doCatRequest(final RestRequest request, NodeClient getComposableTemplatesRequest, getComposableTemplatesStep.delegateResponse((l, e) -> { if (ExceptionsHelper.unwrapCause(e) instanceof ResourceNotFoundException) { - l.onResponse(new GetComposableIndexTemplateAction.Response(Map.of(), null)); + l.onResponse(new GetComposableIndexTemplateAction.Response(Map.of())); } else { l.onFailure(e); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateResponseTests.java index d31c9fddf2712..5f25903aeaa50 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComponentTemplateResponseTests.java @@ -12,7 +12,6 @@ import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.ComponentTemplate; import org.elasticsearch.cluster.metadata.ComponentTemplateTests; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionTests; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.Template; import org.elasticsearch.common.Strings; @@ -45,8 +44,7 @@ protected Writeable.Reader instanceReader() protected GetComponentTemplateAction.Response createTestInstance() { return new GetComponentTemplateAction.Response( randomBoolean() ? Map.of() : randomTemplates(), - RolloverConfigurationTests.randomRolloverConditions(), - DataStreamGlobalRetentionTests.randomGlobalRetention() + RolloverConfigurationTests.randomRolloverConditions() ); } @@ -54,13 +52,11 @@ protected GetComponentTemplateAction.Response createTestInstance() { protected GetComponentTemplateAction.Response mutateInstance(GetComponentTemplateAction.Response instance) { var templates = instance.getComponentTemplates(); var rolloverConditions = instance.getRolloverConfiguration(); - var globalRetention = instance.getGlobalRetention(); - switch (randomInt(2)) { + switch (randomInt(1)) { case 0 -> templates = templates == null ? randomTemplates() : null; case 1 -> rolloverConditions = randomValueOtherThan(rolloverConditions, RolloverConfigurationTests::randomRolloverConditions); - case 2 -> globalRetention = randomValueOtherThan(globalRetention, DataStreamGlobalRetentionTests::randomGlobalRetention); } - return new GetComponentTemplateAction.Response(templates, rolloverConditions, globalRetention); + return new GetComponentTemplateAction.Response(templates, rolloverConditions); } public void testXContentSerializationWithRolloverAndEffectiveRetention() throws IOException { @@ -84,20 +80,15 @@ public void testXContentSerializationWithRolloverAndEffectiveRetention() throws null, false ); - var globalRetention = DataStreamGlobalRetentionTests.randomGlobalRetention(); var rolloverConfiguration = RolloverConfigurationTests.randomRolloverConditions(); - var response = new GetComponentTemplateAction.Response( - Map.of(randomAlphaOfLength(10), template), - rolloverConfiguration, - globalRetention - ); + var response = new GetComponentTemplateAction.Response(Map.of(randomAlphaOfLength(10), template), rolloverConfiguration); try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { builder.humanReadable(true); response.toXContent(builder, EMPTY_PARAMS); String serialized = Strings.toString(builder); assertThat(serialized, containsString("rollover")); - for (String label : rolloverConfiguration.resolveRolloverConditions(lifecycle.getEffectiveDataRetention(globalRetention)) + for (String label : rolloverConfiguration.resolveRolloverConditions(lifecycle.getEffectiveDataRetention(null)) .getConditions() .keySet()) { assertThat(serialized, containsString(label)); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateResponseTests.java index aa9989257aa39..c3deabd849998 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetComposableIndexTemplateResponseTests.java @@ -8,11 +8,13 @@ package org.elasticsearch.action.admin.indices.template.get; +import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration; +import org.elasticsearch.action.admin.indices.rollover.RolloverConfigurationTests; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.ComposableIndexTemplateTests; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionTests; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.test.AbstractWireSerializingTestCase; import java.util.HashMap; @@ -26,19 +28,41 @@ protected Writeable.Reader instanceRe @Override protected GetComposableIndexTemplateAction.Response createTestInstance() { - DataStreamGlobalRetention globalRetention = randomBoolean() ? null : DataStreamGlobalRetentionTests.randomGlobalRetention(); + RolloverConfiguration rolloverConfiguration = randomBoolean() ? null : RolloverConfigurationTests.randomRolloverConditions(); if (randomBoolean()) { - return new GetComposableIndexTemplateAction.Response(Map.of(), globalRetention); + return new GetComposableIndexTemplateAction.Response(Map.of(), rolloverConfiguration); } Map templates = new HashMap<>(); for (int i = 0; i < randomIntBetween(1, 4); i++) { templates.put(randomAlphaOfLength(4), ComposableIndexTemplateTests.randomInstance()); } - return new GetComposableIndexTemplateAction.Response(templates, globalRetention); + return new GetComposableIndexTemplateAction.Response(templates, rolloverConfiguration); } @Override protected GetComposableIndexTemplateAction.Response mutateInstance(GetComposableIndexTemplateAction.Response instance) { - return randomValueOtherThan(instance, this::createTestInstance); + var rolloverConfiguration = instance.getRolloverConfiguration(); + var templates = instance.indexTemplates(); + switch (randomInt(1)) { + case 0 -> rolloverConfiguration = randomBoolean() || rolloverConfiguration == null + ? randomValueOtherThan(rolloverConfiguration, RolloverConfigurationTests::randomRolloverConditions) + : null; + case 1 -> { + var updatedTemplates = new HashMap(); + for (String name : templates.keySet()) { + if (randomBoolean()) { + updatedTemplates.put(name, templates.get(name)); + } + } + updatedTemplates.put(randomAlphaOfLength(4), ComposableIndexTemplateTests.randomInstance()); + templates = updatedTemplates; + } + } + return new GetComposableIndexTemplateAction.Response(templates, rolloverConfiguration); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(IndicesModule.getNamedWriteables()); } } From 06d09fc7aa3248595f4068fb68259f50f1754bac Mon Sep 17 00:00:00 2001 From: Luigi Dell'Aquila Date: Mon, 19 Aug 2024 11:01:06 +0200 Subject: [PATCH 08/15] Add generated evaluators for DateNanos conversion functions (#111961) These are generated files, resulting from [this PR](https://github.com/elastic/elasticsearch/pull/111850). --- .../ToDatetimeFromDateNanosEvaluator.java | 122 ++++++++++++++++++ .../ToStringFromDateNanosEvaluator.java | 109 ++++++++++++++++ 2 files changed, 231 insertions(+) create mode 100644 x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromDateNanosEvaluator.java create mode 100644 x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateNanosEvaluator.java diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromDateNanosEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromDateNanosEvaluator.java new file mode 100644 index 0000000000000..92b629657b95b --- /dev/null +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromDateNanosEvaluator.java @@ -0,0 +1,122 @@ +// 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.esql.expression.function.scalar.convert; + +import java.lang.IllegalArgumentException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.core.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDatetime}. + * This class is generated. Do not edit it. + */ +public final class ToDatetimeFromDateNanosEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDatetimeFromDateNanosEvaluator(EvalOperator.ExpressionEvaluator field, Source source, + DriverContext driverContext) { + super(driverContext, field, source); + } + + @Override + public String name() { + return "ToDatetimeFromDateNanos"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return driverContext.blockFactory().newConstantLongBlockWith(evalValue(vector, 0), positionCount); + } catch (IllegalArgumentException e) { + registerException(e); + return driverContext.blockFactory().newConstantNullBlock(positionCount); + } + } + try (LongBlock.Builder builder = driverContext.blockFactory().newLongBlockBuilder(positionCount)) { + for (int p = 0; p < positionCount; p++) { + try { + builder.appendLong(evalValue(vector, p)); + } catch (IllegalArgumentException e) { + registerException(e); + builder.appendNull(); + } + } + return builder.build(); + } + } + + private static long evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToDatetime.fromDatenanos(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + try (LongBlock.Builder builder = driverContext.blockFactory().newLongBlockBuilder(positionCount)) { + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (IllegalArgumentException e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + } + + private static long evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToDatetime.fromDatenanos(value); + } + + public static class Factory implements EvalOperator.ExpressionEvaluator.Factory { + private final Source source; + + private final EvalOperator.ExpressionEvaluator.Factory field; + + public Factory(EvalOperator.ExpressionEvaluator.Factory field, Source source) { + this.field = field; + this.source = source; + } + + @Override + public ToDatetimeFromDateNanosEvaluator get(DriverContext context) { + return new ToDatetimeFromDateNanosEvaluator(field.get(context), source, context); + } + + @Override + public String toString() { + return "ToDatetimeFromDateNanosEvaluator[field=" + field + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateNanosEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateNanosEvaluator.java new file mode 100644 index 0000000000000..37f13ea340a26 --- /dev/null +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateNanosEvaluator.java @@ -0,0 +1,109 @@ +// 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.esql.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.core.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromDateNanosEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromDateNanosEvaluator(EvalOperator.ExpressionEvaluator field, Source source, + DriverContext driverContext) { + super(driverContext, field, source); + } + + @Override + public String name() { + return "ToStringFromDateNanos"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + return driverContext.blockFactory().newConstantBytesRefBlockWith(evalValue(vector, 0), positionCount); + } + try (BytesRefBlock.Builder builder = driverContext.blockFactory().newBytesRefBlockBuilder(positionCount)) { + for (int p = 0; p < positionCount; p++) { + builder.appendBytesRef(evalValue(vector, p)); + } + return builder.build(); + } + } + + private static BytesRef evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToString.fromDateNanos(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + try (BytesRefBlock.Builder builder = driverContext.blockFactory().newBytesRefBlockBuilder(positionCount)) { + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + } + + private static BytesRef evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToString.fromDateNanos(value); + } + + public static class Factory implements EvalOperator.ExpressionEvaluator.Factory { + private final Source source; + + private final EvalOperator.ExpressionEvaluator.Factory field; + + public Factory(EvalOperator.ExpressionEvaluator.Factory field, Source source) { + this.field = field; + this.source = source; + } + + @Override + public ToStringFromDateNanosEvaluator get(DriverContext context) { + return new ToStringFromDateNanosEvaluator(field.get(context), source, context); + } + + @Override + public String toString() { + return "ToStringFromDateNanosEvaluator[field=" + field + "]"; + } + } +} From d2e667004596544bbbb594c7d42e215483caf9c5 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 11:57:07 +0100 Subject: [PATCH 09/15] Move repo analyzer to its own package (#111963) In preparation for adding more things to the blobstore testkit, this commit moves the repository analyzer implementation from `o.e.r.blobstore.testkit` to `o.e.r.blobstore.testkit.analyze`. --- .../AzureRepositoryAnalysisRestIT.java} | 4 ++-- .../GCSRepositoryAnalysisRestIT.java} | 4 ++-- .../AbstractHdfsRepositoryAnalysisRestIT.java} | 4 ++-- .../HdfsRepositoryAnalysisRestIT.java} | 4 ++-- .../SecureHdfsRepositoryAnalysisRestIT.java} | 4 ++-- .../MinioRepositoryAnalysisRestIT.java} | 4 ++-- x-pack/plugin/snapshot-repo-test-kit/qa/rest/build.gradle | 2 +- .../FsRepositoryAnalysisRestIT.java} | 5 ++--- .../S3RepositoryAnalysisRestIT.java} | 4 ++-- .../blobstore/testkit/{ => analyze}/BytesRegister.java | 2 +- .../{ => analyze}/RepositoryAnalysisFailureIT.java | 7 ++++--- .../{ => analyze}/RepositoryAnalysisSuccessIT.java | 7 ++++--- .../blobstore/testkit/SnapshotRepositoryTestKit.java | 5 ++++- .../testkit/{ => analyze}/BlobAnalyzeAction.java | 2 +- .../testkit/{ => analyze}/BlobWriteAbortedException.java | 2 +- .../{ => analyze}/ContendedRegisterAnalyzeAction.java | 2 +- .../testkit/{ => analyze}/GetBlobChecksumAction.java | 2 +- .../testkit/{ => analyze}/RandomBlobContent.java | 2 +- .../{ => analyze}/RandomBlobContentBytesReference.java | 2 +- .../testkit/{ => analyze}/RandomBlobContentStream.java | 2 +- .../testkit/{ => analyze}/RepositoryAnalyzeAction.java | 8 ++++---- .../{ => analyze}/RepositoryPerformanceSummary.java | 2 +- .../{ => analyze}/RestRepositoryAnalyzeAction.java | 2 +- .../{ => analyze}/UncontendedRegisterAnalyzeAction.java | 6 +++--- .../AbstractRepositoryAnalysisRestTestCase.java} | 4 ++-- .../RandomBlobContentBytesReferenceTests.java | 4 ++-- .../{ => analyze}/RandomBlobContentStreamTests.java | 4 ++-- .../{ => analyze}/RepositoryAnalyzeActionTests.java | 2 +- 28 files changed, 53 insertions(+), 49 deletions(-) rename x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{AzureSnapshotRepoTestKitIT.java => analyze/AzureRepositoryAnalysisRestIT.java} (97%) rename x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{GCSSnapshotRepoTestKitIT.java => analyze/GCSRepositoryAnalysisRestIT.java} (95%) rename x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{AbstractHdfsSnapshotRepoTestKitIT.java => analyze/AbstractHdfsRepositoryAnalysisRestIT.java} (86%) rename x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{HdfsSnapshotRepoTestKitIT.java => analyze/HdfsRepositoryAnalysisRestIT.java} (90%) rename x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{SecureHdfsSnapshotRepoTestKitIT.java => analyze/SecureHdfsRepositoryAnalysisRestIT.java} (93%) rename x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{MinioSnapshotRepoTestKitIT.java => analyze/MinioRepositoryAnalysisRestIT.java} (93%) rename x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{rest/FsSnapshotRepoTestKitIT.java => analyze/FsRepositoryAnalysisRestIT.java} (71%) rename x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/{S3SnapshotRepoTestKitIT.java => analyze/S3RepositoryAnalysisRestIT.java} (94%) rename x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/BytesRegister.java (93%) rename x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RepositoryAnalysisFailureIT.java (98%) rename x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RepositoryAnalysisSuccessIT.java (98%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/BlobAnalyzeAction.java (99%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/BlobWriteAbortedException.java (85%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/ContendedRegisterAnalyzeAction.java (99%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/GetBlobChecksumAction.java (99%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RandomBlobContent.java (98%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RandomBlobContentBytesReference.java (97%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RandomBlobContentStream.java (97%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RepositoryAnalyzeAction.java (99%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RepositoryPerformanceSummary.java (98%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RestRepositoryAnalyzeAction.java (98%) rename x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/UncontendedRegisterAnalyzeAction.java (96%) rename x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/{AbstractSnapshotRepoTestKitRestTestCase.java => analyze/AbstractRepositoryAnalysisRestTestCase.java} (90%) rename x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RandomBlobContentBytesReferenceTests.java (91%) rename x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RandomBlobContentStreamTests.java (97%) rename x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/{ => analyze}/RepositoryAnalyzeActionTests.java (98%) diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AzureSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java similarity index 97% rename from x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AzureSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java index 154b5bec54418..ecc8401e1d79a 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AzureSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import fixture.azure.AzureHttpFixture; @@ -25,7 +25,7 @@ import static org.hamcrest.Matchers.blankOrNullString; import static org.hamcrest.Matchers.not; -public class AzureSnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public class AzureRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { private static final boolean USE_FIXTURE = Booleans.parseBoolean(System.getProperty("test.azure.fixture", "true")); private static final boolean USE_HTTPS_FIXTURE = USE_FIXTURE && ESTestCase.inFipsJvm() == false; // TODO when https://github.com/elastic/elasticsearch/issues/111532 addressed, use a HTTPS fixture in FIPS mode too diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/GCSSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GCSRepositoryAnalysisRestIT.java similarity index 95% rename from x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/GCSSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GCSRepositoryAnalysisRestIT.java index 95b6f4aed5221..7f7540d138825 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/GCSSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/gcs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GCSRepositoryAnalysisRestIT.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import fixture.gcs.GoogleCloudStorageHttpFixture; import fixture.gcs.TestUtils; @@ -23,7 +23,7 @@ import static org.hamcrest.Matchers.blankOrNullString; import static org.hamcrest.Matchers.not; -public class GCSSnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public class GCSRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { private static final boolean USE_FIXTURE = Booleans.parseBoolean(System.getProperty("test.google.fixture", "true")); private static GoogleCloudStorageHttpFixture fixture = new GoogleCloudStorageHttpFixture(USE_FIXTURE, "bucket", "o/oauth2/token"); diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AbstractHdfsSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractHdfsRepositoryAnalysisRestIT.java similarity index 86% rename from x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AbstractHdfsSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractHdfsRepositoryAnalysisRestIT.java index 2810c4801e8dd..2aec22476d6cc 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/AbstractHdfsSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractHdfsRepositoryAnalysisRestIT.java @@ -5,14 +5,14 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.common.settings.Settings; import static org.hamcrest.Matchers.blankOrNullString; import static org.hamcrest.Matchers.not; -public abstract class AbstractHdfsSnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public abstract class AbstractHdfsRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { @Override protected String repositoryType() { diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/HdfsSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/HdfsRepositoryAnalysisRestIT.java similarity index 90% rename from x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/HdfsSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/HdfsRepositoryAnalysisRestIT.java index e9787ecdce854..d60497949ff61 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/HdfsSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/HdfsRepositoryAnalysisRestIT.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; @@ -17,7 +17,7 @@ import org.junit.rules.TestRule; @ThreadLeakFilters(filters = { HdfsClientThreadLeakFilter.class }) -public class HdfsSnapshotRepoTestKitIT extends AbstractHdfsSnapshotRepoTestKitIT { +public class HdfsRepositoryAnalysisRestIT extends AbstractHdfsRepositoryAnalysisRestIT { public static HdfsFixture hdfsFixture = new HdfsFixture(); diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/SecureHdfsSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/SecureHdfsRepositoryAnalysisRestIT.java similarity index 93% rename from x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/SecureHdfsSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/SecureHdfsRepositoryAnalysisRestIT.java index 6d599e41e3b9f..dd388c0a79776 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/SecureHdfsSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/hdfs/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/SecureHdfsRepositoryAnalysisRestIT.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; @@ -22,7 +22,7 @@ import org.junit.rules.TestRule; @ThreadLeakFilters(filters = { HdfsClientThreadLeakFilter.class, TestContainersThreadFilter.class }) -public class SecureHdfsSnapshotRepoTestKitIT extends AbstractHdfsSnapshotRepoTestKitIT { +public class SecureHdfsRepositoryAnalysisRestIT extends AbstractHdfsRepositoryAnalysisRestIT { public static Krb5kDcContainer krb5Fixture = new Krb5kDcContainer(); diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/MinioSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java similarity index 93% rename from x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/MinioSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java index 3e58a8d89ff31..b0068bd7bfdaf 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/MinioSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; @@ -18,7 +18,7 @@ import org.junit.rules.TestRule; @ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) -public class MinioSnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public class MinioRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { public static final MinioTestContainer minioFixture = new MinioTestContainer(); diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/rest/build.gradle b/x-pack/plugin/snapshot-repo-test-kit/qa/rest/build.gradle index 17df249b08cf6..8a5dbca7dd0b2 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/rest/build.gradle +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/rest/build.gradle @@ -21,7 +21,7 @@ testClusters.matching { it.name == "yamlRestTest" }.configureEach { } tasks.named('yamlRestTestTestingConventions').configure { - baseClass 'org.elasticsearch.repositories.blobstore.testkit.AbstractSnapshotRepoTestKitRestTestCase' + baseClass 'org.elasticsearch.repositories.blobstore.testkit.analyze.AbstractRepositoryAnalysisRestTestCase' baseClass 'org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase' } diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/rest/FsSnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/FsRepositoryAnalysisRestIT.java similarity index 71% rename from x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/rest/FsSnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/FsRepositoryAnalysisRestIT.java index 77dfb3902805a..7151b6e80a4d5 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/rest/FsSnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/rest/src/yamlRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/FsRepositoryAnalysisRestIT.java @@ -5,13 +5,12 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit.rest; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.repositories.blobstore.testkit.AbstractSnapshotRepoTestKitRestTestCase; import org.elasticsearch.repositories.fs.FsRepository; -public class FsSnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public class FsRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { @Override protected String repositoryType() { diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/S3SnapshotRepoTestKitIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java similarity index 94% rename from x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/S3SnapshotRepoTestKitIT.java rename to x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java index c38bd1204189f..8986cf1059191 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/S3SnapshotRepoTestKitIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import fixture.s3.S3HttpFixture; @@ -18,7 +18,7 @@ import static org.hamcrest.Matchers.blankOrNullString; import static org.hamcrest.Matchers.not; -public class S3SnapshotRepoTestKitIT extends AbstractSnapshotRepoTestKitRestTestCase { +public class S3RepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { static final boolean USE_FIXTURE = Boolean.parseBoolean(System.getProperty("tests.use.fixture", "true")); diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/BytesRegister.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BytesRegister.java similarity index 93% rename from x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/BytesRegister.java rename to x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BytesRegister.java index 4303fff673359..3f5e406ac797b 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/BytesRegister.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BytesRegister.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisFailureIT.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java rename to x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisFailureIT.java index 73a90f247810e..e61f883abd60f 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisFailureIT.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; @@ -40,6 +40,7 @@ import org.elasticsearch.repositories.RepositoryMissingException; import org.elasticsearch.repositories.RepositoryVerificationException; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.repositories.blobstore.testkit.SnapshotRepositoryTestKit; import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; @@ -66,8 +67,8 @@ import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.bytesFromLong; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.longFromBytes; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.bytesFromLong; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.longFromBytes; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.anyOf; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisSuccessIT.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java rename to x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisSuccessIT.java index e4d9bf9041b4a..bb452ad2a64ce 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisSuccessIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalysisSuccessIT.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.RepositoryMetadata; @@ -36,6 +36,7 @@ import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryMissingException; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.repositories.blobstore.testkit.SnapshotRepositoryTestKit; import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.ObjectPath; @@ -61,8 +62,8 @@ import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.MAX_RESTORE_BYTES_PER_SEC; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.MAX_SNAPSHOT_BYTES_PER_SEC; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.longFromBytes; -import static org.elasticsearch.repositories.blobstore.testkit.RepositoryAnalysisFailureIT.isContendedRegisterKey; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.longFromBytes; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.RepositoryAnalysisFailureIT.isContendedRegisterKey; import static org.elasticsearch.test.XContentTestUtils.convertToMap; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.allOf; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/SnapshotRepositoryTestKit.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/SnapshotRepositoryTestKit.java index 124174a2a025b..04d59906e6db3 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/SnapshotRepositoryTestKit.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/SnapshotRepositoryTestKit.java @@ -20,6 +20,8 @@ import org.elasticsearch.features.NodeFeature; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.blobstore.testkit.analyze.RepositoryAnalyzeAction; +import org.elasticsearch.repositories.blobstore.testkit.analyze.RestRepositoryAnalyzeAction; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; import org.elasticsearch.xcontent.XContentBuilder; @@ -51,7 +53,8 @@ public List getRestHandlers( return List.of(new RestRepositoryAnalyzeAction()); } - static void humanReadableNanos(XContentBuilder builder, String rawFieldName, String readableFieldName, long nanos) throws IOException { + public static void humanReadableNanos(XContentBuilder builder, String rawFieldName, String readableFieldName, long nanos) + throws IOException { assert rawFieldName.equals(readableFieldName) == false : rawFieldName + " vs " + readableFieldName; if (builder.humanReadable()) { diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobAnalyzeAction.java similarity index 99% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobAnalyzeAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobAnalyzeAction.java index aa0cf3e3cfc1b..6007968d7cb4d 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobAnalyzeAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobWriteAbortedException.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobWriteAbortedException.java similarity index 85% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobWriteAbortedException.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobWriteAbortedException.java index 11c73993a3e6e..8a7bbb7255c5a 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/BlobWriteAbortedException.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/BlobWriteAbortedException.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; public class BlobWriteAbortedException extends RuntimeException { public BlobWriteAbortedException() { diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/ContendedRegisterAnalyzeAction.java similarity index 99% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/ContendedRegisterAnalyzeAction.java index 40cb4a45a0339..f527a46371641 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/ContendedRegisterAnalyzeAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/GetBlobChecksumAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GetBlobChecksumAction.java similarity index 99% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/GetBlobChecksumAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GetBlobChecksumAction.java index f706ff79bf073..816f9e860a33a 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/GetBlobChecksumAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/GetBlobChecksumAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContent.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContent.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContent.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContent.java index aa9125f214f58..d5061b303f93d 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContent.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContent.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.repositories.RepositoryVerificationException; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReference.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReference.java similarity index 97% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReference.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReference.java index 44627000a2de9..eee40992cb0d7 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReference.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReference.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStream.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStream.java similarity index 97% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStream.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStream.java index c6163a7ffd82d..15fa370c5fe0a 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStream.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStream.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import java.io.InputStream; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeAction.java similarity index 99% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeAction.java index 30c2d0a89e0ee..5ced0176a4f81 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -83,10 +83,10 @@ import java.util.stream.IntStream; import static org.elasticsearch.core.Strings.format; -import static org.elasticsearch.repositories.blobstore.testkit.BlobAnalyzeAction.MAX_ATOMIC_WRITE_SIZE; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.bytesFromLong; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.longFromBytes; import static org.elasticsearch.repositories.blobstore.testkit.SnapshotRepositoryTestKit.humanReadableNanos; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.BlobAnalyzeAction.MAX_ATOMIC_WRITE_SIZE; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.bytesFromLong; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.longFromBytes; /** * Action which distributes a bunch of {@link BlobAnalyzeAction}s over the nodes in the cluster, with limited concurrency, and collects diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryPerformanceSummary.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryPerformanceSummary.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryPerformanceSummary.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryPerformanceSummary.java index 3ee8805480023..c2625285a8912 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryPerformanceSummary.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryPerformanceSummary.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RestRepositoryAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RestRepositoryAnalyzeAction.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RestRepositoryAnalyzeAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RestRepositoryAnalyzeAction.java index 2a549db8b3255..b0f6b01936ffa 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/RestRepositoryAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RestRepositoryAnalyzeAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.rest.BaseRestHandler; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/UncontendedRegisterAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/UncontendedRegisterAnalyzeAction.java similarity index 96% rename from x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/UncontendedRegisterAnalyzeAction.java rename to x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/UncontendedRegisterAnalyzeAction.java index 1986b47e3188c..23c25e466b917 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/UncontendedRegisterAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/analyze/UncontendedRegisterAnalyzeAction.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -38,8 +38,8 @@ import java.io.IOException; import java.util.Map; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.bytesFromLong; -import static org.elasticsearch.repositories.blobstore.testkit.ContendedRegisterAnalyzeAction.longFromBytes; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.bytesFromLong; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.ContendedRegisterAnalyzeAction.longFromBytes; class UncontendedRegisterAnalyzeAction extends HandledTransportAction { diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/AbstractSnapshotRepoTestKitRestTestCase.java b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractRepositoryAnalysisRestTestCase.java similarity index 90% rename from x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/AbstractSnapshotRepoTestKitRestTestCase.java rename to x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractRepositoryAnalysisRestTestCase.java index 3af8c118803a7..2c96003f7e3d3 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/AbstractSnapshotRepoTestKitRestTestCase.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AbstractRepositoryAnalysisRestTestCase.java @@ -5,14 +5,14 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.apache.http.client.methods.HttpPost; import org.elasticsearch.client.Request; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.rest.ESRestTestCase; -public abstract class AbstractSnapshotRepoTestKitRestTestCase extends ESRestTestCase { +public abstract class AbstractRepositoryAnalysisRestTestCase extends ESRestTestCase { protected abstract String repositoryType(); diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReferenceTests.java b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReferenceTests.java similarity index 91% rename from x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReferenceTests.java rename to x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReferenceTests.java index c85b634083faf..29a6253c031d8 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentBytesReferenceTests.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentBytesReferenceTests.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.test.ESTestCase; @@ -13,7 +13,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.CRC32; -import static org.elasticsearch.repositories.blobstore.testkit.RandomBlobContent.BUFFER_SIZE; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.RandomBlobContent.BUFFER_SIZE; import static org.hamcrest.Matchers.equalTo; public class RandomBlobContentBytesReferenceTests extends ESTestCase { diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStreamTests.java b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStreamTests.java similarity index 97% rename from x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStreamTests.java rename to x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStreamTests.java index 6c353e0937a33..1854d98f7ec79 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RandomBlobContentStreamTests.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RandomBlobContentStreamTests.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.repositories.RepositoryVerificationException; import org.elasticsearch.test.ESTestCase; @@ -14,7 +14,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.zip.CRC32; -import static org.elasticsearch.repositories.blobstore.testkit.RandomBlobContent.BUFFER_SIZE; +import static org.elasticsearch.repositories.blobstore.testkit.analyze.RandomBlobContent.BUFFER_SIZE; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeActionTests.java b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeActionTests.java similarity index 98% rename from x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeActionTests.java rename to x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeActionTests.java index 0d1bdc86002b4..44770e68d714b 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalyzeActionTests.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/test/java/org/elasticsearch/repositories/blobstore/testkit/analyze/RepositoryAnalyzeActionTests.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.repositories.blobstore.testkit; +package org.elasticsearch.repositories.blobstore.testkit.analyze; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.test.ESTestCase; From ca6d41ce2093989dad829ddfe053a1194e0d0b7a Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 19 Aug 2024 12:04:55 +0100 Subject: [PATCH 10/15] Fail `indexDocs()` on rejection (#111962) In 9dc59e29 we relaxed the `indexDocs()` test utility to retry on rejections caused by exceeding the write queue length limit, but then we massively relaxed this limit in #59559. We should not be seeing such rejections any more, so we can revert this special handling and strengthen the tests to assert that the indexing process encounters no failures at all. --- .../elasticsearch/test/ESIntegTestCase.java | 38 +------------------ 1 file changed, 2 insertions(+), 36 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index aad3dcc457241..fa686a0bc753a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.TotalHits; import org.apache.lucene.tests.util.LuceneTestCase; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; @@ -101,7 +100,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.MockBigArrays; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.common.xcontent.XContentHelper; @@ -109,7 +107,6 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.core.Tuple; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.gateway.PersistedClusterStateService; @@ -186,7 +183,6 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; @@ -212,7 +208,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout; import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.emptyIterable; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; @@ -1735,7 +1730,6 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma } } Collections.shuffle(builders, random()); - final CopyOnWriteArrayList> errors = new CopyOnWriteArrayList<>(); List inFlightAsyncOperations = new ArrayList<>(); // If you are indexing just a few documents then frequently do it one at a time. If many then frequently in bulk. final String[] indicesArray = indices.toArray(new String[] {}); @@ -1744,7 +1738,7 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma logger.info("Index [{}] docs async: [{}] bulk: [{}]", builders.size(), true, false); for (IndexRequestBuilder indexRequestBuilder : builders) { indexRequestBuilder.execute( - new PayloadLatchedActionListener<>(indexRequestBuilder, newLatch(inFlightAsyncOperations), errors) + new LatchedActionListener(newLatch(inFlightAsyncOperations)).delegateResponse((l, e) -> fail(e)) ); postIndexAsyncActions(indicesArray, inFlightAsyncOperations, maybeFlush); } @@ -1771,19 +1765,8 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma } } for (CountDownLatch operation : inFlightAsyncOperations) { - operation.await(); - } - final List actualErrors = new ArrayList<>(); - for (Tuple tuple : errors) { - Throwable t = ExceptionsHelper.unwrapCause(tuple.v2()); - if (t instanceof EsRejectedExecutionException) { - logger.debug("Error indexing doc: " + t.getMessage() + ", reindexing."); - tuple.v1().get(); // re-index if rejected - } else { - actualErrors.add(tuple.v2()); - } + safeAwait(operation); } - assertThat(actualErrors, emptyIterable()); if (bogusIds.isEmpty() == false) { // delete the bogus types again - it might trigger merges or at least holes in the segments and enforces deleted docs! for (List doc : bogusIds) { @@ -1957,23 +1940,6 @@ protected void addError(Exception e) {} } - private class PayloadLatchedActionListener extends LatchedActionListener { - private final CopyOnWriteArrayList> errors; - private final T builder; - - PayloadLatchedActionListener(T builder, CountDownLatch latch, CopyOnWriteArrayList> errors) { - super(latch); - this.errors = errors; - this.builder = builder; - } - - @Override - protected void addError(Exception e) { - errors.add(new Tuple<>(builder, e)); - } - - } - /** * Clears the given scroll Ids */ From 7bf730a88f5b1e324d3afbd077cc400ab092de92 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Mon, 19 Aug 2024 15:55:58 +0200 Subject: [PATCH 11/15] Fix: HierarchyCircuitBreakerTelemetryTests testCircuitBreakerTripCountMetric failing (#111831) * Cleanup code and teardown for testCircuitBreakerTripCountMetric * Move to a more appropriate location --- muted-tests.yml | 3 - .../HierarchyCircuitBreakerTelemetryIT.java} | 101 +++++------------- 2 files changed, 24 insertions(+), 80 deletions(-) rename server/src/{test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerTelemetryTests.java => internalClusterTest/java/org/elasticsearch/indices/memory/breaker/HierarchyCircuitBreakerTelemetryIT.java} (58%) diff --git a/muted-tests.yml b/muted-tests.yml index 22adc4a8c44b5..dd4dd2c7f2ec7 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -137,9 +137,6 @@ tests: - class: org.elasticsearch.xpack.restart.CoreFullClusterRestartIT method: testSnapshotRestore {cluster=UPGRADED} issue: https://github.com/elastic/elasticsearch/issues/111799 -- class: org.elasticsearch.indices.breaker.HierarchyCircuitBreakerTelemetryTests - method: testCircuitBreakerTripCountMetric - issue: https://github.com/elastic/elasticsearch/issues/111778 - class: org.elasticsearch.xpack.esql.qa.mixed.MixedClusterEsqlSpecIT method: test {comparison.RangeVersion SYNC} issue: https://github.com/elastic/elasticsearch/issues/111814 diff --git a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerTelemetryTests.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/memory/breaker/HierarchyCircuitBreakerTelemetryIT.java similarity index 58% rename from server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerTelemetryTests.java rename to server/src/internalClusterTest/java/org/elasticsearch/indices/memory/breaker/HierarchyCircuitBreakerTelemetryIT.java index 2cbe1202520df..ff2117ea93bb9 100644 --- a/server/src/test/java/org/elasticsearch/indices/breaker/HierarchyCircuitBreakerTelemetryTests.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/memory/breaker/HierarchyCircuitBreakerTelemetryIT.java @@ -6,25 +6,23 @@ * Side Public License, v 1. */ -package org.elasticsearch.indices.breaker; +package org.elasticsearch.indices.memory.breaker; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.indices.breaker.CircuitBreakerMetrics; +import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.telemetry.Measurement; -import org.elasticsearch.telemetry.RecordingInstruments; -import org.elasticsearch.telemetry.RecordingMeterRegistry; import org.elasticsearch.telemetry.TestTelemetryPlugin; -import org.elasticsearch.telemetry.metric.LongCounter; -import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESIntegTestCase; import org.hamcrest.Matchers; +import org.junit.After; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -41,54 +39,11 @@ import static org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0, numClientNodes = 0, supportsDedicatedMasters = true) -public class HierarchyCircuitBreakerTelemetryTests extends ESIntegTestCase { +public class HierarchyCircuitBreakerTelemetryIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return List.of(TestCircuitBreakerTelemetryPlugin.class); - } - - public static class TestCircuitBreakerTelemetryPlugin extends TestTelemetryPlugin { - protected final MeterRegistry meter = new RecordingMeterRegistry() { - private final LongCounter tripCount = new RecordingInstruments.RecordingLongCounter( - CircuitBreakerMetrics.ES_BREAKER_TRIP_COUNT_TOTAL, - recorder - ) { - @Override - public void incrementBy(long inc) { - throw new UnsupportedOperationException(); - } - - @Override - public void incrementBy(long inc, Map attributes) { - throw new UnsupportedOperationException(); - } - }; - - @Override - protected LongCounter buildLongCounter(String name, String description, String unit) { - if (name.equals(tripCount.getName())) { - return tripCount; - } - throw new IllegalArgumentException("Unknown counter metric name [" + name + "]"); - } - - @Override - public LongCounter registerLongCounter(String name, String description, String unit) { - assertCircuitBreakerName(name); - return super.registerLongCounter(name, description, unit); - } - - @Override - public LongCounter getLongCounter(String name) { - assertCircuitBreakerName(name); - return super.getLongCounter(name); - } - - private void assertCircuitBreakerName(final String name) { - assertThat(name, Matchers.oneOf(CircuitBreakerMetrics.ES_BREAKER_TRIP_COUNT_TOTAL)); - } - }; + return List.of(TestTelemetryPlugin.class); } public void testCircuitBreakerTripCountMetric() { @@ -142,37 +97,29 @@ public void testCircuitBreakerTripCountMetric() { fail("Expected exception not thrown"); } - private List getMeasurements(String dataNodeName) { - final TestTelemetryPlugin dataNodeTelemetryPlugin = internalCluster().getInstance(PluginsService.class, dataNodeName) - .filterPlugins(TestCircuitBreakerTelemetryPlugin.class) + @After + public void resetClusterSetting() { + final var circuitBreakerSettings = Settings.builder() + .putNull(FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.getKey()) + .putNull(FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING.getKey()) + .putNull(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey()) + .putNull(REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.getKey()) + .putNull(IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey()) + .putNull(IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING.getKey()) + .putNull(TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.getKey()) + .putNull(HierarchyCircuitBreakerService.USE_REAL_MEMORY_USAGE_SETTING.getKey()); + updateClusterSettings(circuitBreakerSettings); + } + + private List getMeasurements(String nodeName) { + final TestTelemetryPlugin telemetryPlugin = internalCluster().getInstance(PluginsService.class, nodeName) + .filterPlugins(TestTelemetryPlugin.class) .toList() .get(0); return Measurement.combine( - Stream.of(dataNodeTelemetryPlugin.getLongCounterMeasurement(CircuitBreakerMetrics.ES_BREAKER_TRIP_COUNT_TOTAL).stream()) + Stream.of(telemetryPlugin.getLongCounterMeasurement(CircuitBreakerMetrics.ES_BREAKER_TRIP_COUNT_TOTAL).stream()) .flatMap(Function.identity()) .toList() ); } - - // Make sure circuit breaker telemetry on trip count reports the same values as circuit breaker stats - private void assertCircuitBreakerTripCount( - final HierarchyCircuitBreakerService circuitBreakerService, - final String circuitBreakerName, - int firstBytesEstimate, - int secondBytesEstimate, - long expectedTripCountValue - ) { - try { - circuitBreakerService.getBreaker(circuitBreakerName).addEstimateBytesAndMaybeBreak(firstBytesEstimate, randomAlphaOfLength(5)); - circuitBreakerService.getBreaker(circuitBreakerName).addEstimateBytesAndMaybeBreak(secondBytesEstimate, randomAlphaOfLength(5)); - } catch (final CircuitBreakingException cbex) { - final CircuitBreakerStats circuitBreakerStats = Arrays.stream(circuitBreakerService.stats().getAllStats()) - .filter(stats -> circuitBreakerName.equals(stats.getName())) - .findAny() - .get(); - assertThat(circuitBreakerService.getBreaker(circuitBreakerName).getTrippedCount(), Matchers.equalTo(expectedTripCountValue)); - assertThat(circuitBreakerStats.getTrippedCount(), Matchers.equalTo(expectedTripCountValue)); - } - } - } From 8b0a1aa7ebac47af865f4fbc732cc4a09835906a Mon Sep 17 00:00:00 2001 From: Artem Prigoda Date: Mon, 19 Aug 2024 16:14:09 +0200 Subject: [PATCH 12/15] [cache] Support async RangeMissingHandler callbacks (#111340) (#111896) Change `fillCacheRange` method to accept a completion listener that must be called by `RangeMissingHandler` implementations when they finish fetching data. By doing so, we support asynchronously fetching the data from a third party storage. We also support asynchronous `SourceInputStreamFactory` for reading gaps from the storage. --- .../shared/SharedBlobCacheService.java | 101 +++++--- .../shared/SharedBlobCacheServiceTests.java | 216 ++++++++++++------ .../store/input/FrozenIndexInput.java | 59 ++--- 3 files changed, 253 insertions(+), 123 deletions(-) diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java index 3242a02dff525..8ca62a3b95023 100644 --- a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java @@ -650,13 +650,14 @@ private RangeMissingHandler writerWithOffset(RangeMissingHandler writer, int wri // no need to allocate a new capturing lambda if the offset isn't adjusted return writer; } - return (channel, channelPos, streamFactory, relativePos, len, progressUpdater) -> writer.fillCacheRange( + return (channel, channelPos, streamFactory, relativePos, len, progressUpdater, completionListener) -> writer.fillCacheRange( channel, channelPos, streamFactory, relativePos - writeOffset, len, - progressUpdater + progressUpdater, + completionListener ); } @@ -991,16 +992,17 @@ void populateAndRead( executor.execute(fillGapRunnable(gap, writer, null, refs.acquireListener())); } } else { - final List gapFillingTasks = gaps.stream() - .map(gap -> fillGapRunnable(gap, writer, streamFactory, refs.acquireListener())) - .toList(); - executor.execute(() -> { - try (streamFactory) { + var gapFillingListener = refs.acquireListener(); + try (var gfRefs = new RefCountingRunnable(ActionRunnable.run(gapFillingListener, streamFactory::close))) { + final List gapFillingTasks = gaps.stream() + .map(gap -> fillGapRunnable(gap, writer, streamFactory, gfRefs.acquireListener())) + .toList(); + executor.execute(() -> { // Fill the gaps in order. If a gap fails to fill for whatever reason, the task for filling the next // gap will still be executed. gapFillingTasks.forEach(Runnable::run); - } - }); + }); + } } } } @@ -1009,13 +1011,13 @@ void populateAndRead( } } - private AbstractRunnable fillGapRunnable( + private Runnable fillGapRunnable( SparseFileTracker.Gap gap, RangeMissingHandler writer, @Nullable SourceInputStreamFactory streamFactory, ActionListener listener ) { - return ActionRunnable.run(listener.delegateResponse((l, e) -> failGapAndListener(gap, l, e)), () -> { + return () -> ActionListener.run(listener, l -> { var ioRef = io; assert regionOwners.get(ioRef) == CacheFileRegion.this; assert CacheFileRegion.this.hasReferences() : CacheFileRegion.this; @@ -1026,10 +1028,15 @@ private AbstractRunnable fillGapRunnable( streamFactory, start, Math.toIntExact(gap.end() - start), - progress -> gap.onProgress(start + progress) + progress -> gap.onProgress(start + progress), + l.map(unused -> { + assert regionOwners.get(ioRef) == CacheFileRegion.this; + assert CacheFileRegion.this.hasReferences() : CacheFileRegion.this; + writeCount.increment(); + gap.onCompletion(); + return null; + }).delegateResponse((delegate, e) -> failGapAndListener(gap, delegate, e)) ); - writeCount.increment(); - gap.onCompletion(); }); } @@ -1117,12 +1124,23 @@ public void fillCacheRange( SourceInputStreamFactory streamFactory, int relativePos, int length, - IntConsumer progressUpdater + IntConsumer progressUpdater, + ActionListener completionListener ) throws IOException { - writer.fillCacheRange(channel, channelPos, streamFactory, relativePos, length, progressUpdater); - var elapsedTime = TimeUnit.NANOSECONDS.toMillis(relativeTimeInNanosSupplier.getAsLong() - startTime); - SharedBlobCacheService.this.blobCacheMetrics.getCacheMissLoadTimes().record(elapsedTime); - SharedBlobCacheService.this.blobCacheMetrics.getCacheMissCounter().increment(); + writer.fillCacheRange( + channel, + channelPos, + streamFactory, + relativePos, + length, + progressUpdater, + completionListener.map(unused -> { + var elapsedTime = TimeUnit.NANOSECONDS.toMillis(relativeTimeInNanosSupplier.getAsLong() - startTime); + blobCacheMetrics.getCacheMissLoadTimes().record(elapsedTime); + blobCacheMetrics.getCacheMissCounter().increment(); + return null; + }) + ); } }; if (rangeToRead.isEmpty()) { @@ -1215,9 +1233,18 @@ public void fillCacheRange( SourceInputStreamFactory streamFactory, int relativePos, int len, - IntConsumer progressUpdater + IntConsumer progressUpdater, + ActionListener completionListener ) throws IOException { - delegate.fillCacheRange(channel, channelPos, streamFactory, relativePos - writeOffset, len, progressUpdater); + delegate.fillCacheRange( + channel, + channelPos, + streamFactory, + relativePos - writeOffset, + len, + progressUpdater, + completionListener + ); } }; } @@ -1230,14 +1257,25 @@ public void fillCacheRange( SourceInputStreamFactory streamFactory, int relativePos, int len, - IntConsumer progressUpdater + IntConsumer progressUpdater, + ActionListener completionListener ) throws IOException { assert assertValidRegionAndLength(fileRegion, channelPos, len); - delegate.fillCacheRange(channel, channelPos, streamFactory, relativePos, len, progressUpdater); - assert regionOwners.get(fileRegion.io) == fileRegion - : "File chunk [" + fileRegion.regionKey + "] no longer owns IO [" + fileRegion.io + "]"; + delegate.fillCacheRange( + channel, + channelPos, + streamFactory, + relativePos, + len, + progressUpdater, + Assertions.ENABLED ? ActionListener.runBefore(completionListener, () -> { + assert regionOwners.get(fileRegion.io) == fileRegion + : "File chunk [" + fileRegion.regionKey + "] no longer owns IO [" + fileRegion.io + "]"; + }) : completionListener + ); } }; + } return adjustedWriter; } @@ -1324,6 +1362,7 @@ default SourceInputStreamFactory sharedInputStreamFactory(List completionListener ) throws IOException; } @@ -1343,9 +1383,9 @@ public interface SourceInputStreamFactory extends Releasable { /** * Create the input stream at the specified position. * @param relativePos the relative position in the remote storage to read from. - * @return the input stream ready to be read from. + * @param listener listener for the input stream ready to be read from. */ - InputStream create(int relativePos) throws IOException; + void create(int relativePos, ActionListener listener) throws IOException; } private abstract static class DelegatingRangeMissingHandler implements RangeMissingHandler { @@ -1367,9 +1407,10 @@ public void fillCacheRange( SourceInputStreamFactory streamFactory, int relativePos, int length, - IntConsumer progressUpdater + IntConsumer progressUpdater, + ActionListener completionListener ) throws IOException { - delegate.fillCacheRange(channel, channelPos, streamFactory, relativePos, length, progressUpdater); + delegate.fillCacheRange(channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener); } } diff --git a/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java b/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java index e477673c90d6d..6c49b50c06e82 100644 --- a/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java +++ b/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.StoppableExecutorServiceWrapper; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.TestEnvironment; @@ -72,6 +73,13 @@ private static long size(long numPages) { return numPages * SharedBytes.PAGE_SIZE; } + private static void completeWith(ActionListener listener, CheckedRunnable runnable) { + ActionListener.completeWith(listener, () -> { + runnable.run(); + return null; + }); + } + public void testBasicEviction() throws IOException { Settings settings = Settings.builder() .put(NODE_NAME_SETTING.getKey(), "node") @@ -115,7 +123,10 @@ public void testBasicEviction() throws IOException { ByteRange.of(0L, 1L), ByteRange.of(0L, 1L), (channel, channelPos, relativePos, length) -> 1, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> progressUpdater.accept(length), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> progressUpdater.accept(length) + ), taskQueue.getThreadPool().generic(), bytesReadFuture ); @@ -552,11 +563,14 @@ public void execute(Runnable command) { cacheService.maybeFetchFullEntry( cacheKey, size, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - assert streamFactory == null : streamFactory; - bytesRead.addAndGet(-length); - progressUpdater.accept(length); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + assert streamFactory == null : streamFactory; + bytesRead.addAndGet(-length); + progressUpdater.accept(length); + } + ), bulkExecutor, future ); @@ -570,9 +584,15 @@ public void execute(Runnable command) { // a download that would use up all regions should not run final var cacheKey = generateCacheKey(); assertEquals(2, cacheService.freeRegionCount()); - var configured = cacheService.maybeFetchFullEntry(cacheKey, size(500), (ch, chPos, streamFactory, relPos, len, update) -> { - throw new AssertionError("Should never reach here"); - }, bulkExecutor, ActionListener.noop()); + var configured = cacheService.maybeFetchFullEntry( + cacheKey, + size(500), + (ch, chPos, streamFactory, relPos, len, update, completionListener) -> completeWith(completionListener, () -> { + throw new AssertionError("Should never reach here"); + }), + bulkExecutor, + ActionListener.noop() + ); assertFalse(configured); assertEquals(2, cacheService.freeRegionCount()); } @@ -613,9 +633,14 @@ public void testFetchFullCacheEntryConcurrently() throws Exception { (ActionListener listener) -> cacheService.maybeFetchFullEntry( cacheKey, size, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> progressUpdater.accept( - length - ), + ( + channel, + channelPos, + streamFactory, + relativePos, + length, + progressUpdater, + completionListener) -> completeWith(completionListener, () -> progressUpdater.accept(length)), bulkExecutor, listener ) @@ -859,7 +884,10 @@ public void testMaybeEvictLeastUsed() throws Exception { var entry = cacheService.get(cacheKey, regionSize, 0); entry.populate( ByteRange.of(0L, regionSize), - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> progressUpdater.accept(length), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> progressUpdater.accept(length) + ), taskQueue.getThreadPool().generic(), ActionListener.noop() ); @@ -954,11 +982,14 @@ public void execute(Runnable command) { cacheKey, 0, blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - assert streamFactory == null : streamFactory; - bytesRead.addAndGet(length); - progressUpdater.accept(length); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + assert streamFactory == null : streamFactory; + bytesRead.addAndGet(length); + progressUpdater.accept(length); + } + ), bulkExecutor, future ); @@ -985,11 +1016,14 @@ public void execute(Runnable command) { cacheKey, region, blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - assert streamFactory == null : streamFactory; - bytesRead.addAndGet(length); - progressUpdater.accept(length); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + assert streamFactory == null : streamFactory; + bytesRead.addAndGet(length); + progressUpdater.accept(length); + } + ), bulkExecutor, listener ); @@ -1010,9 +1044,12 @@ public void execute(Runnable command) { cacheKey, randomIntBetween(0, 10), randomLongBetween(1L, regionSize), - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - throw new AssertionError("should not be executed"); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + throw new AssertionError("should not be executed"); + } + ), bulkExecutor, future ); @@ -1032,11 +1069,14 @@ public void execute(Runnable command) { cacheKey, 0, blobLength, - (channel, channelPos, ignore, relativePos, length, progressUpdater) -> { - assert ignore == null : ignore; - bytesRead.addAndGet(length); - progressUpdater.accept(length); - }, + (channel, channelPos, ignore, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + assert ignore == null : ignore; + bytesRead.addAndGet(length); + progressUpdater.accept(length); + } + ), bulkExecutor, future ); @@ -1110,12 +1150,15 @@ public void execute(Runnable command) { region, range, blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - assertThat(range.start() + relativePos, equalTo(cacheService.getRegionStart(region) + regionRange.start())); - assertThat(channelPos, equalTo(Math.toIntExact(regionRange.start()))); - assertThat(length, equalTo(Math.toIntExact(regionRange.length()))); - bytesCopied.addAndGet(length); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + assertThat(range.start() + relativePos, equalTo(cacheService.getRegionStart(region) + regionRange.start())); + assertThat(channelPos, equalTo(Math.toIntExact(regionRange.start()))); + assertThat(length, equalTo(Math.toIntExact(regionRange.length()))); + bytesCopied.addAndGet(length); + } + ), bulkExecutor, future ); @@ -1150,7 +1193,10 @@ public void execute(Runnable command) { region, ByteRange.of(0L, blobLength), blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> bytesCopied.addAndGet(length), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> bytesCopied.addAndGet(length) + ), bulkExecutor, listener ); @@ -1173,9 +1219,12 @@ public void execute(Runnable command) { randomIntBetween(0, 10), ByteRange.of(0L, blobLength), blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - throw new AssertionError("should not be executed"); - }, + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + throw new AssertionError("should not be executed"); + } + ), bulkExecutor, future ); @@ -1196,7 +1245,10 @@ public void execute(Runnable command) { 0, ByteRange.of(0L, blobLength), blobLength, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> bytesCopied.addAndGet(length), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> bytesCopied.addAndGet(length) + ), bulkExecutor, future ); @@ -1237,10 +1289,18 @@ public void testPopulate() throws Exception { var entry = cacheService.get(cacheKey, blobLength, 0); AtomicLong bytesWritten = new AtomicLong(0L); final PlainActionFuture future1 = new PlainActionFuture<>(); - entry.populate(ByteRange.of(0, regionSize - 1), (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - bytesWritten.addAndGet(length); - progressUpdater.accept(length); - }, taskQueue.getThreadPool().generic(), future1); + entry.populate( + ByteRange.of(0, regionSize - 1), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + bytesWritten.addAndGet(length); + progressUpdater.accept(length); + } + ), + taskQueue.getThreadPool().generic(), + future1 + ); assertThat(future1.isDone(), is(false)); assertThat(taskQueue.hasRunnableTasks(), is(true)); @@ -1248,18 +1308,34 @@ public void testPopulate() throws Exception { // start populating the second region entry = cacheService.get(cacheKey, blobLength, 1); final PlainActionFuture future2 = new PlainActionFuture<>(); - entry.populate(ByteRange.of(0, regionSize - 1), (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - bytesWritten.addAndGet(length); - progressUpdater.accept(length); - }, taskQueue.getThreadPool().generic(), future2); + entry.populate( + ByteRange.of(0, regionSize - 1), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + bytesWritten.addAndGet(length); + progressUpdater.accept(length); + } + ), + taskQueue.getThreadPool().generic(), + future2 + ); // start populating again the first region, listener should be called immediately entry = cacheService.get(cacheKey, blobLength, 0); final PlainActionFuture future3 = new PlainActionFuture<>(); - entry.populate(ByteRange.of(0, regionSize - 1), (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> { - bytesWritten.addAndGet(length); - progressUpdater.accept(length); - }, taskQueue.getThreadPool().generic(), future3); + entry.populate( + ByteRange.of(0, regionSize - 1), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> { + bytesWritten.addAndGet(length); + progressUpdater.accept(length); + } + ), + taskQueue.getThreadPool().generic(), + future3 + ); assertThat(future3.isDone(), is(true)); var written = future3.get(10L, TimeUnit.SECONDS); @@ -1377,7 +1453,10 @@ public void testSharedSourceInputStreamFactory() throws Exception { range, range, (channel, channelPos, relativePos, length) -> length, - (channel, channelPos, streamFactory, relativePos, length, progressUpdater) -> progressUpdater.accept(length), + (channel, channelPos, streamFactory, relativePos, length, progressUpdater, completionListener) -> completeWith( + completionListener, + () -> progressUpdater.accept(length) + ), EsExecutors.DIRECT_EXECUTOR_SERVICE, future ); @@ -1394,8 +1473,8 @@ public void testSharedSourceInputStreamFactory() throws Exception { final var factoryClosed = new AtomicBoolean(false); final var dummyStreamFactory = new SourceInputStreamFactory() { @Override - public InputStream create(int relativePos) { - return null; + public void create(int relativePos, ActionListener listener) { + listener.onResponse(null); } @Override @@ -1420,17 +1499,20 @@ public void fillCacheRange( SourceInputStreamFactory streamFactory, int relativePos, int length, - IntConsumer progressUpdater + IntConsumer progressUpdater, + ActionListener completion ) throws IOException { - if (invocationCounter.incrementAndGet() == 1) { - final Thread witness = invocationThread.compareAndExchange(null, Thread.currentThread()); - assertThat(witness, nullValue()); - } else { - assertThat(invocationThread.get(), sameInstance(Thread.currentThread())); - } - assertThat(streamFactory, sameInstance(dummyStreamFactory)); - assertThat(position.getAndSet(relativePos), lessThan(relativePos)); - progressUpdater.accept(length); + completeWith(completion, () -> { + if (invocationCounter.incrementAndGet() == 1) { + final Thread witness = invocationThread.compareAndExchange(null, Thread.currentThread()); + assertThat(witness, nullValue()); + } else { + assertThat(invocationThread.get(), sameInstance(Thread.currentThread())); + } + assertThat(streamFactory, sameInstance(dummyStreamFactory)); + assertThat(position.getAndSet(relativePos), lessThan(relativePos)); + progressUpdater.accept(length); + }); } }; diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInput.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInput.java index 56efc72f2f6f7..d7cf22a05981f 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInput.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInput.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.blobcache.BlobCacheUtils; import org.elasticsearch.blobcache.common.ByteBufferReference; import org.elasticsearch.blobcache.common.ByteRange; @@ -146,32 +147,38 @@ private void readWithoutBlobCacheSlow(ByteBuffer b, long position, int length) t final int read = SharedBytes.readCacheFile(channel, pos, relativePos, len, byteBufferReference); stats.addCachedBytesRead(read); return read; - }, (channel, channelPos, streamFactory, relativePos, len, progressUpdater) -> { - assert streamFactory == null : streamFactory; - final long startTimeNanos = stats.currentTimeNanos(); - try (InputStream input = openInputStreamFromBlobStore(rangeToWrite.start() + relativePos, len)) { - assert ThreadPool.assertCurrentThreadPool(SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME); - logger.trace( - "{}: writing channel {} pos {} length {} (details: {})", - fileInfo.physicalName(), - channelPos, - relativePos, - len, - cacheFile - ); - SharedBytes.copyToCacheFileAligned( - channel, - input, - channelPos, - relativePos, - len, - progressUpdater, - writeBuffer.get().clear() - ); - final long endTimeNanos = stats.currentTimeNanos(); - stats.addCachedBytesWritten(len, endTimeNanos - startTimeNanos); - } - }); + }, + (channel, channelPos, streamFactory, relativePos, len, progressUpdater, completionListener) -> ActionListener.completeWith( + completionListener, + () -> { + assert streamFactory == null : streamFactory; + final long startTimeNanos = stats.currentTimeNanos(); + try (InputStream input = openInputStreamFromBlobStore(rangeToWrite.start() + relativePos, len)) { + assert ThreadPool.assertCurrentThreadPool(SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME); + logger.trace( + "{}: writing channel {} pos {} length {} (details: {})", + fileInfo.physicalName(), + channelPos, + relativePos, + len, + cacheFile + ); + SharedBytes.copyToCacheFileAligned( + channel, + input, + channelPos, + relativePos, + len, + progressUpdater, + writeBuffer.get().clear() + ); + final long endTimeNanos = stats.currentTimeNanos(); + stats.addCachedBytesWritten(len, endTimeNanos - startTimeNanos); + return null; + } + } + ) + ); assert bytesRead == length : bytesRead + " vs " + length; byteBufferReference.finish(bytesRead); } finally { From cf034c03df532ef353ff5d09f3cccbf109af53d6 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 19 Aug 2024 10:55:34 -0400 Subject: [PATCH 13/15] Add a new random rerank retriever (#111851) * Add a new random rerank retriever, that reranks results in random order without requiring inference * Update docs/changelog/111851.yaml * PR feedback - remove null checks for field as it can never be null * Update docs * Revert "Update docs" This reverts commit 3d61676e8c9ab76472f824554efd607ddd1c5678. * Remove minScore * Random seed * Delete docs/changelog/111851.yaml * PR feedback * Add optional seed to request, YAML test * PR feedback --- .../org/elasticsearch/TransportVersions.java | 1 + .../xpack/inference/InferenceFeatures.java | 6 +- .../xpack/inference/InferencePlugin.java | 6 +- .../rank/random/RandomRankBuilder.java | 165 ++++++++++++++++++ ...ankFeaturePhaseRankCoordinatorContext.java | 55 ++++++ .../random/RandomRankRetrieverBuilder.java | 124 +++++++++++++ .../rank/random/RandomRankBuilderTests.java | 70 ++++++++ .../RandomRankRetrieverBuilderTests.java | 104 +++++++++++ .../inference/80_random_rerank_retriever.yml | 94 ++++++++++ 9 files changed, 623 insertions(+), 2 deletions(-) create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilder.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankFeaturePhaseRankCoordinatorContext.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilder.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilderTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilderTests.java create mode 100644 x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/80_random_rerank_retriever.yml diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index fd3a3d8672966..1009d9e2ae7d1 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -191,6 +191,7 @@ static TransportVersion def(int id) { public static final TransportVersion INGEST_PIPELINE_EXCEPTION_ADDED = def(8_721_00_0); public static final TransportVersion ZDT_NANOS_SUPPORT = def(8_722_00_0); public static final TransportVersion REMOVE_GLOBAL_RETENTION_FROM_TEMPLATES = def(8_723_00_0); + public static final TransportVersion RANDOM_RERANKER_RETRIEVER = def(8_724_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java index 4cc7f5b502ba9..12a32ecdc6d4f 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java @@ -9,6 +9,7 @@ import org.elasticsearch.features.FeatureSpecification; import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.xpack.inference.rank.random.RandomRankRetrieverBuilder; import org.elasticsearch.xpack.inference.rank.textsimilarity.TextSimilarityRankRetrieverBuilder; import java.util.Set; @@ -20,7 +21,10 @@ public class InferenceFeatures implements FeatureSpecification { @Override public Set getFeatures() { - return Set.of(TextSimilarityRankRetrieverBuilder.TEXT_SIMILARITY_RERANKER_RETRIEVER_SUPPORTED); + return Set.of( + TextSimilarityRankRetrieverBuilder.TEXT_SIMILARITY_RERANKER_RETRIEVER_SUPPORTED, + RandomRankRetrieverBuilder.RANDOM_RERANKER_RETRIEVER_SUPPORTED + ); } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java index f6d4a9f774a91..9d85bbf751250 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java @@ -63,6 +63,8 @@ import org.elasticsearch.xpack.inference.logging.ThrottlerManager; import org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper; import org.elasticsearch.xpack.inference.queries.SemanticQueryBuilder; +import org.elasticsearch.xpack.inference.rank.random.RandomRankBuilder; +import org.elasticsearch.xpack.inference.rank.random.RandomRankRetrieverBuilder; import org.elasticsearch.xpack.inference.rank.textsimilarity.TextSimilarityRankBuilder; import org.elasticsearch.xpack.inference.rank.textsimilarity.TextSimilarityRankRetrieverBuilder; import org.elasticsearch.xpack.inference.registry.ModelRegistry; @@ -243,6 +245,7 @@ public List getInferenceServiceFactories() { public List getNamedWriteables() { var entries = new ArrayList<>(InferenceNamedWriteablesProvider.getNamedWriteables()); entries.add(new NamedWriteableRegistry.Entry(RankBuilder.class, TextSimilarityRankBuilder.NAME, TextSimilarityRankBuilder::new)); + entries.add(new NamedWriteableRegistry.Entry(RankBuilder.class, RandomRankBuilder.NAME, RandomRankBuilder::new)); return entries; } @@ -336,7 +339,8 @@ public List> getQueries() { @Override public List> getRetrievers() { return List.of( - new RetrieverSpec<>(new ParseField(TextSimilarityRankBuilder.NAME), TextSimilarityRankRetrieverBuilder::fromXContent) + new RetrieverSpec<>(new ParseField(TextSimilarityRankBuilder.NAME), TextSimilarityRankRetrieverBuilder::fromXContent), + new RetrieverSpec<>(new ParseField(RandomRankBuilder.NAME), RandomRankRetrieverBuilder::fromXContent) ); } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilder.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilder.java new file mode 100644 index 0000000000000..fdb5503e491eb --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilder.java @@ -0,0 +1,165 @@ +/* + * 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.inference.rank.random; + +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.Query; +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.rank.RankBuilder; +import org.elasticsearch.search.rank.RankDoc; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; +import org.elasticsearch.search.rank.context.RankFeaturePhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.RankFeaturePhaseRankShardContext; +import org.elasticsearch.search.rank.feature.RankFeatureDoc; +import org.elasticsearch.search.rank.rerank.RerankingQueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.rerank.RerankingQueryPhaseRankShardContext; +import org.elasticsearch.search.rank.rerank.RerankingRankFeaturePhaseRankShardContext; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; +import static org.elasticsearch.xpack.inference.rank.random.RandomRankRetrieverBuilder.FIELD_FIELD; +import static org.elasticsearch.xpack.inference.rank.random.RandomRankRetrieverBuilder.SEED_FIELD; + +/** + * A {@code RankBuilder} that performs reranking with random scores, used for testing. + */ +public class RandomRankBuilder extends RankBuilder { + + public static final String NAME = "random_reranker"; + + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, args -> { + Integer rankWindowSize = args[0] == null ? DEFAULT_RANK_WINDOW_SIZE : (Integer) args[0]; + String field = (String) args[1]; + Integer seed = (Integer) args[2]; + + return new RandomRankBuilder(rankWindowSize, field, seed); + }); + + static { + PARSER.declareInt(optionalConstructorArg(), RANK_WINDOW_SIZE_FIELD); + PARSER.declareString(constructorArg(), FIELD_FIELD); + PARSER.declareInt(optionalConstructorArg(), SEED_FIELD); + } + + private final String field; + private final Integer seed; + + public RandomRankBuilder(int rankWindowSize, String field, Integer seed) { + super(rankWindowSize); + + if (field == null || field.isEmpty()) { + throw new IllegalArgumentException("field is required"); + } + + this.field = field; + this.seed = seed; + } + + public RandomRankBuilder(StreamInput in) throws IOException { + super(in); + // rankWindowSize deserialization is handled by the parent class RankBuilder + this.field = in.readString(); + this.seed = in.readOptionalInt(); + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return TransportVersions.RANDOM_RERANKER_RETRIEVER; + } + + @Override + public void doWriteTo(StreamOutput out) throws IOException { + // rankWindowSize serialization is handled by the parent class RankBuilder + out.writeString(field); + out.writeOptionalInt(seed); + } + + @Override + public void doXContent(XContentBuilder builder, Params params) throws IOException { + // rankWindowSize serialization is handled by the parent class RankBuilder + builder.field(FIELD_FIELD.getPreferredName(), field); + if (seed != null) { + builder.field(SEED_FIELD.getPreferredName(), seed); + } + } + + @Override + public boolean isCompoundBuilder() { + return false; + } + + @Override + public Explanation explainHit(Explanation baseExplanation, RankDoc scoreDoc, List queryNames) { + if (scoreDoc == null) { + return baseExplanation; + } + if (false == baseExplanation.isMatch()) { + return baseExplanation; + } + + assert scoreDoc instanceof RankFeatureDoc : "ScoreDoc is not an instance of RankFeatureDoc"; + RankFeatureDoc rankFeatureDoc = (RankFeatureDoc) scoreDoc; + + return Explanation.match( + rankFeatureDoc.score, + "rank after reranking: [" + rankFeatureDoc.rank + "] using seed [" + seed + "] with score: [" + rankFeatureDoc.score + "]", + baseExplanation + ); + } + + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new RerankingQueryPhaseRankShardContext(queries, rankWindowSize()); + } + + @Override + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new RerankingQueryPhaseRankCoordinatorContext(rankWindowSize()); + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RerankingRankFeaturePhaseRankShardContext(field); + } + + @Override + public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext(int size, int from, Client client) { + return new RandomRankFeaturePhaseRankCoordinatorContext(size, from, rankWindowSize(), seed); + } + + public String field() { + return field; + } + + @Override + protected boolean doEquals(RankBuilder other) { + RandomRankBuilder that = (RandomRankBuilder) other; + return Objects.equals(field, that.field) && Objects.equals(seed, that.seed); + } + + @Override + protected int doHashCode() { + return Objects.hash(field, seed); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankFeaturePhaseRankCoordinatorContext.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankFeaturePhaseRankCoordinatorContext.java new file mode 100644 index 0000000000000..446d8e5862dd2 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankFeaturePhaseRankCoordinatorContext.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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.rank.random; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.search.rank.context.RankFeaturePhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.feature.RankFeatureDoc; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Random; + +/** + * A {@code RankFeaturePhaseRankCoordinatorContext} that performs a rerank inference call to determine relevance scores for documents within + * the provided rank window. + */ +public class RandomRankFeaturePhaseRankCoordinatorContext extends RankFeaturePhaseRankCoordinatorContext { + + private final Integer seed; + + public RandomRankFeaturePhaseRankCoordinatorContext(int size, int from, int rankWindowSize, Integer seed) { + super(size, from, rankWindowSize); + this.seed = seed; + } + + @Override + protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { + // Generate random scores seeded by doc + float[] scores = new float[featureDocs.length]; + for (int i = 0; i < featureDocs.length; i++) { + RankFeatureDoc featureDoc = featureDocs[i]; + int doc = featureDoc.doc; + long docSeed = seed != null ? seed + doc : doc; + scores[i] = new Random(docSeed).nextFloat(); + } + scoreListener.onResponse(scores); + } + + /** + * Sorts documents by score descending. + * @param originalDocs documents to process + */ + @Override + protected RankFeatureDoc[] preprocess(RankFeatureDoc[] originalDocs) { + return Arrays.stream(originalDocs) + .sorted(Comparator.comparing((RankFeatureDoc doc) -> doc.score).reversed()) + .toArray(RankFeatureDoc[]::new); + } + +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilder.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilder.java new file mode 100644 index 0000000000000..ab8c85cac00e3 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilder.java @@ -0,0 +1,124 @@ +/* + * 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.inference.rank.random; + +import org.elasticsearch.common.ParsingException; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.retriever.RetrieverBuilder; +import org.elasticsearch.search.retriever.RetrieverParserContext; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.search.rank.RankBuilder.DEFAULT_RANK_WINDOW_SIZE; +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; + +/** + * A {@code RetrieverBuilder} for parsing and constructing a text similarity reranker retriever. + */ +public class RandomRankRetrieverBuilder extends RetrieverBuilder { + + public static final NodeFeature RANDOM_RERANKER_RETRIEVER_SUPPORTED = new NodeFeature("random_reranker_retriever_supported"); + + public static final ParseField RETRIEVER_FIELD = new ParseField("retriever"); + public static final ParseField FIELD_FIELD = new ParseField("field"); + public static final ParseField RANK_WINDOW_SIZE_FIELD = new ParseField("rank_window_size"); + public static final ParseField SEED_FIELD = new ParseField("seed"); + + public static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>(RandomRankBuilder.NAME, args -> { + RetrieverBuilder retrieverBuilder = (RetrieverBuilder) args[0]; + String field = (String) args[1]; + int rankWindowSize = args[2] == null ? DEFAULT_RANK_WINDOW_SIZE : (int) args[2]; + Integer seed = (Integer) args[3]; + + return new RandomRankRetrieverBuilder(retrieverBuilder, field, rankWindowSize, seed); + }); + + static { + PARSER.declareNamedObject(constructorArg(), (p, c, n) -> p.namedObject(RetrieverBuilder.class, n, c), RETRIEVER_FIELD); + PARSER.declareString(optionalConstructorArg(), FIELD_FIELD); + PARSER.declareInt(optionalConstructorArg(), RANK_WINDOW_SIZE_FIELD); + PARSER.declareInt(optionalConstructorArg(), SEED_FIELD); + + RetrieverBuilder.declareBaseParserFields(RandomRankBuilder.NAME, PARSER); + } + + public static RandomRankRetrieverBuilder fromXContent(XContentParser parser, RetrieverParserContext context) throws IOException { + if (context.clusterSupportsFeature(RANDOM_RERANKER_RETRIEVER_SUPPORTED) == false) { + throw new ParsingException(parser.getTokenLocation(), "unknown retriever [" + RandomRankBuilder.NAME + "]"); + } + return PARSER.apply(parser, context); + } + + private final RetrieverBuilder retrieverBuilder; + private final String field; + private final int rankWindowSize; + private final Integer seed; + + public RandomRankRetrieverBuilder(RetrieverBuilder retrieverBuilder, String field, int rankWindowSize, Integer seed) { + this.retrieverBuilder = retrieverBuilder; + this.field = field; + this.rankWindowSize = rankWindowSize; + this.seed = seed; + } + + @Override + public void extractToSearchSourceBuilder(SearchSourceBuilder searchSourceBuilder, boolean compoundUsed) { + retrieverBuilder.extractToSearchSourceBuilder(searchSourceBuilder, compoundUsed); + + // Combining with other rank builder (such as RRF) is not supported + if (searchSourceBuilder.rankBuilder() != null) { + throw new IllegalArgumentException("random rank builder cannot be combined with other rank builders"); + } + + searchSourceBuilder.rankBuilder(new RandomRankBuilder(this.rankWindowSize, this.field, this.seed)); + } + + @Override + public String getName() { + return RandomRankBuilder.NAME; + } + + public int rankWindowSize() { + return rankWindowSize; + } + + @Override + protected void doToXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(RETRIEVER_FIELD.getPreferredName()); + builder.startObject(); + builder.field(retrieverBuilder.getName(), retrieverBuilder); + builder.endObject(); + builder.field(FIELD_FIELD.getPreferredName(), field); + builder.field(RANK_WINDOW_SIZE_FIELD.getPreferredName(), rankWindowSize); + if (seed != null) { + builder.field(SEED_FIELD.getPreferredName(), seed); + } + } + + @Override + protected boolean doEquals(Object other) { + RandomRankRetrieverBuilder that = (RandomRankRetrieverBuilder) other; + return Objects.equals(retrieverBuilder, that.retrieverBuilder) + && Objects.equals(field, that.field) + && Objects.equals(rankWindowSize, that.rankWindowSize) + && Objects.equals(seed, that.seed); + } + + @Override + protected int doHashCode() { + return Objects.hash(retrieverBuilder, field, rankWindowSize, seed); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilderTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilderTests.java new file mode 100644 index 0000000000000..c464dbaea47cd --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankBuilderTests.java @@ -0,0 +1,70 @@ +/* + * 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.inference.rank.random; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractXContentSerializingTestCase; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.json.JsonXContent; + +import java.io.IOException; + +import static org.elasticsearch.search.rank.RankBuilder.DEFAULT_RANK_WINDOW_SIZE; + +public class RandomRankBuilderTests extends AbstractXContentSerializingTestCase { + + @Override + protected RandomRankBuilder createTestInstance() { + return new RandomRankBuilder(randomIntBetween(1, 1000), "my-field", randomBoolean() ? randomIntBetween(1, 1000) : null); + } + + @Override + protected RandomRankBuilder mutateInstance(RandomRankBuilder instance) throws IOException { + String field = instance.field() + randomAlphaOfLength(2); + int rankWindowSize = randomValueOtherThan(instance.rankWindowSize(), this::randomRankWindowSize); + Integer seed = randomBoolean() ? randomIntBetween(1, 1000) : null; + return new RandomRankBuilder(rankWindowSize, field, seed); + } + + @Override + protected Writeable.Reader instanceReader() { + return RandomRankBuilder::new; + } + + @Override + protected RandomRankBuilder doParseInstance(XContentParser parser) throws IOException { + parser.nextToken(); + assertEquals(parser.currentToken(), XContentParser.Token.START_OBJECT); + parser.nextToken(); + assertEquals(parser.currentToken(), XContentParser.Token.FIELD_NAME); + assertEquals(parser.currentName(), RandomRankBuilder.NAME); + RandomRankBuilder builder = RandomRankBuilder.PARSER.parse(parser, null); + parser.nextToken(); + assertEquals(parser.currentToken(), XContentParser.Token.END_OBJECT); + parser.nextToken(); + assertNull(parser.currentToken()); + return builder; + } + + private int randomRankWindowSize() { + return randomIntBetween(0, 1000); + } + + public void testParserDefaults() throws IOException { + String json = """ + { + "field": "my-field" + }"""; + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, json)) { + RandomRankBuilder parsed = RandomRankBuilder.PARSER.parse(parser, null); + assertEquals(DEFAULT_RANK_WINDOW_SIZE, parsed.rankWindowSize()); + } + } + +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilderTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilderTests.java new file mode 100644 index 0000000000000..c33f30d461350 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rank/random/RandomRankRetrieverBuilderTests.java @@ -0,0 +1,104 @@ +/* + * 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.inference.rank.random; + +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.search.retriever.RetrieverBuilder; +import org.elasticsearch.search.retriever.RetrieverParserContext; +import org.elasticsearch.search.retriever.TestRetrieverBuilder; +import org.elasticsearch.test.AbstractXContentTestCase; +import org.elasticsearch.usage.SearchUsage; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.inference.rank.textsimilarity.TextSimilarityRankBuilder; +import org.elasticsearch.xpack.inference.rank.textsimilarity.TextSimilarityRankRetrieverBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.search.rank.RankBuilder.DEFAULT_RANK_WINDOW_SIZE; + +public class RandomRankRetrieverBuilderTests extends AbstractXContentTestCase { + + /** + * Creates a random {@link RandomRankRetrieverBuilder}. The created instance + * is not guaranteed to pass {@link SearchRequest} validation. This is purely + * for x-content testing. + */ + public static RandomRankRetrieverBuilder createRandomRankRetrieverBuilder() { + return new RandomRankRetrieverBuilder( + TestRetrieverBuilder.createRandomTestRetrieverBuilder(), + randomAlphaOfLength(10), + randomIntBetween(1, 10000), + randomBoolean() ? randomIntBetween(1, 1000) : null + ); + } + + @Override + protected RandomRankRetrieverBuilder createTestInstance() { + return createRandomRankRetrieverBuilder(); + } + + @Override + protected RandomRankRetrieverBuilder doParseInstance(XContentParser parser) { + return RandomRankRetrieverBuilder.PARSER.apply( + parser, + new RetrieverParserContext( + new SearchUsage(), + nf -> nf == RetrieverBuilder.RETRIEVERS_SUPPORTED || nf == RandomRankRetrieverBuilder.RANDOM_RERANKER_RETRIEVER_SUPPORTED + ) + ); + } + + @Override + protected boolean supportsUnknownFields() { + return false; + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + List entries = new ArrayList<>(); + entries.add( + new NamedXContentRegistry.Entry( + RetrieverBuilder.class, + TestRetrieverBuilder.TEST_SPEC.getName(), + (p, c) -> TestRetrieverBuilder.TEST_SPEC.getParser().fromXContent(p, (RetrieverParserContext) c), + TestRetrieverBuilder.TEST_SPEC.getName().getForRestApiVersion() + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + RetrieverBuilder.class, + new ParseField(TextSimilarityRankBuilder.NAME), + (p, c) -> TextSimilarityRankRetrieverBuilder.PARSER.apply(p, (RetrieverParserContext) c) + ) + ); + return new NamedXContentRegistry(entries); + } + + public void testParserDefaults() throws IOException { + String json = """ + { + "retriever": { + "test": { + "value": "my-test-retriever" + } + }, + "field": "my-field" + }"""; + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, json)) { + RandomRankRetrieverBuilder parsed = RandomRankRetrieverBuilder.PARSER.parse(parser, null); + assertEquals(DEFAULT_RANK_WINDOW_SIZE, parsed.rankWindowSize()); + } + } + +} diff --git a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/80_random_rerank_retriever.yml b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/80_random_rerank_retriever.yml new file mode 100644 index 0000000000000..d33f57f763db8 --- /dev/null +++ b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/80_random_rerank_retriever.yml @@ -0,0 +1,94 @@ +setup: + - requires: + cluster_features: "gte_v8.16.0" + reason: random rerank retriever introduced in 8.16.0 + test_runner_features: "close_to" + + - do: + indices.create: + index: test-index + body: + mappings: + properties: + text: + type: text + topic: + type: keyword + subtopic: + type: keyword + + - do: + bulk: + refresh: true + index: test-index + body: | + {"index": { "_id": "doc_1" } } + { "text": "Pugs are proof that even nature has a sense of humor." } + {"index": { "_id": "doc_2" } } + { "text": "A pugs snore can rival a chainsaw, but it's somehow adorable." } + {"index": { "_id": "doc_3" } } + { "text": "Pugs are like potato chips; you can't have just one wrinkle." } + {"index": { "_id": "doc_4" } } + { "text": "Pugs don't walk; pugs waddle majestically." } + {"index": { "_id": "doc_5" } } + { "text": "A pugs life goal: be the ultimate couch potato, and they're crushing it." } +--- +"Random rerank retriever predictably shuffles results": + + - do: + search: + index: test-index + body: + query: + query_string: + query: "pugs" + size: 10 + + - match: { hits.total.value: 5 } + - length: { hits.hits: 5 } + + - match: { hits.hits.0._id: "doc_4" } + - close_to: { hits.hits.0._score: { value: 0.136, error: 0.001 } } + + - do: + search: + index: test-index + body: + retriever: + random_reranker: + retriever: + standard: + query: + query_string: + query: "pugs" + field: text + seed: 42 + rank_window_size: 10 + size: 10 + + - match: { hits.total.value: 5 } + - length: { hits.hits: 5 } + + - match: { hits.hits.0._id: "doc_1" } + - close_to: { hits.hits.0._score: { value: 0.727, error: 0.001 } } + + - do: + search: + index: test-index + body: + retriever: + random_reranker: + retriever: + standard: + query: + query_string: + query: "pugs" + field: text + rank_window_size: 10 + size: 10 + + - match: { hits.total.value: 5 } + - length: { hits.hits: 5 } + + - match: { hits.hits.0._id: "doc_3" } + - close_to: { hits.hits.0._score: { value: 0.731, error: 0.001 } } From ba8590ba13b898909eb2418671478d9e9643e09d Mon Sep 17 00:00:00 2001 From: Quentin Pradet Date: Mon, 19 Aug 2024 18:57:39 +0400 Subject: [PATCH 14/15] Add analysis-common YAML tests to rest-resources-zip (#111974) --- modules/analysis-common/build.gradle | 3 +++ .../test/indices.analyze/{10_analyze.yml => 15_analyze.yml} | 0 x-pack/rest-resources-zip/build.gradle | 1 + 3 files changed, 4 insertions(+) rename modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/indices.analyze/{10_analyze.yml => 15_analyze.yml} (100%) diff --git a/modules/analysis-common/build.gradle b/modules/analysis-common/build.gradle index 77fd095806d10..1fc42a1b294fe 100644 --- a/modules/analysis-common/build.gradle +++ b/modules/analysis-common/build.gradle @@ -36,3 +36,6 @@ tasks.named("yamlRestTestV7CompatTransform").configure { task -> task.skipTest("search.query/50_queries_with_synonyms/Test common terms query with stacked tokens", "#42654 - `common` query throws an exception") } +artifacts { + restTests(new File(projectDir, "src/yamlRestTest/resources/rest-api-spec/test")) +} diff --git a/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/indices.analyze/10_analyze.yml b/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/indices.analyze/15_analyze.yml similarity index 100% rename from modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/indices.analyze/10_analyze.yml rename to modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/indices.analyze/15_analyze.yml diff --git a/x-pack/rest-resources-zip/build.gradle b/x-pack/rest-resources-zip/build.gradle index 3d0533b4ec57e..cc5bddf12d801 100644 --- a/x-pack/rest-resources-zip/build.gradle +++ b/x-pack/rest-resources-zip/build.gradle @@ -20,6 +20,7 @@ dependencies { apis project(path: ':rest-api-spec', configuration: 'restSpecs') freeTests project(path: ':rest-api-spec', configuration: 'restTests') freeTests project(path: ':modules:aggregations', configuration: 'restTests') + freeTests project(path: ':modules:analysis-common', configuration: 'restTests') compatApis project(path: ':rest-api-spec', configuration: 'restCompatSpecs') compatApis project(path: ':x-pack:plugin', configuration: 'restCompatSpecs') freeCompatTests project(path: ':rest-api-spec', configuration: 'restCompatTests') From aa959e69cc507a16f7f725240db2e7453c0a8320 Mon Sep 17 00:00:00 2001 From: Luigi Dell'Aquila Date: Mon, 19 Aug 2024 17:18:15 +0200 Subject: [PATCH 15/15] ES|QL: shorten error messages for UnsupportedAttributes (#111973) When dealing with index patterns, eg. `FROM logs-*`, some fields can have the same name but different types in different indices. In this case we build an error message like ``` Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types: [ip] in [test1, test2], [keyword] in [test3]" ``` With this PR, in case of many indices involved, we avoid listing them all, but we only list three of them and provide information about how many other indices are affected, eg. ``` Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types: [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6] ``` (see the `and [2] other indices`) Since these error messages are stored in `UnspportedAttributes` and serialized, this PR reduces significantly the size of a serialized execution plan with many type conflicts. Fixes https://github.com/elastic/elasticsearch/issues/111964 Related to https://github.com/elastic/elasticsearch/issues/111358 --- .../esql/core/type/InvalidMappedField.java | 8 +- .../xpack/esql/action/EsqlCapabilities.java | 7 +- .../xpack/esql/analysis/VerifierTests.java | 31 +++-- .../test/esql/51_many_indexes.yml | 126 ++++++++++++++++++ 4 files changed, 156 insertions(+), 16 deletions(-) create mode 100644 x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/51_many_indexes.yml diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/InvalidMappedField.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/InvalidMappedField.java index 9b3d7950c2a01..8b15893f8a056 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/InvalidMappedField.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/InvalidMappedField.java @@ -130,7 +130,13 @@ private static String makeErrorMessage(Map> typesToIndices) errorMessage.append("["); errorMessage.append(e.getKey()); errorMessage.append("] in "); - errorMessage.append(e.getValue()); + if (e.getValue().size() <= 3) { + errorMessage.append(e.getValue()); + } else { + errorMessage.append(e.getValue().stream().sorted().limit(3).collect(Collectors.toList())); + errorMessage.append(" and [" + (e.getValue().size() - 3) + "] other "); + errorMessage.append(e.getValue().size() == 4 ? "index" : "indices"); + } } return errorMessage.toString(); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 3abbb655dadd3..996c5ac2ea319 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -229,7 +229,12 @@ public enum Cap { /** * Consider the upper bound when computing the interval in BUCKET auto mode. */ - BUCKET_INCLUSIVE_UPPER_BOUND; + BUCKET_INCLUSIVE_UPPER_BOUND, + + /** + * Changed error messages for fields with conflicting types in different indices. + */ + SHORT_ERROR_MESSAGES_FOR_UNSUPPORTED_FIELDS; private final boolean snapshotOnly; private final FeatureFlag featureFlag; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 904308ef64d58..9b0c32b8ade2e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -64,9 +64,12 @@ public void testUnsupportedAndMultiTypedFields() { LinkedHashSet ipIndices = new LinkedHashSet<>(); ipIndices.add("test1"); ipIndices.add("test2"); + ipIndices.add("test3"); + ipIndices.add("test4"); + ipIndices.add("test5"); LinkedHashMap> typesToIndices = new LinkedHashMap<>(); typesToIndices.put("ip", ipIndices); - typesToIndices.put("keyword", Set.of("test3")); + typesToIndices.put("keyword", Set.of("test6")); EsField multiTypedField = new InvalidMappedField(multiTyped, typesToIndices); // Also add an unsupported/multityped field under the names `int` and `double` so we can use `LOOKUP int_number_names ...` and @@ -85,7 +88,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:22: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | dissect multi_typed \"%{foo}\"", analyzer) ); @@ -95,7 +98,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:19: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | grok multi_typed \"%{WORD:foo}\"", analyzer) ); @@ -115,7 +118,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:23: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | eval x = multi_typed", analyzer) ); @@ -125,7 +128,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:32: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | eval x = to_lower(multi_typed)", analyzer) ); @@ -135,7 +138,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:32: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | stats count(1) by multi_typed", analyzer) ); if (EsqlCapabilities.Cap.INLINESTATS.isEnabled()) { @@ -145,7 +148,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:38: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | inlinestats count(1) by multi_typed", analyzer) ); } @@ -156,7 +159,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:27: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | stats values(multi_typed)", analyzer) ); if (EsqlCapabilities.Cap.INLINESTATS.isEnabled()) { @@ -166,7 +169,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:33: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | inlinestats values(multi_typed)", analyzer) ); } @@ -177,7 +180,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:27: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | stats values(multi_typed)", analyzer) ); @@ -200,7 +203,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:24: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | mv_expand multi_typed", analyzer) ); @@ -210,7 +213,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:21: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | rename multi_typed as x", analyzer) ); @@ -220,7 +223,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:19: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | sort multi_typed desc", analyzer) ); @@ -230,7 +233,7 @@ public void testUnsupportedAndMultiTypedFields() { ); assertEquals( "1:20: Cannot use field [multi_typed] due to ambiguities being mapped as [2] incompatible types:" - + " [ip] in [test1, test2], [keyword] in [test3]", + + " [ip] in [test1, test2, test3] and [2] other indices, [keyword] in [test6]", error("from test* | where multi_typed is not null", analyzer) ); } diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/51_many_indexes.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/51_many_indexes.yml new file mode 100644 index 0000000000000..eb589cb810cc3 --- /dev/null +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/51_many_indexes.yml @@ -0,0 +1,126 @@ +setup: + - requires: + capabilities: + - method: POST + path: /_query + parameters: [method, path, parameters, capabilities] + capabilities: [short_error_messages_for_unsupported_fields] + reason: "We changed error messages for unsupported fields in v 8.16" + test_runner_features: [capabilities, allowed_warnings_regex] + + - do: + indices.create: + index: ambiguous_1 + body: + mappings: + properties: + "name": + type: keyword + + - do: + indices.create: + index: ambiguous_2 + body: + mappings: + properties: + "name": + type: keyword + + - do: + indices.create: + index: ambiguous_3 + body: + mappings: + properties: + "name": + type: keyword + + - do: + indices.create: + index: ambiguous_4 + body: + mappings: + properties: + "name": + type: integer + + - do: + indices.create: + index: ambiguous_5 + body: + mappings: + properties: + "name": + type: integer + + - do: + indices.create: + index: ambiguous_6 + body: + mappings: + properties: + "name": + type: integer + + - do: + indices.create: + index: ambiguous_7 + body: + mappings: + properties: + "name": + type: integer + + - do: + indices.create: + index: ambiguous_8 + body: + mappings: + properties: + "name": + type: ip + + - do: + indices.create: + index: ambiguous_9 + body: + mappings: + properties: + "name": + type: ip + + - do: + indices.create: + index: ambiguous_10 + body: + mappings: + properties: + "name": + type: ip + + - do: + indices.create: + index: ambiguous_11 + body: + mappings: + properties: + "name": + type: ip + + - do: + indices.create: + index: ambiguous_12 + body: + mappings: + properties: + "name": + type: ip + +--- +load many indices with ambiguities: + - do: + catch: '/Cannot use field \[name\] due to ambiguities being mapped as \[3\] incompatible types: \[integer\] in \[ambiguous_4, ambiguous_5, ambiguous_6\] and \[1\] other index, \[ip\] in \[ambiguous_10, ambiguous_11, ambiguous_12\] and \[2\] other indices, \[keyword\] in \[ambiguous_1, ambiguous_2, ambiguous_3\]/' + esql.query: + body: + query: 'FROM ambiguous* | SORT name' +