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/build-tools-internal/version.properties b/build-tools-internal/version.properties index e22fb9b788fcb..99a135480b97b 100644 --- a/build-tools-internal/version.properties +++ b/build-tools-internal/version.properties @@ -1,5 +1,5 @@ elasticsearch = 8.16.0 -lucene = 9.12.0-snapshot-4a0a09a8bbb +lucene = 9.12.0-snapshot-a9a70fa97cc bundled_jdk_vendor = openjdk bundled_jdk = 22.0.1+8@c7ec1332f7bb44aeba2eb341ae18aca4 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/docs/reference/release-notes/8.15.0.asciidoc b/docs/reference/release-notes/8.15.0.asciidoc index 1df0969ecc629..e2314381a4b06 100644 --- a/docs/reference/release-notes/8.15.0.asciidoc +++ b/docs/reference/release-notes/8.15.0.asciidoc @@ -16,6 +16,12 @@ 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 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] === Breaking changes diff --git a/gradle/verification-metadata.xml b/gradle/verification-metadata.xml index 73c42c2a2066d..2d030e02c265e 100644 --- a/gradle/verification-metadata.xml +++ b/gradle/verification-metadata.xml @@ -953,11 +953,6 @@ - - - - - @@ -1746,16 +1741,16 @@ - - - - - + + + + + @@ -2821,129 +2816,129 @@ - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + - - - + + + 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/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/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); + } } 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/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/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; + } + }; +} 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()); } } 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) + ); + } + +} 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/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 + "]"; + } + } +} 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); }