From f0ea05811f7ac6027bc507b31202f39dc6cd495a Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 3 May 2024 07:46:03 +0100 Subject: [PATCH 01/17] Async close of `IndexShard` (#108145) Moves the work to close an `IndexShard`, including any final flush and waiting for merges to complete, off the cluster applier thread to avoid delaying the application of cluster state updates. Relates #89821 Relates #107513 Relates #108096 Relates ES-8334 --- docs/changelog/108145.yaml | 5 + .../index/shard/IndexShardIT.java | 4 + .../indices/recovery/DanglingIndicesIT.java | 1 + .../service/ClusterApplierService.java | 10 +- .../cluster/IndicesClusterStateService.java | 217 ++++++++++++++---- .../indices/store/IndicesStore.java | 10 +- .../indices/IndicesServiceTests.java | 1 + .../search/SearchServiceTests.java | 2 + ...actIndicesClusterStateServiceTestCase.java | 7 + .../test/ESSingleNodeTestCase.java | 10 + .../test/InternalTestCluster.java | 18 ++ 11 files changed, 229 insertions(+), 56 deletions(-) create mode 100644 docs/changelog/108145.yaml rename {server/src/test => test/framework/src/main}/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java (98%) diff --git a/docs/changelog/108145.yaml b/docs/changelog/108145.yaml new file mode 100644 index 0000000000000..b8c9428c1e3a8 --- /dev/null +++ b/docs/changelog/108145.yaml @@ -0,0 +1,5 @@ +pr: 108145 +summary: Async close of `IndexShard` +area: Engine +type: bug +issues: [] diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java index c01d945ca2a1a..3f7ed48b714fb 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -96,6 +96,7 @@ import static org.elasticsearch.index.shard.IndexShardTestCase.closeShardNoCheck; import static org.elasticsearch.index.shard.IndexShardTestCase.getTranslog; import static org.elasticsearch.index.shard.IndexShardTestCase.recoverFromStore; +import static org.elasticsearch.indices.cluster.AbstractIndicesClusterStateServiceTestCase.awaitIndexShardCloseAsyncTasks; import static org.elasticsearch.test.LambdaMatchers.falseWith; import static org.elasticsearch.test.LambdaMatchers.trueWith; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -226,6 +227,7 @@ public void testIndexDirIsDeletedWhenShardRemoved() throws Exception { prepareIndex("test").setId("1").setSource("{}", XContentType.JSON).setRefreshPolicy(IMMEDIATE).get(); assertHitCount(client().prepareSearch("test"), 1L); indicesAdmin().prepareDelete("test").get(); + awaitIndexShardCloseAsyncTasks(); assertAllIndicesRemovedAndDeletionCompleted(Collections.singleton(getInstanceFromNode(IndicesService.class))); assertPathHasBeenCleared(idxPath); } @@ -273,6 +275,7 @@ public void testIndexCanChangeCustomDataPath() throws Exception { // Now, try closing and changing the settings logger.info("--> closing the index [{}] before updating data_path", index); assertAcked(indicesAdmin().prepareClose(index)); + awaitIndexShardCloseAsyncTasks(); final Path newIndexDataPath = sharedDataPath.resolve("end-" + randomAlphaOfLength(10)); IOUtils.rm(newIndexDataPath); @@ -307,6 +310,7 @@ public void testIndexCanChangeCustomDataPath() throws Exception { assertHitCount(client().prepareSearch(index).setSize(0), 1L); assertAcked(indicesAdmin().prepareDelete(index)); + awaitIndexShardCloseAsyncTasks(); assertAllIndicesRemovedAndDeletionCompleted(Collections.singleton(getInstanceFromNode(IndicesService.class))); assertPathHasBeenCleared(newIndexDataPath.toAbsolutePath()); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/DanglingIndicesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/DanglingIndicesIT.java index 3582fa6930f54..c47ada432f4b1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/DanglingIndicesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/DanglingIndicesIT.java @@ -190,6 +190,7 @@ public void testDanglingIndexCanBeDeleted() throws Exception { // tombstone has been pushed out of the graveyard. createIndex("additional"); assertAcked(indicesAdmin().prepareDelete("additional")); + internalCluster().awaitIndexShardCloseAsyncTasks(); assertThat(listDanglingIndices(), is(empty())); } diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java index 7357c52b28bea..ae39e5e9dcd12 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java @@ -678,25 +678,21 @@ public int getTimeoutClusterStateListenersSize() { isApplyingClusterState = Assertions.ENABLED ? new ThreadLocal<>() : null; } - public static boolean assertIsApplyingClusterState() { - assert isApplyingClusterState == null || isApplyingClusterState.get() != null - : "operation not permitted unless applying cluster state on thread " + Thread.currentThread().getName(); - return true; - } - public static boolean assertNotApplyingClusterState() { assert isApplyingClusterState == null || isApplyingClusterState.get() == null - : "operation not permitted while applying cluster state on thread " + Thread.currentThread().getName(); + : "operation not permitted while applying cluster state, currently on thread " + Thread.currentThread().getName(); return true; } public static void setIsApplyingClusterState() { + assert ThreadPool.assertCurrentThreadPool(CLUSTER_UPDATE_THREAD_NAME); if (isApplyingClusterState != null) { isApplyingClusterState.set(Boolean.TRUE); } } public static void clearIsApplyingClusterState() { + assert ThreadPool.assertCurrentThreadPool(CLUSTER_UPDATE_THREAD_NAME); if (isApplyingClusterState != null) { isApplyingClusterState.remove(); } diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 3ceeb5f49a9ae..fa2475921aa93 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -18,6 +18,7 @@ import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; @@ -42,10 +43,13 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.util.concurrent.ThrottledTaskRunner; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.gateway.GatewayService; +import org.elasticsearch.index.CloseUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; @@ -130,16 +134,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final TimeValue shardLockRetryInterval; private final TimeValue shardLockRetryTimeout; - private final Executor shardCloseExecutor = r -> { - // ES-8334 TODO move this work onto a background thread - try { - assert ClusterApplierService.assertIsApplyingClusterState(); - ClusterApplierService.clearIsApplyingClusterState(); - r.run(); - } finally { - ClusterApplierService.setIsApplyingClusterState(); - } - }; + private final Executor shardCloseExecutor; @Inject public IndicesClusterStateService( @@ -203,6 +198,7 @@ public IndicesClusterStateService( this.client = client; this.shardLockRetryInterval = SHARD_LOCK_RETRY_INTERVAL_SETTING.get(settings); this.shardLockRetryTimeout = SHARD_LOCK_RETRY_TIMEOUT_SETTING.get(settings); + this.shardCloseExecutor = new ShardCloseExecutor(settings, threadPool.generic()); } @Override @@ -223,8 +219,52 @@ protected void doStop() { @Override protected void doClose() {} + /** + * Completed when all the shards removed by earlier-applied cluster states have fully closed. + *

+ * Kind of a hack tbh, we can't be sure the shard locks are fully released when this is completed so there's all sorts of retries and + * other lenience to handle that. It'd be better to wait for the shard locks to be released and then delete the data. See #74149. + */ + private volatile SubscribableListener lastClusterStateShardsClosedListener = SubscribableListener.newSucceeded(null); + + @Nullable // if not currently applying a cluster state + private RefCountingListener currentClusterStateShardsClosedListeners; + + private ActionListener getShardsClosedListener() { + assert ThreadPool.assertCurrentThreadPool(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME); + if (currentClusterStateShardsClosedListeners == null) { + assert false : "not currently applying cluster state"; + return ActionListener.noop(); + } else { + return currentClusterStateShardsClosedListeners.acquire(); + } + } + + /** + * @param action Action to run when all the shards removed by earlier-applied cluster states have fully closed. May run on the calling + * thread, or on the thread that completed the closing of the last such shard. + */ + public void onClusterStateShardsClosed(Runnable action) { + // In practice, must be called from the applier thread and we should validate the last-applied cluster state version too + // ES-8334 TODO validate this is called properly + lastClusterStateShardsClosedListener.andThenAccept(ignored -> action.run()); + } + @Override public synchronized void applyClusterState(final ClusterChangedEvent event) { + final var previousShardsClosedListener = lastClusterStateShardsClosedListener; + lastClusterStateShardsClosedListener = new SubscribableListener<>(); + currentClusterStateShardsClosedListeners = new RefCountingListener(lastClusterStateShardsClosedListener); + try { + previousShardsClosedListener.addListener(currentClusterStateShardsClosedListeners.acquire()); + doApplyClusterState(event); + } finally { + currentClusterStateShardsClosedListeners.close(); + currentClusterStateShardsClosedListeners = null; + } + } + + private void doApplyClusterState(final ClusterChangedEvent event) { if (lifecycle.started() == false) { return; } @@ -249,7 +289,7 @@ public synchronized void applyClusterState(final ClusterChangedEvent event) { NO_LONGER_ASSIGNED, "cleaning index (disabled block persistence)", shardCloseExecutor, - ActionListener.noop() + getShardsClosedListener() ); } return; @@ -332,17 +372,15 @@ private void deleteIndices(final ClusterChangedEvent event) { } AllocatedIndex indexService = indicesService.indexService(index); final IndexSettings indexSettings; + final SubscribableListener indexServiceClosedListener; if (indexService != null) { indexSettings = indexService.getIndexSettings(); - indicesService.removeIndex( - index, - DELETED, - "index no longer part of the metadata", - shardCloseExecutor, - ActionListener.noop() + indexServiceClosedListener = SubscribableListener.newForked( + l -> indicesService.removeIndex(index, DELETED, "index no longer part of the metadata", shardCloseExecutor, l) ); } else if (previousState.metadata().hasIndex(index)) { // The deleted index was part of the previous cluster state, but not loaded on the local node + indexServiceClosedListener = SubscribableListener.newSucceeded(null); final IndexMetadata metadata = previousState.metadata().index(index); indexSettings = new IndexSettings(metadata, settings); indicesService.deleteUnassignedIndex("deleted index was not assigned to local node", metadata, state); @@ -356,6 +394,7 @@ private void deleteIndices(final ClusterChangedEvent event) { // previous cluster state is not initialized/recovered. assert state.metadata().indexGraveyard().containsIndex(index) || previousState.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK); + indexServiceClosedListener = SubscribableListener.newSucceeded(null); final IndexMetadata metadata = indicesService.verifyIndexIsDeleted(index, event.state()); if (metadata != null) { indexSettings = new IndexSettings(metadata, settings); @@ -364,7 +403,7 @@ private void deleteIndices(final ClusterChangedEvent event) { } } if (indexSettings != null) { - threadPool.generic().execute(new AbstractRunnable() { + indexServiceClosedListener.andThenAccept(ignored -> threadPool.generic().execute(new AbstractRunnable() { @Override public void onFailure(Exception e) { logger.warn(() -> "[" + index + "] failed to complete pending deletion for index", e); @@ -390,7 +429,8 @@ protected void doRun() throws Exception { public String toString() { return "processPendingDeletes[" + index + "]"; } - }); + })); + indexServiceClosedListener.addListener(getShardsClosedListener()); } } } @@ -432,7 +472,7 @@ private void removeIndicesAndShards(final ClusterChangedEvent event) { if (reason != null) { logger.debug("{} removing index ({})", index, reason); - indicesService.removeIndex(index, reason, "removing index (" + reason + ")", shardCloseExecutor, ActionListener.noop()); + indicesService.removeIndex(index, reason, "removing index (" + reason + ")", shardCloseExecutor, getShardsClosedListener()); } else { // remove shards based on routing nodes (no deletion of data) for (Shard shard : indexService) { @@ -443,7 +483,12 @@ private void removeIndicesAndShards(final ClusterChangedEvent event) { // we can just remove the shard without cleaning it locally, since we will clean it in IndicesStore // once all shards are allocated logger.debug("{} removing shard (not allocated)", shardId); - indexService.removeShard(shardId.id(), "removing shard (not allocated)", shardCloseExecutor, ActionListener.noop()); + indexService.removeShard( + shardId.id(), + "removing shard (not allocated)", + shardCloseExecutor, + getShardsClosedListener() + ); } else if (newShardRouting.isSameAllocation(currentRoutingEntry) == false) { logger.debug( "{} removing shard (stale allocation id, stale {}, new {})", @@ -451,20 +496,35 @@ private void removeIndicesAndShards(final ClusterChangedEvent event) { currentRoutingEntry, newShardRouting ); - indexService.removeShard(shardId.id(), "removing shard (stale copy)", shardCloseExecutor, ActionListener.noop()); + indexService.removeShard( + shardId.id(), + "removing shard (stale copy)", + shardCloseExecutor, + getShardsClosedListener() + ); } else if (newShardRouting.initializing() && currentRoutingEntry.active()) { // this can happen if the node was isolated/gc-ed, rejoins the cluster and a new shard with the same allocation id // is assigned to it. Batch cluster state processing or if shard fetching completes before the node gets a new // cluster state may result in a new shard being initialized while having the same allocation id as the currently // started shard. logger.debug("{} removing shard (not active, current {}, new {})", shardId, currentRoutingEntry, newShardRouting); - indexService.removeShard(shardId.id(), "removing shard (stale copy)", shardCloseExecutor, ActionListener.noop()); + indexService.removeShard( + shardId.id(), + "removing shard (stale copy)", + shardCloseExecutor, + getShardsClosedListener() + ); } else if (newShardRouting.primary() && currentRoutingEntry.primary() == false && newShardRouting.initializing()) { assert currentRoutingEntry.initializing() : currentRoutingEntry; // see above if clause // this can happen when cluster state batching batches activation of the shard, closing an index, reopening it // and assigning an initializing primary to this node logger.debug("{} removing shard (not active, current {}, new {})", shardId, currentRoutingEntry, newShardRouting); - indexService.removeShard(shardId.id(), "removing shard (stale copy)", shardCloseExecutor, ActionListener.noop()); + indexService.removeShard( + shardId.id(), + "removing shard (stale copy)", + shardCloseExecutor, + getShardsClosedListener() + ); } } } @@ -529,7 +589,7 @@ private void createIndicesAndUpdateShards(final ClusterState state) { FAILURE, "removing index (mapping update failed)", shardCloseExecutor, - ActionListener.noop() + getShardsClosedListener() ); } for (ShardRouting shardRouting : entry.getValue()) { @@ -583,7 +643,7 @@ private void updateIndices(ClusterChangedEvent event) { FAILURE, "removing index (" + reason + ")", shardCloseExecutor, - ActionListener.noop() + getShardsClosedListener() ); // fail shards that would be created or updated by createOrUpdateShards @@ -644,7 +704,7 @@ public void onFailure(Exception e) { e, state, shardCloseExecutor, - ActionListener.noop() + ActionListener.noop() // on the failure path, did not create the shard, so don't need to wait for it to close ); } }, () -> { @@ -655,7 +715,7 @@ public void onFailure(Exception e) { } catch (Exception e) { assert pendingShardCreations.get(shardId) == null || pendingShardCreations.get(shardId).clusterStateUUID().equals(state.stateUUID()) == false; - failAndRemoveShard(shardRouting, true, "failed to create shard", e, state, shardCloseExecutor, ActionListener.noop()); + failAndRemoveShard(shardRouting, true, "failed to create shard", e, state, shardCloseExecutor, getShardsClosedListener()); } } @@ -819,7 +879,7 @@ private void updateShard(ShardRouting shardRouting, Shard shard, ClusterState cl e, clusterState, shardCloseExecutor, - ActionListener.noop() + getShardsClosedListener() ); return; } @@ -926,15 +986,24 @@ public void onRecoveryFailure(RecoveryFailedException e, boolean sendShardFailur // package-private for testing synchronized void handleRecoveryFailure(ShardRouting shardRouting, boolean sendShardFailure, Exception failure) { - failAndRemoveShard( - shardRouting, - sendShardFailure, - "failed recovery", - failure, - clusterService.state(), - EsExecutors.DIRECT_EXECUTOR_SERVICE, - ActionListener.noop() - ); + try { + CloseUtils.executeDirectly( + l -> failAndRemoveShard( + shardRouting, + sendShardFailure, + "failed recovery", + failure, + clusterService.state(), + EsExecutors.DIRECT_EXECUTOR_SERVICE, + l + ) + ); + } catch (Exception e) { + // should not be possible + final var wrappedException = new IllegalStateException("unexpected failure in handleRecoveryFailure on " + shardRouting, e); + logger.error(wrappedException.getMessage(), e); + assert false : e; + } } private void failAndRemoveShard( @@ -998,15 +1067,27 @@ public void accept(final IndexShard.ShardFailure shardFailure) { final ShardRouting shardRouting = shardFailure.routing(); threadPool.generic().execute(() -> { synchronized (IndicesClusterStateService.this) { - failAndRemoveShard( - shardRouting, - true, - "shard failure, reason [" + shardFailure.reason() + "]", - shardFailure.cause(), - clusterService.state(), - EsExecutors.DIRECT_EXECUTOR_SERVICE /* NB holding mutex while closing shard, ES-8334 TODO revisit this? */, - ActionListener.noop() - ); + try { + CloseUtils.executeDirectly( + l -> failAndRemoveShard( + shardRouting, + true, + "shard failure, reason [" + shardFailure.reason() + "]", + shardFailure.cause(), + clusterService.state(), + EsExecutors.DIRECT_EXECUTOR_SERVICE /* NB holding mutex while closing shard, ES-8334 TODO revisit this? */, + l + ) + ); + } catch (Exception e) { + // should not be possible + final var wrappedException = new IllegalStateException( + "unexpected failure in FailedShardHandler on " + shardRouting, + e + ); + logger.error(wrappedException.getMessage(), e); + assert false : e; + } } }); } @@ -1236,4 +1317,46 @@ enum IndexRemovalReason { SHUTDOWN, } } + + private static class ShardCloseExecutor implements Executor { + + private final ThrottledTaskRunner throttledTaskRunner; + + ShardCloseExecutor(Settings settings, Executor delegate) { + // Closing shards may involve IO so we don't want to do too many at once. We also currently have no backpressure mechanism so + // could build up an unbounded queue of shards to close. We think it's unlikely in practice to see this: we won't see very many + // of these tasks in nodes which are running normally, there's not that many shards moving off such nodes, and on a + // shutting-down node we won't be starting up any new shards so the number of these tasks is bounded by the number of shards to + // close. The bad case would be a normally-running node with very high churn in shards, starting up new shards so fast that it + // can't close the old ones down fast enough. Maybe we could block or throttle new shards starting while old shards are still + // shutting down, given that starting new shards is already async. Since this seems unlikely in practice, we opt for the simple + // approach here. + final var maxThreads = Math.max(EsExecutors.NODE_PROCESSORS_SETTING.get(settings).roundUp(), 10); + throttledTaskRunner = new ThrottledTaskRunner(IndicesClusterStateService.class.getCanonicalName(), maxThreads, delegate); + } + + @Override + public void execute(Runnable command) { + throttledTaskRunner.enqueueTask(new ActionListener<>() { + @Override + public void onResponse(Releasable releasable) { + try (releasable) { + command.run(); + } + } + + @Override + public void onFailure(Exception e) { + // should be impossible, GENERIC pool doesn't reject anything + logger.error("unexpected failure running " + command.toString(), e); + assert false : new AssertionError("unexpected failure running " + command, e); + } + + @Override + public String toString() { + return command.toString(); + } + }); + } + } } diff --git a/server/src/main/java/org/elasticsearch/indices/store/IndicesStore.java b/server/src/main/java/org/elasticsearch/indices/store/IndicesStore.java index 9f1726822f6e5..b8fd05f5b5224 100644 --- a/server/src/main/java/org/elasticsearch/indices/store/IndicesStore.java +++ b/server/src/main/java/org/elasticsearch/indices/store/IndicesStore.java @@ -41,6 +41,7 @@ import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportChannel; @@ -82,6 +83,7 @@ public final class IndicesStore implements ClusterStateListener, Closeable { private final ClusterService clusterService; private final TransportService transportService; private final ThreadPool threadPool; + private final IndicesClusterStateService indicesClusterStateService; // Cache successful shard deletion checks to prevent unnecessary file system lookups private final Set folderNotFoundCache = new HashSet<>(); @@ -94,13 +96,15 @@ public IndicesStore( IndicesService indicesService, ClusterService clusterService, TransportService transportService, - ThreadPool threadPool + ThreadPool threadPool, + IndicesClusterStateService indicesClusterStateService ) { this.settings = settings; this.indicesService = indicesService; this.clusterService = clusterService; this.transportService = transportService; this.threadPool = threadPool; + this.indicesClusterStateService = indicesClusterStateService; transportService.registerRequestHandler( ACTION_SHARD_EXISTS, EsExecutors.DIRECT_EXECUTOR_SERVICE, @@ -169,7 +173,9 @@ public void clusterChanged(ClusterChangedEvent event) { ); switch (shardDeletionCheckResult) { case FOLDER_FOUND_CAN_DELETE: - deleteShardIfExistElseWhere(event.state(), indexShardRoutingTable); + indicesClusterStateService.onClusterStateShardsClosed( + () -> deleteShardIfExistElseWhere(event.state(), indexShardRoutingTable) + ); break; case NO_FOLDER_FOUND: folderNotFoundCache.add(shardId); diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 14c6e7f3c6a00..d5359d4510436 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -307,6 +307,7 @@ public void testDeleteIndexStore() throws Exception { assertNotNull(meta); assertNotNull(meta.index("test")); assertAcked(client().admin().indices().prepareDelete("test")); + awaitIndexShardCloseAsyncTasks(); assertFalse(firstPath.exists()); diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 26c3f5831ec8c..d2c6c55634ec6 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -136,6 +136,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.singletonList; import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; +import static org.elasticsearch.indices.cluster.AbstractIndicesClusterStateServiceTestCase.awaitIndexShardCloseAsyncTasks; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.search.SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED; import static org.elasticsearch.search.SearchService.SEARCH_WORKER_THREADS_ENABLED; @@ -282,6 +283,7 @@ public void testClearIndexDelete() { assertEquals(1, service.getActiveContexts()); assertAcked(indicesAdmin().prepareDelete("index")); + awaitIndexShardCloseAsyncTasks(); assertEquals(0, service.getActiveContexts()); } diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/test/framework/src/main/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java similarity index 98% rename from server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java rename to test/framework/src/main/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index a288ee6f1d840..50e723ebd49d2 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -47,6 +47,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -424,4 +425,10 @@ public ShardLongFieldRange getTimestampRange() { } } + + public static void awaitIndexShardCloseAsyncTasks(IndicesClusterStateService indicesClusterStateService) { + final var latch = new CountDownLatch(1); + indicesClusterStateService.onClusterStateShardsClosed(latch::countDown); + safeAwait(latch); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 5abca85ac0f42..415c97c78d56e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -41,6 +41,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; +import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeValidationException; @@ -63,6 +64,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -126,6 +128,7 @@ public void setUp() throws Exception { @Override public void tearDown() throws Exception { logger.trace("[{}#{}]: cleaning up after test", getTestClass().getSimpleName(), getTestName()); + awaitIndexShardCloseAsyncTasks(); ensureNoInitializingShards(); SearchService searchService = getInstanceFromNode(SearchService.class); assertThat(searchService.getActiveContexts(), equalTo(0)); @@ -460,4 +463,11 @@ protected void ensureNoInitializingShards() { protected boolean enableConcurrentSearch() { return true; } + + protected void awaitIndexShardCloseAsyncTasks() { + // ES-8334 TODO build this wait into the relevant APIs (especially, delete-index and close-index) + final var latch = new CountDownLatch(1); + getInstanceFromNode(IndicesClusterStateService.class).onClusterStateShardsClosed(latch::countDown); + safeAwait(latch); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 53601caa8a1d2..66e2664c7b8b9 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -25,6 +25,7 @@ import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.elasticsearch.action.support.DestructiveOperations; +import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; @@ -86,6 +87,7 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; +import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.node.MockNode; @@ -122,6 +124,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -145,6 +148,7 @@ import static org.elasticsearch.node.Node.INITIAL_STATE_TIMEOUT_SETTING; import static org.elasticsearch.test.ESTestCase.assertBusy; import static org.elasticsearch.test.ESTestCase.randomFrom; +import static org.elasticsearch.test.ESTestCase.safeAwait; import static org.elasticsearch.test.NodeRoles.dataOnlyNode; import static org.elasticsearch.test.NodeRoles.masterOnlyNode; import static org.elasticsearch.test.NodeRoles.noRoles; @@ -2432,6 +2436,7 @@ public Settings getDefaultSettings() { @Override public void ensureEstimatedStats() { if (size() > 0) { + awaitIndexShardCloseAsyncTasks(); // Checks that the breakers have been reset without incurring a // network request, because a network request can increment one // of the breakers @@ -2509,6 +2514,7 @@ public synchronized void assertAfterTest() throws Exception { assertRequestsFinished(); assertSearchContextsReleased(); assertNoInFlightDocsInEngine(); + awaitIndexShardCloseAsyncTasks(); for (NodeAndClient nodeAndClient : nodes.values()) { NodeEnvironment env = nodeAndClient.node().getNodeEnvironment(); Set shardIds = env.lockedShards(); @@ -2574,4 +2580,16 @@ private void assertSearchContextsReleased() { } } } + + public void awaitIndexShardCloseAsyncTasks() { + // ES-8334 TODO build this wait into the relevant APIs (especially, delete-index and close-index) + final var latch = new CountDownLatch(1); + try (var refs = new RefCountingRunnable(latch::countDown)) { + for (final var nodeAndClient : nodes.values()) { + final var ref = refs.acquire(); + getInstanceFromNode(IndicesClusterStateService.class, nodeAndClient.node()).onClusterStateShardsClosed(ref::close); + } + } + safeAwait(latch); + } } From 9a62dba53ced990014705297e8c6c098d3f101f8 Mon Sep 17 00:00:00 2001 From: Liam Thompson <32779855+leemthompo@users.noreply.github.com> Date: Fri, 3 May 2024 09:12:37 +0200 Subject: [PATCH 02/17] [DOCS] Remove remaining beta flags for RCS (#108201) --- docs/reference/cluster/remote-info.asciidoc | 1 - docs/reference/modules/network.asciidoc | 1 - docs/reference/rest-api/security.asciidoc | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/reference/cluster/remote-info.asciidoc b/docs/reference/cluster/remote-info.asciidoc index 240aab04e82d1..8f2923846df0e 100644 --- a/docs/reference/cluster/remote-info.asciidoc +++ b/docs/reference/cluster/remote-info.asciidoc @@ -69,7 +69,6 @@ Refer to <>. mode is configured. `cluster_credentials`:: -beta:[] This field presents and has value of `::es_redacted::` only when the <>. Otherwise, the field is not present. diff --git a/docs/reference/modules/network.asciidoc b/docs/reference/modules/network.asciidoc index d5392a204299e..55c236ce43574 100644 --- a/docs/reference/modules/network.asciidoc +++ b/docs/reference/modules/network.asciidoc @@ -67,7 +67,6 @@ Defaults to `9300-9400`. [[remote_cluster.port]] `remote_cluster.port`:: (<>, integer) -beta:[] The port to bind for remote cluster client communication. Accepts a single value. + Defaults to `9443`. diff --git a/docs/reference/rest-api/security.asciidoc b/docs/reference/rest-api/security.asciidoc index e5c42a93d34b1..4571d963179a6 100644 --- a/docs/reference/rest-api/security.asciidoc +++ b/docs/reference/rest-api/security.asciidoc @@ -71,7 +71,7 @@ without requiring basic authentication: * <> * <> -beta:[] Use the following APIs to create and update cross-cluster API keys for +Use the following APIs to create and update cross-cluster API keys for <>: * <> From 5b28d3bff4264e598cb4d7bfd82856f1c8fb8b2f Mon Sep 17 00:00:00 2001 From: Stef Nestor <26751266+stefnestor@users.noreply.github.com> Date: Fri, 3 May 2024 01:13:42 -0600 Subject: [PATCH 03/17] [Doc+] Add Secure Connection to Setup CCR Tutorial (#103237) --- docs/reference/ccr/getting-started.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/reference/ccr/getting-started.asciidoc b/docs/reference/ccr/getting-started.asciidoc index a9fe8be93d018..2a0e3bcc5681f 100644 --- a/docs/reference/ccr/getting-started.asciidoc +++ b/docs/reference/ccr/getting-started.asciidoc @@ -90,6 +90,7 @@ image::images/ccr-tutorial-clusters.png[ClusterA contains the leader index and C To configure a remote cluster from Stack Management in {kib}: +. Set up a <> as needed. . Select *Remote Clusters* from the side navigation. . Specify the {es} endpoint URL, or the IP address or host name of the remote cluster (`ClusterA`) followed by the transport port (defaults to `9300`). For From fe857b3908c80ebff86fa68a79733fe389d4cc6a Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Fri, 3 May 2024 10:09:04 +0200 Subject: [PATCH 04/17] Optimise cardinality aggregations for single value fields (#107892) --- docs/changelog/107892.yaml | 5 + .../metrics/CardinalityAggregator.java | 190 +++++++++++++++--- .../GlobalOrdCardinalityAggregator.java | 137 ++++++++----- 3 files changed, 255 insertions(+), 77 deletions(-) create mode 100644 docs/changelog/107892.yaml diff --git a/docs/changelog/107892.yaml b/docs/changelog/107892.yaml new file mode 100644 index 0000000000000..5fd5404c48d02 --- /dev/null +++ b/docs/changelog/107892.yaml @@ -0,0 +1,5 @@ +pr: 107892 +summary: Optimise cardinality aggregations for single value fields +area: Aggregations +type: enhancement +issues: [] diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java index ecdaefc2a95e7..8187e0f1cbb0c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/CardinalityAggregator.java @@ -8,7 +8,10 @@ package org.elasticsearch.search.aggregations.metrics; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.search.ScoreMode; @@ -23,6 +26,8 @@ import org.elasticsearch.common.util.ObjectArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.fielddata.FieldData; +import org.elasticsearch.index.fielddata.NumericDoubleValues; import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.search.aggregations.AggregationExecutionContext; @@ -79,11 +84,24 @@ public ScoreMode scoreMode() { private Collector pickCollector(LeafReaderContext ctx) throws IOException { if (valuesSource instanceof ValuesSource.Numeric source) { - MurmurHash3Values hashValues = source.isFloatingPoint() - ? MurmurHash3Values.hash(source.doubleValues(ctx)) - : MurmurHash3Values.hash(source.longValues(ctx)); numericCollectorsUsed++; - return new DirectCollector(counts, hashValues); + if (source.isFloatingPoint()) { + SortedNumericDoubleValues values = source.doubleValues(ctx); + NumericDoubleValues singleton = FieldData.unwrapSingleton(values); + if (singleton != null) { + return new DirectSingleValuesCollector(counts, MurmurHash3SingleValues.hash(singleton)); + } else { + return new DirectMultiValuesCollector(counts, MurmurHash3MultiValues.hash(values)); + } + } else { + SortedNumericDocValues values = source.longValues(ctx); + NumericDocValues singleton = DocValues.unwrapSingleton(values); + if (singleton != null) { + return new DirectSingleValuesCollector(counts, MurmurHash3SingleValues.hash(singleton)); + } else { + return new DirectMultiValuesCollector(counts, MurmurHash3MultiValues.hash(values)); + } + } } if (valuesSource instanceof ValuesSource.Bytes.WithOrdinals source) { @@ -104,9 +122,14 @@ private Collector pickCollector(LeafReaderContext ctx) throws IOException { ordinalsCollectorsOverheadTooHigh++; } } - stringHashingCollectorsUsed++; - return new DirectCollector(counts, MurmurHash3Values.hash(valuesSource.bytesValues(ctx))); + final SortedBinaryDocValues values = valuesSource.bytesValues(ctx); + final BinaryDocValues singleton = FieldData.unwrapSingleton(values); + if (singleton != null) { + return new DirectSingleValuesCollector(counts, MurmurHash3SingleValues.hash(singleton)); + } else { + return new DirectMultiValuesCollector(counts, MurmurHash3MultiValues.hash(values)); + } } @Override @@ -193,13 +216,29 @@ public void close() { } } - private static class DirectCollector extends Collector { - - private final MurmurHash3Values hashes; - private final HyperLogLogPlusPlus counts; + private abstract static class DirectCollector extends Collector { + protected final HyperLogLogPlusPlus counts; - DirectCollector(HyperLogLogPlusPlus counts, MurmurHash3Values values) { + DirectCollector(HyperLogLogPlusPlus counts) { this.counts = counts; + } + + @Override + public void postCollect() { + // no-op + } + + @Override + public void close() { + // no-op: the HyperLogLogPlusPlus object is closed as part of the aggregator itself. + } + } + + private static class DirectMultiValuesCollector extends DirectCollector { + private final MurmurHash3MultiValues hashes; + + DirectMultiValuesCollector(HyperLogLogPlusPlus counts, MurmurHash3MultiValues values) { + super(counts); this.hashes = values; } @@ -212,17 +251,22 @@ public void collect(int doc, long bucketOrd) throws IOException { } } } + } - @Override - public void postCollect() { - // no-op + private static class DirectSingleValuesCollector extends DirectCollector { + private final MurmurHash3SingleValues hashes; + + DirectSingleValuesCollector(HyperLogLogPlusPlus counts, MurmurHash3SingleValues values) { + super(counts); + this.hashes = values; } @Override - public void close() { - // no-op + public void collect(int doc, long bucketOrd) throws IOException { + if (hashes.advanceExact(doc)) { + counts.collect(bucketOrd, hashes.longValue()); + } } - } static class OrdinalsCollector extends Collector { @@ -314,7 +358,7 @@ public void close() { /** * Representation of a list of hash values. There might be dups and there is no guarantee on the order. */ - abstract static class MurmurHash3Values { + private abstract static class MurmurHash3MultiValues { public abstract boolean advanceExact(int docId) throws IOException; @@ -323,27 +367,27 @@ abstract static class MurmurHash3Values { public abstract long nextValue() throws IOException; /** - * Return a {@link MurmurHash3Values} instance that computes hashes on the fly for each double value. + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each double value. */ - public static MurmurHash3Values hash(SortedNumericDoubleValues values) { + public static MurmurHash3MultiValues hash(SortedNumericDoubleValues values) { return new Double(values); } /** - * Return a {@link MurmurHash3Values} instance that computes hashes on the fly for each long value. + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each long value. */ - public static MurmurHash3Values hash(SortedNumericDocValues values) { + public static MurmurHash3MultiValues hash(SortedNumericDocValues values) { return new Long(values); } /** - * Return a {@link MurmurHash3Values} instance that computes hashes on the fly for each binary value. + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each binary value. */ - public static MurmurHash3Values hash(SortedBinaryDocValues values) { + public static MurmurHash3MultiValues hash(SortedBinaryDocValues values) { return new Bytes(values); } - private static class Long extends MurmurHash3Values { + private static class Long extends MurmurHash3MultiValues { private final SortedNumericDocValues values; @@ -367,7 +411,7 @@ public long nextValue() throws IOException { } } - private static class Double extends MurmurHash3Values { + private static class Double extends MurmurHash3MultiValues { private final SortedNumericDoubleValues values; @@ -391,7 +435,7 @@ public long nextValue() throws IOException { } } - private static class Bytes extends MurmurHash3Values { + private static class Bytes extends MurmurHash3MultiValues { private final MurmurHash3.Hash128 hash = new MurmurHash3.Hash128(); @@ -419,4 +463,96 @@ public long nextValue() throws IOException { } } } + + /** + * Representation of a list of hash values. There might be dups and there is no guarantee on the order. + */ + private abstract static class MurmurHash3SingleValues { + + public abstract boolean advanceExact(int docId) throws IOException; + + public abstract long longValue() throws IOException; + + /** + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each double value. + */ + public static MurmurHash3SingleValues hash(NumericDoubleValues values) { + return new Double(values); + } + + /** + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each long value. + */ + public static MurmurHash3SingleValues hash(NumericDocValues values) { + return new Long(values); + } + + /** + * Return a {@link MurmurHash3MultiValues} instance that computes hashes on the fly for each binary value. + */ + public static MurmurHash3SingleValues hash(BinaryDocValues values) { + return new Bytes(values); + } + + private static class Long extends MurmurHash3SingleValues { + + private final NumericDocValues values; + + Long(NumericDocValues values) { + this.values = values; + } + + @Override + public boolean advanceExact(int docId) throws IOException { + return values.advanceExact(docId); + } + + @Override + public long longValue() throws IOException { + return BitMixer.mix64(values.longValue()); + } + } + + private static class Double extends MurmurHash3SingleValues { + + private final NumericDoubleValues values; + + Double(NumericDoubleValues values) { + this.values = values; + } + + @Override + public boolean advanceExact(int docId) throws IOException { + return values.advanceExact(docId); + } + + @Override + public long longValue() throws IOException { + return BitMixer.mix64(java.lang.Double.doubleToLongBits(values.doubleValue())); + } + } + + private static class Bytes extends MurmurHash3SingleValues { + + private final MurmurHash3.Hash128 hash = new MurmurHash3.Hash128(); + + private final BinaryDocValues values; + + Bytes(BinaryDocValues values) { + this.values = values; + } + + @Override + public boolean advanceExact(int docId) throws IOException { + return values.advanceExact(docId); + } + + @Override + public long longValue() throws IOException { + final BytesRef bytes = values.binaryValue(); + MurmurHash3.hash128(bytes.bytes, bytes.offset, bytes.length, 0, hash); + return hash.h1; + } + } + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java index 56d627712db22..32cf3c7d24115 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GlobalOrdCardinalityAggregator.java @@ -8,9 +8,11 @@ package org.elasticsearch.search.aggregations.metrics; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; @@ -201,55 +203,71 @@ void onVisitedOrdinal(long ordinal) throws IOException { @Override public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException { values = valuesSource.globalOrdinalsValues(aggCtx.getLeafReaderContext()); - + final SortedDocValues singleton = DocValues.unwrapSingleton(values); if (parent == null && field != null) { // This optimization only applies to top-level cardinality aggregations that apply to fields indexed with an inverted index. final Terms indexTerms = aggCtx.getLeafReaderContext().reader().terms(field); if (indexTerms != null) { visitedOrds = bigArrays.grow(visitedOrds, 1); - BitArray bits = visitedOrds.get(0); - final int numNonVisitedOrds = maxOrd - (bits == null ? 0 : (int) bits.cardinality()); + final int numNonVisitedOrds; + { + final BitArray bits = visitedOrds.get(0); + numNonVisitedOrds = maxOrd - (bits == null ? 0 : (int) bits.cardinality()); + } if (maxOrd <= MAX_FIELD_CARDINALITY_FOR_DYNAMIC_PRUNING || numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) { dynamicPruningAttempts++; - return new LeafBucketCollector() { - final SortedSetDocValues docValues = values; - - final BitArray bits; - final CompetitiveIterator competitiveIterator; - - { - // This optimization only works for top-level cardinality aggregations that collect bucket 0, so we can retrieve - // the appropriate BitArray ahead of time. - visitedOrds = bigArrays.grow(visitedOrds, 1); - BitArray bits = visitedOrds.get(0); - if (bits == null) { - bits = new BitArray(maxOrd, bigArrays); - visitedOrds.set(0, bits); - } - this.bits = bits; - final DocIdSetIterator docsWithField = valuesSource.ordinalsValues(aggCtx.getLeafReaderContext()); - competitiveIterator = new CompetitiveIterator(numNonVisitedOrds, bits, indexTerms, docsWithField); - if (numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) { - competitiveIterator.startPruning(); - } + final BitArray bits = getNewOrExistingBitArray(0L); + final CompetitiveIterator competitiveIterator; + { + // This optimization only works for top-level cardinality aggregations that collect bucket 0, so we can retrieve + // the appropriate BitArray ahead of time. + final DocIdSetIterator docsWithField = valuesSource.ordinalsValues(aggCtx.getLeafReaderContext()); + competitiveIterator = new CompetitiveIterator(numNonVisitedOrds, bits, indexTerms, docsWithField); + if (numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) { + competitiveIterator.startPruning(); } - - @Override - public void collect(int doc, long bucketOrd) throws IOException { - if (docValues.advanceExact(doc)) { - for (long ord = docValues.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = docValues.nextOrd()) { + } + if (singleton != null) { + return new LeafBucketCollector() { + final SortedDocValues docValues = singleton; + + @Override + public void collect(int doc, long bucketOrd) throws IOException { + if (docValues.advanceExact(doc)) { + final int ord = docValues.ordValue(); if (bits.getAndSet(ord) == false) { competitiveIterator.onVisitedOrdinal(ord); } } } - } - @Override - public CompetitiveIterator competitiveIterator() { - return competitiveIterator; - } - }; + @Override + public CompetitiveIterator competitiveIterator() { + return competitiveIterator; + } + }; + } else { + return new LeafBucketCollector() { + final SortedSetDocValues docValues = values; + + @Override + public void collect(int doc, long bucketOrd) throws IOException { + if (docValues.advanceExact(doc)) { + for (long ord = docValues.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = docValues + .nextOrd()) { + if (bits.getAndSet(ord) == false) { + competitiveIterator.onVisitedOrdinal(ord); + } + } + } + } + + @Override + public CompetitiveIterator competitiveIterator() { + return competitiveIterator; + } + }; + } } } else { final FieldInfo fi = aggCtx.getLeafReaderContext().reader().getFieldInfos().fieldInfo(field); @@ -265,24 +283,43 @@ public CompetitiveIterator competitiveIterator() { } bruteForce++; - return new LeafBucketCollector() { - final SortedSetDocValues docValues = values; - - @Override - public void collect(int doc, long bucketOrd) throws IOException { - if (docValues.advanceExact(doc)) { - visitedOrds = bigArrays.grow(visitedOrds, bucketOrd + 1); - BitArray bits = visitedOrds.get(bucketOrd); - if (bits == null) { - bits = new BitArray(maxOrd, bigArrays); - visitedOrds.set(bucketOrd, bits); + if (singleton != null) { + return new LeafBucketCollector() { + final SortedDocValues docValues = singleton; + + @Override + public void collect(int doc, long bucketOrd) throws IOException { + if (docValues.advanceExact(doc)) { + final BitArray bits = getNewOrExistingBitArray(bucketOrd); + bits.set(docValues.ordValue()); } - for (long ord = docValues.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = docValues.nextOrd()) { - bits.set((int) ord); + } + }; + } else { + return new LeafBucketCollector() { + final SortedSetDocValues docValues = values; + + @Override + public void collect(int doc, long bucketOrd) throws IOException { + if (docValues.advanceExact(doc)) { + final BitArray bits = getNewOrExistingBitArray(bucketOrd); + for (long ord = docValues.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = docValues.nextOrd()) { + bits.set((int) ord); + } } } - } - }; + }; + } + } + + private BitArray getNewOrExistingBitArray(long bucketOrd) { + visitedOrds = bigArrays.grow(visitedOrds, bucketOrd + 1); + BitArray bits = visitedOrds.get(bucketOrd); + if (bits == null) { + bits = new BitArray(maxOrd, bigArrays); + visitedOrds.set(bucketOrd, bits); + } + return bits; } protected void doPostCollection() throws IOException { From 63294e7f38ec5de12cbe7c0607ffdf0d299d7f4d Mon Sep 17 00:00:00 2001 From: Artem Prigoda Date: Fri, 3 May 2024 10:45:43 +0200 Subject: [PATCH 05/17] Unmute SnapshotStatusApisIT#testInfiniteTimeout (#108178) This test doesn't fail anymore, I've run it 1000 times locally. This test got introduced in #107050, and I believe the test got fixed in #107675. Unfortunately, the got muted before #107675 got merged, so I can't confirm that PR actually fixed the test on CI. --- .../java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java index f49e46e9b1971..9ed0f981ef666 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java @@ -689,7 +689,6 @@ public void testConcurrentCreateAndStatusAPICalls() throws Exception { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/107405") public void testInfiniteTimeout() throws Exception { createRepository("test-repo", "mock"); createIndex("test-idx", 1, 0); From a8166ddb6bf42c38b45a059d487aebe6b3275aa0 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Fri, 3 May 2024 10:49:53 +0100 Subject: [PATCH 06/17] [ML] Allow deletion of the ELSER inference service when referenced in ingest (#108146) --- docs/changelog/108146.yaml | 5 +++++ .../services/elasticsearch/ElasticsearchInternalService.java | 4 +++- .../xpack/inference/services/elser/ElserInternalService.java | 4 +++- 3 files changed, 11 insertions(+), 2 deletions(-) create mode 100644 docs/changelog/108146.yaml diff --git a/docs/changelog/108146.yaml b/docs/changelog/108146.yaml new file mode 100644 index 0000000000000..2a4f917134090 --- /dev/null +++ b/docs/changelog/108146.yaml @@ -0,0 +1,5 @@ +pr: 108146 +summary: Allow deletion of the ELSER inference service when reference in ingest +area: Machine Learning +type: bug +issues: [] diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java index d0f9814540627..cceeb59284c1b 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java @@ -330,9 +330,11 @@ public void start(Model model, ActionListener listener) { @Override public void stop(String inferenceEntityId, ActionListener listener) { + var request = new StopTrainedModelDeploymentAction.Request(inferenceEntityId); + request.setForce(true); client.execute( StopTrainedModelDeploymentAction.INSTANCE, - new StopTrainedModelDeploymentAction.Request(inferenceEntityId), + request, listener.delegateFailureAndWrap((delegatedResponseListener, response) -> delegatedResponseListener.onResponse(Boolean.TRUE)) ); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java index 2d05baa4df604..01829bfee5241 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java @@ -246,9 +246,11 @@ public void onFailure(Exception e) { @Override public void stop(String inferenceEntityId, ActionListener listener) { + var request = new StopTrainedModelDeploymentAction.Request(inferenceEntityId); + request.setForce(true); client.execute( StopTrainedModelDeploymentAction.INSTANCE, - new StopTrainedModelDeploymentAction.Request(inferenceEntityId), + request, listener.delegateFailureAndWrap((delegatedResponseListener, response) -> delegatedResponseListener.onResponse(Boolean.TRUE)) ); } From 000594988476d905ce8b994f66f19a30261ed12c Mon Sep 17 00:00:00 2001 From: David Kyle Date: Fri, 3 May 2024 10:52:16 +0100 Subject: [PATCH 07/17] [ML] Inference Processor: skip inference when all fields are missing (#108131) If all the configured input_output fields are missing then skip the inference request. --- docs/changelog/108131.yaml | 5 ++ .../inference/ingest/InferenceProcessor.java | 18 +++++++ .../ingest/InferenceProcessorTests.java | 49 +++++++++++++++++++ 3 files changed, 72 insertions(+) create mode 100644 docs/changelog/108131.yaml diff --git a/docs/changelog/108131.yaml b/docs/changelog/108131.yaml new file mode 100644 index 0000000000000..7a4286c1e44a0 --- /dev/null +++ b/docs/changelog/108131.yaml @@ -0,0 +1,5 @@ +pr: 108131 +summary: "Inference Processor: skip inference when all fields are missing" +area: Machine Learning +type: enhancement +issues: [] diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java index 32c85eb4e335e..dab66f887b9a3 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java @@ -194,6 +194,10 @@ public void execute(IngestDocument ingestDocument, BiConsumer requestInputs = new ArrayList<>(); + boolean anyFieldsPresent = false; for (var inputFields : inputs) { try { var inputText = ingestDocument.getFieldValue(inputFields.inputField, String.class, ignoreMissing); // field is missing and ignoreMissing == true then a null value is returned. + anyFieldsPresent = anyFieldsPresent || inputText != null; if (inputText == null) { inputText = ""; // need to send a non-null request to the same number of results back } @@ -245,6 +259,10 @@ CoordinatedInferenceAction.Request buildRequest(IngestDocument ingestDocument) { throw e; } } + + if (anyFieldsPresent == false) { + return null; + } } var request = CoordinatedInferenceAction.Request.forTextInput( modelId, diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java index 6feb014309fe9..6b0b589ace606 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java @@ -34,6 +34,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; @@ -687,4 +688,52 @@ public void testBuildRequestWithInputFields_MissingField() { assertThat(requestInputs, contains("body_text", "")); } } + + public void testBuildRequestReturnsNullWhenAllFieldsMissing() { + List inputs = new ArrayList<>(); + inputs.add(new InferenceProcessor.Factory.InputConfig("body.text", "ml.results", "body_tokens", Map.of())); + inputs.add(new InferenceProcessor.Factory.InputConfig("title.text", "ml.results", "title_tokens", Map.of())); + + InferenceProcessor inferenceProcessor = InferenceProcessor.fromInputFieldConfiguration( + client, + auditor, + "my_processor_tag", + "description", + "elser", + new EmptyConfigUpdate(), + inputs, + true + ); + + IngestDocument document = TestIngestDocument.emptyIngestDocument(); + assertNull(inferenceProcessor.buildRequest(document)); + } + + public void testInferenceNotCalledWhenAllFieldsMissing() { + List inputs = new ArrayList<>(); + inputs.add(new InferenceProcessor.Factory.InputConfig("body.text", "ml.results", "body_tokens", Map.of())); + inputs.add(new InferenceProcessor.Factory.InputConfig("title.text", "ml.results", "title_tokens", Map.of())); + + InferenceProcessor inferenceProcessor = InferenceProcessor.fromInputFieldConfiguration( + client, + auditor, + "my_processor_tag", + "description", + "elser", + new EmptyConfigUpdate(), + inputs, + true + ); + + IngestDocument document = TestIngestDocument.emptyIngestDocument(); + var capturedDoc = new AtomicReference(); + var capturedError = new AtomicReference(); + inferenceProcessor.execute(document, (d, e) -> { + capturedDoc.set(d); + capturedError.set(e); + }); + + assertSame(document, capturedDoc.get()); + assertNull(capturedError.get()); + } } From 0bb7dc07880bf91ce7cafacfbbe81f49e2cbb1c0 Mon Sep 17 00:00:00 2001 From: Andrew Wilkins Date: Fri, 3 May 2024 17:54:24 +0800 Subject: [PATCH 08/17] apm-data: improve indexing resilience (#108227) In 8.11 we were not dynamically mapping any stack trace fields, exception attributes, {error,transaction}.custom, cookies, or HTTP request bodies. In 8.12 we changed to using `flattened` in some of these and mirrored the change in the apm-data plugin. It has since come to light that we're seeing indexing failures using `flattened` where the field contents are not completely unsanitised, e.g. in stack traces with deeply nested stack frame variables, or source lines that exceed 32KB. We're reverting this use of `flattened` since there is no requirement for these fields to be searchable by default, and users can override this with a custom component template. On the other hand we are making HTTP request and response headers `flattened`, since: - headers cannot be deeply nested - these have a natural length limit imposed by server implementations --- docs/changelog/108227.yaml | 5 + .../logs-apm.error@mappings.yaml | 22 ++- .../traces-apm@mappings.yaml | 16 ++- .../test/20_flattened_fields.yml | 87 ++++-------- .../rest-api-spec/test/20_object_fields.yml | 129 ++++++++++++++++++ 5 files changed, 190 insertions(+), 69 deletions(-) create mode 100644 docs/changelog/108227.yaml create mode 100644 x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_object_fields.yml diff --git a/docs/changelog/108227.yaml b/docs/changelog/108227.yaml new file mode 100644 index 0000000000000..79f69bc4aaff6 --- /dev/null +++ b/docs/changelog/108227.yaml @@ -0,0 +1,5 @@ +pr: 108227 +summary: "Apm-data: improve indexing resilience" +area: Data streams +type: enhancement +issues: [] diff --git a/x-pack/plugin/apm-data/src/main/resources/component-templates/logs-apm.error@mappings.yaml b/x-pack/plugin/apm-data/src/main/resources/component-templates/logs-apm.error@mappings.yaml index e6353853bc4d5..1e2a6a679dc30 100644 --- a/x-pack/plugin/apm-data/src/main/resources/component-templates/logs-apm.error@mappings.yaml +++ b/x-pack/plugin/apm-data/src/main/resources/component-templates/logs-apm.error@mappings.yaml @@ -8,13 +8,17 @@ template: properties: # error.* error.custom: - type: flattened + type: object + dynamic: false error.exception.attributes: - type: flattened + type: object + dynamic: false error.exception.stacktrace: - type: flattened + type: object + dynamic: false error.log.stacktrace: - type: flattened + type: object + dynamic: false error.grouping_name: type: keyword script: | @@ -30,5 +34,13 @@ template: } # http.* - http.request.body: + http.request.headers: + type: flattened + http.response.headers: type: flattened + http.request.cookies: + type: object + dynamic: false + http.request.body: + type: object + dynamic: false diff --git a/x-pack/plugin/apm-data/src/main/resources/component-templates/traces-apm@mappings.yaml b/x-pack/plugin/apm-data/src/main/resources/component-templates/traces-apm@mappings.yaml index 780fce37e1d40..6f60dc45b3d8c 100644 --- a/x-pack/plugin/apm-data/src/main/resources/component-templates/traces-apm@mappings.yaml +++ b/x-pack/plugin/apm-data/src/main/resources/component-templates/traces-apm@mappings.yaml @@ -20,8 +20,16 @@ template: index: false # http.* - http.request.body: + http.request.headers: + type: flattened + http.response.headers: type: flattened + http.request.cookies: + type: object + dynamic: false + http.request.body: + type: object + dynamic: false http.response.transfer_size: type: long index: false @@ -40,11 +48,13 @@ template: scaling_factor: 1000 index: false span.stacktrace: - type: flattened + type: object + dynamic: false # transaction.* transaction.custom: - type: flattened + type: object + dynamic: false transaction.duration.us: type: long transaction.representative_count: diff --git a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_flattened_fields.yml b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_flattened_fields.yml index adb248b23fe5b..6a740cf571cbc 100644 --- a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_flattened_fields.yml +++ b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_flattened_fields.yml @@ -11,29 +11,19 @@ setup: index: traces-apm-testing refresh: true body: - # http.request.body should be mapped as flattened, allowing - # differing types to be used in http.request.body.original. + # http.request.headers should be mapped as flattened, allowing + # differing types to be used, and to avoid creating arbitrarily + # many fields. - create: {} - - '{"@timestamp": "2017-06-22", "http.request.body": {"original": "text"}}' + - '{"@timestamp": "2017-06-22", "http.request.headers": {"a": "b"}}' - create: {} - - '{"@timestamp": "2017-06-22", "http.request.body": {"original": {"field": "value"}}}' + - '{"@timestamp": "2017-06-22", "http.request.headers": {"c": ["d", "e"]}}' - # span.stacktrace is a complex object whose structure may - # change over time, and which is always treated as an object. - # Moreover, stacktraces may contain dynamic "vars" whose - # types may change from one document to the next. + # http.response.headers has the same requirements as http.request.headers. - create: {} - - '{"@timestamp": "2017-06-22", "span.stacktrace": [{"vars": {"a": 123}}]}' + - '{"@timestamp": "2017-06-22", "http.response.headers": {"a": "b"}}' - create: {} - - '{"@timestamp": "2017-06-22", "span.stacktrace": [{"vars": {"a": "b"}}]}' - - # transaction.custom is a complex object of fields with - # arbitrary field types that may change from one document - # to the next. - - create: {} - - '{"@timestamp": "2017-06-22", "transaction.custom": {"a": {"b": 123}}}' - - create: {} - - '{"@timestamp": "2017-06-22", "transaction.custom": {"a": "b"}}' + - '{"@timestamp": "2017-06-22", "http.response.headers": {"c": ["d", "e"]}}' - is_false: errors @@ -41,14 +31,12 @@ setup: search: index: traces-apm-testing body: - fields: ["http.request.body", "span.stacktrace", "transaction.custom"] - - length: { hits.hits: 6 } - - match: { hits.hits.0.fields: {"http.request.body": [{"original": "text"}]} } - - match: { hits.hits.1.fields: {"http.request.body": [{"original": {"field": "value"}}]} } - - match: { hits.hits.2.fields: {"span.stacktrace": [{"vars": {"a": 123}}]} } - - match: { hits.hits.3.fields: {"span.stacktrace": [{"vars": {"a": "b"}}]} } - - match: { hits.hits.4.fields: {"transaction.custom": [{"a": {"b": 123}}]} } - - match: { hits.hits.5.fields: {"transaction.custom": [{"a": "b"}]} } + fields: ["http.request.headers", "http.response.headers"] + - length: { hits.hits: 4 } + - match: { hits.hits.0.fields: {"http.request.headers": [{"a": "b"}]} } + - match: { hits.hits.1.fields: {"http.request.headers": [{"c": ["d", "e"]}]} } + - match: { hits.hits.2.fields: {"http.response.headers": [{"a": "b"}]} } + - match: { hits.hits.3.fields: {"http.response.headers": [{"c": ["d", "e"]}]} } --- "Test logs-apm.error-* flattened fields": @@ -57,35 +45,18 @@ setup: index: logs-apm.error-testing refresh: true body: - # http.request.body has the same requirements as http.request.body + # http.request.headers has the same requirements as http.request.headers # in traces-apm-* data streams. - create: {} - - '{"@timestamp": "2017-06-22", "http.request.body": {"original": "text"}}' - - create: {} - - '{"@timestamp": "2017-06-22", "http.request.body": {"original": {"field": "value"}}}' - - # error.{exception,log}.stacktrace have the same requirements as span.stacktrace. - - create: {} - - '{"@timestamp": "2017-06-22", "error.exception.stacktrace": [{"vars": {"a": 123}}]}' - - create: {} - - '{"@timestamp": "2017-06-22", "error.exception.stacktrace": [{"vars": {"a": "b"}}]}' - - create: {} - - '{"@timestamp": "2017-06-22", "error.log.stacktrace": [{"vars": {"a": 123}}]}' - - create: {} - - '{"@timestamp": "2017-06-22", "error.log.stacktrace": [{"vars": {"a": "b"}}]}' - - # error.exception.attributes is a complex object with arbitrary field types - # that may change from one document to the next. - - create: {} - - '{"@timestamp": "2017-06-22", "error.exception": [{"attributes": {"a": 123}}]}' + - '{"@timestamp": "2017-06-22", "http.request.headers": {"a": "b"}}' - create: {} - - '{"@timestamp": "2017-06-22", "error.exception": [{"attributes": {"a": "b"}}]}' + - '{"@timestamp": "2017-06-22", "http.request.headers": {"c": ["d", "e"]}}' - # error.custom has the same requirements as transaction.custom. + # http.response.headers has the same requirements as http.request.headers. - create: {} - - '{"@timestamp": "2017-06-22", "error.custom": {"a": {"b": 123}}}' + - '{"@timestamp": "2017-06-22", "http.response.headers": {"a": "b"}}' - create: {} - - '{"@timestamp": "2017-06-22", "error.custom": {"a": "b"}}' + - '{"@timestamp": "2017-06-22", "http.response.headers": {"c": ["d", "e"]}}' - is_false: errors @@ -93,15 +64,9 @@ setup: search: index: logs-apm.error-testing body: - fields: ["http.request.body", "error.log.*", "error.exception.*", "error.custom"] - - length: { hits.hits: 10 } - - match: { hits.hits.0.fields: {"http.request.body": [{"original": "text"}]} } - - match: { hits.hits.1.fields: {"http.request.body": [{"original": {"field": "value"}}]} } - - match: { hits.hits.2.fields: {"error.exception.stacktrace": [{"vars": {"a": 123}}]} } - - match: { hits.hits.3.fields: {"error.exception.stacktrace": [{"vars": {"a": "b"}}]} } - - match: { hits.hits.4.fields: {"error.log.stacktrace": [{"vars": {"a": 123}}]} } - - match: { hits.hits.5.fields: {"error.log.stacktrace": [{"vars": {"a": "b"}}]} } - - match: { hits.hits.6.fields: {"error.exception.attributes": [{"a": 123}]} } - - match: { hits.hits.7.fields: {"error.exception.attributes": [{"a": "b"}]} } - - match: { hits.hits.8.fields: {"error.custom": [{"a": {"b": 123}}]} } - - match: { hits.hits.9.fields: {"error.custom": [{"a": "b"}]} } + fields: ["http.request.headers", "http.response.headers"] + - length: { hits.hits: 4 } + - match: { hits.hits.0.fields: {"http.request.headers": [{"a": "b"}]} } + - match: { hits.hits.1.fields: {"http.request.headers": [{"c": ["d", "e"]}]} } + - match: { hits.hits.2.fields: {"http.response.headers": [{"a": "b"}]} } + - match: { hits.hits.3.fields: {"http.response.headers": [{"c": ["d", "e"]}]} } diff --git a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_object_fields.yml b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_object_fields.yml new file mode 100644 index 0000000000000..253c1344a6207 --- /dev/null +++ b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/20_object_fields.yml @@ -0,0 +1,129 @@ +--- +setup: + - do: + cluster.health: + wait_for_events: languid + +--- +"Test traces-apm-* unmapped object fields": + - do: + bulk: + index: traces-apm-testing + refresh: true + body: + # http.request.body should be mapped as an object without dynamic mapping, + # allowing differing types to be used in http.request.body.original, and + # to avoid creating arbitrarily many fields. + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.body": {"original": "text"}}' + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.body": {"original": {"field": "value"}}}' + + # http.request.cookies should be mapped as an object without dynamic mapping, + # to avoid creating arbitrarily many fields. + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.cookies": {"a": "b"}}' + + # span.stacktrace is a complex object whose structure may + # change over time, and which is always treated as an object. + # Moreover, stacktraces may contain dynamic "vars" whose + # types may change from one document to the next. + - create: {} + - '{"@timestamp": "2017-06-22", "span.stacktrace": [{"vars": {"a": 123}}]}' + - create: {} + - '{"@timestamp": "2017-06-22", "span.stacktrace": [{"vars": {"a": "b"}}]}' + + # transaction.custom is a complex object of fields with + # arbitrary field types that may change from one document + # to the next. + - create: {} + - '{"@timestamp": "2017-06-22", "transaction.custom": {"a": {"b": 123}}}' + - create: {} + - '{"@timestamp": "2017-06-22", "transaction.custom": {"a": "b"}}' + + - is_false: errors + + - do: + search: + index: traces-apm-testing + body: + fields: + - "http.request.body" + - "http.request.cookies" + - "http.request.env" + - "span.stacktrace" + - "transaction.custom" + - length: { hits.hits: 7 } + - match: { hits.hits.0.fields: null } + - match: { hits.hits.1.fields: null } + - match: { hits.hits.2.fields: null } + - match: { hits.hits.3.fields: null } + - match: { hits.hits.4.fields: null } + - match: { hits.hits.5.fields: null } + - match: { hits.hits.6.fields: null } + +--- +"Test logs-apm.error-* unmapped object fields": + - do: + bulk: + index: logs-apm.error-testing + refresh: true + body: + # http.request.body has the same requirements as http.request.body + # in traces-apm-* data streams. + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.body": {"original": "text"}}' + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.body": {"original": {"field": "value"}}}' + + # http.request.cookies has the same requirements as http.request.bookies + # in traces-apm-* data streams. + - create: {} + - '{"@timestamp": "2017-06-22", "http.request.cookies": {"a": "b"}}' + + # error.{exception,log}.stacktrace have the same requirements as span.stacktrace. + - create: {} + - '{"@timestamp": "2017-06-22", "error.exception.stacktrace": [{"vars": {"a": 123}}]}' + - create: {} + - '{"@timestamp": "2017-06-22", "error.exception.stacktrace": [{"vars": {"a": "b"}}]}' + - create: {} + - '{"@timestamp": "2017-06-22", "error.log.stacktrace": [{"vars": {"a": 123}}]}' + - create: {} + - '{"@timestamp": "2017-06-22", "error.log.stacktrace": [{"vars": {"a": "b"}}]}' + + # error.exception.attributes is a complex object with arbitrary field types + # that may change from one document to the next, and should be mapped as an + # object without dynamic mapping to avoid creating arbitrarily many fields. + - create: {} + - '{"@timestamp": "2017-06-22", "error.exception": [{"attributes": {"a": 123}}]}' + - create: {} + - '{"@timestamp": "2017-06-22", "error.exception": [{"attributes": {"a": "b"}}]}' + + # error.custom has the same requirements as transaction.custom. + - create: {} + - '{"@timestamp": "2017-06-22", "error.custom": {"a": {"b": 123}}}' + - create: {} + - '{"@timestamp": "2017-06-22", "error.custom": {"a": "b"}}' + + - is_false: errors + + - do: + search: + index: logs-apm.error-testing + body: + size: 1000 + fields: + - "http.*" + - "error.*" + - length: { hits.hits: 11 } + - match: { hits.hits.0.fields: null } + - match: { hits.hits.1.fields: null } + - match: { hits.hits.2.fields: null } + - match: { hits.hits.3.fields: null } + - match: { hits.hits.4.fields: null } + - match: { hits.hits.5.fields: null } + - match: { hits.hits.6.fields: null } + - match: { hits.hits.7.fields: null } + - match: { hits.hits.8.fields: null } + - match: { hits.hits.9.fields: null } + - match: { hits.hits.10.fields: null } From 8c8063be63170bac12d4ebfd1bae71f850c4ceb7 Mon Sep 17 00:00:00 2001 From: Howard Date: Fri, 3 May 2024 18:03:33 +0800 Subject: [PATCH 09/17] Drop shards close timeout when stopping node. (#107978) Closing https://github.com/elastic/elasticsearch/issues/107938 to drop shards close timeout. --- docs/changelog/107978.yaml | 6 ++++++ .../java/org/elasticsearch/indices/IndicesService.java | 10 +++------- 2 files changed, 9 insertions(+), 7 deletions(-) create mode 100644 docs/changelog/107978.yaml diff --git a/docs/changelog/107978.yaml b/docs/changelog/107978.yaml new file mode 100644 index 0000000000000..50115df9ee092 --- /dev/null +++ b/docs/changelog/107978.yaml @@ -0,0 +1,6 @@ +pr: 107978 +summary: Drop shards close timeout when stopping node. +area: Engine +type: enhancement +issues: + - 107938 diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index d3fccebb705ed..94582a0dd9862 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -190,7 +190,6 @@ public class IndicesService extends AbstractLifecycleComponent IndexService.ShardStoreDeleter { private static final Logger logger = LogManager.getLogger(IndicesService.class); - public static final String INDICES_SHARDS_CLOSED_TIMEOUT = "indices.shards_closed_timeout"; public static final Setting INDICES_CACHE_CLEAN_INTERVAL_SETTING = Setting.positiveTimeSetting( "indices.cache.cleanup_interval", TimeValue.timeValueMinutes(1), @@ -218,7 +217,6 @@ public class IndicesService extends AbstractLifecycleComponent private final PluginsService pluginsService; private final NodeEnvironment nodeEnv; private final XContentParserConfiguration parserConfig; - private final TimeValue shardsClosedTimeout; private final AnalysisRegistry analysisRegistry; private final IndexNameExpressionResolver indexNameExpressionResolver; private final IndexScopedSettings indexScopedSettings; @@ -280,7 +278,6 @@ protected void doStart() { this.parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE) .withRegistry(builder.xContentRegistry); this.valuesSourceRegistry = builder.valuesSourceRegistry; - this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS)); this.analysisRegistry = builder.analysisRegistry; this.indexNameExpressionResolver = builder.indexNameExpressionResolver; this.indicesRequestCache = new IndicesRequestCache(settings); @@ -412,11 +409,10 @@ protected void doStop() { ); } try { - if (latch.await(shardsClosedTimeout.seconds(), TimeUnit.SECONDS) == false) { - logger.warn("Not all shards are closed yet, waited {}sec - stopping service", shardsClosedTimeout.seconds()); - } + latch.await(); } catch (InterruptedException e) { - // ignore + // continue with shutdown + Thread.currentThread().interrupt(); } finally { indicesStopExecutor.shutdown(); } From a6d715f9569ccb6cc818ef0838dc66557c255877 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Fri, 3 May 2024 14:25:36 +0200 Subject: [PATCH 10/17] Strength ccs testing of aggregations in mixed clusters setup (#108235) --- .../upgrades/AggregationsIT.java | 230 ++++++++++++++++++ .../test/rest/ESRestTestCase.java | 4 + 2 files changed, 234 insertions(+) create mode 100644 qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/AggregationsIT.java diff --git a/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/AggregationsIT.java b/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/AggregationsIT.java new file mode 100644 index 0000000000000..f752915d4a371 --- /dev/null +++ b/qa/ccs-rolling-upgrade-remote-cluster/src/test/java/org/elasticsearch/upgrades/AggregationsIT.java @@ -0,0 +1,230 @@ +/* + * 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.upgrades; + +import org.apache.http.HttpHost; +import org.elasticsearch.Version; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.ObjectPath; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.not; + +/** + * This test ensure that we keep the search states of a CCS request correctly when the local and remote clusters + * have different but compatible versions. See SearchService#createAndPutReaderContext + */ +public class AggregationsIT extends ESRestTestCase { + + private static final String CLUSTER_ALIAS = "remote_cluster"; + private static final String localIndex = "test_bwc_index"; + private static final String remoteIndex = "test_bwc_remote_index"; + private static final String queryIndices = URLEncoder.encode(localIndex + ",remote_cluster:" + remoteIndex, StandardCharsets.UTF_8); + private static int docs; + + @Override + protected boolean preserveClusterUponCompletion() { + return true; + } + + static List getNodes(RestClient restClient) throws IOException { + Response response = restClient.performRequest(new Request("GET", "_nodes")); + ObjectPath objectPath = ObjectPath.createFromResponse(response); + final Map nodeMap = objectPath.evaluate("nodes"); + final List nodes = new ArrayList<>(); + for (String id : nodeMap.keySet()) { + final String name = objectPath.evaluate("nodes." + id + ".name"); + final Version version = Version.fromString(objectPath.evaluate("nodes." + id + ".version")); + final String transportAddress = objectPath.evaluate("nodes." + id + ".transport.publish_address"); + final String httpAddress = objectPath.evaluate("nodes." + id + ".http.publish_address"); + final Map attributes = objectPath.evaluate("nodes." + id + ".attributes"); + nodes.add(new SearchStatesIT.Node(id, name, version, transportAddress, httpAddress, attributes)); + } + return nodes; + } + + static List parseHosts(String props) { + final String address = System.getProperty(props); + assertNotNull("[" + props + "] is not configured", address); + String[] stringUrls = address.split(","); + List hosts = new ArrayList<>(stringUrls.length); + for (String stringUrl : stringUrls) { + int portSeparator = stringUrl.lastIndexOf(':'); + if (portSeparator < 0) { + throw new IllegalArgumentException("Illegal cluster url [" + stringUrl + "]"); + } + String host = stringUrl.substring(0, portSeparator); + int port = Integer.parseInt(stringUrl.substring(portSeparator + 1)); + hosts.add(new HttpHost(host, port, "http")); + } + assertThat("[" + props + "] is empty", hosts, not(empty())); + return hosts; + } + + static RestClient newLocalClient() { + return RestClient.builder(randomFrom(parseHosts("tests.rest.cluster"))).build(); + } + + static RestClient newRemoteClient() { + return RestClient.builder(randomFrom(parseHosts("tests.rest.remote_cluster"))).build(); + } + + @Before + private void configureClusters() throws Exception { + if (docs == 0) { + try (RestClient localClient = newLocalClient(); RestClient remoteClient = newRemoteClient()) { + configureRemoteClusters(localClient, getNodes(remoteClient)); + docs = between(10, 100); + createindex(localClient, localIndex); + createindex(remoteClient, remoteIndex); + } + } + } + + @After + private void clearClusters() throws Exception { + try (RestClient localClient = newLocalClient(); RestClient remoteClient = newRemoteClient()) { + deleteIndex(localClient, localIndex); + deleteIndex(remoteClient, remoteIndex); + docs = 0; + } + } + + private void createindex(RestClient client, String index) throws IOException { + final String mapping = """ + "properties": { + "date": { "type": "date" }, + "number": { "type": "integer" }, + "keyword": { "type": "keyword" } + } + """; + createIndex(client, index, Settings.EMPTY, mapping); + for (int i = 0; i < docs; i++) { + Request createDoc = new Request("POST", "/" + index + "/_doc/id_" + i); + createDoc.setJsonEntity(Strings.format(""" + { "date": %s, "number": %s, "keyword" : %s } + """, i * 1000 * 60, i, "" + i)); + assertOK(client.performRequest(createDoc)); + } + refresh(client, index); + } + + private static void configureRemoteClusters(RestClient localClient, List remoteNodes) throws Exception { + final String remoteClusterSettingPrefix = "cluster.remote." + CLUSTER_ALIAS + "."; + final Settings remoteConnectionSettings; + final List seeds = remoteNodes.stream() + .filter(n -> n.attributes().containsKey("gateway")) + .map(n -> n.transportAddress()) + .collect(Collectors.toList()); + remoteConnectionSettings = Settings.builder() + .putNull(remoteClusterSettingPrefix + "proxy_address") + .put(remoteClusterSettingPrefix + "mode", "sniff") + .putList(remoteClusterSettingPrefix + "seeds", seeds) + .build(); + updateClusterSettings(localClient, remoteConnectionSettings); + assertBusy(() -> { + final Response resp = localClient.performRequest(new Request("GET", "/_remote/info")); + assertOK(resp); + final ObjectPath objectPath = ObjectPath.createFromResponse(resp); + assertNotNull(objectPath.evaluate(CLUSTER_ALIAS)); + assertTrue(objectPath.evaluate(CLUSTER_ALIAS + ".connected")); + }, 60, TimeUnit.SECONDS); + } + + public void testDateHistogram() throws Exception { + for (int i = 0; i < 3; i++) { + try (RestClient localClient = newLocalClient()) { + Request request = new Request("POST", "/" + queryIndices + "/_search"); + request.setJsonEntity(""" + { + "aggs": { + "hist": { + "date_histogram": { + "field": "date", + "calendar_interval": "minute" + } + } + } + } + """); + ObjectPath response = ObjectPath.createFromResponse(localClient.performRequest(request)); + assertEquals(docs, response.evaluateArraySize("aggregations.hist.buckets")); + for (int j = 0; j < docs; j++) { + assertEquals(2, (int) response.evaluate("aggregations.hist.buckets." + j + ".doc_count")); + } + } + } + } + + public void testHistogram() throws Exception { + for (int i = 0; i < 3; i++) { + try (RestClient localClient = newLocalClient()) { + Request request = new Request("POST", "/" + queryIndices + "/_search"); + request.setJsonEntity(""" + { + "aggs": { + "hist": { + "histogram": { + "field": "number", + "interval": 1 + } + } + } + } + """); + ObjectPath response = ObjectPath.createFromResponse(localClient.performRequest(request)); + assertEquals(docs, response.evaluateArraySize("aggregations.hist.buckets")); + for (int j = 0; j < docs; j++) { + assertEquals(2, (int) response.evaluate("aggregations.hist.buckets." + j + ".doc_count")); + } + } + } + } + + public void testTerms() throws Exception { + for (int i = 0; i < 3; i++) { + try (RestClient localClient = newLocalClient()) { + Request request = new Request("POST", "/" + queryIndices + "/_search"); + request.setJsonEntity(""" + { + "aggs": { + "terms": { + "terms": { + "field": "keyword", + "size": 1000 + } + } + } + } + """); + ObjectPath response = ObjectPath.createFromResponse(localClient.performRequest(request)); + assertEquals(docs, response.evaluateArraySize("aggregations.terms.buckets")); + for (int j = 0; j < docs; j++) { + assertEquals(2, (int) response.evaluate("aggregations.terms.buckets." + j + ".doc_count")); + } + } + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index fba04181d5e79..8997844bdbba1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -1746,6 +1746,10 @@ protected static CreateIndexResponse createIndex(RestClient client, String name, return createIndex(client, name, settings, null, null); } + protected static CreateIndexResponse createIndex(RestClient client, String name, Settings settings, String mapping) throws IOException { + return createIndex(client, name, settings, mapping, null); + } + protected static CreateIndexResponse createIndex(String name, Settings settings, String mapping) throws IOException { return createIndex(name, settings, mapping, null); } From ac49ec995a0965924ebe9d7c2e70d2e672a8dc21 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tim=20R=C3=BChsen?= Date: Fri, 3 May 2024 14:59:10 +0200 Subject: [PATCH 11/17] [Profiling] Use synthetic source for events and metrics (#108234) * [Profiling] Use synthetic source for events and metrics * Bump template and data stream versions --- .../profiling/component-template/profiling-events.json | 2 +- .../profiling/component-template/profiling-metrics.json | 2 +- .../persistence/ProfilingIndexTemplateRegistry.java | 7 ++++--- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-events.json b/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-events.json index 8e1b8478d90ba..c540a61c28f05 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-events.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-events.json @@ -22,7 +22,7 @@ }, "mappings": { "_source": { - "enabled": false + "mode": "synthetic" }, "_meta": { "index-template-version": ${xpack.profiling.template.version}, diff --git a/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-metrics.json b/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-metrics.json index ac4a6def2a70b..e933aa117a6b3 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-metrics.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/profiling/component-template/profiling-metrics.json @@ -17,7 +17,7 @@ }, "mappings": { "_source": { - "enabled": false + "mode": "synthetic" }, "_meta": { "index-template-version": ${xpack.profiling.template.version}, diff --git a/x-pack/plugin/profiling/src/main/java/org/elasticsearch/xpack/profiling/persistence/ProfilingIndexTemplateRegistry.java b/x-pack/plugin/profiling/src/main/java/org/elasticsearch/xpack/profiling/persistence/ProfilingIndexTemplateRegistry.java index 61d3010bddf77..066a975d7de7d 100644 --- a/x-pack/plugin/profiling/src/main/java/org/elasticsearch/xpack/profiling/persistence/ProfilingIndexTemplateRegistry.java +++ b/x-pack/plugin/profiling/src/main/java/org/elasticsearch/xpack/profiling/persistence/ProfilingIndexTemplateRegistry.java @@ -48,12 +48,13 @@ public class ProfilingIndexTemplateRegistry extends IndexTemplateRegistry { // version 5: Add optional component template '@custom' to all index templates that reference component templates // version 6: Added 'host.arch' keyword mapping to profiling-hosts // version 7: Added 'host.type', 'cloud.provider', 'cloud.region' keyword mappings to profiling-hosts - public static final int INDEX_TEMPLATE_VERSION = 7; + // version 8: Changed from disabled _source to synthetic _source for profiling-events-* and profiling-metrics + public static final int INDEX_TEMPLATE_VERSION = 8; // history for individual indices / index templates. Only bump these for breaking changes that require to create a new index - public static final int PROFILING_EVENTS_VERSION = 2; + public static final int PROFILING_EVENTS_VERSION = 3; public static final int PROFILING_EXECUTABLES_VERSION = 1; - public static final int PROFILING_METRICS_VERSION = 1; + public static final int PROFILING_METRICS_VERSION = 2; public static final int PROFILING_HOSTS_VERSION = 2; public static final int PROFILING_STACKFRAMES_VERSION = 1; public static final int PROFILING_STACKTRACES_VERSION = 1; From 5f4ef87c4726f2b21012e5a80c4b50e706a6977d Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Fri, 3 May 2024 15:37:22 +0200 Subject: [PATCH 12/17] Fix docs generation of signatures for variadic functions (#107865) This fixes the generation of the signatures for variadic functions, except for those that take a list as last argument; i.e. functions with optional arguments (like ROUND) or functions with overloading-like signatures (like BUCKET). --- docs/changelog/107865.yaml | 5 +++++ docs/reference/esql/functions/types/bucket.asciidoc | 5 +++++ docs/reference/esql/functions/types/coalesce.asciidoc | 5 +++++ docs/reference/esql/functions/types/greatest.asciidoc | 5 +++++ docs/reference/esql/functions/types/least.asciidoc | 5 +++++ docs/reference/esql/functions/types/locate.asciidoc | 4 ++++ docs/reference/esql/functions/types/log.asciidoc | 4 ++++ docs/reference/esql/functions/types/round.asciidoc | 4 ++++ .../esql/expression/function/AbstractFunctionTestCase.java | 7 ++++--- 9 files changed, 41 insertions(+), 3 deletions(-) create mode 100644 docs/changelog/107865.yaml diff --git a/docs/changelog/107865.yaml b/docs/changelog/107865.yaml new file mode 100644 index 0000000000000..f7bb1d869eed5 --- /dev/null +++ b/docs/changelog/107865.yaml @@ -0,0 +1,5 @@ +pr: 107865 +summary: Fix docs generation of signatures for variadic functions +area: ES|QL +type: bug +issues: [] diff --git a/docs/reference/esql/functions/types/bucket.asciidoc b/docs/reference/esql/functions/types/bucket.asciidoc index c4b997d0e124d..d1ce8e499eb07 100644 --- a/docs/reference/esql/functions/types/bucket.asciidoc +++ b/docs/reference/esql/functions/types/bucket.asciidoc @@ -5,7 +5,10 @@ [%header.monospaced.styled,format=dsv,separator=|] |=== field | buckets | from | to | result +datetime | date_period | | | datetime datetime | integer | datetime | datetime | datetime +datetime | time_duration | | | datetime +double | double | | | double double | integer | double | double | double double | integer | double | integer | double double | integer | double | long | double @@ -15,6 +18,7 @@ double | integer | integer | long | double double | integer | long | double | double double | integer | long | integer | double double | integer | long | long | double +integer | double | | | double integer | integer | double | double | double integer | integer | double | integer | double integer | integer | double | long | double @@ -24,6 +28,7 @@ integer | integer | integer | long | double integer | integer | long | double | double integer | integer | long | integer | double integer | integer | long | long | double +long | double | | | double long | integer | double | double | double long | integer | double | integer | double long | integer | double | long | double diff --git a/docs/reference/esql/functions/types/coalesce.asciidoc b/docs/reference/esql/functions/types/coalesce.asciidoc index 97ac47c2bb505..e7d513f2aad86 100644 --- a/docs/reference/esql/functions/types/coalesce.asciidoc +++ b/docs/reference/esql/functions/types/coalesce.asciidoc @@ -6,8 +6,13 @@ |=== first | rest | result boolean | boolean | boolean +boolean | | boolean integer | integer | integer +integer | | integer keyword | keyword | keyword +keyword | | keyword long | long | long +long | | long text | text | text +text | | text |=== diff --git a/docs/reference/esql/functions/types/greatest.asciidoc b/docs/reference/esql/functions/types/greatest.asciidoc index 2a14b6280aa0a..537be55cd17ef 100644 --- a/docs/reference/esql/functions/types/greatest.asciidoc +++ b/docs/reference/esql/functions/types/greatest.asciidoc @@ -6,11 +6,16 @@ |=== first | rest | result boolean | boolean | boolean +boolean | | boolean double | double | double integer | integer | integer +integer | | integer ip | ip | ip keyword | keyword | keyword +keyword | | keyword long | long | long +long | | long text | text | text +text | | text version | version | version |=== diff --git a/docs/reference/esql/functions/types/least.asciidoc b/docs/reference/esql/functions/types/least.asciidoc index 2a14b6280aa0a..537be55cd17ef 100644 --- a/docs/reference/esql/functions/types/least.asciidoc +++ b/docs/reference/esql/functions/types/least.asciidoc @@ -6,11 +6,16 @@ |=== first | rest | result boolean | boolean | boolean +boolean | | boolean double | double | double integer | integer | integer +integer | | integer ip | ip | ip keyword | keyword | keyword +keyword | | keyword long | long | long +long | | long text | text | text +text | | text version | version | version |=== diff --git a/docs/reference/esql/functions/types/locate.asciidoc b/docs/reference/esql/functions/types/locate.asciidoc index 895dce1335813..8faea386bd254 100644 --- a/docs/reference/esql/functions/types/locate.asciidoc +++ b/docs/reference/esql/functions/types/locate.asciidoc @@ -6,7 +6,11 @@ |=== string | substring | start | result keyword | keyword | integer | integer +keyword | keyword | | integer keyword | text | integer | integer +keyword | text | | integer text | keyword | integer | integer +text | keyword | | integer text | text | integer | integer +text | text | | integer |=== diff --git a/docs/reference/esql/functions/types/log.asciidoc b/docs/reference/esql/functions/types/log.asciidoc index 0a59e51e45c72..032ff9f36c557 100644 --- a/docs/reference/esql/functions/types/log.asciidoc +++ b/docs/reference/esql/functions/types/log.asciidoc @@ -9,16 +9,20 @@ double | double | double double | integer | double double | long | double double | unsigned_long | double +double | | double integer | double | double integer | integer | double integer | long | double integer | unsigned_long | double +integer | | double long | double | double long | integer | double long | long | double long | unsigned_long | double +long | | double unsigned_long | double | double unsigned_long | integer | double unsigned_long | long | double unsigned_long | unsigned_long | double +unsigned_long | | double |=== diff --git a/docs/reference/esql/functions/types/round.asciidoc b/docs/reference/esql/functions/types/round.asciidoc index 8c13e14e73b01..2c0fe768741f6 100644 --- a/docs/reference/esql/functions/types/round.asciidoc +++ b/docs/reference/esql/functions/types/round.asciidoc @@ -6,6 +6,10 @@ |=== number | decimals | result double | integer | double +double | | double integer | integer | integer +integer | | integer long | integer | long +long | | long +unsigned_long | | unsigned_long |=== diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java index 772dea0ef4557..286216c41a8b8 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java @@ -1133,6 +1133,7 @@ private static Map, DataType> signatures() { if (signatures != null && classGeneratingSignatures == testClass) { return signatures; } + classGeneratingSignatures = testClass; signatures = new HashMap<>(); Set paramsFactories = new ClassModel(testClass).getAnnotatedLeafMethods(ParametersFactory.class).keySet(); assertThat(paramsFactories, hasSize(1)); @@ -1214,13 +1215,14 @@ private static void renderTypes(List argNames) throws IOException { List table = new ArrayList<>(); for (Map.Entry, DataType> sig : signatures().entrySet()) { // TODO flip to using sortedSignatures - if (sig.getKey().size() != argNames.size()) { + if (sig.getKey().size() > argNames.size()) { // skip variadic [test] cases (but not those with optional parameters) continue; } StringBuilder b = new StringBuilder(); for (DataType arg : sig.getKey()) { b.append(arg.typeName()).append(" | "); } + b.append("| ".repeat(argNames.size() - sig.getKey().size())); b.append(sig.getValue().typeName()); table.add(b.toString()); } @@ -1389,8 +1391,7 @@ private static void renderKibanaFunctionDefinition( // For variadic functions we test much longer signatures, let's just stop at the last one continue; } - // TODO make constants for auto_bucket so the signatures get recognized - if (name.equals("auto_bucket") == false && sig.getKey().size() < minArgCount) { + if (sig.getKey().size() < minArgCount) { throw new IllegalArgumentException("signature " + sig.getKey() + " is missing non-optional arg for " + args); } builder.startObject(); From a5619589858c7c61eb5d93c4ca13ba027b5dd6b8 Mon Sep 17 00:00:00 2001 From: Keith Massey Date: Fri, 3 May 2024 09:09:29 -0500 Subject: [PATCH 13/17] Excluding system data streams from global and factory retention (#108038) --- .../DataStreamLifecycleServiceIT.java | 173 +++++++++++++++++- .../ExplainDataStreamLifecycleIT.java | 72 +++++++- .../lifecycle/DataStreamLifecycleService.java | 6 +- ...pdateDataStreamGlobalRetentionService.java | 6 +- ...sportExplainDataStreamLifecycleAction.java | 3 +- ...TransportGetDataStreamLifecycleAction.java | 3 +- ...DataStreamGlobalRetentionServiceTests.java | 82 +++++++-- .../org/elasticsearch/TransportVersions.java | 1 + .../datastreams/GetDataStreamAction.java | 3 +- .../ExplainIndexDataStreamLifecycle.java | 14 +- .../GetDataStreamLifecycleAction.java | 20 +- .../cluster/metadata/DataStream.java | 8 +- .../cluster/metadata/DataStreamLifecycle.java | 10 +- .../metadata/MetadataDataStreamsService.java | 7 +- .../datastreams/GetDataStreamActionTests.java | 108 +++++++++++ ...plainDataStreamLifecycleResponseTests.java | 2 + .../ExplainIndexDataStreamLifecycleTests.java | 82 +++++++++ .../GetDataStreamLifecycleActionTests.java | 86 +++++++++ .../cluster/metadata/DataStreamTests.java | 8 +- 19 files changed, 650 insertions(+), 44 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/action/datastreams/GetDataStreamActionTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleActionTests.java diff --git a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceIT.java b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceIT.java index 5ebdbd272f3fe..7252d31d838c5 100644 --- a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceIT.java +++ b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.datastreams.CreateDataStreamAction; +import org.elasticsearch.action.datastreams.DeleteDataStreamAction; import org.elasticsearch.action.datastreams.GetDataStreamAction; import org.elasticsearch.action.datastreams.ModifyDataStreamsAction; import org.elasticsearch.action.datastreams.lifecycle.ErrorEntry; @@ -41,12 +42,16 @@ import org.elasticsearch.cluster.metadata.Template; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.datastreams.DataStreamsPlugin; +import org.elasticsearch.datastreams.lifecycle.action.DeleteDataStreamGlobalRetentionAction; +import org.elasticsearch.datastreams.lifecycle.action.PutDataStreamGlobalRetentionAction; import org.elasticsearch.datastreams.lifecycle.health.DataStreamLifecycleHealthIndicatorService; import org.elasticsearch.health.Diagnosis; import org.elasticsearch.health.GetHealthAction; @@ -58,14 +63,20 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.indices.ExecutorNames; +import org.elasticsearch.indices.SystemDataStreamDescriptor; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentType; import org.junit.After; import java.io.IOException; +import java.time.Clock; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -73,6 +84,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.backingIndexEqualTo; @@ -82,6 +94,8 @@ import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleService.DATA_STREAM_MERGE_POLICY_TARGET_FLOOR_SEGMENT_SETTING; import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleService.ONE_HUNDRED_MB; import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleService.TARGET_MERGE_FACTOR_VALUE; +import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleServiceIT.TestSystemDataStreamPlugin.SYSTEM_DATA_STREAM_NAME; +import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleServiceIT.TestSystemDataStreamPlugin.SYSTEM_DATA_STREAM_RETENTION_DAYS; import static org.elasticsearch.datastreams.lifecycle.health.DataStreamLifecycleHealthIndicatorService.STAGNATING_BACKING_INDICES_DIAGNOSIS_DEF; import static org.elasticsearch.datastreams.lifecycle.health.DataStreamLifecycleHealthIndicatorService.STAGNATING_INDEX_IMPACT; import static org.elasticsearch.index.IndexSettings.LIFECYCLE_ORIGINATION_DATE; @@ -102,7 +116,7 @@ public class DataStreamLifecycleServiceIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return List.of(DataStreamsPlugin.class, MockTransportService.TestPlugin.class); + return List.of(DataStreamsPlugin.class, MockTransportService.TestPlugin.class, TestSystemDataStreamPlugin.class); } @Override @@ -173,6 +187,116 @@ public void testRolloverAndRetention() throws Exception { }); } + @SuppressWarnings("unchecked") + public void testSystemDataStreamRetention() throws Exception { + /* + * This test makes sure that global data stream retention is ignored by system data streams, and that the configured retention + * for a system data stream is respected instead. + */ + Iterable dataStreamLifecycleServices = internalCluster().getInstances(DataStreamLifecycleService.class); + Clock clock = Clock.systemUTC(); + AtomicLong now = new AtomicLong(clock.millis()); + dataStreamLifecycleServices.forEach(dataStreamLifecycleService -> dataStreamLifecycleService.setNowSupplier(now::get)); + try { + // Putting in place a global retention that we expect will be ignored by the system data stream: + final int globalRetentionSeconds = 10; + client().execute( + PutDataStreamGlobalRetentionAction.INSTANCE, + new PutDataStreamGlobalRetentionAction.Request( + TimeValue.timeValueSeconds(globalRetentionSeconds), + TimeValue.timeValueSeconds(globalRetentionSeconds) + ) + ).actionGet(); + try { + + CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(SYSTEM_DATA_STREAM_NAME); + client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).actionGet(); + indexDocs(SYSTEM_DATA_STREAM_NAME, 1); + /* + * First we advance the time to well beyond the global retention (10s) but well under the configured retention (100d). + * We expect to see that rollover has occurred but that the old index has not been deleted since the global retention is + * ignored. + */ + now.addAndGet(TimeValue.timeValueSeconds(3 * globalRetentionSeconds).millis()); + assertBusy(() -> { + GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request( + new String[] { SYSTEM_DATA_STREAM_NAME } + ); + GetDataStreamAction.Response getDataStreamResponse = client().execute( + GetDataStreamAction.INSTANCE, + getDataStreamRequest + ).actionGet(); + assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(1)); + assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getName(), equalTo(SYSTEM_DATA_STREAM_NAME)); + List backingIndices = getDataStreamResponse.getDataStreams().get(0).getDataStream().getIndices(); + assertThat(backingIndices.size(), equalTo(2)); // global retention is ignored + // we expect the data stream to have two backing indices since the effective retention is 100 days + String writeIndex = backingIndices.get(1).getName(); + assertThat(writeIndex, backingIndexEqualTo(SYSTEM_DATA_STREAM_NAME, 2)); + }); + + // Now we advance the time to well beyond the configured retention. We expect that the older index will have been deleted. + now.addAndGet(TimeValue.timeValueDays(3 * TestSystemDataStreamPlugin.SYSTEM_DATA_STREAM_RETENTION_DAYS).millis()); + assertBusy(() -> { + GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request( + new String[] { SYSTEM_DATA_STREAM_NAME } + ); + GetDataStreamAction.Response getDataStreamResponse = client().execute( + GetDataStreamAction.INSTANCE, + getDataStreamRequest + ).actionGet(); + assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(1)); + assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getName(), equalTo(SYSTEM_DATA_STREAM_NAME)); + List backingIndices = getDataStreamResponse.getDataStreams().get(0).getDataStream().getIndices(); + assertThat(backingIndices.size(), equalTo(1)); // global retention is ignored + // we expect the data stream to have only one backing index, the write one, with generation 2 + // as generation 1 would've been deleted by the data stream lifecycle given the configuration + String writeIndex = backingIndices.get(0).getName(); + assertThat(writeIndex, backingIndexEqualTo(SYSTEM_DATA_STREAM_NAME, 2)); + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + builder.humanReadable(true); + ToXContent.Params withEffectiveRetention = new ToXContent.MapParams( + DataStreamLifecycle.INCLUDE_EFFECTIVE_RETENTION_PARAMS + ); + getDataStreamResponse.getDataStreams() + .get(0) + .toXContent( + builder, + withEffectiveRetention, + getDataStreamResponse.getRolloverConfiguration(), + getDataStreamResponse.getGlobalRetention() + ); + String serialized = Strings.toString(builder); + Map resultMap = XContentHelper.convertToMap( + XContentType.JSON.xContent(), + serialized, + randomBoolean() + ); + assertNotNull(resultMap); + Map lifecycleMap = (Map) resultMap.get("lifecycle"); + assertNotNull(lifecycleMap); + assertThat( + lifecycleMap.get("data_retention"), + equalTo(TimeValue.timeValueDays(SYSTEM_DATA_STREAM_RETENTION_DAYS).getStringRep()) + ); + assertThat( + lifecycleMap.get("effective_retention"), + equalTo(TimeValue.timeValueDays(SYSTEM_DATA_STREAM_RETENTION_DAYS).getStringRep()) + ); + assertThat(lifecycleMap.get("retention_determined_by"), equalTo("data_stream_configuration")); + assertThat(lifecycleMap.get("enabled"), equalTo(true)); + } + }); + + client().execute(DeleteDataStreamAction.INSTANCE, new DeleteDataStreamAction.Request(SYSTEM_DATA_STREAM_NAME)).actionGet(); + } finally { + client().execute(DeleteDataStreamGlobalRetentionAction.INSTANCE, new DeleteDataStreamGlobalRetentionAction.Request()); + } + } finally { + dataStreamLifecycleServices.forEach(dataStreamLifecycleService -> dataStreamLifecycleService.setNowSupplier(clock::millis)); + } + } + public void testOriginationDate() throws Exception { /* * In this test, we set up a datastream with 7 day retention. Then we add two indices to it -- one with an origination date 365 @@ -880,4 +1004,51 @@ static void updateLifecycle(String dataStreamName, TimeValue dataRetention) { ); assertAcked(client().execute(PutDataStreamLifecycleAction.INSTANCE, putDataLifecycleRequest)); } + + /* + * This test plugin adds `.system-test` as a known system data stream. The data stream is not created by this plugin. But if it is + * created, it will be a system data stream. + */ + public static class TestSystemDataStreamPlugin extends Plugin implements SystemIndexPlugin { + public static final String SYSTEM_DATA_STREAM_NAME = ".system-test"; + public static final int SYSTEM_DATA_STREAM_RETENTION_DAYS = 100; + + @Override + public String getFeatureName() { + return "test"; + } + + @Override + public String getFeatureDescription() { + return "test"; + } + + @Override + public Collection getSystemDataStreamDescriptors() { + return List.of( + new SystemDataStreamDescriptor( + SYSTEM_DATA_STREAM_NAME, + "test", + SystemDataStreamDescriptor.Type.INTERNAL, + ComposableIndexTemplate.builder() + .dataStreamTemplate(new ComposableIndexTemplate.DataStreamTemplate()) + .indexPatterns(List.of(DataStream.BACKING_INDEX_PREFIX + SYSTEM_DATA_STREAM_NAME + "*")) + .template( + new Template( + Settings.EMPTY, + null, + null, + DataStreamLifecycle.newBuilder() + .dataRetention(TimeValue.timeValueDays(SYSTEM_DATA_STREAM_RETENTION_DAYS)) + .build() + ) + ) + .build(), + Map.of(), + List.of(), + ExecutorNames.DEFAULT_SYSTEM_INDEX_THREAD_POOLS + ) + ); + } + } } diff --git a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/ExplainDataStreamLifecycleIT.java b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/ExplainDataStreamLifecycleIT.java index 7120196176928..2723637b2959b 100644 --- a/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/ExplainDataStreamLifecycleIT.java +++ b/modules/data-streams/src/internalClusterTest/java/org/elasticsearch/datastreams/lifecycle/ExplainDataStreamLifecycleIT.java @@ -28,7 +28,10 @@ import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.datastreams.DataStreamsPlugin; +import org.elasticsearch.datastreams.lifecycle.action.DeleteDataStreamGlobalRetentionAction; +import org.elasticsearch.datastreams.lifecycle.action.PutDataStreamGlobalRetentionAction; import org.elasticsearch.index.Index; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.plugins.Plugin; @@ -46,6 +49,8 @@ import static org.elasticsearch.cluster.metadata.DataStreamTestHelper.backingIndexEqualTo; import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.DEFAULT_TIMESTAMP_FIELD; +import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleServiceIT.TestSystemDataStreamPlugin.SYSTEM_DATA_STREAM_NAME; +import static org.elasticsearch.datastreams.lifecycle.DataStreamLifecycleServiceIT.TestSystemDataStreamPlugin.SYSTEM_DATA_STREAM_RETENTION_DAYS; import static org.elasticsearch.indices.ShardLimitValidator.SETTING_CLUSTER_MAX_SHARDS_PER_NODE; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -59,7 +64,11 @@ public class ExplainDataStreamLifecycleIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return List.of(DataStreamsPlugin.class, MockTransportService.TestPlugin.class); + return List.of( + DataStreamsPlugin.class, + MockTransportService.TestPlugin.class, + DataStreamLifecycleServiceIT.TestSystemDataStreamPlugin.class + ); } @Override @@ -194,6 +203,67 @@ public void testExplainLifecycle() throws Exception { } } + public void testSystemExplainLifecycle() throws Exception { + /* + * This test makes sure that for system data streams, we correctly ignore the global retention when calling + * ExplainDataStreamLifecycle. It is very similar to testExplainLifecycle, but only focuses on the retention for a system index. + */ + // Putting in place a global retention that we expect will be ignored by the system data stream: + final int globalRetentionSeconds = 10; + client().execute( + PutDataStreamGlobalRetentionAction.INSTANCE, + new PutDataStreamGlobalRetentionAction.Request( + TimeValue.timeValueSeconds(globalRetentionSeconds), + TimeValue.timeValueSeconds(globalRetentionSeconds) + ) + ).actionGet(); + try { + String dataStreamName = SYSTEM_DATA_STREAM_NAME; + CreateDataStreamAction.Request createDataStreamRequest = new CreateDataStreamAction.Request(dataStreamName); + client().execute(CreateDataStreamAction.INSTANCE, createDataStreamRequest).get(); + + indexDocs(dataStreamName, 1); + + assertBusy(() -> { + GetDataStreamAction.Request getDataStreamRequest = new GetDataStreamAction.Request(new String[] { dataStreamName }); + GetDataStreamAction.Response getDataStreamResponse = client().execute(GetDataStreamAction.INSTANCE, getDataStreamRequest) + .actionGet(); + assertThat(getDataStreamResponse.getDataStreams().size(), equalTo(1)); + assertThat(getDataStreamResponse.getDataStreams().get(0).getDataStream().getName(), equalTo(dataStreamName)); + List backingIndices = getDataStreamResponse.getDataStreams().get(0).getDataStream().getIndices(); + assertThat(backingIndices.size(), equalTo(2)); + String backingIndex = backingIndices.get(0).getName(); + assertThat(backingIndex, backingIndexEqualTo(dataStreamName, 1)); + String writeIndex = backingIndices.get(1).getName(); + assertThat(writeIndex, backingIndexEqualTo(dataStreamName, 2)); + }); + + ExplainDataStreamLifecycleAction.Request explainIndicesRequest = new ExplainDataStreamLifecycleAction.Request( + new String[] { + DataStream.getDefaultBackingIndexName(dataStreamName, 1), + DataStream.getDefaultBackingIndexName(dataStreamName, 2) } + ); + ExplainDataStreamLifecycleAction.Response response = client().execute( + ExplainDataStreamLifecycleAction.INSTANCE, + explainIndicesRequest + ).actionGet(); + assertThat(response.getIndices().size(), is(2)); + // we requested the explain for indices with the default include_details=false + assertThat(response.getRolloverConfiguration(), nullValue()); + for (ExplainIndexDataStreamLifecycle explainIndex : response.getIndices()) { + assertThat(explainIndex.isManagedByLifecycle(), is(true)); + assertThat(explainIndex.getIndexCreationDate(), notNullValue()); + assertThat(explainIndex.getLifecycle(), notNullValue()); + assertThat( + explainIndex.getLifecycle().getDataStreamRetention(), + equalTo(TimeValue.timeValueDays(SYSTEM_DATA_STREAM_RETENTION_DAYS)) + ); + } + } finally { + client().execute(DeleteDataStreamGlobalRetentionAction.INSTANCE, new DeleteDataStreamGlobalRetentionAction.Request()); + } + } + public void testExplainLifecycleForIndicesWithErrors() throws Exception { // empty lifecycle contains the default rollover DataStreamLifecycle lifecycle = new DataStreamLifecycle(); diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleService.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleService.java index 9e3dd5cc1a3ba..27beb92e23180 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleService.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleService.java @@ -787,7 +787,8 @@ private Set maybeExecuteRollover(ClusterState state, DataStream dataStrea RolloverRequest rolloverRequest = getDefaultRolloverRequest( rolloverConfiguration, dataStream.getName(), - dataStream.getLifecycle().getEffectiveDataRetention(globalRetentionResolver.resolve(state)) + dataStream.getLifecycle() + .getEffectiveDataRetention(dataStream.isSystem() ? null : globalRetentionResolver.resolve(state)) ); transportActionsDeduplicator.executeOnce( rolloverRequest, @@ -840,7 +841,8 @@ private Set maybeExecuteRetention(ClusterState state, DataStream dataStre Set indicesToBeRemoved = new HashSet<>(); // We know that there is lifecycle and retention because there are indices to be deleted assert dataStream.getLifecycle() != null; - TimeValue effectiveDataRetention = dataStream.getLifecycle().getEffectiveDataRetention(globalRetention); + TimeValue effectiveDataRetention = dataStream.getLifecycle() + .getEffectiveDataRetention(dataStream.isSystem() ? null : globalRetention); for (Index index : backingIndicesOlderThanRetention) { if (indicesToExcludeForRemainingRun.contains(index) == false) { IndexMetadata backingIndex = metadata.index(index); diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionService.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionService.java index 4c54189ee0111..a18095c555f12 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionService.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionService.java @@ -105,8 +105,10 @@ public List determin List affectedDataStreams = new ArrayList<>(); for (DataStream dataStream : clusterState.metadata().dataStreams().values()) { if (dataStream.getLifecycle() != null) { - TimeValue previousEffectiveRetention = dataStream.getLifecycle().getEffectiveDataRetention(previousGlobalRetention); - TimeValue newEffectiveRetention = dataStream.getLifecycle().getEffectiveDataRetention(newGlobalRetention); + TimeValue previousEffectiveRetention = dataStream.getLifecycle() + .getEffectiveDataRetention(dataStream.isSystem() ? null : previousGlobalRetention); + TimeValue newEffectiveRetention = dataStream.getLifecycle() + .getEffectiveDataRetention(dataStream.isSystem() ? null : newGlobalRetention); if (Objects.equals(previousEffectiveRetention, newEffectiveRetention) == false) { affectedDataStreams.add( new UpdateDataStreamGlobalRetentionResponse.AffectedDataStream( diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportExplainDataStreamLifecycleAction.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportExplainDataStreamLifecycleAction.java index ac5f46edb5ccc..fe5b3a1a378ff 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportExplainDataStreamLifecycleAction.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportExplainDataStreamLifecycleAction.java @@ -94,7 +94,7 @@ protected void masterOperation( DataStream parentDataStream = indexAbstraction.getParentDataStream(); if (parentDataStream == null || parentDataStream.isIndexManagedByDataStreamLifecycle(idxMetadata.getIndex(), metadata::index) == false) { - explainIndices.add(new ExplainIndexDataStreamLifecycle(index, false, null, null, null, null, null)); + explainIndices.add(new ExplainIndexDataStreamLifecycle(index, false, false, null, null, null, null, null)); continue; } @@ -103,6 +103,7 @@ protected void masterOperation( ExplainIndexDataStreamLifecycle explainIndexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( index, true, + parentDataStream.isSystem(), idxMetadata.getCreationDate(), rolloverInfo == null ? null : rolloverInfo.getTime(), generationDate, diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportGetDataStreamLifecycleAction.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportGetDataStreamLifecycleAction.java index deff083579800..7ac9eaae41a50 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportGetDataStreamLifecycleAction.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/TransportGetDataStreamLifecycleAction.java @@ -89,7 +89,8 @@ protected void masterOperation( .map( dataStream -> new GetDataStreamLifecycleAction.Response.DataStreamLifecycle( dataStream.getName(), - dataStream.getLifecycle() + dataStream.getLifecycle(), + dataStream.isSystem() ) ) .sorted(Comparator.comparing(GetDataStreamLifecycleAction.Response.DataStreamLifecycle::dataStreamName)) diff --git a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionServiceTests.java b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionServiceTests.java index 41e3e3a28ed5a..b9dc6d349873c 100644 --- a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionServiceTests.java +++ b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/UpdateDataStreamGlobalRetentionServiceTests.java @@ -15,10 +15,10 @@ import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; -import org.elasticsearch.cluster.metadata.DataStreamTestHelper; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.test.ClusterServiceUtils; @@ -31,6 +31,7 @@ import org.junit.BeforeClass; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.hamcrest.CoreMatchers.equalTo; @@ -129,7 +130,7 @@ public void testUpdateClusterState() { public void testDetermineAffectedDataStreams() { Metadata.Builder builder = Metadata.builder(); - DataStream dataStreamWithoutLifecycle = DataStreamTestHelper.newInstance( + DataStream dataStreamWithoutLifecycle = newDataStreamInstance( "ds-no-lifecycle", List.of(new Index(randomAlphaOfLength(10), randomAlphaOfLength(10))), 1, @@ -140,7 +141,7 @@ public void testDetermineAffectedDataStreams() { ); builder.put(dataStreamWithoutLifecycle); String dataStreamNoRetention = "ds-no-retention"; - DataStream dataStreamWithLifecycleNoRetention = DataStreamTestHelper.newInstance( + DataStream dataStreamWithLifecycleNoRetention = newDataStreamInstance( dataStreamNoRetention, List.of(new Index(randomAlphaOfLength(10), randomAlphaOfLength(10))), 1, @@ -151,7 +152,7 @@ public void testDetermineAffectedDataStreams() { ); builder.put(dataStreamWithLifecycleNoRetention); - DataStream dataStreamWithLifecycleShortRetention = DataStreamTestHelper.newInstance( + DataStream dataStreamWithLifecycleShortRetention = newDataStreamInstance( "ds-no-short-retention", List.of(new Index(randomAlphaOfLength(10), randomAlphaOfLength(10))), 1, @@ -162,7 +163,7 @@ public void testDetermineAffectedDataStreams() { ); builder.put(dataStreamWithLifecycleShortRetention); String dataStreamLongRetention = "ds-long-retention"; - DataStream dataStreamWithLifecycleLongRetention = DataStreamTestHelper.newInstance( + DataStream dataStreamWithLifecycleLongRetention = newDataStreamInstance( dataStreamLongRetention, List.of(new Index(randomAlphaOfLength(10), randomAlphaOfLength(10))), 1, @@ -191,25 +192,45 @@ public void testDetermineAffectedDataStreams() { { var globalRetention = new DataStreamGlobalRetention(TimeValue.timeValueDays(randomIntBetween(1, 10)), null); var affectedDataStreams = service.determineAffectedDataStreams(globalRetention, clusterState); - assertThat(affectedDataStreams.size(), is(1)); - var dataStream = affectedDataStreams.get(0); - assertThat(dataStream.dataStreamName(), equalTo(dataStreamNoRetention)); - assertThat(dataStream.previousEffectiveRetention(), nullValue()); - assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getDefaultRetention())); + if (dataStreamWithLifecycleNoRetention.isSystem()) { + assertThat(affectedDataStreams.size(), is(0)); + } else { + assertThat(affectedDataStreams.size(), is(1)); + var dataStream = affectedDataStreams.get(0); + assertThat(dataStream.dataStreamName(), equalTo(dataStreamNoRetention)); + assertThat(dataStream.previousEffectiveRetention(), nullValue()); + assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getDefaultRetention())); + } } // Max retention in effect { var globalRetention = new DataStreamGlobalRetention(null, TimeValue.timeValueDays(randomIntBetween(10, 90))); var affectedDataStreams = service.determineAffectedDataStreams(globalRetention, clusterState); - assertThat(affectedDataStreams.size(), is(2)); - var dataStream = affectedDataStreams.get(0); - assertThat(dataStream.dataStreamName(), equalTo(dataStreamLongRetention)); - assertThat(dataStream.previousEffectiveRetention(), notNullValue()); - assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); - dataStream = affectedDataStreams.get(1); - assertThat(dataStream.dataStreamName(), equalTo(dataStreamNoRetention)); - assertThat(dataStream.previousEffectiveRetention(), nullValue()); - assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); + if (dataStreamWithLifecycleLongRetention.isSystem() && dataStreamWithLifecycleNoRetention.isSystem()) { + assertThat(affectedDataStreams.size(), is(0)); + } else if (dataStreamWithLifecycleLongRetention.isSystem() == false && dataStreamWithLifecycleNoRetention.isSystem() == false) { + assertThat(affectedDataStreams.size(), is(2)); + var dataStream = affectedDataStreams.get(0); + assertThat(dataStream.dataStreamName(), equalTo(dataStreamLongRetention)); + assertThat(dataStream.previousEffectiveRetention(), notNullValue()); + assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); + dataStream = affectedDataStreams.get(1); + assertThat(dataStream.dataStreamName(), equalTo(dataStreamNoRetention)); + assertThat(dataStream.previousEffectiveRetention(), nullValue()); + assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); + } else if (dataStreamWithLifecycleLongRetention.isSystem() == false) { + assertThat(affectedDataStreams.size(), is(1)); + var dataStream = affectedDataStreams.get(0); + assertThat(dataStream.dataStreamName(), equalTo(dataStreamLongRetention)); + assertThat(dataStream.previousEffectiveRetention(), notNullValue()); + assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); + } else { + assertThat(affectedDataStreams.size(), is(1)); + var dataStream = affectedDataStreams.get(0); + assertThat(dataStream.dataStreamName(), equalTo(dataStreamNoRetention)); + assertThat(dataStream.previousEffectiveRetention(), nullValue()); + assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); + } } // Requested global retention match the factory retention, so no affected data streams @@ -225,6 +246,29 @@ public void testDetermineAffectedDataStreams() { } } + private static DataStream newDataStreamInstance( + String name, + List indices, + long generation, + Map metadata, + boolean replicated, + @Nullable DataStreamLifecycle lifecycle, + List failureStores + ) { + DataStream.Builder builder = DataStream.builder(name, indices) + .setGeneration(generation) + .setMetadata(metadata) + .setReplicated(replicated) + .setLifecycle(lifecycle) + .setFailureStoreEnabled(failureStores.isEmpty() == false) + .setFailureIndices(failureStores); + if (randomBoolean()) { + builder.setSystem(true); + builder.setHidden(true); + } + return builder.build(); + } + private static DataStreamGlobalRetention randomNonEmptyGlobalRetention() { boolean withDefault = randomBoolean(); return new DataStreamGlobalRetention( diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 707da53b69e51..fb7458662edf7 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -188,6 +188,7 @@ static TransportVersion def(int id) { public static final TransportVersion SECURITY_ROLE_MAPPINGS_IN_CLUSTER_STATE = def(8_647_00_0); public static final TransportVersion ESQL_REQUEST_TABLES = def(8_648_00_0); public static final TransportVersion ROLE_REMOTE_CLUSTER_PRIVS = def(8_649_00_0); + public static final TransportVersion NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS = def(8_650_00_0); /* * STOP! READ THIS FIRST! No, really, * ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _ diff --git a/server/src/main/java/org/elasticsearch/action/datastreams/GetDataStreamAction.java b/server/src/main/java/org/elasticsearch/action/datastreams/GetDataStreamAction.java index 01ce7cbd3346b..1517b368e21ea 100644 --- a/server/src/main/java/org/elasticsearch/action/datastreams/GetDataStreamAction.java +++ b/server/src/main/java/org/elasticsearch/action/datastreams/GetDataStreamAction.java @@ -346,7 +346,8 @@ public XContentBuilder toXContent( } if (dataStream.getLifecycle() != null) { builder.field(LIFECYCLE_FIELD.getPreferredName()); - dataStream.getLifecycle().toXContent(builder, params, rolloverConfiguration, globalRetention); + dataStream.getLifecycle() + .toXContent(builder, params, rolloverConfiguration, dataStream.isSystem() ? null : globalRetention); } if (ilmPolicyName != null) { builder.field(ILM_POLICY_FIELD.getPreferredName(), ilmPolicyName); diff --git a/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycle.java b/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycle.java index bb6c3f90f1b0a..32be73a7b0960 100644 --- a/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycle.java +++ b/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycle.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.datastreams.lifecycle; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; @@ -44,6 +45,7 @@ public class ExplainIndexDataStreamLifecycle implements Writeable, ToXContentObj private final String index; private final boolean managedByLifecycle; + private final boolean isSystemDataStream; @Nullable private final Long indexCreationDate; @Nullable @@ -59,6 +61,7 @@ public class ExplainIndexDataStreamLifecycle implements Writeable, ToXContentObj public ExplainIndexDataStreamLifecycle( String index, boolean managedByLifecycle, + boolean isSystemDataStream, @Nullable Long indexCreationDate, @Nullable Long rolloverDate, @Nullable TimeValue generationDate, @@ -67,6 +70,7 @@ public ExplainIndexDataStreamLifecycle( ) { this.index = index; this.managedByLifecycle = managedByLifecycle; + this.isSystemDataStream = isSystemDataStream; this.indexCreationDate = indexCreationDate; this.rolloverDate = rolloverDate; this.generationDateMillis = generationDate == null ? null : generationDate.millis(); @@ -77,6 +81,11 @@ public ExplainIndexDataStreamLifecycle( public ExplainIndexDataStreamLifecycle(StreamInput in) throws IOException { this.index = in.readString(); this.managedByLifecycle = in.readBoolean(); + if (in.getTransportVersion().onOrAfter(TransportVersions.NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS)) { + this.isSystemDataStream = in.readBoolean(); + } else { + this.isSystemDataStream = false; + } if (managedByLifecycle) { this.indexCreationDate = in.readOptionalLong(); this.rolloverDate = in.readOptionalLong(); @@ -132,7 +141,7 @@ public XContentBuilder toXContent( } if (this.lifecycle != null) { builder.field(LIFECYCLE_FIELD.getPreferredName()); - lifecycle.toXContent(builder, params, rolloverConfiguration, globalRetention); + lifecycle.toXContent(builder, params, rolloverConfiguration, isSystemDataStream ? null : globalRetention); } if (this.error != null) { if (error.firstOccurrenceTimestamp() != -1L && error.recordedTimestamp() != -1L && error.retryCount() != -1) { @@ -151,6 +160,9 @@ public XContentBuilder toXContent( public void writeTo(StreamOutput out) throws IOException { out.writeString(index); out.writeBoolean(managedByLifecycle); + if (out.getTransportVersion().onOrAfter(TransportVersions.NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS)) { + out.writeBoolean(isSystemDataStream); + } if (managedByLifecycle) { out.writeOptionalLong(indexCreationDate); out.writeOptionalLong(rolloverDate); diff --git a/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleAction.java b/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleAction.java index c7384e7003963..d0bc1ee9dc011 100644 --- a/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleAction.java +++ b/server/src/main/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleAction.java @@ -137,22 +137,30 @@ public Request includeDefaults(boolean includeDefaults) { public static class Response extends ActionResponse implements ChunkedToXContentObject { public static final ParseField DATA_STREAMS_FIELD = new ParseField("data_streams"); - public record DataStreamLifecycle(String dataStreamName, @Nullable org.elasticsearch.cluster.metadata.DataStreamLifecycle lifecycle) - implements - Writeable, - ToXContentObject { + public record DataStreamLifecycle( + String dataStreamName, + @Nullable org.elasticsearch.cluster.metadata.DataStreamLifecycle lifecycle, + boolean isSystemDataStream + ) implements Writeable, ToXContentObject { public static final ParseField NAME_FIELD = new ParseField("name"); public static final ParseField LIFECYCLE_FIELD = new ParseField("lifecycle"); DataStreamLifecycle(StreamInput in) throws IOException { - this(in.readString(), in.readOptionalWriteable(org.elasticsearch.cluster.metadata.DataStreamLifecycle::new)); + this( + in.readString(), + in.readOptionalWriteable(org.elasticsearch.cluster.metadata.DataStreamLifecycle::new), + in.getTransportVersion().onOrAfter(TransportVersions.NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS) && in.readBoolean() + ); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(dataStreamName); out.writeOptionalWriteable(lifecycle); + if (out.getTransportVersion().onOrAfter(TransportVersions.NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS)) { + out.writeBoolean(isSystemDataStream); + } } @Override @@ -178,7 +186,7 @@ public XContentBuilder toXContent( builder, org.elasticsearch.cluster.metadata.DataStreamLifecycle.maybeAddEffectiveRetentionParams(params), rolloverConfiguration, - globalRetention + isSystemDataStream ? null : globalRetention ); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java index 33dab20a81494..16ad072f271ff 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStream.java @@ -765,12 +765,14 @@ public List getIndicesPastRetention( LongSupplier nowSupplier, DataStreamGlobalRetention globalRetention ) { - if (lifecycle == null || lifecycle.isEnabled() == false || lifecycle.getEffectiveDataRetention(globalRetention) == null) { + if (lifecycle == null + || lifecycle.isEnabled() == false + || lifecycle.getEffectiveDataRetention(isSystem() ? null : globalRetention) == null) { return List.of(); } List indicesPastRetention = getNonWriteIndicesOlderThan( - lifecycle.getEffectiveDataRetention(globalRetention), + lifecycle.getEffectiveDataRetention(isSystem() ? null : globalRetention), indexMetadataSupplier, this::isIndexManagedByDataStreamLifecycle, nowSupplier @@ -1150,7 +1152,7 @@ public XContentBuilder toXContent( } if (lifecycle != null) { builder.field(LIFECYCLE.getPreferredName()); - lifecycle.toXContent(builder, params, rolloverConfiguration, globalRetention); + lifecycle.toXContent(builder, params, rolloverConfiguration, isSystem() ? null : globalRetention); } builder.field(ROLLOVER_ON_WRITE_FIELD.getPreferredName(), rolloverOnWrite); if (autoShardingEvent != null) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamLifecycle.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamLifecycle.java index d816da900a083..3fb5e92cb3359 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamLifecycle.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamLifecycle.java @@ -145,7 +145,10 @@ public boolean isEnabled() { } /** - * The least amount of time data should be kept by elasticsearch. + * The least amount of time data should be kept by elasticsearch. If a caller does not want the global retention considered (for + * example, when evaluating the effective retention for a system data stream or a template) then null should be given for + * globalRetention. + * @param globalRetention The global retention, or null if global retention does not exist or should not be applied * @return the time period or null, null represents that data should never be deleted. */ @Nullable @@ -154,7 +157,10 @@ public TimeValue getEffectiveDataRetention(@Nullable DataStreamGlobalRetention g } /** - * The least amount of time data should be kept by elasticsearch. + * The least amount of time data should be kept by elasticsearch. If a caller does not want the global retention considered (for + * example, when evaluating the effective retention for a system data stream or a template) then null should be given for + * globalRetention. + * @param globalRetention The global retention, or null if global retention does not exist or should not be applied * @return A tuple containing the time period or null as v1 (where null represents that data should never be deleted), and the non-null * retention source as v2. */ diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDataStreamsService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDataStreamsService.java index 20afd7f9eb3ed..a018f3d93a9bc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDataStreamsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDataStreamsService.java @@ -208,12 +208,17 @@ static ClusterState modifyDataStream( ClusterState updateDataLifecycle(ClusterState currentState, List dataStreamNames, @Nullable DataStreamLifecycle lifecycle) { Metadata metadata = currentState.metadata(); Metadata.Builder builder = Metadata.builder(metadata); + boolean atLeastOneDataStreamIsNotSystem = false; for (var dataStreamName : dataStreamNames) { var dataStream = validateDataStream(metadata, dataStreamName); builder.put(dataStream.copy().setLifecycle(lifecycle).build()); + atLeastOneDataStreamIsNotSystem = atLeastOneDataStreamIsNotSystem || dataStream.isSystem() == false; } if (lifecycle != null) { - lifecycle.addWarningHeaderIfDataRetentionNotEffective(globalRetentionResolver.resolve(currentState)); + if (atLeastOneDataStreamIsNotSystem) { + // We don't issue any warnings if all data streams are system data streams + lifecycle.addWarningHeaderIfDataRetentionNotEffective(globalRetentionResolver.resolve(currentState)); + } } return ClusterState.builder(currentState).metadata(builder.build()).build(); } diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/GetDataStreamActionTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/GetDataStreamActionTests.java new file mode 100644 index 0000000000000..285a41f976393 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/datastreams/GetDataStreamActionTests.java @@ -0,0 +1,108 @@ +/* + * 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.action.datastreams; + +import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamLifecycle; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.Index; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class GetDataStreamActionTests extends ESTestCase { + + @SuppressWarnings("unchecked") + public void testDataStreamInfoToXContent() throws IOException { + TimeValue configuredRetention = TimeValue.timeValueDays(100); + TimeValue globalDefaultRetention = TimeValue.timeValueDays(10); + TimeValue globalMaxRetention = TimeValue.timeValueDays(50); + + { + // Since this is a system data stream, we expect the global retention to be ignored + boolean isSystem = true; + GetDataStreamAction.Response.DataStreamInfo dataStreamInfo = newDataStreamInfo(isSystem, configuredRetention); + Map resultMap = getXContentMap(dataStreamInfo, globalDefaultRetention, globalMaxRetention); + assertThat(resultMap.get("hidden"), equalTo(true)); + assertThat(resultMap.get("system"), equalTo(true)); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("data_stream_configuration")); + } + { + // Since this is not a system data stream, we expect the global retention to override the configured retention + boolean isSystem = false; + GetDataStreamAction.Response.DataStreamInfo dataStreamInfo = newDataStreamInfo(isSystem, configuredRetention); + Map resultMap = getXContentMap(dataStreamInfo, globalDefaultRetention, globalMaxRetention); + assertThat(resultMap.get("hidden"), equalTo(false)); + assertThat(resultMap.get("system"), equalTo(false)); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(globalMaxRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("max_global_retention")); + } + } + + /* + * Calls toXContent on the given dataStreamInfo, and converts the response to a Map + */ + private Map getXContentMap( + GetDataStreamAction.Response.DataStreamInfo dataStreamInfo, + TimeValue globalDefaultRetention, + TimeValue globalMaxRetention + ) throws IOException { + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + ToXContent.Params params = new ToXContent.MapParams(DataStreamLifecycle.INCLUDE_EFFECTIVE_RETENTION_PARAMS); + RolloverConfiguration rolloverConfiguration = null; + DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(globalDefaultRetention, globalMaxRetention); + dataStreamInfo.toXContent(builder, params, rolloverConfiguration, globalRetention); + String serialized = Strings.toString(builder); + return XContentHelper.convertToMap(XContentType.JSON.xContent(), serialized, randomBoolean()); + } + } + + private static GetDataStreamAction.Response.DataStreamInfo newDataStreamInfo(boolean isSystem, TimeValue retention) { + DataStream dataStream = newDataStreamInstance(isSystem, retention); + return new GetDataStreamAction.Response.DataStreamInfo( + dataStream, + randomFrom(ClusterHealthStatus.values()), + null, + null, + null, + Map.of(), + randomBoolean() + ); + } + + private static DataStream newDataStreamInstance(boolean isSystem, TimeValue retention) { + List indices = List.of(new Index(randomAlphaOfLength(10), randomAlphaOfLength(10))); + DataStreamLifecycle lifecycle = new DataStreamLifecycle(new DataStreamLifecycle.Retention(retention), null, null); + return DataStream.builder(randomAlphaOfLength(50), indices) + .setGeneration(randomLongBetween(1, 1000)) + .setMetadata(Map.of()) + .setSystem(isSystem) + .setHidden(isSystem) + .setReplicated(randomBoolean()) + .setLifecycle(lifecycle) + .build(); + } +} diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainDataStreamLifecycleResponseTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainDataStreamLifecycleResponseTests.java index a47eca7692842..8e920e618e7c5 100644 --- a/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainDataStreamLifecycleResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainDataStreamLifecycleResponseTests.java @@ -204,6 +204,7 @@ public void testToXContent() throws IOException { ExplainIndexDataStreamLifecycle explainIndexWithNullGenerationDate = new ExplainIndexDataStreamLifecycle( index, true, + randomBoolean(), now, randomBoolean() ? now + TimeValue.timeValueDays(1).getMillis() : null, null, @@ -263,6 +264,7 @@ private static ExplainIndexDataStreamLifecycle createRandomIndexDataStreamLifecy return new ExplainIndexDataStreamLifecycle( index, true, + randomBoolean(), now, randomBoolean() ? now + TimeValue.timeValueDays(1).getMillis() : null, randomBoolean() ? TimeValue.timeValueMillis(now) : null, diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycleTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycleTests.java index 7087b677673e7..7f202a6258082 100644 --- a/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycleTests.java +++ b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/ExplainIndexDataStreamLifecycleTests.java @@ -8,14 +8,23 @@ package org.elasticsearch.action.datastreams.lifecycle; +import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentType; import java.io.IOException; +import java.util.Map; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; @@ -27,6 +36,7 @@ public void testGetGenerationTime() { ExplainIndexDataStreamLifecycle explainIndexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( randomAlphaOfLengthBetween(10, 30), true, + randomBoolean(), now, randomBoolean() ? now + TimeValue.timeValueDays(1).getMillis() : null, null, @@ -44,6 +54,7 @@ public void testGetGenerationTime() { explainIndexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( randomAlphaOfLengthBetween(10, 30), true, + randomBoolean(), now, randomBoolean() ? now + TimeValue.timeValueDays(1).getMillis() : null, TimeValue.timeValueMillis(now + 100), @@ -64,6 +75,7 @@ public void testGetGenerationTime() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", false, + randomBoolean(), null, null, null, @@ -78,6 +90,7 @@ public void testGetGenerationTime() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", true, + randomBoolean(), now, now + 80L, // rolled over in the future (clocks are funny that way) TimeValue.timeValueMillis(now + 100L), @@ -105,6 +118,7 @@ public void testGetTimeSinceIndexCreation() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", false, + randomBoolean(), null, null, null, @@ -119,6 +133,7 @@ public void testGetTimeSinceIndexCreation() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", true, + randomBoolean(), now + 80L, // created in the future (clocks are funny that way) null, null, @@ -153,6 +168,7 @@ public void testGetTimeSinceRollover() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", false, + randomBoolean(), null, null, null, @@ -167,6 +183,7 @@ public void testGetTimeSinceRollover() { ExplainIndexDataStreamLifecycle indexDataStreamLifecycle = new ExplainIndexDataStreamLifecycle( "my-index", true, + randomBoolean(), now - 50L, now + 100L, // rolled over in the future TimeValue.timeValueMillis(now), @@ -177,6 +194,62 @@ public void testGetTimeSinceRollover() { } } + @SuppressWarnings("unchecked") + public void testToXContent() throws Exception { + TimeValue configuredRetention = TimeValue.timeValueDays(100); + TimeValue globalDefaultRetention = TimeValue.timeValueDays(10); + TimeValue globalMaxRetention = TimeValue.timeValueDays(50); + DataStreamLifecycle dataStreamLifecycle = new DataStreamLifecycle( + new DataStreamLifecycle.Retention(configuredRetention), + null, + null + ); + { + boolean isSystemDataStream = true; + ExplainIndexDataStreamLifecycle explainIndexDataStreamLifecycle = createManagedIndexDataStreamLifecycleExplanation( + System.currentTimeMillis(), + dataStreamLifecycle, + isSystemDataStream + ); + Map resultMap = getXContentMap(explainIndexDataStreamLifecycle, globalDefaultRetention, globalMaxRetention); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("data_stream_configuration")); + } + { + boolean isSystemDataStream = false; + ExplainIndexDataStreamLifecycle explainIndexDataStreamLifecycle = createManagedIndexDataStreamLifecycleExplanation( + System.currentTimeMillis(), + dataStreamLifecycle, + isSystemDataStream + ); + Map resultMap = getXContentMap(explainIndexDataStreamLifecycle, globalDefaultRetention, globalMaxRetention); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(globalMaxRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("max_global_retention")); + } + } + + /* + * Calls toXContent on the given explainIndexDataStreamLifecycle, and converts the response to a Map + */ + private Map getXContentMap( + ExplainIndexDataStreamLifecycle explainIndexDataStreamLifecycle, + TimeValue globalDefaultRetention, + TimeValue globalMaxRetention + ) throws IOException { + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + ToXContent.Params params = new ToXContent.MapParams(DataStreamLifecycle.INCLUDE_EFFECTIVE_RETENTION_PARAMS); + RolloverConfiguration rolloverConfiguration = null; + DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(globalDefaultRetention, globalMaxRetention); + explainIndexDataStreamLifecycle.toXContent(builder, params, rolloverConfiguration, globalRetention); + String serialized = Strings.toString(builder); + return XContentHelper.convertToMap(XContentType.JSON.xContent(), serialized, randomBoolean()); + } + } + @Override protected Writeable.Reader instanceReader() { return ExplainIndexDataStreamLifecycle::new; @@ -195,10 +268,19 @@ protected ExplainIndexDataStreamLifecycle mutateInstance(ExplainIndexDataStreamL private static ExplainIndexDataStreamLifecycle createManagedIndexDataStreamLifecycleExplanation( long now, @Nullable DataStreamLifecycle lifecycle + ) { + return createManagedIndexDataStreamLifecycleExplanation(now, lifecycle, randomBoolean()); + } + + private static ExplainIndexDataStreamLifecycle createManagedIndexDataStreamLifecycleExplanation( + long now, + @Nullable DataStreamLifecycle lifecycle, + boolean isSystemDataStream ) { return new ExplainIndexDataStreamLifecycle( randomAlphaOfLengthBetween(10, 30), true, + isSystemDataStream, now, randomBoolean() ? now + TimeValue.timeValueDays(1).getMillis() : null, TimeValue.timeValueMillis(now), diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleActionTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleActionTests.java new file mode 100644 index 0000000000000..c769e504ef15b --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/datastreams/lifecycle/GetDataStreamLifecycleActionTests.java @@ -0,0 +1,86 @@ +/* + * 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.action.datastreams.lifecycle; + +import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamLifecycle; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.util.Map; + +import static org.elasticsearch.rest.RestRequest.PATH_RESTRICTED; +import static org.hamcrest.Matchers.equalTo; + +public class GetDataStreamLifecycleActionTests extends ESTestCase { + + @SuppressWarnings("unchecked") + public void testDataStreamLifecycleToXContent() throws Exception { + TimeValue configuredRetention = TimeValue.timeValueDays(100); + TimeValue globalDefaultRetention = TimeValue.timeValueDays(10); + TimeValue globalMaxRetention = TimeValue.timeValueDays(50); + DataStreamLifecycle lifecycle = new DataStreamLifecycle(new DataStreamLifecycle.Retention(configuredRetention), null, null); + { + boolean isSystemDataStream = true; + GetDataStreamLifecycleAction.Response.DataStreamLifecycle explainIndexDataStreamLifecycle = createDataStreamLifecycle( + lifecycle, + isSystemDataStream + ); + Map resultMap = getXContentMap(explainIndexDataStreamLifecycle, globalDefaultRetention, globalMaxRetention); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("data_stream_configuration")); + } + { + boolean isSystemDataStream = false; + GetDataStreamLifecycleAction.Response.DataStreamLifecycle explainIndexDataStreamLifecycle = createDataStreamLifecycle( + lifecycle, + isSystemDataStream + ); + Map resultMap = getXContentMap(explainIndexDataStreamLifecycle, globalDefaultRetention, globalMaxRetention); + Map lifecycleResult = (Map) resultMap.get("lifecycle"); + assertThat(lifecycleResult.get("data_retention"), equalTo(configuredRetention.getStringRep())); + assertThat(lifecycleResult.get("effective_retention"), equalTo(globalMaxRetention.getStringRep())); + assertThat(lifecycleResult.get("retention_determined_by"), equalTo("max_global_retention")); + } + } + + private GetDataStreamLifecycleAction.Response.DataStreamLifecycle createDataStreamLifecycle( + DataStreamLifecycle lifecycle, + boolean isSystemDataStream + ) { + return new GetDataStreamLifecycleAction.Response.DataStreamLifecycle(randomAlphaOfLength(50), lifecycle, isSystemDataStream); + } + + /* + * Calls toXContent on the given dataStreamLifecycle, and converts the response to a Map + */ + private Map getXContentMap( + GetDataStreamLifecycleAction.Response.DataStreamLifecycle dataStreamLifecycle, + TimeValue globalDefaultRetention, + TimeValue globalMaxRetention + ) throws IOException { + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + ToXContent.Params params = new ToXContent.MapParams(Map.of(PATH_RESTRICTED, "serverless")); + RolloverConfiguration rolloverConfiguration = null; + DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention(globalDefaultRetention, globalMaxRetention); + dataStreamLifecycle.toXContent(builder, params, rolloverConfiguration, globalRetention); + String serialized = Strings.toString(builder); + return XContentHelper.convertToMap(XContentType.JSON.xContent(), serialized, randomBoolean()); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java index 083d6e651dd1e..d42b6096b6e32 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamTests.java @@ -1760,14 +1760,15 @@ public void testXContentSerializationWithRolloverAndEffectiveRetention() throws } DataStreamLifecycle lifecycle = new DataStreamLifecycle(); + boolean isSystem = randomBoolean(); DataStream dataStream = new DataStream( dataStreamName, indices, generation, metadata, + isSystem, randomBoolean(), - randomBoolean(), - false, // Some tests don't work well with system data streams, since these data streams require special handling + isSystem, System::currentTimeMillis, randomBoolean(), randomBoolean() ? IndexMode.STANDARD : null, // IndexMode.TIME_SERIES triggers validation that many unit tests doesn't pass @@ -1794,7 +1795,8 @@ public void testXContentSerializationWithRolloverAndEffectiveRetention() throws } // We check that even if there was no retention provided by the user, the global retention applies assertThat(serialized, not(containsString("data_retention"))); - if (globalRetention.getDefaultRetention() != null || globalRetention.getMaxRetention() != null) { + if (dataStream.isSystem() == false + && (globalRetention.getDefaultRetention() != null || globalRetention.getMaxRetention() != null)) { assertThat(serialized, containsString("effective_retention")); } else { assertThat(serialized, not(containsString("effective_retention"))); From 864f68cfedcbb931d4769d9cf43e644c3b6f59c2 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 3 May 2024 15:12:16 +0000 Subject: [PATCH 14/17] Bump versions after 8.13.3 release --- .buildkite/pipelines/intake.yml | 2 +- .buildkite/pipelines/periodic-packaging.yml | 6 +++--- .buildkite/pipelines/periodic.yml | 10 +++++----- .ci/bwcVersions | 2 +- .ci/snapshotBwcVersions | 2 +- server/src/main/java/org/elasticsearch/Version.java | 1 + .../resources/org/elasticsearch/TransportVersions.csv | 1 + .../org/elasticsearch/index/IndexVersions.csv | 1 + 8 files changed, 14 insertions(+), 11 deletions(-) diff --git a/.buildkite/pipelines/intake.yml b/.buildkite/pipelines/intake.yml index b1f05ea23da4c..fbac4f75985c4 100644 --- a/.buildkite/pipelines/intake.yml +++ b/.buildkite/pipelines/intake.yml @@ -56,7 +56,7 @@ steps: timeout_in_minutes: 300 matrix: setup: - BWC_VERSION: ["7.17.21", "8.13.3", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.buildkite/pipelines/periodic-packaging.yml b/.buildkite/pipelines/periodic-packaging.yml index 96a852be8df04..eee61ef2a9cce 100644 --- a/.buildkite/pipelines/periodic-packaging.yml +++ b/.buildkite/pipelines/periodic-packaging.yml @@ -529,8 +529,8 @@ steps: env: BWC_VERSION: 8.12.2 - - label: "{{matrix.image}} / 8.13.3 / packaging-tests-upgrade" - command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.13.3 + - label: "{{matrix.image}} / 8.13.4 / packaging-tests-upgrade" + command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.13.4 timeout_in_minutes: 300 matrix: setup: @@ -543,7 +543,7 @@ steps: machineType: custom-16-32768 buildDirectory: /dev/shm/bk env: - BWC_VERSION: 8.13.3 + BWC_VERSION: 8.13.4 - label: "{{matrix.image}} / 8.14.0 / packaging-tests-upgrade" command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.14.0 diff --git a/.buildkite/pipelines/periodic.yml b/.buildkite/pipelines/periodic.yml index 48e54eb1667f0..97d08e6ad9d06 100644 --- a/.buildkite/pipelines/periodic.yml +++ b/.buildkite/pipelines/periodic.yml @@ -591,8 +591,8 @@ steps: - signal_reason: agent_stop limit: 3 - - label: 8.13.3 / bwc - command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.13.3#bwcTest + - label: 8.13.4 / bwc + command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.13.4#bwcTest timeout_in_minutes: 300 agents: provider: gcp @@ -601,7 +601,7 @@ steps: buildDirectory: /dev/shm/bk preemptible: true env: - BWC_VERSION: 8.13.3 + BWC_VERSION: 8.13.4 retry: automatic: - exit_status: "-1" @@ -714,7 +714,7 @@ steps: setup: ES_RUNTIME_JAVA: - openjdk17 - BWC_VERSION: ["7.17.21", "8.13.3", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 @@ -760,7 +760,7 @@ steps: - openjdk17 - openjdk21 - openjdk22 - BWC_VERSION: ["7.17.21", "8.13.3", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.ci/bwcVersions b/.ci/bwcVersions index 32a5ef8f8d1e5..7be802285b300 100644 --- a/.ci/bwcVersions +++ b/.ci/bwcVersions @@ -30,6 +30,6 @@ BWC_VERSION: - "8.10.4" - "8.11.4" - "8.12.2" - - "8.13.3" + - "8.13.4" - "8.14.0" - "8.15.0" diff --git a/.ci/snapshotBwcVersions b/.ci/snapshotBwcVersions index 6ee9691a9e5ee..7b85a5c341e43 100644 --- a/.ci/snapshotBwcVersions +++ b/.ci/snapshotBwcVersions @@ -1,5 +1,5 @@ BWC_VERSION: - "7.17.21" - - "8.13.3" + - "8.13.4" - "8.14.0" - "8.15.0" diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index ab7b26570a665..2af65ffac3d1f 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -172,6 +172,7 @@ public class Version implements VersionId, ToXContentFragment { public static final Version V_8_13_1 = new Version(8_13_01_99); public static final Version V_8_13_2 = new Version(8_13_02_99); public static final Version V_8_13_3 = new Version(8_13_03_99); + public static final Version V_8_13_4 = new Version(8_13_04_99); public static final Version V_8_14_0 = new Version(8_14_00_99); public static final Version V_8_15_0 = new Version(8_15_00_99); public static final Version CURRENT = V_8_15_0; diff --git a/server/src/main/resources/org/elasticsearch/TransportVersions.csv b/server/src/main/resources/org/elasticsearch/TransportVersions.csv index dbc170828fabc..842d651a14fbe 100644 --- a/server/src/main/resources/org/elasticsearch/TransportVersions.csv +++ b/server/src/main/resources/org/elasticsearch/TransportVersions.csv @@ -117,3 +117,4 @@ 8.13.0,8595000 8.13.1,8595000 8.13.2,8595000 +8.13.3,8595000 diff --git a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv index f379ac81b9009..432c6d8a08816 100644 --- a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv +++ b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv @@ -117,3 +117,4 @@ 8.13.0,8503000 8.13.1,8503000 8.13.2,8503000 +8.13.3,8503000 From f02761f729445922cf9dd46a5389d3ecc8f661d2 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 3 May 2024 15:14:10 +0000 Subject: [PATCH 15/17] Prune changelogs after 8.13.3 release --- docs/changelog/106077.yaml | 7 ------- docs/changelog/106824.yaml | 5 ----- docs/changelog/106975.yaml | 5 ----- docs/changelog/107122.yaml | 5 ----- docs/changelog/107131.yaml | 6 ------ docs/changelog/107328.yaml | 7 ------- docs/changelog/107333.yaml | 18 ------------------ docs/changelog/107355.yaml | 6 ------ docs/changelog/107432.yaml | 6 ------ docs/changelog/107785.yaml | 5 ----- 10 files changed, 70 deletions(-) delete mode 100644 docs/changelog/106077.yaml delete mode 100644 docs/changelog/106824.yaml delete mode 100644 docs/changelog/106975.yaml delete mode 100644 docs/changelog/107122.yaml delete mode 100644 docs/changelog/107131.yaml delete mode 100644 docs/changelog/107328.yaml delete mode 100644 docs/changelog/107333.yaml delete mode 100644 docs/changelog/107355.yaml delete mode 100644 docs/changelog/107432.yaml delete mode 100644 docs/changelog/107785.yaml diff --git a/docs/changelog/106077.yaml b/docs/changelog/106077.yaml deleted file mode 100644 index eb987cd9617f8..0000000000000 --- a/docs/changelog/106077.yaml +++ /dev/null @@ -1,7 +0,0 @@ -pr: 106077 -summary: Fix merging component templates with a mix of dotted and nested object mapper - definitions -area: Mapping -type: bug -issues: - - 105482 diff --git a/docs/changelog/106824.yaml b/docs/changelog/106824.yaml deleted file mode 100644 index 0a2001df5039a..0000000000000 --- a/docs/changelog/106824.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 106824 -summary: "ESQL: Introduce language versioning to REST API" -area: ES|QL -type: enhancement -issues: [] diff --git a/docs/changelog/106975.yaml b/docs/changelog/106975.yaml deleted file mode 100644 index bd32b3574c4f9..0000000000000 --- a/docs/changelog/106975.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 106975 -summary: GET /_all should return hidden indices with visible aliases -area: Indices APIs -type: bug -issues: [] diff --git a/docs/changelog/107122.yaml b/docs/changelog/107122.yaml deleted file mode 100644 index e227bfd45b939..0000000000000 --- a/docs/changelog/107122.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 107122 -summary: Avoid unintentionally clearing the `DataStream.rolloverOnWrite` flag -area: Data streams -type: bug -issues: [] diff --git a/docs/changelog/107131.yaml b/docs/changelog/107131.yaml deleted file mode 100644 index ebb696931777b..0000000000000 --- a/docs/changelog/107131.yaml +++ /dev/null @@ -1,6 +0,0 @@ -pr: 107131 -summary: "ESQL: Fix bug when combining projections" -area: ES|QL -type: bug -issues: - - 107083 diff --git a/docs/changelog/107328.yaml b/docs/changelog/107328.yaml deleted file mode 100644 index a608d7567ddef..0000000000000 --- a/docs/changelog/107328.yaml +++ /dev/null @@ -1,7 +0,0 @@ -pr: 107328 -summary: "ESQL: Fix missing refs due to pruning renamed grouping columns" -area: ES|QL -type: bug -issues: - - 107083 - - 107166 diff --git a/docs/changelog/107333.yaml b/docs/changelog/107333.yaml deleted file mode 100644 index 0762e9a19795c..0000000000000 --- a/docs/changelog/107333.yaml +++ /dev/null @@ -1,18 +0,0 @@ -pr: 107333 -summary: Limit how much space some string functions can use -area: SQL -type: breaking -issues: [] -breaking: - title: Limit how much space some string functions can use - area: REST API - details: "Before this change, some of the string functions could return a result\ - \ of any arbitrary length, which could force the VM to allocate large chunks of\ - \ memory or even make it exit. Any user with access to the SQL API can invoke\ - \ these functions. This change introduces a limitation of how much memory the\ - \ result returned by a function call can consume. The functions affected by this\ - \ change are: CONCAT, INSERT, REPEAT, REPLACE and SPACE." - impact: "The affected functions used to return a result of any length. After this\ - \ change, a result can no longer exceed 1MB in length. Note that this is a bytes\ - \ length, the character count may be lower." - notable: false diff --git a/docs/changelog/107355.yaml b/docs/changelog/107355.yaml deleted file mode 100644 index 1d4813b877e58..0000000000000 --- a/docs/changelog/107355.yaml +++ /dev/null @@ -1,6 +0,0 @@ -pr: 107355 -summary: Handle exceptions thrown by HTTP header validation -area: Network -type: bug -issues: - - 107338 diff --git a/docs/changelog/107432.yaml b/docs/changelog/107432.yaml deleted file mode 100644 index c492644c5baf2..0000000000000 --- a/docs/changelog/107432.yaml +++ /dev/null @@ -1,6 +0,0 @@ -pr: 107432 -summary: "Percolator named queries: rewrite for matched info" -area: Percolator -type: bug -issues: - - 107176 diff --git a/docs/changelog/107785.yaml b/docs/changelog/107785.yaml deleted file mode 100644 index fae01a7da597d..0000000000000 --- a/docs/changelog/107785.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 107785 -summary: Fix `minimized_round_trips` in lookup runtime fields -area: Search -type: bug -issues: [] From 75e02bcb7a64c828904a3097da24cb6a1e3a7a17 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 3 May 2024 15:39:47 +0000 Subject: [PATCH 16/17] Bump versions after 7.17.21 release --- .buildkite/pipelines/intake.yml | 2 +- .buildkite/pipelines/periodic-packaging.yml | 6 +++--- .buildkite/pipelines/periodic.yml | 10 +++++----- .ci/bwcVersions | 2 +- .ci/snapshotBwcVersions | 2 +- server/src/main/java/org/elasticsearch/Version.java | 1 + .../resources/org/elasticsearch/TransportVersions.csv | 1 + .../org/elasticsearch/index/IndexVersions.csv | 1 + 8 files changed, 14 insertions(+), 11 deletions(-) diff --git a/.buildkite/pipelines/intake.yml b/.buildkite/pipelines/intake.yml index fbac4f75985c4..e702c97248cdc 100644 --- a/.buildkite/pipelines/intake.yml +++ b/.buildkite/pipelines/intake.yml @@ -56,7 +56,7 @@ steps: timeout_in_minutes: 300 matrix: setup: - BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.22", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.buildkite/pipelines/periodic-packaging.yml b/.buildkite/pipelines/periodic-packaging.yml index eee61ef2a9cce..d013780b1fd0a 100644 --- a/.buildkite/pipelines/periodic-packaging.yml +++ b/.buildkite/pipelines/periodic-packaging.yml @@ -305,8 +305,8 @@ steps: env: BWC_VERSION: 7.16.3 - - label: "{{matrix.image}} / 7.17.21 / packaging-tests-upgrade" - command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v7.17.21 + - label: "{{matrix.image}} / 7.17.22 / packaging-tests-upgrade" + command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v7.17.22 timeout_in_minutes: 300 matrix: setup: @@ -319,7 +319,7 @@ steps: machineType: custom-16-32768 buildDirectory: /dev/shm/bk env: - BWC_VERSION: 7.17.21 + BWC_VERSION: 7.17.22 - label: "{{matrix.image}} / 8.0.1 / packaging-tests-upgrade" command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.0.1 diff --git a/.buildkite/pipelines/periodic.yml b/.buildkite/pipelines/periodic.yml index 97d08e6ad9d06..7e9f6872b9146 100644 --- a/.buildkite/pipelines/periodic.yml +++ b/.buildkite/pipelines/periodic.yml @@ -325,8 +325,8 @@ steps: - signal_reason: agent_stop limit: 3 - - label: 7.17.21 / bwc - command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v7.17.21#bwcTest + - label: 7.17.22 / bwc + command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v7.17.22#bwcTest timeout_in_minutes: 300 agents: provider: gcp @@ -335,7 +335,7 @@ steps: buildDirectory: /dev/shm/bk preemptible: true env: - BWC_VERSION: 7.17.21 + BWC_VERSION: 7.17.22 retry: automatic: - exit_status: "-1" @@ -714,7 +714,7 @@ steps: setup: ES_RUNTIME_JAVA: - openjdk17 - BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.22", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 @@ -760,7 +760,7 @@ steps: - openjdk17 - openjdk21 - openjdk22 - BWC_VERSION: ["7.17.21", "8.13.4", "8.14.0", "8.15.0"] + BWC_VERSION: ["7.17.22", "8.13.4", "8.14.0", "8.15.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.ci/bwcVersions b/.ci/bwcVersions index 7be802285b300..77e51005d5ace 100644 --- a/.ci/bwcVersions +++ b/.ci/bwcVersions @@ -16,7 +16,7 @@ BWC_VERSION: - "7.14.2" - "7.15.2" - "7.16.3" - - "7.17.21" + - "7.17.22" - "8.0.1" - "8.1.3" - "8.2.3" diff --git a/.ci/snapshotBwcVersions b/.ci/snapshotBwcVersions index 7b85a5c341e43..49f3708ce4af9 100644 --- a/.ci/snapshotBwcVersions +++ b/.ci/snapshotBwcVersions @@ -1,5 +1,5 @@ BWC_VERSION: - - "7.17.21" + - "7.17.22" - "8.13.4" - "8.14.0" - "8.15.0" diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index 2af65ffac3d1f..a2e04d0bf3d48 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -121,6 +121,7 @@ public class Version implements VersionId, ToXContentFragment { public static final Version V_7_17_19 = new Version(7_17_19_99); public static final Version V_7_17_20 = new Version(7_17_20_99); public static final Version V_7_17_21 = new Version(7_17_21_99); + public static final Version V_7_17_22 = new Version(7_17_22_99); public static final Version V_8_0_0 = new Version(8_00_00_99); public static final Version V_8_0_1 = new Version(8_00_01_99); diff --git a/server/src/main/resources/org/elasticsearch/TransportVersions.csv b/server/src/main/resources/org/elasticsearch/TransportVersions.csv index 842d651a14fbe..e6f0da6a45452 100644 --- a/server/src/main/resources/org/elasticsearch/TransportVersions.csv +++ b/server/src/main/resources/org/elasticsearch/TransportVersions.csv @@ -68,6 +68,7 @@ 7.17.18,7171899 7.17.19,7171999 7.17.20,7172099 +7.17.21,7172199 8.0.0,8000099 8.0.1,8000199 8.1.0,8010099 diff --git a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv index 432c6d8a08816..bc6523c98761c 100644 --- a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv +++ b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv @@ -68,6 +68,7 @@ 7.17.18,7171899 7.17.19,7171999 7.17.20,7172099 +7.17.21,7172199 8.0.0,8000099 8.0.1,8000199 8.1.0,8010099 From eb90e362350a8d19617bbbb3b1a0a68e4732884f Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 3 May 2024 16:57:20 +0100 Subject: [PATCH 17/17] Fix serialization of put/delete shutdown requests (#107862) Co-authored-by: Simon Cooper --- docs/changelog/107862.yaml | 6 + .../org/elasticsearch/TransportVersions.java | 3 + .../shutdown/DeleteShutdownNodeAction.java | 15 ++ .../xpack/shutdown/PutShutdownNodeAction.java | 18 ++ .../shutdown/DeleteShutdownRequestTests.java | 82 ++++++++ .../shutdown/PutShutdownRequestTests.java | 196 ++++++++++++++++++ 6 files changed, 320 insertions(+) create mode 100644 docs/changelog/107862.yaml create mode 100644 x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/DeleteShutdownRequestTests.java create mode 100644 x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/PutShutdownRequestTests.java diff --git a/docs/changelog/107862.yaml b/docs/changelog/107862.yaml new file mode 100644 index 0000000000000..77f7a8c9fb02a --- /dev/null +++ b/docs/changelog/107862.yaml @@ -0,0 +1,6 @@ +pr: 107862 +summary: Fix serialization of put-shutdown request +area: Infra/Node Lifecycle +type: bug +issues: + - 107857 diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index fb7458662edf7..f9ab7944714a4 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -133,6 +133,7 @@ static TransportVersion def(int id) { public static final TransportVersion INDEX_REQUEST_NORMALIZED_BYTES_PARSED = def(8_593_00_0); public static final TransportVersion INGEST_GRAPH_STRUCTURE_EXCEPTION = def(8_594_00_0); public static final TransportVersion V_8_13_0 = def(8_595_00_0); + public static final TransportVersion SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_13 = def(8_595_00_1); // 8.14.0+ public static final TransportVersion RANDOM_AGG_SHARD_SEED = def(8_596_00_0); public static final TransportVersion ESQL_TIMINGS = def(8_597_00_0); @@ -175,6 +176,7 @@ static TransportVersion def(int id) { public static final TransportVersion ML_INFERENCE_AZURE_OPENAI_EMBEDDINGS = def(8_634_00_0); public static final TransportVersion ILM_SHRINK_ENABLE_WRITE = def(8_635_00_0); public static final TransportVersion GEOIP_CACHE_STATS = def(8_636_00_0); + public static final TransportVersion SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_14 = def(8_636_00_1); public static final TransportVersion WATERMARK_THRESHOLDS_STATS = def(8_637_00_0); public static final TransportVersion ENRICH_CACHE_ADDITIONAL_STATS = def(8_638_00_0); public static final TransportVersion ML_INFERENCE_RATE_LIMIT_SETTINGS_ADDED = def(8_639_00_0); @@ -189,6 +191,7 @@ static TransportVersion def(int id) { public static final TransportVersion ESQL_REQUEST_TABLES = def(8_648_00_0); public static final TransportVersion ROLE_REMOTE_CLUSTER_PRIVS = def(8_649_00_0); public static final TransportVersion NO_GLOBAL_RETENTION_FOR_SYSTEM_DATA_STREAMS = def(8_650_00_0); + public static final TransportVersion SHUTDOWN_REQUEST_TIMEOUTS_FIX = def(8_651_00_0); /* * STOP! READ THIS FIRST! No, really, * ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _ diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/DeleteShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/DeleteShutdownNodeAction.java index 75c36f063f805..4f7b16380d0f8 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/DeleteShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/DeleteShutdownNodeAction.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.shutdown; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedRequest; @@ -14,6 +15,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.TaskId; import java.io.IOException; @@ -35,11 +37,24 @@ public Request(String nodeId) { } public Request(StreamInput in) throws IOException { + if (in.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_13) + || in.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_14) + || in.getTransportVersion().onOrAfter(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX)) { + // effectively super(in): + setParentTask(TaskId.readFromStream(in)); + masterNodeTimeout(in.readTimeValue()); + ackTimeout(in.readTimeValue()); + } this.nodeId = in.readString(); } @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_13) + || out.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_14) + || out.getTransportVersion().onOrAfter(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX)) { + super.writeTo(out); + } out.writeString(this.nodeId); } diff --git a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java index d05b60cd947f5..bff2b0b1793b1 100644 --- a/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java +++ b/x-pack/plugin/shutdown/src/main/java/org/elasticsearch/xpack/shutdown/PutShutdownNodeAction.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.shutdown; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.master.AcknowledgedRequest; @@ -17,6 +18,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.UpdateForV9; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentParser; @@ -95,7 +98,16 @@ public Request( this.gracePeriod = gracePeriod; } + @UpdateForV9 // TODO call super(in) instead of explicitly reading superclass contents once bwc no longer needed public Request(StreamInput in) throws IOException { + if (in.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_13) + || in.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_14) + || in.getTransportVersion().onOrAfter(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX)) { + // effectively super(in): + setParentTask(TaskId.readFromStream(in)); + masterNodeTimeout(in.readTimeValue()); + ackTimeout(in.readTimeValue()); + } this.nodeId = in.readString(); this.type = in.readEnum(SingleNodeShutdownMetadata.Type.class); this.reason = in.readString(); @@ -114,6 +126,11 @@ public Request(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_13) + || out.getTransportVersion().isPatchFrom(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX_8_14) + || out.getTransportVersion().onOrAfter(TransportVersions.SHUTDOWN_REQUEST_TIMEOUTS_FIX)) { + super.writeTo(out); + } out.writeString(nodeId); if (out.getTransportVersion().before(REPLACE_SHUTDOWN_TYPE_ADDED_VERSION) && this.type == SingleNodeShutdownMetadata.Type.REPLACE) { @@ -207,5 +224,6 @@ public ActionRequestValidationException validate() { return null; } } + } } diff --git a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/DeleteShutdownRequestTests.java b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/DeleteShutdownRequestTests.java new file mode 100644 index 0000000000000..59c1e674731b9 --- /dev/null +++ b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/DeleteShutdownRequestTests.java @@ -0,0 +1,82 @@ +/* + * 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.shutdown; + +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +public class DeleteShutdownRequestTests extends AbstractWireSerializingTestCase { + + /** + * Wraps a {@link DeleteShutdownNodeAction.Request} to add proper equality checks + */ + record RequestWrapper(String nodeId, TaskId parentTask, TimeValue masterNodeTimeout, TimeValue ackTimeout) implements Writeable { + @Override + public void writeTo(StreamOutput out) throws IOException { + final var request = new DeleteShutdownNodeAction.Request(nodeId); + request.setParentTask(parentTask); + request.ackTimeout(ackTimeout); + request.masterNodeTimeout(masterNodeTimeout); + request.writeTo(out); + } + } + + @Override + protected Writeable.Reader instanceReader() { + return in -> { + final var request = new DeleteShutdownNodeAction.Request(in); + return new RequestWrapper(request.getNodeId(), request.getParentTask(), request.masterNodeTimeout(), request.ackTimeout()); + }; + } + + @Override + protected RequestWrapper createTestInstance() { + return new RequestWrapper(randomIdentifier(), randomTaskId(), randomTimeValue(), randomTimeValue()); + } + + private static TaskId randomTaskId() { + return randomBoolean() ? TaskId.EMPTY_TASK_ID : new TaskId(randomIdentifier(), randomNonNegativeLong()); + } + + @Override + protected RequestWrapper mutateInstance(RequestWrapper instance) { + return switch (between(1, 4)) { + case 1 -> new RequestWrapper( + randomValueOtherThan(instance.nodeId, ESTestCase::randomIdentifier), + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 2 -> new RequestWrapper( + instance.nodeId, + randomValueOtherThan(instance.parentTask, DeleteShutdownRequestTests::randomTaskId), + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 3 -> new RequestWrapper( + instance.nodeId, + instance.parentTask, + randomValueOtherThan(instance.ackTimeout, ESTestCase::randomTimeValue), + instance.masterNodeTimeout + ); + case 4 -> new RequestWrapper( + instance.nodeId, + instance.parentTask, + instance.ackTimeout, + randomValueOtherThan(instance.masterNodeTimeout, ESTestCase::randomTimeValue) + ); + default -> throw new AssertionError("impossible"); + }; + } +} diff --git a/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/PutShutdownRequestTests.java b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/PutShutdownRequestTests.java new file mode 100644 index 0000000000000..516431f40c170 --- /dev/null +++ b/x-pack/plugin/shutdown/src/test/java/org/elasticsearch/xpack/shutdown/PutShutdownRequestTests.java @@ -0,0 +1,196 @@ +/* + * 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.shutdown; + +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; + +public class PutShutdownRequestTests extends AbstractWireSerializingTestCase { + + /** + * Wraps a {@link org.elasticsearch.xpack.shutdown.PutShutdownNodeAction.Request} to add proper equality checks + */ + record RequestWrapper( + String nodeId, + SingleNodeShutdownMetadata.Type type, + String reason, + TimeValue allocationDelay, + String targetNodeName, + TimeValue gracePeriod, + TaskId parentTask, + TimeValue masterNodeTimeout, + TimeValue ackTimeout + ) implements Writeable { + @Override + public void writeTo(StreamOutput out) throws IOException { + final var request = new PutShutdownNodeAction.Request(nodeId, type, reason, allocationDelay, targetNodeName, gracePeriod); + request.setParentTask(parentTask); + request.ackTimeout(ackTimeout); + request.masterNodeTimeout(masterNodeTimeout); + request.writeTo(out); + } + } + + @Override + protected Writeable.Reader instanceReader() { + return in -> { + final var request = new PutShutdownNodeAction.Request(in); + return new RequestWrapper( + request.getNodeId(), + request.getType(), + request.getReason(), + request.getAllocationDelay(), + request.getTargetNodeName(), + request.getGracePeriod(), + request.getParentTask(), + request.masterNodeTimeout(), + request.ackTimeout() + ); + }; + } + + @Override + protected RequestWrapper createTestInstance() { + return new RequestWrapper( + randomIdentifier(), + randomFrom(SingleNodeShutdownMetadata.Type.values()), + randomIdentifier(), + randomOptionalTimeValue(), + randomOptionalIdentifier(), + randomOptionalTimeValue(), + randomTaskId(), + randomTimeValue(), + randomTimeValue() + ); + } + + private static String randomOptionalIdentifier() { + return randomBoolean() ? null : randomIdentifier(); + } + + private static TimeValue randomOptionalTimeValue() { + return randomBoolean() ? null : randomTimeValue(); + } + + private static TaskId randomTaskId() { + return randomBoolean() ? TaskId.EMPTY_TASK_ID : new TaskId(randomIdentifier(), randomNonNegativeLong()); + } + + @Override + protected RequestWrapper mutateInstance(RequestWrapper instance) { + return switch (between(1, 9)) { + case 1 -> new RequestWrapper( + randomValueOtherThan(instance.nodeId, ESTestCase::randomIdentifier), + instance.type, + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 2 -> new RequestWrapper( + instance.nodeId, + randomValueOtherThan(instance.type, () -> randomFrom(SingleNodeShutdownMetadata.Type.values())), + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 3 -> new RequestWrapper( + instance.nodeId, + instance.type, + randomValueOtherThan(instance.reason, ESTestCase::randomIdentifier), + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 4 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + randomValueOtherThan(instance.allocationDelay, PutShutdownRequestTests::randomOptionalTimeValue), + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 5 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + instance.allocationDelay, + randomValueOtherThan(instance.targetNodeName, PutShutdownRequestTests::randomOptionalIdentifier), + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 6 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + randomValueOtherThan(instance.gracePeriod, PutShutdownRequestTests::randomOptionalTimeValue), + instance.parentTask, + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 7 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + randomValueOtherThan(instance.parentTask, PutShutdownRequestTests::randomTaskId), + instance.ackTimeout, + instance.masterNodeTimeout + ); + case 8 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + randomValueOtherThan(instance.ackTimeout, ESTestCase::randomTimeValue), + instance.masterNodeTimeout + ); + case 9 -> new RequestWrapper( + instance.nodeId, + instance.type, + instance.reason, + instance.allocationDelay, + instance.targetNodeName, + instance.gracePeriod, + instance.parentTask, + instance.ackTimeout, + randomValueOtherThan(instance.masterNodeTimeout, ESTestCase::randomTimeValue) + ); + default -> throw new AssertionError("impossible"); + }; + } +}