From 75b5efede488f130e5fafc25bae7a648772ffdc4 Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Tue, 23 Apr 2024 08:24:47 -0700 Subject: [PATCH 01/27] Binary field enables doc values by default for index mode with synthetic source (#107739) Binary field enables doc values by default for index mode with synthetic source --- docs/changelog/107739.yaml | 6 +++ .../percolator/PercolatorFieldMapper.java | 3 +- .../index/mapper/BinaryFieldMapper.java | 25 +++++++---- .../index/mapper/DynamicFieldsBuilder.java | 5 ++- .../fielddata/AbstractFieldDataTestCase.java | 5 ++- .../index/mapper/BinaryFieldMapperTests.java | 43 +++++++++++++++++++ .../CountedKeywordFieldMapper.java | 5 ++- 7 files changed, 79 insertions(+), 13 deletions(-) create mode 100644 docs/changelog/107739.yaml diff --git a/docs/changelog/107739.yaml b/docs/changelog/107739.yaml new file mode 100644 index 000000000000..c55a0e332b4f --- /dev/null +++ b/docs/changelog/107739.yaml @@ -0,0 +1,6 @@ +pr: 107739 +summary: Binary field enables doc values by default for index mode with synthe… +area: Mapping +type: enhancement +issues: + - 107554 diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java index b571766e12b8..954dd9db0d37 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolatorFieldMapper.java @@ -189,7 +189,8 @@ static KeywordFieldMapper createExtractQueryFieldBuilder( } static BinaryFieldMapper createQueryBuilderFieldBuilder(MapperBuilderContext context) { - BinaryFieldMapper.Builder builder = new BinaryFieldMapper.Builder(QUERY_BUILDER_FIELD_NAME, true); + BinaryFieldMapper.Builder builder = new BinaryFieldMapper.Builder(QUERY_BUILDER_FIELD_NAME, context.isSourceSynthetic()) + .docValues(true); return builder.build(context); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BinaryFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/BinaryFieldMapper.java index 1159bdd64d9c..3d72d5f5d55a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/BinaryFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/BinaryFieldMapper.java @@ -46,16 +46,15 @@ private static BinaryFieldMapper toType(FieldMapper in) { public static class Builder extends FieldMapper.Builder { private final Parameter stored = Parameter.storeParam(m -> toType(m).stored, false); - private final Parameter hasDocValues = Parameter.docValuesParam(m -> toType(m).hasDocValues, false); private final Parameter> meta = Parameter.metaParam(); - public Builder(String name) { - this(name, false); - } + private final boolean isSyntheticSourceEnabledViaIndexMode; + private final Parameter hasDocValues; - public Builder(String name, boolean hasDocValues) { + public Builder(String name, boolean isSyntheticSourceEnabledViaIndexMode) { super(name); - this.hasDocValues.setValue(hasDocValues); + this.isSyntheticSourceEnabledViaIndexMode = isSyntheticSourceEnabledViaIndexMode; + this.hasDocValues = Parameter.docValuesParam(m -> toType(m).hasDocValues, isSyntheticSourceEnabledViaIndexMode); } @Override @@ -63,6 +62,11 @@ public Parameter[] getParameters() { return new Parameter[] { meta, stored, hasDocValues }; } + public BinaryFieldMapper.Builder docValues(boolean hasDocValues) { + this.hasDocValues.setValue(hasDocValues); + return this; + } + @Override public BinaryFieldMapper build(MapperBuilderContext context) { return new BinaryFieldMapper( @@ -75,10 +79,11 @@ public BinaryFieldMapper build(MapperBuilderContext context) { } } - public static final TypeParser PARSER = new TypeParser((n, c) -> new Builder(n)); + public static final TypeParser PARSER = new TypeParser( + (n, c) -> new Builder(n, c.getIndexSettings().getMode().isSyntheticSourceEnabled()) + ); public static final class BinaryFieldType extends MappedFieldType { - private BinaryFieldType(String name, boolean isStored, boolean hasDocValues, Map meta) { super(name, false, isStored, hasDocValues, TextSearchInfo.NONE, meta); } @@ -135,6 +140,7 @@ public Query termQuery(Object value, SearchExecutionContext context) { private final boolean stored; private final boolean hasDocValues; + private final boolean isSyntheticSourceEnabledViaIndexMode; protected BinaryFieldMapper( String simpleName, @@ -146,6 +152,7 @@ protected BinaryFieldMapper( super(simpleName, mappedFieldType, multiFields, copyTo); this.stored = builder.stored.getValue(); this.hasDocValues = builder.hasDocValues.getValue(); + this.isSyntheticSourceEnabledViaIndexMode = builder.isSyntheticSourceEnabledViaIndexMode; } @Override @@ -185,7 +192,7 @@ public void indexValue(DocumentParserContext context, byte[] value) { @Override public FieldMapper.Builder getMergeBuilder() { - return new BinaryFieldMapper.Builder(simpleName()).init(this); + return new BinaryFieldMapper.Builder(simpleName(), isSyntheticSourceEnabledViaIndexMode).init(this); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java b/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java index 799042b4f3a8..d479cb97e3fd 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DynamicFieldsBuilder.java @@ -411,7 +411,10 @@ public boolean newDynamicDateField(DocumentParserContext context, String name, D } boolean newDynamicBinaryField(DocumentParserContext context, String name) throws IOException { - return createDynamicField(new BinaryFieldMapper.Builder(name), context); + return createDynamicField( + new BinaryFieldMapper.Builder(name, context.indexSettings().getMode().isSyntheticSourceEnabled()), + context + ); } } diff --git a/server/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java b/server/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java index 683bfb19aac2..7790dff360b3 100644 --- a/server/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java @@ -161,7 +161,10 @@ public > IFD getForField(String type, String field docValues ).build(context).fieldType(); } else if (type.equals("binary")) { - fieldType = new BinaryFieldMapper.Builder(fieldName, docValues).build(context).fieldType(); + fieldType = new BinaryFieldMapper.Builder(fieldName, indexService.getIndexSettings().getMode().isSyntheticSourceEnabled()) + .docValues(docValues) + .build(context) + .fieldType(); } else { throw new UnsupportedOperationException(type); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/BinaryFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/BinaryFieldMapperTests.java index e5d35855d605..aef02494b9a1 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/BinaryFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/BinaryFieldMapperTests.java @@ -8,14 +8,18 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.IndexMode; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.xcontent.XContentBuilder; import org.junit.AssumptionViolatedException; @@ -134,6 +138,45 @@ public void testStoredValue() throws IOException { } } + public void testDefaultsForTimeSeriesIndex() throws IOException { + var isStored = randomBoolean(); + + var indexSettings = getIndexSettingsBuilder().put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES) + .putList(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "dimension") + .put(IndexSettings.TIME_SERIES_START_TIME.getKey(), "2000-01-08T23:40:53.384Z") + .put(IndexSettings.TIME_SERIES_END_TIME.getKey(), "2106-01-08T23:40:53.384Z") + .build(); + + var mapping = mapping(b -> { + b.startObject("field"); + b.field("type", "binary"); + if (isStored) { + b.field("store", "true"); + } + b.endObject(); + + b.startObject("@timestamp"); + b.field("type", "date"); + b.endObject(); + b.startObject("dimension"); + b.field("type", "keyword"); + b.field("time_series_dimension", "true"); + b.endObject(); + }); + DocumentMapper mapper = createMapperService(getVersion(), indexSettings, () -> true, mapping).documentMapper(); + + var source = source(TimeSeriesRoutingHashFieldMapper.DUMMY_ENCODED_VALUE, b -> { + b.field("field", Base64.getEncoder().encodeToString(randomByteArrayOfLength(10))); + b.field("@timestamp", randomMillisUpToYear9999()); + b.field("dimension", "dimension1"); + }, null); + ParsedDocument doc = mapper.parse(source); + + List fields = doc.rootDoc().getFields("field"); + var docValuesField = fields.stream().filter(f -> f.fieldType().docValuesType() == DocValuesType.BINARY).findFirst(); + assertTrue("Doc values are not present", docValuesField.isPresent()); + } + @Override protected Object generateRandomInputValue(MappedFieldType ft) { if (rarely()) return null; diff --git a/x-pack/plugin/mapper-counted-keyword/src/main/java/org/elasticsearch/xpack/countedkeyword/CountedKeywordFieldMapper.java b/x-pack/plugin/mapper-counted-keyword/src/main/java/org/elasticsearch/xpack/countedkeyword/CountedKeywordFieldMapper.java index 878a949a6984..0025d3f7dd7b 100644 --- a/x-pack/plugin/mapper-counted-keyword/src/main/java/org/elasticsearch/xpack/countedkeyword/CountedKeywordFieldMapper.java +++ b/x-pack/plugin/mapper-counted-keyword/src/main/java/org/elasticsearch/xpack/countedkeyword/CountedKeywordFieldMapper.java @@ -289,7 +289,10 @@ protected Parameter[] getParameters() { @Override public FieldMapper build(MapperBuilderContext context) { - BinaryFieldMapper countFieldMapper = new BinaryFieldMapper.Builder(name() + COUNT_FIELD_NAME_SUFFIX, true).build(context); + BinaryFieldMapper countFieldMapper = new BinaryFieldMapper.Builder( + name() + COUNT_FIELD_NAME_SUFFIX, + context.isSourceSynthetic() + ).docValues(true).build(context); boolean isIndexed = indexed.getValue(); FieldType ft = isIndexed ? FIELD_TYPE_INDEXED : FIELD_TYPE_NOT_INDEXED; return new CountedKeywordFieldMapper( From a97e8fa2c843486134a68caabbef4736f117f71f Mon Sep 17 00:00:00 2001 From: Max Hniebergall <137079448+maxhniebergall@users.noreply.github.com> Date: Tue, 23 Apr 2024 12:24:58 -0400 Subject: [PATCH 02/27] [ML] Fix LearningToRankRescorerIT (#107592) * Changed tests to avoid errors --- .../integration/LearningToRankRescorerIT.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/LearningToRankRescorerIT.java b/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/LearningToRankRescorerIT.java index ecfa86804627..f6aca48a3f49 100644 --- a/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/LearningToRankRescorerIT.java +++ b/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/LearningToRankRescorerIT.java @@ -9,6 +9,7 @@ import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.junit.Before; @@ -196,7 +197,6 @@ public void setupModelAndData() throws IOException { adminClient().performRequest(new Request("POST", INDEX_NAME + "/_refresh")); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103072") public void testLearningToRankRescore() throws Exception { Request request = new Request("GET", "store/_search?size=3&error_trace"); request.setJsonEntity(""" @@ -232,7 +232,6 @@ public void testLearningToRankRescore() throws Exception { assertHitScores(client().performRequest(request), List.of(9.0, 9.0, 6.0)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103072") public void testLearningToRankRescoreSmallWindow() throws Exception { Request request = new Request("GET", "store/_search?size=5"); request.setJsonEntity(""" @@ -242,30 +241,33 @@ public void testLearningToRankRescoreSmallWindow() throws Exception { "learning_to_rank": { "model_id": "ltr-model" } } }"""); - assertHitScores(client().performRequest(request), List.of(20.0, 20.0, 1.0, 1.0, 1.0)); + assertThrows( + "Rescore window is too small and should be at least the value of from + size but was [2]", + ResponseException.class, + () -> client().performRequest(request) + ); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103072") public void testLearningToRankRescorerWithChainedRescorers() throws IOException { Request request = new Request("GET", "store/_search?size=5"); request.setJsonEntity(""" { "rescore": [ { - "window_size": 4, + "window_size": 15, "query": { "rescore_query" : { "script_score": { "query": { "match_all": {} }, "script": { "source": "return 4" } } } } }, { - "window_size": 3, + "window_size": 25, "learning_to_rank": { "model_id": "ltr-model" } }, { - "window_size": 2, + "window_size": 35, "query": { "rescore_query": { "script_score": { "query": { "match_all": {} }, "script": { "source": "return 20"} } } } } ] }"""); - assertHitScores(client().performRequest(request), List.of(40.0, 40.0, 17.0, 5.0, 1.0)); + assertHitScores(client().performRequest(request), List.of(40.0, 40.0, 37.0, 29.0, 29.0)); } private void indexData(String data) throws IOException { From a912cb0371bde1e224abf2c8fdafd12d365c17c5 Mon Sep 17 00:00:00 2001 From: James Baiera Date: Tue, 23 Apr 2024 16:09:59 -0400 Subject: [PATCH 03/27] Fix bulk NPE when retrying failure redirect after cluster block (#107598) This PR fixes a bug in the bulk operation when retrying blocked cluster states before executing a failure store write by correctly wrapping the retry runnable to keep it from prematurely returning a null response. --- docs/changelog/107598.yaml | 5 + .../action/bulk/BulkOperation.java | 6 +- .../action/bulk/BulkOperationTests.java | 247 ++++++++++++++---- 3 files changed, 203 insertions(+), 55 deletions(-) create mode 100644 docs/changelog/107598.yaml diff --git a/docs/changelog/107598.yaml b/docs/changelog/107598.yaml new file mode 100644 index 000000000000..125bbe759d2e --- /dev/null +++ b/docs/changelog/107598.yaml @@ -0,0 +1,5 @@ +pr: 107598 +summary: Fix bulk NPE when retrying failure redirect after cluster block +area: Data streams +type: bug +issues: [] diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java index ea4d27822784..fcad07d0696f 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkOperation.java @@ -161,7 +161,11 @@ private void doRedirectFailures() { assert failureStoreRedirects.isEmpty() != true : "Attempting to redirect failures, but none were present in the queue"; final ClusterState clusterState = observer.setAndGetObservedState(); // If the cluster is blocked at this point, discard the failure store redirects and complete the response with the original failures - if (handleBlockExceptions(clusterState, ActionRunnable.run(listener, this::doRedirectFailures), this::discardRedirectsAndFinish)) { + if (handleBlockExceptions( + clusterState, + ActionRunnable.wrap(listener, (l) -> this.doRedirectFailures()), + this::discardRedirectsAndFinish + )) { return; } Map> requestsByShard = drainAndGroupRedirectsByShards(clusterState); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkOperationTests.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkOperationTests.java index b662f439a0e6..742e0207b2cd 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkOperationTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.mapper.MapperException; @@ -49,6 +48,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.client.NoOpNodeClient; import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Assume; import org.junit.Before; @@ -58,8 +58,12 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import java.util.function.Supplier; import static org.hamcrest.CoreMatchers.equalTo; @@ -168,10 +172,7 @@ public void tearDownThreadpool() { * If a bulk operation begins and the cluster is experiencing a non-retryable block, the bulk operation should fail */ public void testClusterBlockedFailsBulk() { - NodeClient client = getNodeClient((r) -> { - fail("Should not have executed shard action on blocked cluster"); - return null; - }); + NodeClient client = getNodeClient(assertNoClientInteraction()); CompletableFuture future = new CompletableFuture<>(); ActionListener listener = ActionListener.wrap(future::complete, future::completeExceptionally); @@ -196,10 +197,7 @@ public void testClusterBlockedFailsBulk() { * If a bulk operation times out while waiting for cluster blocks to be cleared, it should fail the request. */ public void testTimeoutOnRetryableClusterBlockedFailsBulk() { - NodeClient client = getNodeClient((r) -> { - fail("Should not have executed shard action on blocked cluster"); - return null; - }); + NodeClient client = getNodeClient(assertNoClientInteraction()); CompletableFuture future = new CompletableFuture<>(); ActionListener listener = ActionListener.wrap(future::complete, future::completeExceptionally); @@ -234,10 +232,7 @@ public void testTimeoutOnRetryableClusterBlockedFailsBulk() { * If the cluster service closes while a bulk operation is waiting for cluster blocks to be cleared, it should fail the request. */ public void testNodeClosedOnRetryableClusterBlockedFailsBulk() { - NodeClient client = getNodeClient((r) -> { - fail("Should not have executed shard action on blocked cluster"); - return null; - }); + NodeClient client = getNodeClient(assertNoClientInteraction()); CompletableFuture future = new CompletableFuture<>(); ActionListener listener = ActionListener.wrap(future::complete, future::completeExceptionally); @@ -272,7 +267,7 @@ public void testBulkToIndex() throws Exception { bulkRequest.add(new IndexRequest(indexName).id("1").source(Map.of("key", "val"))); bulkRequest.add(new IndexRequest(indexName).id("3").source(Map.of("key", "val"))); - NodeClient client = getNodeClient(this::acceptAllShardWrites); + NodeClient client = getNodeClient(acceptAllShardWrites()); CompletableFuture future = new CompletableFuture<>(); ActionListener listener = ActionListener.wrap(future::complete, future::completeExceptionally); @@ -293,7 +288,7 @@ public void testBulkToIndexFailingEntireShard() throws Exception { bulkRequest.add(new IndexRequest(indexName).id("3").source(Map.of("key", "val"))); NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(indexMetadata.getIndex(), 0), () -> new MapperException("test"))) + shardSpecificResponse(Map.of(new ShardId(indexMetadata.getIndex(), 0), failWithException(() -> new MapperException("test")))) ); CompletableFuture future = new CompletableFuture<>(); @@ -320,7 +315,7 @@ public void testBulkToDataStream() throws Exception { bulkRequest.add(new IndexRequest(dataStreamName).id("1").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); bulkRequest.add(new IndexRequest(dataStreamName).id("3").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); - NodeClient client = getNodeClient(this::acceptAllShardWrites); + NodeClient client = getNodeClient(acceptAllShardWrites()); CompletableFuture future = new CompletableFuture<>(); ActionListener listener = ActionListener.wrap(future::complete, future::completeExceptionally); @@ -341,7 +336,7 @@ public void testBulkToDataStreamFailingEntireShard() throws Exception { bulkRequest.add(new IndexRequest(dataStreamName).id("3").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(ds1BackingIndex2.getIndex(), 0), () -> new MapperException("test"))) + shardSpecificResponse(Map.of(new ShardId(ds1BackingIndex2.getIndex(), 0), failWithException(() -> new MapperException("test")))) ); CompletableFuture future = new CompletableFuture<>(); @@ -371,7 +366,7 @@ public void testFailingEntireShardRedirectsToFailureStore() throws Exception { bulkRequest.add(new IndexRequest(fsDataStreamName).id("3").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), () -> new MapperException("test"))) + shardSpecificResponse(Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), failWithException(() -> new MapperException("test")))) ); CompletableFuture future = new CompletableFuture<>(); @@ -433,12 +428,12 @@ public void testFailureStoreShardFailureRejectsDocument() throws Exception { // Mock client that rejects all shard requests on the first shard in the backing index, and all requests to the only shard of // the failure store index. NodeClient client = getNodeClient( - failingShards( + shardSpecificResponse( Map.of( new ShardId(ds2BackingIndex1.getIndex(), 0), - () -> new MapperException("root cause"), + failWithException(() -> new MapperException("root cause")), new ShardId(ds2FailureStore1.getIndex(), 0), - () -> new MapperException("failure store test failure") + failWithException(() -> new MapperException("failure store test failure")) ) ) ); @@ -500,6 +495,101 @@ public void testFailedDocumentCanNotBeConvertedFails() throws Exception { assertThat(failedItem.getFailure().getCause().getSuppressed()[0].getMessage(), is(equalTo("Could not serialize json"))); } + /** + * A bulk operation to a data stream with a failure store enabled could still succeed if the cluster is experiencing a + * retryable block when the redirected documents would be sent to the shard-level action. If the cluster state observer + * returns an unblocked cluster, the redirection of failure documents should proceed and not return early. + */ + public void testRetryableBlockAcceptsFailureStoreDocument() throws Exception { + Assume.assumeTrue(DataStream.isFailureStoreFeatureFlagEnabled()); + + // Requests that go to two separate shards + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(new IndexRequest(fsDataStreamName).id("1").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); + bulkRequest.add(new IndexRequest(fsDataStreamName).id("3").source(Map.of("key", "val")).opType(DocWriteRequest.OpType.CREATE)); + + // We want to make sure that going async during the write operation won't cause correctness + // issues, so use a real executor for the test + ExecutorService writeExecutor = threadPool.executor(ThreadPool.Names.WRITE); + + // Create a pair of countdown latches to synchronize our test code and the write operation we're testing: + // One to notify the test that the write operation has been reached, and one for the test to signal that + // the write operation should proceed + CountDownLatch readyToPerformFailureStoreWrite = new CountDownLatch(1); + CountDownLatch beginFailureStoreWrite = new CountDownLatch(1); + + // A mock client that: + // 1) Rejects an entire shard level request for the backing index and + // 2) When the followup write is submitted for the failure store, will go async and wait until the above latch is counted down + // before accepting the request. + NodeClient client = getNodeClient( + shardSpecificResponse( + Map.of( + new ShardId(ds2BackingIndex1.getIndex(), 0), + failWithException(() -> new MapperException("root cause")), + new ShardId(ds2FailureStore1.getIndex(), 0), + goAsyncAndWait(writeExecutor, readyToPerformFailureStoreWrite, beginFailureStoreWrite, acceptAllShardWrites()) + ) + ) + ); + + // Create a new cluster state that has a retryable cluster block on it + ClusterState blockedState = ClusterState.builder(DEFAULT_STATE) + .blocks(ClusterBlocks.builder().addGlobalBlock(NoMasterBlockService.NO_MASTER_BLOCK_WRITES).build()) + .build(); + + // Cluster state observer logic: + // First time we will return the normal cluster state (before normal writes) which skips any further interactions, + // Second time we will return a blocked cluster state (before the redirects) causing us to start observing the cluster + // Then, when waiting for next state change, we will emulate the observer receiving an unblocked state to continue the processing + // Finally, third time we will return the normal cluster state again since the cluster will be "unblocked" after waiting + ClusterStateObserver observer = mock(ClusterStateObserver.class); + when(observer.setAndGetObservedState()).thenReturn(DEFAULT_STATE).thenReturn(blockedState).thenReturn(DEFAULT_STATE); + when(observer.isTimedOut()).thenReturn(false); + doAnswer(invocation -> { + ClusterStateObserver.Listener l = invocation.getArgument(0); + l.onNewClusterState(DEFAULT_STATE); + return null; + }).when(observer).waitForNextChange(any()); + + CompletableFuture future = new CompletableFuture<>(); + ActionListener listener = ActionListener.notifyOnce( + ActionListener.wrap(future::complete, future::completeExceptionally) + ); + + newBulkOperation(client, bulkRequest, DEFAULT_STATE, observer, listener).run(); + + // The operation will attempt to write the documents in the request, receive a failure, wait for a stable cluster state, and then + // redirect the failed documents to the failure store. Wait for that failure store write to start: + if (readyToPerformFailureStoreWrite.await(30, TimeUnit.SECONDS) == false) { + // we're going to fail the test, but be a good citizen and unblock the other thread first + beginFailureStoreWrite.countDown(); + fail("timed out waiting for failure store write operation to begin"); + } + + // Check to make sure there is no response yet + if (future.isDone()) { + // we're going to fail the test, but be a good citizen and unblock the other thread first + beginFailureStoreWrite.countDown(); + fail("bulk operation completed prematurely"); + } + + // Operation is still correctly in flight. Allow the write operation to continue + beginFailureStoreWrite.countDown(); + + // Await final result and verify + BulkResponse bulkItemResponses = future.get(); + assertThat(bulkItemResponses.hasFailures(), is(false)); + BulkItemResponse failedItem = Arrays.stream(bulkItemResponses.getItems()) + .filter(item -> item.getIndex().equals(ds2FailureStore1.getIndex().getName())) + .findFirst() + .orElseThrow(() -> new AssertionError("Could not find redirected item")); + assertThat(failedItem, is(notNullValue())); + + verify(observer, times(1)).isTimedOut(); + verify(observer, times(1)).waitForNextChange(any()); + } + /** * A bulk operation to a data stream with a failure store enabled may still partially fail if the cluster is experiencing a * non-retryable block when the redirected documents would be sent to the shard-level action. @@ -515,7 +605,9 @@ public void testBlockedClusterRejectsFailureStoreDocument() throws Exception { // Mock client that rejects all shard requests on the first shard in the backing index, and all requests to the only shard of // the failure store index. NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), () -> new MapperException("root cause"))) + shardSpecificResponse( + Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), failWithException(() -> new MapperException("root cause"))) + ) ); // Create a new cluster state that has a non-retryable cluster block on it @@ -570,7 +662,9 @@ public void testOperationTimeoutRejectsFailureStoreDocument() throws Exception { // Mock client that rejects all shard requests on the first shard in the backing index, and all requests to the only shard of // the failure store index. NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), () -> new MapperException("root cause"))) + shardSpecificResponse( + Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), failWithException(() -> new MapperException("root cause"))) + ) ); // Create a new cluster state that has a retryable cluster block on it @@ -633,7 +727,9 @@ public void testNodeClosureRejectsFailureStoreDocument() { // Mock client that rejects all shard requests on the first shard in the backing index, and all requests to the only shard of // the failure store index. NodeClient client = getNodeClient( - failingShards(Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), () -> new MapperException("root cause"))) + shardSpecificResponse( + Map.of(new ShardId(ds2BackingIndex1.getIndex(), 0), failWithException(() -> new MapperException("root cause"))) + ) ); // Create a new cluster state that has a retryable cluster block on it @@ -663,29 +759,74 @@ public void testNodeClosureRejectsFailureStoreDocument() { verify(observer, times(1)).waitForNextChange(any()); } + /** + * Throws an assertion error with the given message if the client operation executes + */ + private static BiConsumer> assertNoClientInteraction() { + return (r, l) -> fail("Should not have executed shard action on blocked cluster"); + } + /** * Accepts all write operations from the given request object when it is encountered in the mock shard bulk action */ - private BulkShardResponse acceptAllShardWrites(BulkShardRequest request) { - return new BulkShardResponse( - request.shardId(), - Arrays.stream(request.items()).map(item -> requestToResponse(request.shardId(), item)).toArray(BulkItemResponse[]::new) - ); + private static BiConsumer> acceptAllShardWrites() { + return (BulkShardRequest request, ActionListener listener) -> { + listener.onResponse( + new BulkShardResponse( + request.shardId(), + Arrays.stream(request.items()).map(item -> requestToResponse(request.shardId(), item)).toArray(BulkItemResponse[]::new) + ) + ); + }; + } + + /** + * When the request is received, it is marked as failed with an exception created by the supplier + */ + private BiConsumer> failWithException(Supplier exceptionSupplier) { + return (BulkShardRequest request, ActionListener listener) -> { listener.onFailure(exceptionSupplier.get()); }; } /** - * Maps an entire shard id to an exception to throw when it is encountered in the mock shard bulk action + * Maps an entire shard id to a consumer when it is encountered in the mock shard bulk action */ - private CheckedFunction failingShards(Map> shardsToFail) { - return (BulkShardRequest request) -> { - if (shardsToFail.containsKey(request.shardId())) { - throw shardsToFail.get(request.shardId()).get(); + private BiConsumer> shardSpecificResponse( + Map>> shardsToResponders + ) { + return (BulkShardRequest request, ActionListener listener) -> { + if (shardsToResponders.containsKey(request.shardId())) { + shardsToResponders.get(request.shardId()).accept(request, listener); } else { - return acceptAllShardWrites(request); + acceptAllShardWrites().accept(request, listener); } }; } + /** + * When the consumer is called, it goes async on the given executor. It will signal that it has reached the operation by counting down + * the readyLatch, then wait on the provided continueLatch before executing the delegate consumer. + */ + private BiConsumer> goAsyncAndWait( + Executor executor, + CountDownLatch readyLatch, + CountDownLatch continueLatch, + BiConsumer> delegate + ) { + return (final BulkShardRequest request, final ActionListener listener) -> { + executor.execute(() -> { + try { + readyLatch.countDown(); + if (continueLatch.await(30, TimeUnit.SECONDS) == false) { + listener.onFailure(new RuntimeException("Timeout in client operation waiting for test to signal a continuation")); + } + } catch (InterruptedException e) { + listener.onFailure(new RuntimeException(e)); + } + delegate.accept(request, listener); + }); + }; + } + /** * Index name / id tuple */ @@ -694,17 +835,19 @@ private record IndexAndId(String indexName, String id) {} /** * Maps a document to an exception to thrown when it is encountered in the mock shard bulk action */ - private CheckedFunction thatFailsDocuments( + private BiConsumer> thatFailsDocuments( Map> documentsToFail ) { - return (BulkShardRequest request) -> new BulkShardResponse(request.shardId(), Arrays.stream(request.items()).map(item -> { - IndexAndId key = new IndexAndId(request.index(), item.request().id()); - if (documentsToFail.containsKey(key)) { - return requestToFailedResponse(item, documentsToFail.get(key).get()); - } else { - return requestToResponse(request.shardId(), item); - } - }).toArray(BulkItemResponse[]::new)); + return (BulkShardRequest request, ActionListener listener) -> { + listener.onResponse(new BulkShardResponse(request.shardId(), Arrays.stream(request.items()).map(item -> { + IndexAndId key = new IndexAndId(request.index(), item.request().id()); + if (documentsToFail.containsKey(key)) { + return requestToFailedResponse(item, documentsToFail.get(key).get()); + } else { + return requestToResponse(request.shardId(), item); + } + }).toArray(BulkItemResponse[]::new))); + }; } /** @@ -734,7 +877,7 @@ private static BulkItemResponse requestToFailedResponse(BulkItemRequest itemRequ * @param onShardAction Called when TransportShardBulkAction is executed. * @return A node client for the test. */ - private NodeClient getNodeClient(CheckedFunction onShardAction) { + private NodeClient getNodeClient(BiConsumer> onShardAction) { return new NoOpNodeClient(threadPool) { @Override @SuppressWarnings("unchecked") @@ -744,17 +887,13 @@ public Task exe ActionListener listener ) { if (TransportShardBulkAction.TYPE.equals(action)) { - Response response = null; - Exception exception = null; + ActionListener notifyOnceListener = ActionListener.notifyOnce( + (ActionListener) listener + ); try { - response = (Response) onShardAction.apply((BulkShardRequest) request); + onShardAction.accept((BulkShardRequest) request, notifyOnceListener); } catch (Exception responseException) { - exception = responseException; - } - if (response != null) { - listener.onResponse(response); - } else { - listener.onFailure(exception); + notifyOnceListener.onFailure(responseException); } } else { fail("Unexpected client call to " + action.name()); From a5970da6bf06852124e74f5ac2e7d0920c6883cb Mon Sep 17 00:00:00 2001 From: Keith Massey Date: Tue, 23 Apr 2024 15:27:25 -0500 Subject: [PATCH 04/27] Removing invalid assertion from SnapshotLifecycleServiceTests (#107736) --- .../elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java index b65f3587ffbf..5b59ac9efc0a 100644 --- a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java +++ b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java @@ -279,7 +279,6 @@ public void testPolicyCRUD() throws Exception { clock.fastForwardSeconds(2); // The existing job should be cancelled and no longer trigger - assertBusy(() -> assertThat(triggerCount.get(), equalTo(currentCount2))); assertThat(sls.getScheduler().scheduledJobIds(), equalTo(Collections.emptySet())); // When the service is no longer master, all jobs should be automatically cancelled From 982c567d9acdcd9702467fb72421178c5aa84d3c Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Tue, 23 Apr 2024 16:29:31 -0700 Subject: [PATCH 05/27] Increase container size for machine dependent heap docker tests (#107797) --- .../java/org/elasticsearch/packaging/test/DockerTests.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/qa/packaging/src/test/java/org/elasticsearch/packaging/test/DockerTests.java b/qa/packaging/src/test/java/org/elasticsearch/packaging/test/DockerTests.java index ece49fbb15e5..b1240747b1a6 100644 --- a/qa/packaging/src/test/java/org/elasticsearch/packaging/test/DockerTests.java +++ b/qa/packaging/src/test/java/org/elasticsearch/packaging/test/DockerTests.java @@ -1011,12 +1011,11 @@ public void test140CgroupOsStatsAreAvailable() throws Exception { * Check that when available system memory is constrained by Docker, the machine-dependant heap sizing * logic sets the correct heap size, based on the container limits. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/104786") public void test150MachineDependentHeap() throws Exception { - final List xArgs = machineDependentHeapTest("942m", List.of()); + final List xArgs = machineDependentHeapTest("1536m", List.of()); - // This is roughly 0.4 * 942 - assertThat(xArgs, hasItems("-Xms376m", "-Xmx376m")); + // This is roughly 0.5 * 1536 + assertThat(xArgs, hasItems("-Xms768m", "-Xmx768m")); } /** From eb153446903916bea537374210e5b5a55b014e63 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 23 Apr 2024 16:38:34 -0700 Subject: [PATCH 06/27] Fix minimized_round_trips in lookup runtime fields (#107785) Today, we have disabled ccs_minimized_round_trips for lookup requests, under the assumption that cross-cluster lookups occur when ccs_minimized_round_trips is disabled in the main search request. However, this assumption does not hold true for cases where the search is local but the lookup happens remotely. --- docs/changelog/107785.yaml | 5 ++ .../search/ccs/CrossClusterIT.java | 60 +++++++++++++++++++ .../action/search/FetchLookupFieldsPhase.java | 2 +- 3 files changed, 66 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/107785.yaml diff --git a/docs/changelog/107785.yaml b/docs/changelog/107785.yaml new file mode 100644 index 000000000000..fae01a7da597 --- /dev/null +++ b/docs/changelog/107785.yaml @@ -0,0 +1,5 @@ +pr: 107785 +summary: Fix `minimized_round_trips` in lookup runtime fields +area: Search +type: bug +issues: [] diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java index eedda05dcb10..501f46fb52b4 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java @@ -50,6 +50,7 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.NodeRoles; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.XContentParser; @@ -62,6 +63,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -446,6 +448,64 @@ public void testLookupFields() throws Exception { assertThat(hit2.field("to").getValues(), contains(Map.of("name", List.of("Local B")), Map.of("name", List.of("Local C")))); }); } + // Search locally, but lookup fields on remote clusters + { + final String remoteLookupFields = """ + { + "from": { + "type": "lookup", + "target_index": "cluster_a:users", + "input_field": "from_user", + "target_field": "_id", + "fetch_fields": ["name"] + }, + "to": { + "type": "lookup", + "target_index": "cluster_a:users", + "input_field": "to_user", + "target_field": "_id", + "fetch_fields": ["name"] + } + } + """; + final Map remoteRuntimeMappings; + try (XContentParser parser = createParser(JsonXContent.jsonXContent, remoteLookupFields)) { + remoteRuntimeMappings = parser.map(); + } + AtomicInteger searchSearchRequests = new AtomicInteger(0); + for (TransportService ts : cluster("cluster_a").getInstances(TransportService.class)) { + MockTransportService transportService = (MockTransportService) ts; + transportService.addRequestHandlingBehavior(TransportSearchShardsAction.NAME, (handler, request, channel, task) -> { + handler.messageReceived(request, channel, task); + searchSearchRequests.incrementAndGet(); + }); + } + for (boolean ccsMinimizeRoundtrips : List.of(true, false)) { + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder().query(new TermQueryBuilder("to_user", "c")) + .runtimeMappings(remoteRuntimeMappings) + .sort(new FieldSortBuilder("duration")) + .fetchField("from") + .fetchField("to"); + SearchRequest request = new SearchRequest("local_calls").source(searchSourceBuilder); + request.setCcsMinimizeRoundtrips(ccsMinimizeRoundtrips); + assertResponse(client().search(request), response -> { + assertHitCount(response, 1); + SearchHit hit = response.getHits().getHits()[0]; + assertThat(hit.getIndex(), equalTo("local_calls")); + assertThat(hit.field("from").getValues(), contains(Map.of("name", List.of("Remote A")))); + assertThat( + hit.field("to").getValues(), + contains(Map.of("name", List.of("Remote B")), Map.of("name", List.of("Remote C"))) + ); + }); + if (ccsMinimizeRoundtrips) { + assertThat(searchSearchRequests.get(), equalTo(0)); + } else { + assertThat(searchSearchRequests.get(), greaterThan(0)); + searchSearchRequests.set(0); + } + } + } } @Override diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index 0605e23fc343..417e5e40f6fd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -87,7 +87,7 @@ private void doRun(List clusters) { : "lookup across clusters only if [ccs_minimize_roundtrips] is disabled"; for (LookupField lookupField : cluster.lookupFields) { final SearchRequest searchRequest = lookupField.toSearchRequest(clusterAlias); - searchRequest.setCcsMinimizeRoundtrips(false); + searchRequest.setCcsMinimizeRoundtrips(context.getRequest().isCcsMinimizeRoundtrips()); multiSearchRequest.add(searchRequest); } } From 7402f93839b700d587c519210fe3c2e84aed5565 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Tue, 23 Apr 2024 17:17:34 -0700 Subject: [PATCH 07/27] Expose server process to server cli subclasses (#107729) This commit gives protected access to subclasses of ServerCli (ie for tests or serverless). --- .../main/java/org/elasticsearch/server/cli/ServerCli.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/distribution/tools/server-cli/src/main/java/org/elasticsearch/server/cli/ServerCli.java b/distribution/tools/server-cli/src/main/java/org/elasticsearch/server/cli/ServerCli.java index aac5f718081b..6dbff2fbfff9 100644 --- a/distribution/tools/server-cli/src/main/java/org/elasticsearch/server/cli/ServerCli.java +++ b/distribution/tools/server-cli/src/main/java/org/elasticsearch/server/cli/ServerCli.java @@ -237,6 +237,11 @@ public void close() { } } + // allow subclasses to access the started process + protected ServerProcess getServer() { + return server; + } + // protected to allow tests to override protected Command loadTool(String toolname, String libs) { return CliToolProvider.load(toolname, libs).create(); From 7dacb7c5e1d238412efe3e2934029b888bed3ee7 Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Wed, 24 Apr 2024 16:03:08 +1000 Subject: [PATCH 08/27] Include default client settings in Datastream ITs (#107740) In #107341 we added a new system property that is used to config the rest client in rest integration tests. This commit changes two datastream ITs to include the default client settings so that this property is effective. --- .../elasticsearch/datastreams/LazyRolloverDataStreamIT.java | 2 +- .../lifecycle/DataStreamLifecyclePermissionsTestCase.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LazyRolloverDataStreamIT.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LazyRolloverDataStreamIT.java index 5389ad3a00b4..1f2a5b250860 100644 --- a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LazyRolloverDataStreamIT.java +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LazyRolloverDataStreamIT.java @@ -71,7 +71,7 @@ protected Settings restClientSettings() { private Settings simpleUserRestClientSettings() { // Note: This user is assigned the role "under_privilged". That role is defined in roles.yml. String token = basicAuthHeaderValue("test_simple_user", new SecureString(PASSWORD.toCharArray())); - return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + return Settings.builder().put(super.restClientSettings()).put(ThreadContext.PREFIX + ".Authorization", token).build(); } @Before diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecyclePermissionsTestCase.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecyclePermissionsTestCase.java index 598948a74624..cd21a31d343c 100644 --- a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecyclePermissionsTestCase.java +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecyclePermissionsTestCase.java @@ -97,13 +97,13 @@ protected Settings restAdminSettings() { Settings restPrivilegedClientSettings() { // Note: This user is assigned the role "under_privilged". That role is defined in roles.yml. String token = basicAuthHeaderValue("test_data_stream_lifecycle", new SecureString(PASSWORD.toCharArray())); - return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + return Settings.builder().put(super.restClientSettings()).put(ThreadContext.PREFIX + ".Authorization", token).build(); } Settings restUnprivilegedClientSettings() { // Note: This user is assigned the role "under_privilged". That role is defined in roles.yml. String token = basicAuthHeaderValue("test_non_privileged", new SecureString(PASSWORD.toCharArray())); - return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + return Settings.builder().put(super.restClientSettings()).put(ThreadContext.PREFIX + ".Authorization", token).build(); } @Before From 94041d3cdd084e16a03342912689c037d765e835 Mon Sep 17 00:00:00 2001 From: Jedr Blaszyk Date: Wed, 24 Apr 2024 08:35:40 +0200 Subject: [PATCH 09/27] [Connector API] Address docs feedback (#107774) --- .../cancel-connector-sync-job-api.asciidoc | 3 +-- .../apis/check-in-connector-api.asciidoc | 2 +- .../check-in-connector-sync-job-api.asciidoc | 2 +- .../connector/apis/connector-apis.asciidoc | 2 +- .../apis/create-connector-api.asciidoc | 12 +++++------ .../create-connector-sync-job-api.asciidoc | 2 +- .../apis/delete-connector-api.asciidoc | 6 ++++-- .../delete-connector-sync-job-api.asciidoc | 2 +- .../connector/apis/get-connector-api.asciidoc | 2 +- .../apis/get-connector-sync-job-api.asciidoc | 2 +- .../list-connector-sync-jobs-api.asciidoc | 2 +- .../apis/list-connectors-api.asciidoc | 20 +++++++++---------- .../set-connector-sync-job-error-api.asciidoc | 2 +- .../set-connector-sync-job-stats-api.asciidoc | 2 +- .../update-connector-api-key-id-api.asciidoc | 2 +- ...pdate-connector-configuration-api.asciidoc | 2 +- .../apis/update-connector-error-api.asciidoc | 2 +- .../update-connector-filtering-api.asciidoc | 2 +- .../update-connector-index-name-api.asciidoc | 2 +- .../update-connector-last-sync-api.asciidoc | 2 +- ...te-connector-name-description-api.asciidoc | 2 +- .../update-connector-pipeline-api.asciidoc | 2 +- .../update-connector-scheduling-api.asciidoc | 2 +- ...update-connector-service-type-api.asciidoc | 2 +- .../apis/update-connector-status-api.asciidoc | 2 +- 25 files changed, 42 insertions(+), 41 deletions(-) diff --git a/docs/reference/connector/apis/cancel-connector-sync-job-api.asciidoc b/docs/reference/connector/apis/cancel-connector-sync-job-api.asciidoc index bd886bf923af..ac22f2c4adf6 100644 --- a/docs/reference/connector/apis/cancel-connector-sync-job-api.asciidoc +++ b/docs/reference/connector/apis/cancel-connector-sync-job-api.asciidoc @@ -15,7 +15,7 @@ Cancels a connector sync job. [[cancel-connector-sync-job-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_sync_job_id` parameter should reference an existing connector sync job. [[cancel-connector-sync-job-api-desc]] @@ -49,4 +49,3 @@ The following example cancels the connector sync job with ID `my-connector-sync- PUT _connector/_sync_job/my-connector-sync-job-id/_cancel ---- // TEST[skip:there's no way to clean up after creating a connector sync job, as we don't know the id ahead of time. Therefore, skip this test.] - diff --git a/docs/reference/connector/apis/check-in-connector-api.asciidoc b/docs/reference/connector/apis/check-in-connector-api.asciidoc index 9f88c595e3a6..e8119028ea24 100644 --- a/docs/reference/connector/apis/check-in-connector-api.asciidoc +++ b/docs/reference/connector/apis/check-in-connector-api.asciidoc @@ -16,7 +16,7 @@ Updates the `last_seen` field of a connector with current timestamp. [[check-in-connector-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[check-in-connector-api-path-params]] diff --git a/docs/reference/connector/apis/check-in-connector-sync-job-api.asciidoc b/docs/reference/connector/apis/check-in-connector-sync-job-api.asciidoc index d3cc34bf025e..482c2e8dd4a2 100644 --- a/docs/reference/connector/apis/check-in-connector-sync-job-api.asciidoc +++ b/docs/reference/connector/apis/check-in-connector-sync-job-api.asciidoc @@ -15,7 +15,7 @@ Checks in a connector sync job (updates `last_seen` to the current time). [[check-in-connector-sync-job-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_sync_job_id` parameter should reference an existing connector sync job. [[check-in-connector-sync-job-api-desc]] diff --git a/docs/reference/connector/apis/connector-apis.asciidoc b/docs/reference/connector/apis/connector-apis.asciidoc index 6f03ded09347..b5f3d1a1aa87 100644 --- a/docs/reference/connector/apis/connector-apis.asciidoc +++ b/docs/reference/connector/apis/connector-apis.asciidoc @@ -5,7 +5,7 @@ preview::[] The connector and sync jobs APIs provide a convenient way to create and manage Elastic {enterprise-search-ref}/connectors.html[connectors^] and sync jobs in an internal index. -Connectors are third-party {es} integrations which can be deployed on {ecloud} or hosted on your own infrastructure: +Connectors are {es} integrations that bring content from third-party data sources, which can be deployed on {ecloud} or hosted on your own infrastructure: * *Native connectors* are a managed service on {ecloud} * *Connector clients* are self-managed on your infrastructure diff --git a/docs/reference/connector/apis/create-connector-api.asciidoc b/docs/reference/connector/apis/create-connector-api.asciidoc index 56e0b378aa1c..15dc4ed43c72 100644 --- a/docs/reference/connector/apis/create-connector-api.asciidoc +++ b/docs/reference/connector/apis/create-connector-api.asciidoc @@ -7,7 +7,7 @@ preview::[] Creates an Elastic connector. -Connectors are third-party {es} integrations which can be deployed on {ecloud} or hosted on your own infrastructure: +Connectors are {es} integrations that bring content from third-party data sources, which can be deployed on {ecloud} or hosted on your own infrastructure: * *Native connectors* are a managed service on {ecloud} * *Connector clients* are self-managed on your infrastructure @@ -41,8 +41,8 @@ DELETE _connector/my-connector [[create-connector-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. -* The `service_type` parameter should reference an existing connector service type. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. +* The `service_type` parameter should reference a supported third-party service. See the available service types for {enterprise-search-ref}/native-connectors.html[native] and {enterprise-search-ref}/build-connector.html[self-managed] connectors. This can also reference the service type of your custom connector. [[create-connector-api-desc]] @@ -65,10 +65,10 @@ Creates a connector document in the internal index and initializes its configura (Optional, string) The description of the connector. `index_name`:: -(Required, string) The target index for syncing data by the connector. +(Optional, string) The target index to sync data. If the index doesn't exist, it will be created upon the first sync. `name`:: -(Optional, string) The name of the connector. +(Optional, string) The name of the connector. Setting the connector name is recommended when managing connectors in {kib}. `is_native`:: (Optional, boolean) Indicates if it's a native connector. Defaults to `false`. @@ -77,7 +77,7 @@ Creates a connector document in the internal index and initializes its configura (Optional, string) Language analyzer for the data. Limited to supported languages. `service_type`:: -(Optional, string) Connector service type. Can reference Elastic-supported connector types or a custom connector type. +(Optional, string) Connector service type. Can reference Elastic-supported third-party services or a custom connector type. See the available service types for {enterprise-search-ref}/native-connectors.html[native] and {enterprise-search-ref}/build-connector.html[self-managed] connectors. [role="child_attributes"] diff --git a/docs/reference/connector/apis/create-connector-sync-job-api.asciidoc b/docs/reference/connector/apis/create-connector-sync-job-api.asciidoc index b03648528525..c4fdd362c31c 100644 --- a/docs/reference/connector/apis/create-connector-sync-job-api.asciidoc +++ b/docs/reference/connector/apis/create-connector-sync-job-api.asciidoc @@ -29,7 +29,7 @@ POST _connector/_sync_job [[create-connector-sync-job-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `id` parameter should reference an existing connector. [[create-connector-sync-job-api-desc]] diff --git a/docs/reference/connector/apis/delete-connector-api.asciidoc b/docs/reference/connector/apis/delete-connector-api.asciidoc index c7e9dcd94d2a..b338f1db2a25 100644 --- a/docs/reference/connector/apis/delete-connector-api.asciidoc +++ b/docs/reference/connector/apis/delete-connector-api.asciidoc @@ -6,9 +6,11 @@ preview::[] -Removes a connector and its associated data. +Removes a connector and associated sync jobs. This is a destructive action that is not recoverable. +Note: this action doesn't delete any API key, ingest pipeline or data index associated with the connector. These need to be removed manually. + [[delete-connector-api-request]] ==== {api-request-title} @@ -17,7 +19,7 @@ This is a destructive action that is not recoverable. [[delete-connector-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[delete-connector-api-path-params]] diff --git a/docs/reference/connector/apis/delete-connector-sync-job-api.asciidoc b/docs/reference/connector/apis/delete-connector-sync-job-api.asciidoc index 32df172df758..1e53c7f843af 100644 --- a/docs/reference/connector/apis/delete-connector-sync-job-api.asciidoc +++ b/docs/reference/connector/apis/delete-connector-sync-job-api.asciidoc @@ -17,7 +17,7 @@ This is a destructive action that is not recoverable. [[delete-connector-sync-job-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. [[delete-connector-sync-job-api-path-params]] ==== {api-path-parms-title} diff --git a/docs/reference/connector/apis/get-connector-api.asciidoc b/docs/reference/connector/apis/get-connector-api.asciidoc index 693a9fd76780..3a546ab372b6 100644 --- a/docs/reference/connector/apis/get-connector-api.asciidoc +++ b/docs/reference/connector/apis/get-connector-api.asciidoc @@ -16,7 +16,7 @@ Retrieves the details about a connector. [[get-connector-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. [[get-connector-api-path-params]] ==== {api-path-parms-title} diff --git a/docs/reference/connector/apis/get-connector-sync-job-api.asciidoc b/docs/reference/connector/apis/get-connector-sync-job-api.asciidoc index bfa82ea0d345..0c136f8e037b 100644 --- a/docs/reference/connector/apis/get-connector-sync-job-api.asciidoc +++ b/docs/reference/connector/apis/get-connector-sync-job-api.asciidoc @@ -16,7 +16,7 @@ Retrieves the details about a connector sync job. [[get-connector-sync-job-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. [[get-connector-sync-job-api-path-params]] ==== {api-path-parms-title} diff --git a/docs/reference/connector/apis/list-connector-sync-jobs-api.asciidoc b/docs/reference/connector/apis/list-connector-sync-jobs-api.asciidoc index 3c31638c84a9..303abdaa546b 100644 --- a/docs/reference/connector/apis/list-connector-sync-jobs-api.asciidoc +++ b/docs/reference/connector/apis/list-connector-sync-jobs-api.asciidoc @@ -18,7 +18,7 @@ Returns information about all stored connector sync jobs ordered by their creati [[list-connector-sync-jobs-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. [[list-connector-sync-jobs-api-path-params]] ==== {api-path-parms-title} diff --git a/docs/reference/connector/apis/list-connectors-api.asciidoc b/docs/reference/connector/apis/list-connectors-api.asciidoc index 6c0279fd8b03..94578dbd493e 100644 --- a/docs/reference/connector/apis/list-connectors-api.asciidoc +++ b/docs/reference/connector/apis/list-connectors-api.asciidoc @@ -7,7 +7,7 @@ preview::[] -Returns information about all stored connectors. +Returns information about all created connectors. [[list-connector-api-request]] @@ -18,19 +18,19 @@ Returns information about all stored connectors. [[list-connector-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. [[list-connector-api-path-params]] ==== {api-path-parms-title} `size`:: -(Optional, integer) Maximum number of results to retrieve. +(Optional, integer) Maximum number of results to retrieve. Defaults to `100`. `from`:: -(Optional, integer) The offset from the first result to fetch. +(Optional, integer) The offset from the first result to fetch. Defaults to `0`. `index_name`:: -(Optional, string) A comma-separated list of data index names associated with connectors, used to filter search results. +(Optional, string) A comma-separated list of index names associated with connectors, used to filter search results. `connector_name`:: (Optional, string) A comma-separated list of connector names, used to filter search results. @@ -81,14 +81,14 @@ The following example lists the first two connectors: [source,console] ---- -GET _connector/?from=0&size=2 +GET _connector?from=0&size=2 ---- -An example to list connectors associated with `search-google-drive` data index name: +An example to list a connector associated with the `search-google-drive` Elasticsearch index: [source,console] ---- -GET _connector/?index_name=search-google-drive +GET _connector?index_name=search-google-drive ---- @@ -96,12 +96,12 @@ An example to list all connectors with `sharepoint_online` service type: [source,console] ---- -GET _connector/?service_type=sharepoint_online +GET _connector?service_type=sharepoint_online ---- An example to list all connectors with `sharepoint_online` or `google_drive` service type: [source,console] ---- -GET _connector/?service_type=sharepoint_online,google_drive +GET _connector?service_type=sharepoint_online,google_drive ---- diff --git a/docs/reference/connector/apis/set-connector-sync-job-error-api.asciidoc b/docs/reference/connector/apis/set-connector-sync-job-error-api.asciidoc index a9dbf5ceb1eb..97b5c20f0813 100644 --- a/docs/reference/connector/apis/set-connector-sync-job-error-api.asciidoc +++ b/docs/reference/connector/apis/set-connector-sync-job-error-api.asciidoc @@ -15,7 +15,7 @@ Sets a connector sync job error. [[set-connector-sync-job-error-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_sync_job_id` parameter should reference an existing connector sync job. [[set-connector-sync-job-error-api-desc]] diff --git a/docs/reference/connector/apis/set-connector-sync-job-stats-api.asciidoc b/docs/reference/connector/apis/set-connector-sync-job-stats-api.asciidoc index a417bcf8b9e9..405df07465a2 100644 --- a/docs/reference/connector/apis/set-connector-sync-job-stats-api.asciidoc +++ b/docs/reference/connector/apis/set-connector-sync-job-stats-api.asciidoc @@ -15,7 +15,7 @@ Sets connector sync job stats. [[set-connector-sync-job-stats-api-prereqs]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_sync_job_id` parameter should reference an existing connector sync job. [[set-connector-sync-job-stats-api-desc]] diff --git a/docs/reference/connector/apis/update-connector-api-key-id-api.asciidoc b/docs/reference/connector/apis/update-connector-api-key-id-api.asciidoc index e15f57a3e2b1..9b08ceea0aac 100644 --- a/docs/reference/connector/apis/update-connector-api-key-id-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-api-key-id-api.asciidoc @@ -23,7 +23,7 @@ See the documentation for {enterprise-search-ref}/native-connectors.html#native- [[update-connector-api-key-id-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. * The `api_key_id` parameter should reference an existing API key. * The `api_key_secret_id` parameter should reference an existing Connector Secret containing an encoded API key value. diff --git a/docs/reference/connector/apis/update-connector-configuration-api.asciidoc b/docs/reference/connector/apis/update-connector-configuration-api.asciidoc index 22a823f27cd8..fea22eb8043b 100644 --- a/docs/reference/connector/apis/update-connector-configuration-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-configuration-api.asciidoc @@ -17,7 +17,7 @@ Updates a connector's `configuration`, allowing for complete schema modification [[update-connector-configuration-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. * The configuration fields definition must be compatible with the specific connector type being used. diff --git a/docs/reference/connector/apis/update-connector-error-api.asciidoc b/docs/reference/connector/apis/update-connector-error-api.asciidoc index dbed25f1bf8d..3ec03c6153f4 100644 --- a/docs/reference/connector/apis/update-connector-error-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-error-api.asciidoc @@ -16,7 +16,7 @@ Updates the `error` field of a connector. [[update-connector-error-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[update-connector-error-api-path-params]] diff --git a/docs/reference/connector/apis/update-connector-filtering-api.asciidoc b/docs/reference/connector/apis/update-connector-filtering-api.asciidoc index 04c40ebf9fa4..0f6bd442e78c 100644 --- a/docs/reference/connector/apis/update-connector-filtering-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-filtering-api.asciidoc @@ -17,7 +17,7 @@ Updates the `filtering` configuration of a connector. Learn more about filtering [[update-connector-filtering-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[update-connector-filtering-api-path-params]] diff --git a/docs/reference/connector/apis/update-connector-index-name-api.asciidoc b/docs/reference/connector/apis/update-connector-index-name-api.asciidoc index be0595a22dca..02a4c0e762b2 100644 --- a/docs/reference/connector/apis/update-connector-index-name-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-index-name-api.asciidoc @@ -16,7 +16,7 @@ Updates the `index_name` field of a connector, specifying the index where the da [[update-connector-index-name-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[update-connector-index-name-api-path-params]] diff --git a/docs/reference/connector/apis/update-connector-last-sync-api.asciidoc b/docs/reference/connector/apis/update-connector-last-sync-api.asciidoc index 6f41925e3676..9326855d3c5d 100644 --- a/docs/reference/connector/apis/update-connector-last-sync-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-last-sync-api.asciidoc @@ -18,7 +18,7 @@ This action is used for analytics and monitoring. [[update-connector-last-sync-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[update-connector-last-sync-api-path-params]] diff --git a/docs/reference/connector/apis/update-connector-name-description-api.asciidoc b/docs/reference/connector/apis/update-connector-name-description-api.asciidoc index c54dba8dd72b..7fb5deb74647 100644 --- a/docs/reference/connector/apis/update-connector-name-description-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-name-description-api.asciidoc @@ -17,7 +17,7 @@ Updates the `name` and `description` fields of a connector. [[update-connector-name-description-api-prereq]] ==== {api-prereq-title} -* To sync data using connectors, it's essential to have the Elastic connectors service running. +* To sync data using self-managed connectors, you need to deploy the {enterprise-search-ref}/build-connector.html[Elastic connector service] on your own infrastructure. This service runs automatically on Elastic Cloud for native connectors. * The `connector_id` parameter should reference an existing connector. [[update-connector-name-description-api-path-params]] diff --git a/docs/reference/connector/apis/update-connector-pipeline-api.asciidoc b/docs/reference/connector/apis/update-connector-pipeline-api.asciidoc index 63872bf96aa5..30873ca5f557 100644 --- a/docs/reference/connector/apis/update-connector-pipeline-api.asciidoc +++ b/docs/reference/connector/apis/update-connector-pipeline-api.asciidoc @@ -18,7 +18,7 @@ When you create a new connector, the configuration of an < Date: Wed, 24 Apr 2024 17:10:20 +1000 Subject: [PATCH 10/27] Ensure appendToCopyNoNullElements work with more than one element (#107805) See title --- .../org/elasticsearch/common/util/CollectionUtils.java | 2 +- .../elasticsearch/common/util/CollectionUtilsTests.java | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/CollectionUtils.java b/server/src/main/java/org/elasticsearch/common/util/CollectionUtils.java index 1664b4690bc7..70417481e184 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CollectionUtils.java +++ b/server/src/main/java/org/elasticsearch/common/util/CollectionUtils.java @@ -252,7 +252,7 @@ public static List appendToCopyNoNullElements(Collection collection, E final int addedSize = elements.length; final int size = existingSize + addedSize; final E[] array = collection.toArray((E[]) new Object[size]); - System.arraycopy(elements, 0, array, size - 1, addedSize); + System.arraycopy(elements, 0, array, size - addedSize, addedSize); return List.of(array); } diff --git a/server/src/test/java/org/elasticsearch/common/util/CollectionUtilsTests.java b/server/src/test/java/org/elasticsearch/common/util/CollectionUtilsTests.java index 3b3d205a8ebb..ed5074106d5d 100644 --- a/server/src/test/java/org/elasticsearch/common/util/CollectionUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/CollectionUtilsTests.java @@ -21,6 +21,8 @@ import java.util.Map; import java.util.Set; +import static org.elasticsearch.common.util.CollectionUtils.appendToCopyNoNullElements; +import static org.elasticsearch.common.util.CollectionUtils.concatLists; import static org.elasticsearch.common.util.CollectionUtils.eagerPartition; import static org.elasticsearch.common.util.CollectionUtils.ensureNoSelfReferences; import static org.elasticsearch.common.util.CollectionUtils.limitSize; @@ -192,4 +194,11 @@ public void testLimitSizeOfLongList() { var longList = randomList(10, 100, () -> "item"); assertThat(limitSize(longList, 10), equalTo(longList.subList(0, 10))); } + + public void testAppendToCopyNoNullElements() { + final List oldList = randomList(3, () -> randomAlphaOfLength(10)); + final String[] extraElements = randomArray(2, 4, String[]::new, () -> randomAlphaOfLength(10)); + final List newList = appendToCopyNoNullElements(oldList, extraElements); + assertThat(newList, equalTo(concatLists(oldList, List.of(extraElements)))); + } } From 6c7ff87777778af302ec4caa97cc08910f24a4a6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Apr 2024 10:20:50 +0200 Subject: [PATCH 11/27] Make ByteArrayIndexInput implement RandomAccessInput (#106958) This can obviously work as random access input out of the box. --- .../lucene/store/ByteArrayIndexInput.java | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/store/ByteArrayIndexInput.java b/server/src/main/java/org/elasticsearch/common/lucene/store/ByteArrayIndexInput.java index e0dc4c941187..1cf90d174120 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/store/ByteArrayIndexInput.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/store/ByteArrayIndexInput.java @@ -8,6 +8,7 @@ package org.elasticsearch.common.lucene.store; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; import org.apache.lucene.util.BitUtil; import org.elasticsearch.common.Strings; @@ -17,7 +18,7 @@ /** * Wraps array of bytes into IndexInput */ -public class ByteArrayIndexInput extends IndexInput { +public class ByteArrayIndexInput extends IndexInput implements RandomAccessInput { private final byte[] bytes; private int pos; @@ -47,12 +48,16 @@ public long getFilePointer() { @Override public void seek(long l) throws IOException { - if (l < 0) { + pos = position(l); + } + + private int position(long p) throws EOFException { + if (p < 0) { throw new IllegalArgumentException("Seeking to negative position: " + pos); - } else if (l > length) { + } else if (p > length) { throw new EOFException("seek past EOF"); } - pos = (int) l; + return (int) p; } @Override @@ -60,6 +65,26 @@ public long length() { return length; } + @Override + public byte readByte(long pos) throws IOException { + return bytes[position(pos)]; + } + + @Override + public short readShort(long pos) throws IOException { + return (short) BitUtil.VH_LE_SHORT.get(bytes, position(pos)); + } + + @Override + public int readInt(long pos) throws IOException { + return (int) BitUtil.VH_LE_INT.get(bytes, position(pos)); + } + + @Override + public long readLong(long pos) throws IOException { + return (long) BitUtil.VH_LE_LONG.get(bytes, position(pos)); + } + @Override public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { if (offset >= 0L && length >= 0L && offset + length <= this.length) { From 78cba460a40b1cd032b3d604beb4a2e2cf9766d3 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Wed, 24 Apr 2024 10:24:35 +0200 Subject: [PATCH 12/27] Increase size of big arrays only when there is an actual value in the aggregators (#107764) During aggregation collection, we use BigArrays to hold the values on a compact way for metrics aggregations. We are currently resizing those arrays whenever the collect method is call, regardless if there is an actual value in the provided doc. This can be wasteful for sparse fields as we might never have a value but still we are resizing those arrays. Therefore this commit moves the resize after checking that there is a value in the provided document. --- docs/changelog/107764.yaml | 5 + .../test/aggregations/empty_field_metric.yml | 142 ++++++++++++++++++ .../AbstractHDRPercentilesAggregator.java | 2 +- .../AbstractTDigestPercentilesAggregator.java | 2 +- .../aggregations/metrics/AvgAggregator.java | 9 +- .../metrics/CardinalityAggregator.java | 12 +- .../metrics/ExtendedStatsAggregator.java | 27 ++-- .../metrics/GeoBoundsAggregator.java | 4 +- .../metrics/GeoCentroidAggregator.java | 4 +- .../GlobalOrdCardinalityAggregator.java | 13 +- .../aggregations/metrics/MaxAggregator.java | 15 +- .../MedianAbsoluteDeviationAggregator.java | 16 +- .../aggregations/metrics/MinAggregator.java | 15 +- .../aggregations/metrics/StatsAggregator.java | 23 ++- .../aggregations/metrics/SumAggregator.java | 7 +- .../metrics/ValueCountAggregator.java | 6 +- .../metrics/WeightedAvgAggregator.java | 11 +- 17 files changed, 227 insertions(+), 86 deletions(-) create mode 100644 docs/changelog/107764.yaml create mode 100644 modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/empty_field_metric.yml diff --git a/docs/changelog/107764.yaml b/docs/changelog/107764.yaml new file mode 100644 index 000000000000..3f83efc78901 --- /dev/null +++ b/docs/changelog/107764.yaml @@ -0,0 +1,5 @@ +pr: 107764 +summary: Increase size of big arrays only when there is an actual value in the aggregators +area: Aggregations +type: enhancement +issues: [] diff --git a/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/empty_field_metric.yml b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/empty_field_metric.yml new file mode 100644 index 000000000000..dee047fbdd6c --- /dev/null +++ b/modules/aggregations/src/yamlRestTest/resources/rest-api-spec/test/aggregations/empty_field_metric.yml @@ -0,0 +1,142 @@ +setup: + - do: + indices.create: + index: test_1 + body: + mappings: + properties: + terms_field: + type: keyword + int_field: + type : integer + double_field: + type : double + string_field: + type: keyword + geo_point_field: + type: geo_point + + - do: + bulk: + refresh: true + body: + - index: + _index: test_1 + _id: "1" + - terms_field: foo + - index: + _index: test_1 + _id: "2" + - terms_field: foo + - index: + _index: test_1 + _id: "3" + - terms_field: bar + +--- +"Basic test": + + - do: + search: + index: test_1 + rest_total_hits_as_int: true + body: + aggs: + the_terms: + terms: + field: terms_field + "order": + "_key": "asc" + aggs: + min_agg: + min: + field: double_field + max_agg: + max: + field: double_field + avg_agg: + avg: + field: double_field + sum_agg: + avg: + field: double_field + value_count_agg: + value_count: + field: double_field + cardinality_agg: + cardinality: + field: string_field + weighted_avg_agg: + weighted_avg: + value: + field: int_field + weight: + field: double_field + median_absolute_deviation_agg: + median_absolute_deviation: + field: double_field + stats_agg: + stats: + field: double_field + extended_stats_agg: + extended_stats: + field: double_field + percentiles_agg: + percentiles: + field: double_field + percents: 50 + geo_bounds_agg: + geo_bounds: + field: geo_point_field + geo_centroid_agg: + geo_centroid: + field: geo_point_field + + - match: { hits.total: 3 } + - length: { hits.hits: 3 } + - match: { aggregations.the_terms.buckets.0.key: bar} + - match: { aggregations.the_terms.buckets.0.doc_count: 1} + - match: { aggregations.the_terms.buckets.0.min_agg.value: null } + - match: { aggregations.the_terms.buckets.0.max_agg.value: null } + - match: { aggregations.the_terms.buckets.0.avg_agg.value: null } + - match: { aggregations.the_terms.buckets.0.sum_agg.value: null } + - match: { aggregations.the_terms.buckets.0.value_count_agg.value: 0 } + - match: { aggregations.the_terms.buckets.0.cardinality_agg.value: 0 } + - match: { aggregations.the_terms.buckets.0.weighted_avg_agg.value: null } + - match: { aggregations.the_terms.buckets.0.median_absolute_deviation_agg.value: null } + - match: { aggregations.the_terms.buckets.0.stats_agg.min: null } + - match: { aggregations.the_terms.buckets.0.stats_agg.max: null } + - match: { aggregations.the_terms.buckets.0.stats_agg.avg: null } + - match: { aggregations.the_terms.buckets.0.stats_agg.sum: 0.0 } + - match: { aggregations.the_terms.buckets.0.extended_stats_agg.min: null } + - match: { aggregations.the_terms.buckets.0.extended_stats_agg.max: null } + - match: { aggregations.the_terms.buckets.0.extended_stats_agg.avg: null } + - match: { aggregations.the_terms.buckets.0.extended_stats_agg.sum: 0.0 } + - match: { aggregations.the_terms.buckets.0.extended_stats_agg.sum_of_squares: null } + - match: { aggregations.the_terms.buckets.0.percentiles_agg.50: null } + - match: { aggregations.the_terms.buckets.0.geo_bounds_agg.top_left: null } + - match: { aggregations.the_terms.buckets.0.geo_bounds_agg.bottom_right: null } + - match: { aggregations.the_terms.buckets.0.geo_centroid_agg.count: 0 } + - match: { aggregations.the_terms.buckets.1.key: foo } + - match: { aggregations.the_terms.buckets.1.doc_count: 2 } + - match: { aggregations.the_terms.buckets.1.min_agg.value: null } + - match: { aggregations.the_terms.buckets.1.max_agg.value: null } + - match: { aggregations.the_terms.buckets.1.avg_agg.value: null } + - match: { aggregations.the_terms.buckets.1.sum_agg.value: null } + - match: { aggregations.the_terms.buckets.1.value_count_agg.value: 0 } + - match: { aggregations.the_terms.buckets.1.cardinality_agg.value: 0 } + - match: { aggregations.the_terms.buckets.1.weighted_avg_agg.value: null } + - match: { aggregations.the_terms.buckets.1.median_absolute_deviation_agg.value: null } + - match: { aggregations.the_terms.buckets.1.stats_agg.min: null } + - match: { aggregations.the_terms.buckets.1.stats_agg.max: null } + - match: { aggregations.the_terms.buckets.1.stats_agg.avg: null } + - match: { aggregations.the_terms.buckets.1.stats_agg.sum: 0.0 } + - match: { aggregations.the_terms.buckets.1.extended_stats_agg.min: null } + - match: { aggregations.the_terms.buckets.1.extended_stats_agg.max: null } + - match: { aggregations.the_terms.buckets.1.extended_stats_agg.avg: null } + - match: { aggregations.the_terms.buckets.1.extended_stats_agg.sum: 0.0 } + - match: { aggregations.the_terms.buckets.1.extended_stats_agg.sum_of_squares: null } + - match: { aggregations.the_terms.buckets.1.percentiles_agg.50: null } + - match: { aggregations.the_terms.buckets.1.geo_bounds_agg.top_left: null } + - match: { aggregations.the_terms.buckets.1.geo_bounds_agg.bottom_right: null } + - match: { aggregations.the_terms.buckets.1.geo_centroid_agg.count: 0 } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java index 7f25cdc495e5..670cf08038e0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java @@ -67,8 +67,8 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - DoubleHistogram state = getExistingOrNewHistogram(bigArrays(), bucket); if (values.advanceExact(doc)) { + DoubleHistogram state = getExistingOrNewHistogram(bigArrays(), bucket); final int valueCount = values.docValueCount(); for (int i = 0; i < valueCount; i++) { state.recordValue(values.nextValue()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java index 36bc76f8e15e..5b58d2e26abf 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java @@ -69,8 +69,8 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - TDigestState state = getExistingOrNewHistogram(bigArrays(), bucket); if (values.advanceExact(doc)) { + TDigestState state = getExistingOrNewHistogram(bigArrays(), bucket); final int valueCount = values.docValueCount(); for (int i = 0; i < valueCount; i++) { state.add(values.nextValue()); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java index 67ad387a0fb7..575108951b89 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/AvgAggregator.java @@ -65,11 +65,12 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - counts = bigArrays().grow(counts, bucket + 1); - sums = bigArrays().grow(sums, bucket + 1); - compensations = bigArrays().grow(compensations, bucket + 1); - if (values.advanceExact(doc)) { + if (bucket >= counts.size()) { + counts = bigArrays().grow(counts, bucket + 1); + sums = bigArrays().grow(sums, bucket + 1); + compensations = bigArrays().grow(compensations, bucket + 1); + } final int valueCount = values.docValueCount(); counts.increment(bucket, valueCount); // Compute the sum of double values with Kahan summation algorithm which is more 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 b83449196719..ecdaefc2a95e 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 @@ -255,13 +255,13 @@ public static long memoryOverhead(long maxOrd) { @Override public void collect(int doc, long bucketOrd) throws IOException { - visitedOrds = bigArrays.grow(visitedOrds, bucketOrd + 1); - BitArray bits = visitedOrds.get(bucketOrd); - if (bits == null) { - bits = new BitArray(maxOrd, bigArrays); - visitedOrds.set(bucketOrd, bits); - } if (values.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); + } for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) { bits.set((int) ord); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java index cecd75941bca..194ec2b64175 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsAggregator.java @@ -82,21 +82,20 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, @Override public void collect(int doc, long bucket) throws IOException { - if (bucket >= counts.size()) { - final long from = counts.size(); - final long overSize = BigArrays.overSize(bucket + 1); - counts = bigArrays().resize(counts, overSize); - sums = bigArrays().resize(sums, overSize); - compensations = bigArrays().resize(compensations, overSize); - mins = bigArrays().resize(mins, overSize); - maxes = bigArrays().resize(maxes, overSize); - sumOfSqrs = bigArrays().resize(sumOfSqrs, overSize); - compensationOfSqrs = bigArrays().resize(compensationOfSqrs, overSize); - mins.fill(from, overSize, Double.POSITIVE_INFINITY); - maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); - } - if (values.advanceExact(doc)) { + if (bucket >= counts.size()) { + final long from = counts.size(); + final long overSize = BigArrays.overSize(bucket + 1); + counts = bigArrays().resize(counts, overSize); + sums = bigArrays().resize(sums, overSize); + compensations = bigArrays().resize(compensations, overSize); + mins = bigArrays().resize(mins, overSize); + maxes = bigArrays().resize(maxes, overSize); + sumOfSqrs = bigArrays().resize(sumOfSqrs, overSize); + compensationOfSqrs = bigArrays().resize(compensationOfSqrs, overSize); + mins.fill(from, overSize, Double.POSITIVE_INFINITY); + maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); + } final int valuesCount = values.docValueCount(); counts.increment(bucket, valuesCount); double min = mins.get(bucket); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java index fd967182145a..5f5ef4da1f95 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsAggregator.java @@ -77,8 +77,8 @@ private LeafBucketCollector getLeafCollector(MultiGeoPointValues values, LeafBuc return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - growBucket(bucket); if (values.advanceExact(doc)) { + growBucket(bucket); for (int i = 0; i < values.docValueCount(); ++i) { addPoint(values.nextValue(), bucket); } @@ -91,8 +91,8 @@ private LeafBucketCollector getLeafCollector(GeoPointValues values, LeafBucketCo return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - growBucket(bucket); if (values.advanceExact(doc)) { + growBucket(bucket); addPoint(values.pointValue(), bucket); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java index f766295f12ee..2bab6c672b4e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidAggregator.java @@ -65,8 +65,8 @@ private LeafBucketCollector getLeafCollector(MultiGeoPointValues values, LeafBuc return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - growBucket(bucket); if (values.advanceExact(doc)) { + growBucket(bucket); final int valueCount = values.docValueCount(); // increment by the number of points for this document counts.increment(bucket, valueCount); @@ -96,8 +96,8 @@ private LeafBucketCollector getLeafCollector(GeoPointValues values, LeafBucketCo return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - growBucket(bucket); if (values.advanceExact(doc)) { + growBucket(bucket); // increment by the number of points for this document counts.increment(bucket, 1); // Compute the sum of double values with Kahan summation algorithm which is more 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 7661c3122db0..56d627712db2 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 @@ -206,6 +206,7 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, // 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()); if (maxOrd <= MAX_FIELD_CARDINALITY_FOR_DYNAMIC_PRUNING || numNonVisitedOrds <= MAX_TERMS_FOR_DYNAMIC_PRUNING) { @@ -269,13 +270,13 @@ public CompetitiveIterator competitiveIterator() { @Override public void collect(int doc, long bucketOrd) throws IOException { - visitedOrds = bigArrays.grow(visitedOrds, bucketOrd + 1); - BitArray bits = visitedOrds.get(bucketOrd); - if (bits == null) { - bits = new BitArray(maxOrd, bigArrays); - visitedOrds.set(bucketOrd, bits); - } 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); + } for (long ord = docValues.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = docValues.nextOrd()) { bits.set((int) ord); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java index b42dda576c67..9483fbe3d0d2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MaxAggregator.java @@ -85,16 +85,13 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, @Override public void collect(int doc, long bucket) throws IOException { - if (bucket >= maxes.size()) { - long from = maxes.size(); - maxes = bigArrays().grow(maxes, bucket + 1); - maxes.fill(from, maxes.size(), Double.NEGATIVE_INFINITY); - } if (values.advanceExact(doc)) { - final double value = values.doubleValue(); - double max = maxes.get(bucket); - max = Math.max(max, value); - maxes.set(bucket, max); + if (bucket >= maxes.size()) { + long from = maxes.size(); + maxes = bigArrays().grow(maxes, bucket + 1); + maxes.fill(from, maxes.size(), Double.NEGATIVE_INFINITY); + } + maxes.set(bucket, Math.max(maxes.get(bucket), values.doubleValue())); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java index 2a4003fc9f60..61c2c75a49d7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java @@ -87,16 +87,14 @@ protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCt return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - - valueSketches = bigArrays().grow(valueSketches, bucket + 1); - - TDigestState valueSketch = valueSketches.get(bucket); - if (valueSketch == null) { - valueSketch = TDigestState.create(compression, executionHint); - valueSketches.set(bucket, valueSketch); - } - if (values.advanceExact(doc)) { + valueSketches = bigArrays().grow(valueSketches, bucket + 1); + + TDigestState valueSketch = valueSketches.get(bucket); + if (valueSketch == null) { + valueSketch = TDigestState.create(compression, executionHint); + valueSketches.set(bucket, valueSketch); + } final int valueCount = values.docValueCount(); for (int i = 0; i < valueCount; i++) { final double value = values.nextValue(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java index 9c8562ae91d0..efe9526ab0c7 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/MinAggregator.java @@ -84,16 +84,13 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, allValues) { @Override public void collect(int doc, long bucket) throws IOException { - if (bucket >= mins.size()) { - long from = mins.size(); - mins = bigArrays().grow(mins, bucket + 1); - mins.fill(from, mins.size(), Double.POSITIVE_INFINITY); - } if (values.advanceExact(doc)) { - final double value = values.doubleValue(); - double min = mins.get(bucket); - min = Math.min(min, value); - mins.set(bucket, min); + if (bucket >= mins.size()) { + long from = mins.size(); + mins = bigArrays().grow(mins, bucket + 1); + mins.fill(from, mins.size(), Double.POSITIVE_INFINITY); + } + mins.set(bucket, Math.min(mins.get(bucket), values.doubleValue())); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java index 7a5861eb97fe..8f571a95a145 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/StatsAggregator.java @@ -65,19 +65,18 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - if (bucket >= counts.size()) { - final long from = counts.size(); - final long overSize = BigArrays.overSize(bucket + 1); - counts = bigArrays().resize(counts, overSize); - sums = bigArrays().resize(sums, overSize); - compensations = bigArrays().resize(compensations, overSize); - mins = bigArrays().resize(mins, overSize); - maxes = bigArrays().resize(maxes, overSize); - mins.fill(from, overSize, Double.POSITIVE_INFINITY); - maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); - } - if (values.advanceExact(doc)) { + if (bucket >= counts.size()) { + final long from = counts.size(); + final long overSize = BigArrays.overSize(bucket + 1); + counts = bigArrays().resize(counts, overSize); + sums = bigArrays().resize(sums, overSize); + compensations = bigArrays().resize(compensations, overSize); + mins = bigArrays().resize(mins, overSize); + maxes = bigArrays().resize(maxes, overSize); + mins.fill(from, overSize, Double.POSITIVE_INFINITY); + maxes.fill(from, overSize, Double.NEGATIVE_INFINITY); + } final int valuesCount = values.docValueCount(); counts.increment(bucket, valuesCount); double min = mins.get(bucket); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java index 14760b8adb2e..94d9f311db62 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/SumAggregator.java @@ -59,10 +59,11 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, values) { @Override public void collect(int doc, long bucket) throws IOException { - sums = bigArrays().grow(sums, bucket + 1); - compensations = bigArrays().grow(compensations, bucket + 1); - if (values.advanceExact(doc)) { + if (bucket >= sums.size()) { + sums = bigArrays().grow(sums, bucket + 1); + compensations = bigArrays().grow(compensations, bucket + 1); + } final int valuesCount = values.docValueCount(); // Compute the sum of double values with Kahan summation algorithm which is more // accurate than naive summation. diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java index ae14b4601b55..a4bfaf185059 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/ValueCountAggregator.java @@ -59,8 +59,8 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, @Override public void collect(int doc, long bucket) throws IOException { - counts = bigArrays().grow(counts, bucket + 1); if (values.advanceExact(doc)) { + counts = bigArrays().grow(counts, bucket + 1); counts.increment(bucket, values.docValueCount()); } } @@ -72,8 +72,8 @@ public void collect(int doc, long bucket) throws IOException { @Override public void collect(int doc, long bucket) throws IOException { - counts = bigArrays().grow(counts, bucket + 1); if (values.advanceExact(doc)) { + counts = bigArrays().grow(counts, bucket + 1); counts.increment(bucket, values.docValueCount()); } } @@ -85,8 +85,8 @@ public void collect(int doc, long bucket) throws IOException { @Override public void collect(int doc, long bucket) throws IOException { - counts = bigArrays().grow(counts, bucket + 1); if (values.advanceExact(doc)) { + counts = bigArrays().grow(counts, bucket + 1); counts.increment(bucket, values.docValueCount()); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java index 1a1ffc46946e..98e8434f940b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/WeightedAvgAggregator.java @@ -69,11 +69,6 @@ public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, return new LeafBucketCollectorBase(sub, docValues) { @Override public void collect(int doc, long bucket) throws IOException { - weights = bigArrays().grow(weights, bucket + 1); - valueSums = bigArrays().grow(valueSums, bucket + 1); - valueCompensations = bigArrays().grow(valueCompensations, bucket + 1); - weightCompensations = bigArrays().grow(weightCompensations, bucket + 1); - if (docValues.advanceExact(doc) && docWeights.advanceExact(doc)) { if (docWeights.docValueCount() > 1) { throw new IllegalArgumentException( @@ -81,6 +76,12 @@ public void collect(int doc, long bucket) throws IOException { + "single document. Use a script to combine multiple weights-per-doc into a single value." ); } + if (bucket >= weights.size()) { + weights = bigArrays().grow(weights, bucket + 1); + valueSums = bigArrays().grow(valueSums, bucket + 1); + valueCompensations = bigArrays().grow(valueCompensations, bucket + 1); + weightCompensations = bigArrays().grow(weightCompensations, bucket + 1); + } // There should always be one weight if advanceExact lands us here, either // a real weight or a `missing` weight assert docWeights.docValueCount() == 1; From 27c6fc4794606ea9ede9cdab2f7761f51515ccdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Slobodan=20Adamovi=C4=87?= Date: Wed, 24 Apr 2024 10:47:06 +0200 Subject: [PATCH 13/27] [Test] Fix AsyncSearchResponse resource leak in security tests (#107809) Closes #107759 --- .../elasticsearch/xpack/search/AsyncSearchSecurityIT.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/async-search/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/search/AsyncSearchSecurityIT.java b/x-pack/plugin/async-search/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/search/AsyncSearchSecurityIT.java index 05eddd742960..c5736fb44058 100644 --- a/x-pack/plugin/async-search/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/search/AsyncSearchSecurityIT.java +++ b/x-pack/plugin/async-search/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/search/AsyncSearchSecurityIT.java @@ -291,7 +291,7 @@ private static void userBasedPermissionsAsserts(String user, String other, Strin private SearchHit[] getSearchHits(String asyncId, String user) throws IOException { final Response resp = getAsyncSearch(asyncId, user); assertOK(resp); - SearchResponse searchResponse = ASYNC_SEARCH_RESPONSE_PARSER.apply( + AsyncSearchResponse asyncSearchResponse = ASYNC_SEARCH_RESPONSE_PARSER.apply( XContentHelper.createParser( NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, @@ -299,11 +299,13 @@ private SearchHit[] getSearchHits(String asyncId, String user) throws IOExceptio XContentType.JSON ), null - ).getSearchResponse(); + ); + SearchResponse searchResponse = asyncSearchResponse.getSearchResponse(); try { return searchResponse.getHits().asUnpooled().getHits(); } finally { searchResponse.decRef(); + asyncSearchResponse.decRef(); } } From 119f6e71ce95742cf3205a2c306b7f1d1ab55819 Mon Sep 17 00:00:00 2001 From: Mary Gouseti Date: Wed, 24 Apr 2024 11:52:24 +0300 Subject: [PATCH 14/27] [Data stream lifecycle] Introduce factory retention settings (#107741) We introduce the plumbing so that a plugin can provide factory retention. This retention will take effect if there is no global retention provided by the user. Without a plugin defining the factory retention, elasticsearch will have no factory retention. --- .../datastreams/DataStreamsPlugin.java | 7 +- .../action/GetDataStreamsTransportAction.java | 16 ++-- .../lifecycle/DataStreamLifecycleService.java | 10 ++- ...pdateDataStreamGlobalRetentionService.java | 16 +++- .../GetDataStreamGlobalRetentionAction.java | 8 +- ...sportExplainDataStreamLifecycleAction.java | 9 ++- ...TransportGetDataStreamLifecycleAction.java | 9 ++- .../GetDataStreamsTransportActionTests.java | 20 +++-- .../DataStreamLifecycleServiceTests.java | 11 ++- ...DataStreamGlobalRetentionServiceTests.java | 59 +++++++++++++- server/src/main/java/module-info.java | 1 + .../TransportGetComponentTemplateAction.java | 11 ++- ...sportGetComposableIndexTemplateAction.java | 13 ++-- .../TransportSimulateIndexTemplateAction.java | 8 +- .../post/TransportSimulateTemplateAction.java | 8 +- .../metadata/DataStreamFactoryRetention.java | 76 ++++++++++++++++++ .../metadata/DataStreamGlobalRetention.java | 6 +- .../DataStreamGlobalRetentionResolver.java | 40 ++++++++++ .../elasticsearch/node/NodeConstruction.java | 13 +++- .../org/elasticsearch/plugins/Plugin.java | 7 ++ ...ataStreamGlobalRetentionResolverTests.java | 77 +++++++++++++++++++ 21 files changed, 378 insertions(+), 47 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamFactoryRetention.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolver.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolverTests.java diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/DataStreamsPlugin.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/DataStreamsPlugin.java index a1e65d7784a3..02663731a171 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/DataStreamsPlugin.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/DataStreamsPlugin.java @@ -208,12 +208,15 @@ public Collection createComponents(PluginServices services) { services.threadPool()::absoluteTimeInMillis, errorStoreInitialisationService.get(), services.allocationService(), - dataStreamLifecycleErrorsPublisher.get() + dataStreamLifecycleErrorsPublisher.get(), + services.dataStreamGlobalRetentionResolver() ) ); dataLifecycleInitialisationService.get().init(); dataStreamLifecycleHealthIndicatorService.set(new DataStreamLifecycleHealthIndicatorService()); - dataStreamGlobalRetentionService.set(new UpdateDataStreamGlobalRetentionService(services.clusterService())); + dataStreamGlobalRetentionService.set( + new UpdateDataStreamGlobalRetentionService(services.clusterService(), services.dataStreamGlobalRetentionResolver()) + ); components.add(errorStoreInitialisationService.get()); components.add(dataLifecycleInitialisationService.get()); diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportAction.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportAction.java index 0fc00ad9ebe5..d4d62c282917 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportAction.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportAction.java @@ -21,7 +21,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.health.ClusterStateHealth; import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -57,6 +57,7 @@ public class GetDataStreamsTransportAction extends TransportMasterNodeReadAction private static final Logger LOGGER = LogManager.getLogger(GetDataStreamsTransportAction.class); private final SystemIndices systemIndices; private final ClusterSettings clusterSettings; + private final DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver; @Inject public GetDataStreamsTransportAction( @@ -65,7 +66,8 @@ public GetDataStreamsTransportAction( ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - SystemIndices systemIndices + SystemIndices systemIndices, + DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver ) { super( GetDataStreamAction.NAME, @@ -79,6 +81,7 @@ public GetDataStreamsTransportAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); this.systemIndices = systemIndices; + this.dataStreamGlobalRetentionResolver = dataStreamGlobalRetentionResolver; clusterSettings = clusterService.getClusterSettings(); } @@ -89,7 +92,9 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - listener.onResponse(innerOperation(state, request, indexNameExpressionResolver, systemIndices, clusterSettings)); + listener.onResponse( + innerOperation(state, request, indexNameExpressionResolver, systemIndices, clusterSettings, dataStreamGlobalRetentionResolver) + ); } static GetDataStreamAction.Response innerOperation( @@ -97,7 +102,8 @@ static GetDataStreamAction.Response innerOperation( GetDataStreamAction.Request request, IndexNameExpressionResolver indexNameExpressionResolver, SystemIndices systemIndices, - ClusterSettings clusterSettings + ClusterSettings clusterSettings, + DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver ) { List dataStreams = getDataStreams(state, indexNameExpressionResolver, request); List dataStreamInfos = new ArrayList<>(dataStreams.size()); @@ -202,7 +208,7 @@ static GetDataStreamAction.Response innerOperation( return new GetDataStreamAction.Response( dataStreamInfos, request.includeDefaults() ? clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) : null, - DataStreamGlobalRetention.getFromClusterState(state) + dataStreamGlobalRetentionResolver.resolve(state) ); } 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 8049363f2679..9e3dd5cc1a3b 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 @@ -43,6 +43,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -161,6 +162,7 @@ public class DataStreamLifecycleService implements ClusterStateListener, Closeab final ResultDeduplicator transportActionsDeduplicator; final ResultDeduplicator clusterStateChangesDeduplicator; private final DataStreamLifecycleHealthInfoPublisher dslHealthInfoPublisher; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; private LongSupplier nowSupplier; private final Clock clock; private final DataStreamLifecycleErrorStore errorStore; @@ -208,7 +210,8 @@ public DataStreamLifecycleService( LongSupplier nowSupplier, DataStreamLifecycleErrorStore errorStore, AllocationService allocationService, - DataStreamLifecycleHealthInfoPublisher dataStreamLifecycleHealthInfoPublisher + DataStreamLifecycleHealthInfoPublisher dataStreamLifecycleHealthInfoPublisher, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { this.settings = settings; this.client = client; @@ -219,6 +222,7 @@ public DataStreamLifecycleService( this.clusterStateChangesDeduplicator = new ResultDeduplicator<>(threadPool.getThreadContext()); this.nowSupplier = nowSupplier; this.errorStore = errorStore; + this.globalRetentionResolver = globalRetentionResolver; this.scheduledJob = null; this.pollInterval = DATA_STREAM_LIFECYCLE_POLL_INTERVAL_SETTING.get(settings); this.targetMergePolicyFloorSegment = DATA_STREAM_MERGE_POLICY_TARGET_FLOOR_SEGMENT_SETTING.get(settings); @@ -783,7 +787,7 @@ private Set maybeExecuteRollover(ClusterState state, DataStream dataStrea RolloverRequest rolloverRequest = getDefaultRolloverRequest( rolloverConfiguration, dataStream.getName(), - dataStream.getLifecycle().getEffectiveDataRetention(DataStreamGlobalRetention.getFromClusterState(state)) + dataStream.getLifecycle().getEffectiveDataRetention(globalRetentionResolver.resolve(state)) ); transportActionsDeduplicator.executeOnce( rolloverRequest, @@ -828,7 +832,7 @@ private Set maybeExecuteRollover(ClusterState state, DataStream dataStrea */ private Set maybeExecuteRetention(ClusterState state, DataStream dataStream, Set indicesToExcludeForRemainingRun) { Metadata metadata = state.metadata(); - DataStreamGlobalRetention globalRetention = DataStreamGlobalRetention.getFromClusterState(state); + DataStreamGlobalRetention globalRetention = globalRetentionResolver.resolve(state); List backingIndicesOlderThanRetention = dataStream.getIndicesPastRetention(metadata::index, nowSupplier, globalRetention); if (backingIndicesOlderThanRetention.isEmpty()) { return Set.of(); 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 5ac50e388e13..a906008c1774 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 @@ -18,6 +18,7 @@ import org.elasticsearch.cluster.SimpleBatchedAckListenerTaskExecutor; import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterServiceTaskQueue; @@ -42,9 +43,14 @@ public class UpdateDataStreamGlobalRetentionService { private static final Logger logger = LogManager.getLogger(UpdateDataStreamGlobalRetentionService.class); + private final DataStreamGlobalRetentionResolver globalRetentionResolver; private final MasterServiceTaskQueue taskQueue; - public UpdateDataStreamGlobalRetentionService(ClusterService clusterService) { + public UpdateDataStreamGlobalRetentionService( + ClusterService clusterService, + DataStreamGlobalRetentionResolver globalRetentionResolver + ) { + this.globalRetentionResolver = globalRetentionResolver; ClusterStateTaskExecutor executor = new SimpleBatchedAckListenerTaskExecutor<>() { @Override @@ -95,7 +101,7 @@ public List determin @Nullable DataStreamGlobalRetention newGlobalRetention, ClusterState clusterState ) { - var previousGlobalRetention = DataStreamGlobalRetention.getFromClusterState(clusterState); + var previousGlobalRetention = globalRetentionResolver.resolve(clusterState); if (Objects.equals(newGlobalRetention, previousGlobalRetention)) { return List.of(); } @@ -121,10 +127,12 @@ public List determin // Visible for testing ClusterState updateGlobalRetention(ClusterState clusterState, @Nullable DataStreamGlobalRetention retentionFromRequest) { - final var initialRetention = DataStreamGlobalRetention.getFromClusterState(clusterState); + // Detecting if this update will result in a change in the cluster state, requires to use only the global retention from + // the cluster state and not the factory retention. + final var initialRetentionFromClusterState = DataStreamGlobalRetention.getFromClusterState(clusterState); // Avoid storing empty retention in the cluster state final var newRetention = DataStreamGlobalRetention.EMPTY.equals(retentionFromRequest) ? null : retentionFromRequest; - if (Objects.equals(newRetention, initialRetention)) { + if (Objects.equals(newRetention, initialRetentionFromClusterState)) { return clusterState; } if (newRetention == null) { diff --git a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/GetDataStreamGlobalRetentionAction.java b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/GetDataStreamGlobalRetentionAction.java index b694e1276785..51eb9e7e7e94 100644 --- a/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/GetDataStreamGlobalRetentionAction.java +++ b/modules/data-streams/src/main/java/org/elasticsearch/datastreams/lifecycle/action/GetDataStreamGlobalRetentionAction.java @@ -20,6 +20,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; @@ -121,6 +122,7 @@ public int hashCode() { public static class TransportGetDataStreamGlobalSettingsAction extends TransportMasterNodeReadAction { private final FeatureService featureService; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportGetDataStreamGlobalSettingsAction( @@ -129,7 +131,8 @@ public TransportGetDataStreamGlobalSettingsAction( ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - FeatureService featureService + FeatureService featureService, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( INSTANCE.name(), @@ -143,6 +146,7 @@ public TransportGetDataStreamGlobalSettingsAction( threadPool.executor(ThreadPool.Names.MANAGEMENT) ); this.featureService = featureService; + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -156,7 +160,7 @@ protected void masterOperation(Task task, Request request, ClusterState state, A ); return; } - DataStreamGlobalRetention globalRetention = DataStreamGlobalRetention.getFromClusterState(state); + DataStreamGlobalRetention globalRetention = globalRetentionResolver.resolve(state); listener.onResponse(new Response(globalRetention == null ? DataStreamGlobalRetention.EMPTY : globalRetention)); } 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 e88c023e8996..ac5f46edb5cc 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 @@ -18,7 +18,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -44,6 +44,7 @@ public class TransportExplainDataStreamLifecycleAction extends TransportMasterNo ExplainDataStreamLifecycleAction.Response> { private final DataStreamLifecycleErrorStore errorStore; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportExplainDataStreamLifecycleAction( @@ -52,7 +53,8 @@ public TransportExplainDataStreamLifecycleAction( ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - DataStreamLifecycleErrorStore dataLifecycleServiceErrorStore + DataStreamLifecycleErrorStore dataLifecycleServiceErrorStore, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( ExplainDataStreamLifecycleAction.INSTANCE.name(), @@ -66,6 +68,7 @@ public TransportExplainDataStreamLifecycleAction( threadPool.executor(ThreadPool.Names.MANAGEMENT) ); this.errorStore = dataLifecycleServiceErrorStore; + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -114,7 +117,7 @@ protected void masterOperation( new ExplainDataStreamLifecycleAction.Response( explainIndices, request.includeDefaults() ? clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) : null, - DataStreamGlobalRetention.getFromClusterState(state) + globalRetentionResolver.resolve(state) ) ); } 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 881f472b19d3..deff08357980 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 @@ -16,7 +16,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -40,6 +40,7 @@ public class TransportGetDataStreamLifecycleAction extends TransportMasterNodeRe GetDataStreamLifecycleAction.Request, GetDataStreamLifecycleAction.Response> { private final ClusterSettings clusterSettings; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportGetDataStreamLifecycleAction( @@ -47,7 +48,8 @@ public TransportGetDataStreamLifecycleAction( ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( GetDataStreamLifecycleAction.INSTANCE.name(), @@ -61,6 +63,7 @@ public TransportGetDataStreamLifecycleAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); clusterSettings = clusterService.getClusterSettings(); + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -92,7 +95,7 @@ protected void masterOperation( .sorted(Comparator.comparing(GetDataStreamLifecycleAction.Response.DataStreamLifecycle::dataStreamName)) .toList(), request.includeDefaults() ? clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING) : null, - DataStreamGlobalRetention.getFromClusterState(state) + globalRetentionResolver.resolve(state) ) ); } diff --git a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportActionTests.java b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportActionTests.java index 2a356e3ebb16..f7616482edd1 100644 --- a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportActionTests.java +++ b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/action/GetDataStreamsTransportActionTests.java @@ -11,7 +11,9 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.DataStreamFactoryRetention; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamTestHelper; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.Metadata; @@ -43,6 +45,9 @@ public class GetDataStreamsTransportActionTests extends ESTestCase { private final IndexNameExpressionResolver resolver = TestIndexNameExpressionResolver.newInstance(); private final SystemIndices systemIndices = new SystemIndices(List.of()); + private final DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver = new DataStreamGlobalRetentionResolver( + DataStreamFactoryRetention.emptyFactoryRetention() + ); public void testGetDataStream() { final String dataStreamName = "my-data-stream"; @@ -153,7 +158,8 @@ public void testGetTimeSeriesDataStream() { req, resolver, systemIndices, - ClusterSettings.createBuiltInClusterSettings() + ClusterSettings.createBuiltInClusterSettings(), + dataStreamGlobalRetentionResolver ); assertThat( response.getDataStreams(), @@ -182,7 +188,8 @@ public void testGetTimeSeriesDataStream() { req, resolver, systemIndices, - ClusterSettings.createBuiltInClusterSettings() + ClusterSettings.createBuiltInClusterSettings(), + dataStreamGlobalRetentionResolver ); assertThat( response.getDataStreams(), @@ -231,7 +238,8 @@ public void testGetTimeSeriesMixedDataStream() { req, resolver, systemIndices, - ClusterSettings.createBuiltInClusterSettings() + ClusterSettings.createBuiltInClusterSettings(), + dataStreamGlobalRetentionResolver ); var name1 = DataStream.getDefaultBackingIndexName("ds-1", 1, instant.toEpochMilli()); @@ -275,7 +283,8 @@ public void testPassingGlobalRetention() { req, resolver, systemIndices, - ClusterSettings.createBuiltInClusterSettings() + ClusterSettings.createBuiltInClusterSettings(), + dataStreamGlobalRetentionResolver ); assertThat(response.getGlobalRetention(), nullValue()); DataStreamGlobalRetention globalRetention = new DataStreamGlobalRetention( @@ -288,7 +297,8 @@ public void testPassingGlobalRetention() { req, resolver, systemIndices, - ClusterSettings.createBuiltInClusterSettings() + ClusterSettings.createBuiltInClusterSettings(), + dataStreamGlobalRetentionResolver ); assertThat(response.getGlobalRetention(), equalTo(globalRetention)); } diff --git a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceTests.java b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceTests.java index 9282fd5b2eeb..07fe2132899c 100644 --- a/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceTests.java +++ b/modules/data-streams/src/test/java/org/elasticsearch/datastreams/lifecycle/DataStreamLifecycleServiceTests.java @@ -35,6 +35,8 @@ import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.DataStreamFactoryRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.DataStreamLifecycle.Downsampling; import org.elasticsearch.cluster.metadata.DataStreamLifecycle.Downsampling.Round; @@ -133,6 +135,9 @@ public class DataStreamLifecycleServiceTests extends ESTestCase { private List clientSeenRequests; private DoExecuteDelegate clientDelegate; private ClusterService clusterService; + private final DataStreamGlobalRetentionResolver globalRetentionResolver = new DataStreamGlobalRetentionResolver( + DataStreamFactoryRetention.emptyFactoryRetention() + ); @Before public void setupServices() { @@ -178,7 +183,8 @@ public void setupServices() { clusterService, errorStore, new FeatureService(List.of(new DataStreamFeatures())) - ) + ), + globalRetentionResolver ); clientDelegate = null; dataStreamLifecycleService.init(); @@ -1401,7 +1407,8 @@ public void testTrackingTimeStats() { clusterService, errorStore, new FeatureService(List.of(new DataStreamFeatures())) - ) + ), + globalRetentionResolver ); assertThat(service.getLastRunDuration(), is(nullValue())); assertThat(service.getTimeBetweenStarts(), is(nullValue())); 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 65b8473e4df9..41e3e3a28ed5 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 @@ -11,11 +11,14 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.DataStreamFactoryRetention; 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.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.test.ClusterServiceUtils; @@ -34,6 +37,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.Matchers.empty; public class UpdateDataStreamGlobalRetentionServiceTests extends ESTestCase { private static TestThreadPool threadPool; @@ -48,7 +52,10 @@ public static void setupThreadPool() { @Before public void setupServices() { clusterService = ClusterServiceUtils.createClusterService(threadPool); - service = new UpdateDataStreamGlobalRetentionService(clusterService); + service = new UpdateDataStreamGlobalRetentionService( + clusterService, + new DataStreamGlobalRetentionResolver(DataStreamFactoryRetention.emptyFactoryRetention()) + ); } @After @@ -102,6 +109,22 @@ public void testUpdateClusterState() { ); assertThat(updatedRetention, equalTo(expectedRetention)); } + + // No change means no new cluster state + { + UpdateDataStreamGlobalRetentionService serviceWithRandomFactoryRetention = new UpdateDataStreamGlobalRetentionService( + clusterService, + new DataStreamGlobalRetentionResolver( + randomBoolean() ? DataStreamFactoryRetention.emptyFactoryRetention() : randomNonEmptyFactoryRetention() + ) + ); + var retention = randomBoolean() ? null : randomNonEmptyGlobalRetention(); + ClusterState clusterState = retention == null + ? ClusterState.EMPTY_STATE + : ClusterState.builder(ClusterName.DEFAULT).putCustom(DataStreamGlobalRetention.TYPE, retention).build(); + var updatedClusterState = serviceWithRandomFactoryRetention.updateGlobalRetention(clusterState, retention); + assertThat(updatedClusterState == clusterState, is(true)); + } } public void testDetermineAffectedDataStreams() { @@ -188,6 +211,18 @@ public void testDetermineAffectedDataStreams() { assertThat(dataStream.previousEffectiveRetention(), nullValue()); assertThat(dataStream.newEffectiveRetention(), equalTo(globalRetention.getMaxRetention())); } + + // Requested global retention match the factory retention, so no affected data streams + { + DataStreamFactoryRetention factoryRetention = randomNonEmptyFactoryRetention(); + UpdateDataStreamGlobalRetentionService serviceWithRandomFactoryRetention = new UpdateDataStreamGlobalRetentionService( + clusterService, + new DataStreamGlobalRetentionResolver(factoryRetention) + ); + var globalRetention = new DataStreamGlobalRetention(factoryRetention.getDefaultRetention(), factoryRetention.getMaxRetention()); + var affectedDataStreams = serviceWithRandomFactoryRetention.determineAffectedDataStreams(globalRetention, clusterState); + assertThat(affectedDataStreams, is(empty())); + } } private static DataStreamGlobalRetention randomNonEmptyGlobalRetention() { @@ -197,4 +232,26 @@ private static DataStreamGlobalRetention randomNonEmptyGlobalRetention() { withDefault == false || randomBoolean() ? TimeValue.timeValueDays(randomIntBetween(1000, 2000)) : null ); } + + private static DataStreamFactoryRetention randomNonEmptyFactoryRetention() { + boolean withDefault = randomBoolean(); + TimeValue defaultRetention = withDefault ? TimeValue.timeValueDays(randomIntBetween(10, 20)) : null; + TimeValue maxRetention = withDefault && randomBoolean() ? null : TimeValue.timeValueDays(randomIntBetween(50, 200)); + return new DataStreamFactoryRetention() { + @Override + public TimeValue getMaxRetention() { + return maxRetention; + } + + @Override + public TimeValue getDefaultRetention() { + return defaultRetention; + } + + @Override + public void init(ClusterSettings clusterSettings) { + + } + }; + } } diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 8295edfee12f..ebc50349e7fe 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -414,6 +414,7 @@ uses org.elasticsearch.internal.BuildExtension; uses org.elasticsearch.features.FeatureSpecification; uses org.elasticsearch.plugins.internal.LoggingDataProvider; + uses org.elasticsearch.cluster.metadata.DataStreamFactoryRetention; provides org.elasticsearch.features.FeatureSpecification with diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java index d238209fa88f..0f088e046dd5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java @@ -16,7 +16,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComponentTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -36,6 +36,7 @@ public class TransportGetComponentTemplateAction extends TransportMasterNodeRead GetComponentTemplateAction.Response> { private final ClusterSettings clusterSettings; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportGetComponentTemplateAction( @@ -43,7 +44,8 @@ public TransportGetComponentTemplateAction( ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( GetComponentTemplateAction.NAME, @@ -57,6 +59,7 @@ public TransportGetComponentTemplateAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); clusterSettings = clusterService.getClusterSettings(); + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -98,11 +101,11 @@ protected void masterOperation( new GetComponentTemplateAction.Response( results, clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - DataStreamGlobalRetention.getFromClusterState(state) + globalRetentionResolver.resolve(state) ) ); } else { - listener.onResponse(new GetComponentTemplateAction.Response(results, DataStreamGlobalRetention.getFromClusterState(state))); + listener.onResponse(new GetComponentTemplateAction.Response(results, globalRetentionResolver.resolve(state))); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java index 35a91e70fd78..6d6de026ced5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java @@ -16,7 +16,7 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -36,6 +36,7 @@ public class TransportGetComposableIndexTemplateAction extends TransportMasterNo GetComposableIndexTemplateAction.Response> { private final ClusterSettings clusterSettings; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportGetComposableIndexTemplateAction( @@ -43,7 +44,8 @@ public TransportGetComposableIndexTemplateAction( ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( GetComposableIndexTemplateAction.NAME, @@ -57,6 +59,7 @@ public TransportGetComposableIndexTemplateAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); clusterSettings = clusterService.getClusterSettings(); + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -96,13 +99,11 @@ protected void masterOperation( new GetComposableIndexTemplateAction.Response( results, clusterSettings.get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING), - DataStreamGlobalRetention.getFromClusterState(state) + globalRetentionResolver.resolve(state) ) ); } else { - listener.onResponse( - new GetComposableIndexTemplateAction.Response(results, DataStreamGlobalRetention.getFromClusterState(state)) - ); + listener.onResponse(new GetComposableIndexTemplateAction.Response(results, globalRetentionResolver.resolve(state))); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java index 51e17999da5c..253f02d30465 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateIndexTemplateAction.java @@ -17,6 +17,7 @@ import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -72,6 +73,7 @@ public class TransportSimulateIndexTemplateAction extends TransportMasterNodeRea private final Set indexSettingProviders; private final ClusterSettings clusterSettings; private final boolean isDslOnlyMode; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportSimulateIndexTemplateAction( @@ -84,7 +86,8 @@ public TransportSimulateIndexTemplateAction( NamedXContentRegistry xContentRegistry, IndicesService indicesService, SystemIndices systemIndices, - IndexSettingProviders indexSettingProviders + IndexSettingProviders indexSettingProviders, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( SimulateIndexTemplateAction.NAME, @@ -104,6 +107,7 @@ public TransportSimulateIndexTemplateAction( this.indexSettingProviders = indexSettingProviders.getIndexSettingProviders(); this.clusterSettings = clusterService.getClusterSettings(); this.isDslOnlyMode = isDataStreamsLifecycleOnlyMode(clusterService.getSettings()); + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -113,7 +117,7 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - final DataStreamGlobalRetention globalRetention = DataStreamGlobalRetention.getFromClusterState(state); + final DataStreamGlobalRetention globalRetention = globalRetentionResolver.resolve(state); final ClusterState stateWithTemplate; if (request.getIndexTemplateRequest() != null) { // we'll "locally" add the template defined by the user in the cluster state (as if it existed in the system) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java index 39cf5f43f39e..b1d6d2814f15 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/post/TransportSimulateTemplateAction.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.DataStreamLifecycle; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetadataIndexTemplateService; @@ -59,6 +60,7 @@ public class TransportSimulateTemplateAction extends TransportMasterNodeReadActi private final Set indexSettingProviders; private final ClusterSettings clusterSettings; private final boolean isDslOnlyMode; + private final DataStreamGlobalRetentionResolver globalRetentionResolver; @Inject public TransportSimulateTemplateAction( @@ -71,7 +73,8 @@ public TransportSimulateTemplateAction( NamedXContentRegistry xContentRegistry, IndicesService indicesService, SystemIndices systemIndices, - IndexSettingProviders indexSettingProviders + IndexSettingProviders indexSettingProviders, + DataStreamGlobalRetentionResolver globalRetentionResolver ) { super( SimulateTemplateAction.NAME, @@ -91,6 +94,7 @@ public TransportSimulateTemplateAction( this.indexSettingProviders = indexSettingProviders.getIndexSettingProviders(); this.clusterSettings = clusterService.getClusterSettings(); this.isDslOnlyMode = isDataStreamsLifecycleOnlyMode(clusterService.getSettings()); + this.globalRetentionResolver = globalRetentionResolver; } @Override @@ -100,7 +104,7 @@ protected void masterOperation( ClusterState state, ActionListener listener ) throws Exception { - final DataStreamGlobalRetention globalRetention = DataStreamGlobalRetention.getFromClusterState(state); + final DataStreamGlobalRetention globalRetention = globalRetentionResolver.resolve(state); String uuid = UUIDs.randomBase64UUID().toLowerCase(Locale.ROOT); final String temporaryIndexName = "simulate_template_index_" + uuid; final ClusterState stateWithTemplate; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamFactoryRetention.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamFactoryRetention.java new file mode 100644 index 000000000000..5b96f92193e9 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamFactoryRetention.java @@ -0,0 +1,76 @@ +/* + * 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.cluster.metadata; + +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.plugins.PluginsService; + +/** + * Holds the factory retention configuration. Factory retention is the global retention configuration meant to be + * used if a user hasn't provided other retention configuration via {@link DataStreamGlobalRetention} metadata in the + * cluster state. + */ +public interface DataStreamFactoryRetention { + + @Nullable + TimeValue getMaxRetention(); + + @Nullable + TimeValue getDefaultRetention(); + + /** + * @return true, if at least one of the two settings is not null, false otherwise. + */ + default boolean isDefined() { + return getMaxRetention() != null || getDefaultRetention() != null; + } + + /** + * Applies any post constructor initialisation, for example, listening to cluster setting changes. + */ + void init(ClusterSettings clusterSettings); + + /** + * Loads a single instance of a DataStreamFactoryRetention from the {@link PluginsService} and finalises the + * initialisation by calling {@link DataStreamFactoryRetention#init(ClusterSettings)} + */ + static DataStreamFactoryRetention load(PluginsService pluginsService, ClusterSettings clusterSettings) { + DataStreamFactoryRetention factoryRetention = pluginsService.loadSingletonServiceProvider( + DataStreamFactoryRetention.class, + DataStreamFactoryRetention::emptyFactoryRetention + ); + factoryRetention.init(clusterSettings); + return factoryRetention; + } + + /** + * Returns empty factory global retention settings. + */ + static DataStreamFactoryRetention emptyFactoryRetention() { + return new DataStreamFactoryRetention() { + + @Override + public TimeValue getMaxRetention() { + return null; + } + + @Override + public TimeValue getDefaultRetention() { + return null; + } + + @Override + public void init(ClusterSettings clusterSettings) { + + } + }; + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetention.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetention.java index 3b58acfedbc1..f691151eee95 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetention.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetention.java @@ -118,8 +118,12 @@ public XContentBuilder toXContentFragment(XContentBuilder builder, ToXContent.Pa } /** - * Returns the metadata found in the cluster state or null. + * Returns the metadata found in the cluster state or null. When trying to retrieve the effective global retention, + * prefer to use the {@link DataStreamGlobalRetentionResolver#resolve(ClusterState)} because it takes into account + * the factory retention settings as well. Only use this, if you only want to know the global retention settings + * stored in the cluster metadata. */ + @Nullable public static DataStreamGlobalRetention getFromClusterState(ClusterState clusterState) { return clusterState.custom(DataStreamGlobalRetention.TYPE); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolver.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolver.java new file mode 100644 index 000000000000..ab44595e37c1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolver.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.metadata; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.core.Nullable; + +/** + * Resolves the global retention configuration for data stream lifecycle taking into consideration the + * metadata in the cluster state and the factory settings. + * Currently, we give precedence to the configuration in the metadata and fallback to the factory settings when it's not present. + */ +public class DataStreamGlobalRetentionResolver { + + private final DataStreamFactoryRetention factoryRetention; + + public DataStreamGlobalRetentionResolver(DataStreamFactoryRetention factoryRetention) { + this.factoryRetention = factoryRetention; + } + + /** + * Return the global retention configuration as found in the metadata. If the metadata is null, then it falls back + * to the factory retention. Returns null if both the cluster metadata for global retention and the factory retention + * are null. + */ + @Nullable + public DataStreamGlobalRetention resolve(ClusterState clusterState) { + DataStreamGlobalRetention globalRetentionFromClusterState = DataStreamGlobalRetention.getFromClusterState(clusterState); + if (globalRetentionFromClusterState != null || factoryRetention.isDefined() == false) { + return globalRetentionFromClusterState; + } + return new DataStreamGlobalRetention(factoryRetention.getDefaultRetention(), factoryRetention.getMaxRetention()); + } +} diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index f8a1f06ec985..5f0227535675 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -40,6 +40,8 @@ import org.elasticsearch.cluster.coordination.Coordinator; import org.elasticsearch.cluster.coordination.MasterHistoryService; import org.elasticsearch.cluster.coordination.StableMasterHealthIndicatorService; +import org.elasticsearch.cluster.metadata.DataStreamFactoryRetention; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.IndexMetadataVerifier; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetadataCreateDataStreamService; @@ -774,6 +776,11 @@ private void construct( threadPool ); + DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver = new DataStreamGlobalRetentionResolver( + DataStreamFactoryRetention.load(pluginsService, clusterService.getClusterSettings()) + ); + modules.bindToInstance(DataStreamGlobalRetentionResolver.class, dataStreamGlobalRetentionResolver); + record PluginServiceInstances( Client client, ClusterService clusterService, @@ -791,7 +798,8 @@ record PluginServiceInstances( AllocationService allocationService, IndicesService indicesService, FeatureService featureService, - SystemIndices systemIndices + SystemIndices systemIndices, + DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver ) implements Plugin.PluginServices {} PluginServiceInstances pluginServices = new PluginServiceInstances( client, @@ -810,7 +818,8 @@ record PluginServiceInstances( clusterModule.getAllocationService(), indicesService, featureService, - systemIndices + systemIndices, + dataStreamGlobalRetentionResolver ); Collection pluginComponents = pluginsService.flatMap(p -> p.createComponents(pluginServices)).toList(); diff --git a/server/src/main/java/org/elasticsearch/plugins/Plugin.java b/server/src/main/java/org/elasticsearch/plugins/Plugin.java index 12ad05e2bc71..71365f14853a 100644 --- a/server/src/main/java/org/elasticsearch/plugins/Plugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/Plugin.java @@ -10,6 +10,7 @@ import org.elasticsearch.bootstrap.BootstrapCheck; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexTemplateMetadata; import org.elasticsearch.cluster.routing.RerouteService; @@ -155,6 +156,12 @@ public interface PluginServices { * The system indices for the cluster */ SystemIndices systemIndices(); + + /** + * A service that resolves the data stream global retention that applies to + * data streams managed by the data stream lifecycle. + */ + DataStreamGlobalRetentionResolver dataStreamGlobalRetentionResolver(); } /** diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolverTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolverTests.java new file mode 100644 index 000000000000..5d5317b61a8f --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/DataStreamGlobalRetentionResolverTests.java @@ -0,0 +1,77 @@ +/* + * 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.cluster.metadata; + +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.ESTestCase; + +import java.util.Map; + +import static org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionTests.randomGlobalRetention; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class DataStreamGlobalRetentionResolverTests extends ESTestCase { + + public void testOnlyGlobalRetentionMetadata() { + DataStreamFactoryRetention factoryRetention = randomBoolean() + ? DataStreamFactoryRetention.emptyFactoryRetention() + : randomNonEmptyFactoryRetention(); + DataStreamGlobalRetentionResolver resolver = new DataStreamGlobalRetentionResolver(factoryRetention); + DataStreamGlobalRetention expectedGlobalRetention = randomGlobalRetention(); + DataStreamGlobalRetention globalRetention = resolver.resolve( + ClusterState.builder(ClusterName.DEFAULT).customs(Map.of(DataStreamGlobalRetention.TYPE, expectedGlobalRetention)).build() + ); + assertThat(globalRetention, notNullValue()); + assertThat(globalRetention.getDefaultRetention(), equalTo(expectedGlobalRetention.getDefaultRetention())); + assertThat(globalRetention.getMaxRetention(), equalTo(expectedGlobalRetention.getMaxRetention())); + } + + public void testOnlyFactoryRetentionFallback() { + DataStreamFactoryRetention factoryRetention = randomNonEmptyFactoryRetention(); + DataStreamGlobalRetentionResolver resolver = new DataStreamGlobalRetentionResolver(factoryRetention); + DataStreamGlobalRetention globalRetention = resolver.resolve(ClusterState.EMPTY_STATE); + assertThat(globalRetention, notNullValue()); + assertThat(globalRetention.getDefaultRetention(), equalTo(factoryRetention.getDefaultRetention())); + assertThat(globalRetention.getMaxRetention(), equalTo(factoryRetention.getMaxRetention())); + } + + private static DataStreamFactoryRetention randomNonEmptyFactoryRetention() { + boolean withDefault = randomBoolean(); + TimeValue defaultRetention = withDefault ? TimeValue.timeValueDays(randomIntBetween(10, 20)) : null; + TimeValue maxRetention = withDefault && randomBoolean() ? null : TimeValue.timeValueDays(randomIntBetween(50, 200)); + return new DataStreamFactoryRetention() { + @Override + public TimeValue getMaxRetention() { + return maxRetention; + } + + @Override + public TimeValue getDefaultRetention() { + return defaultRetention; + } + + @Override + public void init(ClusterSettings clusterSettings) { + + } + }; + } + + public void testNoRetentionConfiguration() { + DataStreamGlobalRetentionResolver resolver = new DataStreamGlobalRetentionResolver( + DataStreamFactoryRetention.emptyFactoryRetention() + ); + assertThat(resolver.resolve(ClusterState.EMPTY_STATE), nullValue()); + } +} From 9ae0b0ce921ab59a96fa2c19e1d4b0f242fbaa6b Mon Sep 17 00:00:00 2001 From: Kostas Krikellas <131142368+kkrik-es@users.noreply.github.com> Date: Wed, 24 Apr 2024 11:54:50 +0300 Subject: [PATCH 15/27] [TEST] Update version skip for `fields with ignore_malformed` (#107811) Support for boolean field mappers was added in #94121 Fixes #107810 --- .../resources/rest-api-spec/test/get/100_synthetic_source.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/get/100_synthetic_source.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/get/100_synthetic_source.yml index e8b4556b2998..fc8df138f94a 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/get/100_synthetic_source.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/get/100_synthetic_source.yml @@ -716,8 +716,8 @@ _doc_count: --- fields with ignore_malformed: - requires: - cluster_features: ["gte_v8.6.0"] - reason: introduced in 8.6.0 + cluster_features: ["gte_v8.8.0"] + reason: support for boolean ignore_malformed was added in 8.8.0 - do: indices.create: From cea9f6ab81d86dc4d357d1571bf82e7feb469f10 Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Wed, 24 Apr 2024 11:04:30 +0200 Subject: [PATCH 16/27] Log when no free regions left (#107763) This change logs when loading is skipped as there no free regions left. --- .../elasticsearch/blobcache/shared/SharedBlobCacheService.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java index e41bd1543d7a..bd67e71eac04 100644 --- a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java @@ -551,6 +551,7 @@ public void maybeFetchRegion( ) { if (freeRegionCount() < 1 && maybeEvictLeastUsed() == false) { // no free page available and no old enough unused region to be evicted + logger.info("No free regions, skipping loading region [{}]", region); listener.onResponse(false); return; } From 7937a428f3a62739e03c6944288a52bcc3a1d902 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Wed, 24 Apr 2024 11:13:11 +0200 Subject: [PATCH 17/27] Mute Test simulate (#107815) (#107822) Related: #107815 --- .../resources/rest-api-spec/test/ml/inference_processor.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/inference_processor.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/inference_processor.yml index c04955c666ea..3e0aa531b245 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/inference_processor.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/inference_processor.yml @@ -49,6 +49,9 @@ setup: --- "Test simulate": + - skip: + version: all + reason: "AwaitsFix https://github.com/elastic/elasticsearch/issues/107815" - do: ingest.simulate: body: > From f65fd7d9a01d8dfbb29b8e2807af73c5ea0b5ae6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 24 Apr 2024 10:48:18 +0100 Subject: [PATCH 18/27] Remove unused `AcknowledgedRequest#timeout` methods (#107761) Relates #107652 --- .../support/master/AcknowledgedRequest.java | 39 +------------------ 1 file changed, 2 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequest.java b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequest.java index 5813741bba25..ff39f1e83ba5 100644 --- a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequest.java +++ b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedRequest.java @@ -19,8 +19,8 @@ import static org.elasticsearch.core.TimeValue.timeValueSeconds; /** - * Abstract class that allows to mark action requests that support acknowledgements. - * Facilitates consistency across different api. + * Abstract base class for action requests that track acknowledgements of cluster state updates: such a request is acknowledged only once + * the cluster state update is committed and all relevant nodes have applied it and acknowledged its application to the elected master.. */ public abstract class AcknowledgedRequest> extends MasterNodeRequest implements @@ -53,19 +53,6 @@ protected AcknowledgedRequest(StreamInput in) throws IOException { this.ackTimeout = in.readTimeValue(); } - /** - * Sets the {@link #ackTimeout}, which specifies how long to wait for all relevant nodes to apply a cluster state update and acknowledge - * this to the elected master. - * - * @param timeout timeout as a string - * @return this request, for method chaining. - * @deprecated use {@link #ackTimeout()} instead. - */ - @Deprecated(forRemoval = true) - public final Request timeout(String timeout) { - return ackTimeout(timeout); - } - /** * Sets the {@link #ackTimeout}, which specifies how long to wait for all relevant nodes to apply a cluster state update and acknowledge * this to the elected master. @@ -77,19 +64,6 @@ public final Request ackTimeout(String ackTimeout) { return ackTimeout(TimeValue.parseTimeValue(ackTimeout, this.ackTimeout, getClass().getSimpleName() + ".ackTimeout")); } - /** - * Sets the {@link #ackTimeout}, which specifies how long to wait for all relevant nodes to apply a cluster state update and acknowledge - * this to the elected master. - * - * @param timeout timeout as a {@link TimeValue} - * @return this request, for method chaining. - * @deprecated use {@link #ackTimeout()} instead. - */ - @Deprecated(forRemoval = true) - public final Request timeout(TimeValue timeout) { - return ackTimeout(timeout); - } - /** * Sets the {@link #ackTimeout}, which specifies how long to wait for all relevant nodes to apply a cluster state update and acknowledge * this to the elected master. @@ -103,15 +77,6 @@ public final Request ackTimeout(TimeValue ackTimeout) { return (Request) this; } - /** - * @return the current ack timeout as a {@link TimeValue} - * @deprecated use {@link #ackTimeout()} instead. - */ - @Deprecated(forRemoval = true) - public final TimeValue timeout() { - return ackTimeout(); - } - /** * @return the current ack timeout as a {@link TimeValue} */ From d029d40cea1f9fe7eea27e79d77225733a043b4e Mon Sep 17 00:00:00 2001 From: Panagiotis Bailis Date: Wed, 24 Apr 2024 14:58:16 +0300 Subject: [PATCH 19/27] Adding new RankContext classes per different search phase/node type (#107093) --- server/src/main/java/module-info.java | 1 + .../search/QueryPhaseResultConsumer.java | 12 ++-- .../action/search/SearchPhaseController.java | 21 ++++--- .../search/DefaultSearchContext.java | 12 ++-- .../elasticsearch/search/SearchService.java | 2 +- .../internal/FilteredSearchContext.java | 10 +-- .../search/internal/SearchContext.java | 6 +- .../search/query/QueryPhase.java | 18 ++++-- .../search/rank/RankBuilder.java | 15 ++--- .../search/rank/RankCoordinatorContext.java | 42 ------------- .../elasticsearch/search/rank/RankDoc.java | 2 + .../search/rank/RankSearchContext.java | 9 ++- .../QueryPhaseRankCoordinatorContext.java | 38 ++++++++++++ .../QueryPhaseRankShardContext.java} | 19 +++--- .../search/SearchPhaseControllerTests.java | 21 +++---- .../search/query/QueryPhaseTests.java | 6 +- .../search/rank/TestRankBuilder.java | 6 +- .../elasticsearch/test/TestSearchContext.java | 12 ++-- ... RRFQueryPhaseRankCoordinatorContext.java} | 26 ++++---- ...ava => RRFQueryPhaseRankShardContext.java} | 14 +++-- .../xpack/rank/rrf/RRFRankBuilder.java | 12 ++-- .../xpack/rank/rrf/RRFRankBuilderTests.java | 10 +-- .../xpack/rank/rrf/RRFRankContextTests.java | 62 +++++++++---------- 23 files changed, 196 insertions(+), 180 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/search/rank/RankCoordinatorContext.java create mode 100644 server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankCoordinatorContext.java rename server/src/main/java/org/elasticsearch/search/rank/{RankShardContext.java => context/QueryPhaseRankShardContext.java} (65%) rename x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/{RRFRankCoordinatorContext.java => RRFQueryPhaseRankCoordinatorContext.java} (90%) rename x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/{RRFRankShardContext.java => RRFQueryPhaseRankShardContext.java} (89%) diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index ebc50349e7fe..7560e2ff1a7d 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -360,6 +360,7 @@ exports org.elasticsearch.search.profile.query; exports org.elasticsearch.search.query; exports org.elasticsearch.search.rank; + exports org.elasticsearch.search.rank.context; exports org.elasticsearch.search.rescore; exports org.elasticsearch.search.retriever; exports org.elasticsearch.search.runtime; diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index f5f64229f929..055274069c9c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -26,7 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; import java.util.ArrayDeque; import java.util.ArrayList; @@ -58,7 +58,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults= 0 : "num reduce phases must be >= 0 but was: " + numReducePhases; @@ -633,10 +633,15 @@ static ReducedQueryPhase reducedQueryPhase( final SearchProfileResultsBuilder profileBuilder = profileShardResults.isEmpty() ? null : new SearchProfileResultsBuilder(profileShardResults); - final SortedTopDocs sortedTopDocs = rankCoordinatorContext == null - ? sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions) - : rankCoordinatorContext.rank(queryResults.stream().map(SearchPhaseResult::queryResult).toList(), topDocsStats); - if (rankCoordinatorContext != null) { + final SortedTopDocs sortedTopDocs; + if (queryPhaseRankCoordinatorContext == null) { + sortedTopDocs = sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions); + } else { + ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults( + queryResults.stream().map(SearchPhaseResult::queryResult).toList(), + topDocsStats + ); + sortedTopDocs = new SortedTopDocs(rankedDocs, false, null, null, null, 0); size = sortedTopDocs.scoreDocs.length; } final TotalHits totalHits = topDocsStats.getTotalHits(); @@ -651,7 +656,7 @@ static ReducedQueryPhase reducedQueryPhase( profileBuilder, sortedTopDocs, sortValueFormats, - rankCoordinatorContext, + queryPhaseRankCoordinatorContext, numReducePhases, size, from, @@ -737,7 +742,7 @@ public record ReducedQueryPhase( // sort value formats used to sort / format the result DocValueFormat[] sortValueFormats, // the rank context if ranking is used - RankCoordinatorContext rankCoordinatorContext, + QueryPhaseRankCoordinatorContext rankCoordinatorContext, // the number of reduces phases int numReducePhases, // the size of the top hits to return diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 49ab7bf74ca9..f4067b7eb756 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -65,7 +65,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.slice.SliceBuilder; import org.elasticsearch.search.sort.SortAndFormats; @@ -124,7 +124,7 @@ final class DefaultSearchContext extends SearchContext { // filter for sliced scroll private SliceBuilder sliceBuilder; private SearchShardTask task; - private RankShardContext rankShardContext; + private QueryPhaseRankShardContext queryPhaseRankShardContext; /** * The original query as sent by the user without the types and aliases @@ -512,13 +512,13 @@ public void suggest(SuggestionSearchContext suggest) { } @Override - public RankShardContext rankShardContext() { - return rankShardContext; + public QueryPhaseRankShardContext queryPhaseRankShardContext() { + return queryPhaseRankShardContext; } @Override - public void rankShardContext(RankShardContext rankShardContext) { - this.rankShardContext = rankShardContext; + public void queryPhaseRankShardContext(QueryPhaseRankShardContext queryPhaseRankShardContext) { + this.queryPhaseRankShardContext = queryPhaseRankShardContext; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 5cbb97976dbc..753b6f8cb710 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -1426,7 +1426,7 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc for (SubSearchSourceBuilder subSearchSourceBuilder : source.subSearches()) { queries.add(subSearchSourceBuilder.toSearchQuery(context.getSearchExecutionContext())); } - context.rankShardContext(source.rankBuilder().buildRankShardContext(queries, context.from())); + context.queryPhaseRankShardContext(source.rankBuilder().buildQueryPhaseShardContext(queries, context.from())); } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 7e54eeacffd7..d5c3c00c00ce 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -34,7 +34,7 @@ import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -140,13 +140,13 @@ public SuggestionSearchContext suggest() { } @Override - public RankShardContext rankShardContext() { - return in.rankShardContext(); + public QueryPhaseRankShardContext queryPhaseRankShardContext() { + return in.queryPhaseRankShardContext(); } @Override - public void rankShardContext(RankShardContext rankShardContext) { - in.rankShardContext(rankShardContext); + public void queryPhaseRankShardContext(QueryPhaseRankShardContext queryPhaseRankShardContext) { + in.queryPhaseRankShardContext(queryPhaseRankShardContext); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 9580c450fd92..232c12e944a9 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -41,7 +41,7 @@ import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -137,9 +137,9 @@ public InnerHitsContext innerHits() { public abstract SuggestionSearchContext suggest(); - public abstract RankShardContext rankShardContext(); + public abstract QueryPhaseRankShardContext queryPhaseRankShardContext(); - public abstract void rankShardContext(RankShardContext rankShardContext); + public abstract void queryPhaseRankShardContext(QueryPhaseRankShardContext queryPhaseRankShardContext); /** * @return list of all rescore contexts. empty if there aren't any. diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index 29cf80b75a22..cd8b5494ac31 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -37,7 +37,7 @@ import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.rank.RankSearchContext; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescorePhase; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestPhase; @@ -59,7 +59,7 @@ public class QueryPhase { private QueryPhase() {} public static void execute(SearchContext searchContext) throws QueryPhaseExecutionException { - if (searchContext.rankShardContext() == null) { + if (searchContext.queryPhaseRankShardContext() == null) { executeQuery(searchContext); } else { executeRank(searchContext); @@ -67,7 +67,7 @@ public static void execute(SearchContext searchContext) throws QueryPhaseExecuti } static void executeRank(SearchContext searchContext) throws QueryPhaseExecutionException { - RankShardContext rankShardContext = searchContext.rankShardContext(); + QueryPhaseRankShardContext queryPhaseRankShardContext = searchContext.queryPhaseRankShardContext(); QuerySearchResult querySearchResult = searchContext.queryResult(); // run the combined boolean query total hits or aggregations @@ -89,12 +89,18 @@ static void executeRank(SearchContext searchContext) throws QueryPhaseExecutionE int nodeQueueSize = querySearchResult.nodeQueueSize(); // run each of the rank queries - for (Query rankQuery : rankShardContext.queries()) { + for (Query rankQuery : queryPhaseRankShardContext.queries()) { // if a search timeout occurs, exit with partial results if (searchTimedOut) { break; } - try (RankSearchContext rankSearchContext = new RankSearchContext(searchContext, rankQuery, rankShardContext.windowSize())) { + try ( + RankSearchContext rankSearchContext = new RankSearchContext( + searchContext, + rankQuery, + queryPhaseRankShardContext.windowSize() + ) + ) { QueryPhase.addCollectorsAndSearch(rankSearchContext); QuerySearchResult rrfQuerySearchResult = rankSearchContext.queryResult(); rrfRankResults.add(rrfQuerySearchResult.topDocs().topDocs); @@ -104,7 +110,7 @@ static void executeRank(SearchContext searchContext) throws QueryPhaseExecutionE } } - querySearchResult.setRankShardResult(rankShardContext.combine(rrfRankResults)); + querySearchResult.setRankShardResult(queryPhaseRankShardContext.combineQueryPhaseResults(rrfRankResults)); // record values relevant to all queries querySearchResult.searchTimedOut(searchTimedOut); diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankBuilder.java b/server/src/main/java/org/elasticsearch/search/rank/RankBuilder.java index a58137eac393..e0e04c563a9a 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankBuilder.java @@ -14,6 +14,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.VersionedNamedWriteable; import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; @@ -23,9 +25,8 @@ import java.util.Objects; /** - * {@code RankContextBuilder} is used as a base class to manage input, parsing, - * and subsequent generation of appropriate contexts for handling searches that - * require multiple queries for global rank relevance. + * {@code RankBuilder} is used as a base class to manage input, parsing, and subsequent generation of appropriate contexts + * for handling searches that require multiple queries and/or ranking steps for global rank relevance. */ public abstract class RankBuilder implements VersionedNamedWriteable, ToXContentObject { @@ -68,14 +69,14 @@ public int windowSize() { } /** - * Generates a context used to execute required searches on the shard. + * Generates a context used to execute required searches during the query phase on the shard. */ - public abstract RankShardContext buildRankShardContext(List queries, int from); + public abstract QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from); /** - * Generates a context used to perform global ranking on the coordinator. + * Generates a context used to be executed on the coordinating node, that would combine all individual shard results. */ - public abstract RankCoordinatorContext buildRankCoordinatorContext(int size, int from); + public abstract QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from); @Override public final boolean equals(Object obj) { diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankCoordinatorContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankCoordinatorContext.java deleted file mode 100644 index 87f2f0b404ea..000000000000 --- a/server/src/main/java/org/elasticsearch/search/rank/RankCoordinatorContext.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 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.search.rank; - -import org.elasticsearch.action.search.SearchPhaseController.SortedTopDocs; -import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; -import org.elasticsearch.search.query.QuerySearchResult; - -import java.util.List; - -/** - * {@code RankContext} is a base class used to generate ranking - * results on the coordinator and then set the rank for any - * search hits that are found. - */ -public abstract class RankCoordinatorContext { - - protected final int size; - protected final int from; - protected final int windowSize; - - public RankCoordinatorContext(int size, int from, int windowSize) { - this.size = size; - this.from = from; - this.windowSize = windowSize; - } - - /** - * This is used to pull information passed back from the shards as part - * of {@link QuerySearchResult#getRankShardResult()} and return a {@link SortedTopDocs} - * of the final rank results. Note that {@link TopDocsStats} is included so that - * appropriate stats may be updated based on rank results. This is called at the end - * of the query phase prior to the fetch phase. - */ - public abstract SortedTopDocs rank(List querySearchResults, TopDocsStats topDocStats); -} diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankDoc.java b/server/src/main/java/org/elasticsearch/search/rank/RankDoc.java index 21d53a172c2c..bd177008dd90 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankDoc.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankDoc.java @@ -31,6 +31,8 @@ public abstract class RankDoc extends ScoreDoc implements Writeable { */ public int rank = NO_RANK; + public record RankKey(int doc, int shardIndex) {} + public RankDoc(int doc, float score, int shardIndex) { super(doc, score, shardIndex); } diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java index d144e45becc1..1cb5843dfc7d 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java @@ -42,6 +42,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -282,13 +283,17 @@ public SuggestionSearchContext suggest() { throw new UnsupportedOperationException(); } + public void suggest(SuggestionSearchContext suggest) { + throw new UnsupportedOperationException(); + } + @Override - public RankShardContext rankShardContext() { + public QueryPhaseRankShardContext queryPhaseRankShardContext() { throw new UnsupportedOperationException(); } @Override - public void rankShardContext(RankShardContext rankShardContext) { + public void queryPhaseRankShardContext(QueryPhaseRankShardContext queryPhaseRankShardContext) { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankCoordinatorContext.java b/server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankCoordinatorContext.java new file mode 100644 index 000000000000..181122380f22 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankCoordinatorContext.java @@ -0,0 +1,38 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.search.rank.context; + +import org.apache.lucene.search.ScoreDoc; +import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; +import org.elasticsearch.search.query.QuerySearchResult; + +import java.util.List; + +/** + * {@link QueryPhaseRankCoordinatorContext} is running on the coordinator node and is + * responsible for combining the query phase results from the shards and rank them accordingly. + * The output is a `window_size` ranked list of ordered results from all shards. + * Note: Currently this can use only sort by score; sort by field is not supported. + */ +public abstract class QueryPhaseRankCoordinatorContext { + + protected final int windowSize; + + public QueryPhaseRankCoordinatorContext(int windowSize) { + this.windowSize = windowSize; + } + + /** + * This is used to pull information passed back from the shards as part of {@link QuerySearchResult#getRankShardResult()} + * and return a {@link ScoreDoc[]} of the `window_size` ranked results. Note that {@link TopDocsStats} is included so that + * appropriate stats may be updated based on rank results. + * This is called when reducing query results through {@code SearchPhaseController#reducedQueryPhase()}. + */ + public abstract ScoreDoc[] rankQueryPhaseResults(List querySearchResults, TopDocsStats topDocStats); +} diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankShardContext.java b/server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankShardContext.java similarity index 65% rename from server/src/main/java/org/elasticsearch/search/rank/RankShardContext.java rename to server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankShardContext.java index f8c60b350972..e8bac25009e8 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankShardContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/context/QueryPhaseRankShardContext.java @@ -6,27 +6,26 @@ * Side Public License, v 1. */ -package org.elasticsearch.search.rank; +package org.elasticsearch.search.rank.context; import org.apache.lucene.search.Query; import org.apache.lucene.search.TopDocs; +import org.elasticsearch.search.rank.RankShardResult; import java.util.List; /** - * {@code RankShardContext} is a base class used to generate ranking - * results on each shard where it's responsible for executing any - * queries during the query phase required for its global ranking method. + * {@link QueryPhaseRankShardContext} is used to generate the top {@code window_size} + * results on each shard. It specifies the queries to run during {@code QueryPhase} and is responsible for combining all query scores and + * order all results through the {@link QueryPhaseRankShardContext#combineQueryPhaseResults} method. */ -public abstract class RankShardContext { +public abstract class QueryPhaseRankShardContext { protected final List queries; - protected final int from; protected final int windowSize; - public RankShardContext(List queries, int from, int windowSize) { + public QueryPhaseRankShardContext(List queries, int windowSize) { this.queries = queries; - this.from = from; this.windowSize = windowSize; } @@ -40,10 +39,10 @@ public int windowSize() { /** * This is used to reduce the number of required results that are serialized - * to the coordinating node. Normally we would have to serialize {@code (queries + knns)*window_size} + * to the coordinating node. Normally we would have to serialize {@code queries * window_size} * results, but we can infer that there will likely be overlap of document results. Given that we * know any searches that match the same document must be on the same shard, we can sort on the shard * instead for a top window_size set of results and reduce the amount of data we serialize. */ - public abstract RankShardResult combine(List rankResults); + public abstract RankShardResult combineQueryPhaseResults(List rankResults); } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index 1f81ad2a02e8..43bca4bae2f3 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -56,11 +56,11 @@ import org.elasticsearch.search.profile.SearchProfileQueryPhaseResult; import org.elasticsearch.search.profile.aggregation.AggregationProfileShardResult; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankCoordinatorContext; import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.search.rank.RankShardResult; import org.elasticsearch.search.rank.TestRankDoc; import org.elasticsearch.search.rank.TestRankShardResult; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; import org.elasticsearch.search.suggest.SortBy; import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.completion.CompletionSuggestion; @@ -356,8 +356,10 @@ public void testMerge() { } public void testMergeWithRank() { - int nShards = randomIntBetween(1, 20); - int queryResultSize = randomBoolean() ? 0 : randomIntBetween(1, nShards * 2); + final int nShards = randomIntBetween(1, 20); + final int fsize = randomIntBetween(1, 10); + final int windowSize = randomIntBetween(11, 100); + final int queryResultSize = randomBoolean() ? 0 : randomIntBetween(1, nShards * 2); for (int trackTotalHits : new int[] { SearchContext.TRACK_TOTAL_HITS_DISABLED, SearchContext.TRACK_TOTAL_HITS_ACCURATE }) { AtomicArray queryResults = generateQueryResults(nShards, List.of(), queryResultSize, false, false, true); try { @@ -369,13 +371,10 @@ public void testMergeWithRank() { 0, true, InternalAggregationTestCase.emptyReduceContextBuilder(), - new RankCoordinatorContext(randomIntBetween(1, 10), 0, randomIntBetween(11, 100)) { + new QueryPhaseRankCoordinatorContext(windowSize) { @Override - public SearchPhaseController.SortedTopDocs rank( - List querySearchResults, - TopDocsStats topDocStats - ) { - PriorityQueue queue = new PriorityQueue(windowSize) { + public ScoreDoc[] rankQueryPhaseResults(List querySearchResults, TopDocsStats topDocStats) { + PriorityQueue queue = new PriorityQueue<>(windowSize) { @Override protected boolean lessThan(RankDoc a, RankDoc b) { return a.score < b.score; @@ -389,14 +388,14 @@ protected boolean lessThan(RankDoc a, RankDoc b) { } } } - int size = Math.min(this.size, queue.size()); + int size = Math.min(fsize, queue.size()); RankDoc[] topResults = new RankDoc[size]; for (int rdi = 0; rdi < size; ++rdi) { topResults[rdi] = queue.pop(); topResults[rdi].rank = rdi + 1; } topDocStats.fetchHits = topResults.length; - return new SearchPhaseController.SortedTopDocs(topResults, false, null, null, null, 0); + return topResults; } }, true diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index ffa54184d652..03d525ae2333 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -78,8 +78,8 @@ import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; -import org.elasticsearch.search.rank.RankShardContext; import org.elasticsearch.search.rank.RankShardResult; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.tasks.TaskCancelHelper; import org.elasticsearch.tasks.TaskCancelledException; @@ -1056,9 +1056,9 @@ public ReaderContext readerContext() { context.parsedQuery( new ParsedQuery(new BooleanQuery.Builder().add(queries.get(0), Occur.SHOULD).add(queries.get(1), Occur.SHOULD).build()) ); - context.rankShardContext(new RankShardContext(queries, 0, 100) { + context.queryPhaseRankShardContext(new QueryPhaseRankShardContext(queries, 0) { @Override - public RankShardResult combine(List rankResults) { + public RankShardResult combineQueryPhaseResults(List rankResults) { return null; } }); diff --git a/test/framework/src/main/java/org/elasticsearch/search/rank/TestRankBuilder.java b/test/framework/src/main/java/org/elasticsearch/search/rank/TestRankBuilder.java index 71a943413ba2..691d54191371 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/rank/TestRankBuilder.java +++ b/test/framework/src/main/java/org/elasticsearch/search/rank/TestRankBuilder.java @@ -13,6 +13,8 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentParser; @@ -73,12 +75,12 @@ protected void doXContent(XContentBuilder builder, Params params) throws IOExcep } @Override - public RankShardContext buildRankShardContext(List queries, int from) { + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { throw new UnsupportedOperationException(); } @Override - public RankCoordinatorContext buildRankCoordinatorContext(int size, int from) { + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { throw new UnsupportedOperationException(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index 8db968b59ae1..cba2b41d279b 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -43,7 +43,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.profile.Profilers; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; @@ -69,7 +69,7 @@ public class TestSearchContext extends SearchContext { SortAndFormats sort; boolean trackScores = false; int trackTotalHitsUpTo = SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; - RankShardContext rankShardContext; + QueryPhaseRankShardContext queryPhaseRankShardContext; ContextIndexSearcher searcher; int from; int size; @@ -509,13 +509,13 @@ public boolean isCancelled() { } @Override - public RankShardContext rankShardContext() { - return rankShardContext; + public QueryPhaseRankShardContext queryPhaseRankShardContext() { + return queryPhaseRankShardContext; } @Override - public void rankShardContext(RankShardContext rankShardContext) { - this.rankShardContext = rankShardContext; + public void queryPhaseRankShardContext(QueryPhaseRankShardContext queryPhaseRankContext) { + this.queryPhaseRankShardContext = queryPhaseRankContext; } @Override diff --git a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankCoordinatorContext.java b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankCoordinatorContext.java similarity index 90% rename from x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankCoordinatorContext.java rename to x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankCoordinatorContext.java index 50f3646264a9..1b3ebe19ce49 100644 --- a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankCoordinatorContext.java +++ b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankCoordinatorContext.java @@ -7,12 +7,13 @@ package org.elasticsearch.xpack.rank.rrf; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.util.PriorityQueue; -import org.elasticsearch.action.search.SearchPhaseController.SortedTopDocs; import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; import org.elasticsearch.common.util.Maps; import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankCoordinatorContext; +import org.elasticsearch.search.rank.RankDoc.RankKey; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; import java.util.ArrayList; import java.util.Arrays; @@ -24,22 +25,21 @@ /** * Ranks and decorates search hits for RRF results on the coordinator. */ -public class RRFRankCoordinatorContext extends RankCoordinatorContext { +public class RRFQueryPhaseRankCoordinatorContext extends QueryPhaseRankCoordinatorContext { + private final int size; + private final int from; private final int rankConstant; - public RRFRankCoordinatorContext(int size, int from, int windowSize, int rankConstant) { - super(size, from, windowSize); + public RRFQueryPhaseRankCoordinatorContext(int size, int from, int windowSize, int rankConstant) { + super(windowSize); + this.size = size; + this.from = from; this.rankConstant = rankConstant; } - // used for faster hash lookup in a map of ranked documents - protected record RankKey(int doc, int shardIndex) { - - } - @Override - public SortedTopDocs rank(List querySearchResults, TopDocsStats topDocsStats) { + public ScoreDoc[] rankQueryPhaseResults(List querySearchResults, TopDocsStats topDocsStats) { // for each shard we check to see if it timed out to skip // if it didn't time out then we need to split the results into // a priority queue per query, so we can do global ranking @@ -96,7 +96,7 @@ protected boolean lessThan(RRFRankDoc a, RRFRankDoc b) { // return early if we have no valid results if (queues.isEmpty()) { - return SortedTopDocs.EMPTY; + return new ScoreDoc[0]; } // rank the global doc sets using RRF from the previously @@ -163,6 +163,6 @@ protected boolean lessThan(RRFRankDoc a, RRFRankDoc b) { // return the top results where sort, collapse fields, // and completion suggesters are not allowed - return new SortedTopDocs(topResults, false, null, null, null, 0); + return topResults; } } diff --git a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardContext.java b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankShardContext.java similarity index 89% rename from x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardContext.java rename to x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankShardContext.java index e22e32819370..59307e62872f 100644 --- a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankShardContext.java +++ b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFQueryPhaseRankShardContext.java @@ -11,7 +11,7 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.elasticsearch.common.util.Maps; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import java.util.Arrays; import java.util.List; @@ -22,17 +22,19 @@ /** * Executes queries and generates results on the shard for RRF. */ -public class RRFRankShardContext extends RankShardContext { +public class RRFQueryPhaseRankShardContext extends QueryPhaseRankShardContext { - protected final int rankConstant; + private final int rankConstant; + private final int from; - public RRFRankShardContext(List queries, int from, int windowSize, int rankConstant) { - super(queries, from, windowSize); + public RRFQueryPhaseRankShardContext(List queries, int from, int windowSize, int rankConstant) { + super(queries, windowSize); + this.from = from; this.rankConstant = rankConstant; } @Override - public RRFRankShardResult combine(List rankResults) { + public RRFRankShardResult combineQueryPhaseResults(List rankResults) { // combine the disjointed sets of TopDocs into a single set or RRFRankDocs // each RRFRankDoc will have both the position and score for each query where // it was within the result set for that query diff --git a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilder.java b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilder.java index f0a8e299de63..fee3c7b5e9cf 100644 --- a/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilder.java +++ b/x-pack/plugin/rank-rrf/src/main/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilder.java @@ -14,8 +14,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.search.rank.RankBuilder; -import org.elasticsearch.search.rank.RankCoordinatorContext; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; @@ -95,13 +95,13 @@ public int rankConstant() { } @Override - public RankShardContext buildRankShardContext(List queries, int from) { - return new RRFRankShardContext(queries, from, windowSize(), rankConstant); + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new RRFQueryPhaseRankShardContext(queries, from, windowSize(), rankConstant); } @Override - public RankCoordinatorContext buildRankCoordinatorContext(int size, int from) { - return new RRFRankCoordinatorContext(size, from, windowSize(), rankConstant); + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new RRFQueryPhaseRankCoordinatorContext(size, from, windowSize(), rankConstant); } @Override diff --git a/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilderTests.java b/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilderTests.java index 29c2f2cbebb9..001857385c5d 100644 --- a/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilderTests.java +++ b/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankBuilderTests.java @@ -11,7 +11,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.search.rank.RankShardContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; import org.elasticsearch.test.AbstractXContentSerializingTestCase; import org.elasticsearch.xcontent.XContentParser; import org.junit.Assert; @@ -59,10 +59,10 @@ public void testCreateRankContexts() { RRFRankBuilder rrfRankBuilder = createTestInstance(); List queries = List.of(new TermQuery(new Term("field0", "test0")), new TermQuery(new Term("field1", "test1"))); - RankShardContext rsc = rrfRankBuilder.buildRankShardContext(queries, randomInt()); - assertEquals(queries, rsc.queries()); - assertEquals(rrfRankBuilder.windowSize(), rsc.windowSize()); + QueryPhaseRankShardContext queryPhaseRankShardContext = rrfRankBuilder.buildQueryPhaseShardContext(queries, randomInt()); + assertEquals(queries, queryPhaseRankShardContext.queries()); + assertEquals(rrfRankBuilder.windowSize(), queryPhaseRankShardContext.windowSize()); - assertNotNull(rrfRankBuilder.buildRankCoordinatorContext(randomInt(), randomInt())); + assertNotNull(rrfRankBuilder.buildQueryPhaseCoordinatorContext(randomInt(), randomInt())); } } diff --git a/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankContextTests.java b/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankContextTests.java index 5cb89c071c76..50aa1d257d21 100644 --- a/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankContextTests.java +++ b/x-pack/plugin/rank-rrf/src/test/java/org/elasticsearch/xpack/rank/rrf/RRFRankContextTests.java @@ -9,7 +9,6 @@ import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; -import org.elasticsearch.action.search.SearchPhaseController.SortedTopDocs; import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.test.ESTestCase; @@ -36,8 +35,7 @@ private void assertRDEquals(RRFRankDoc rd0, RRFRankDoc rd1) { } public void testShardCombine() { - RRFRankShardContext context = new RRFRankShardContext(null, 0, 10, 1); - + RRFQueryPhaseRankShardContext context = new RRFQueryPhaseRankShardContext(null, 0, 10, 1); List topDocs = List.of( new TopDocs( null, @@ -69,7 +67,7 @@ public void testShardCombine() { ) ); - RRFRankShardResult result = context.combine(topDocs); + RRFRankShardResult result = context.combineQueryPhaseResults(topDocs); assertEquals(2, result.queryCount); assertEquals(10, result.rrfRankDocs.length); @@ -165,7 +163,7 @@ public void testShardCombine() { } public void testCoordinatorRank() { - RRFRankCoordinatorContext context = new RRFRankCoordinatorContext(4, 0, 5, 1); + RRFQueryPhaseRankCoordinatorContext context = new RRFQueryPhaseRankCoordinatorContext(4, 0, 5, 1); QuerySearchResult qsr0 = new QuerySearchResult(); qsr0.setShardIndex(1); RRFRankDoc rd11 = new RRFRankDoc(1, -1, 2); @@ -225,10 +223,10 @@ public void testCoordinatorRank() { qsr1.setRankShardResult(new RRFRankShardResult(2, new RRFRankDoc[] { rd21, rd22, rd23, rd24, rd25 })); TopDocsStats tds = new TopDocsStats(0); - SortedTopDocs std = context.rank(List.of(qsr0, qsr1), tds); + ScoreDoc[] scoreDocs = context.rankQueryPhaseResults(List.of(qsr0, qsr1), tds); assertEquals(4, tds.fetchHits); - assertEquals(4, std.scoreDocs().length); + assertEquals(4, scoreDocs.length); RRFRankDoc expected = new RRFRankDoc(1, 2, 2); expected.rank = 1; @@ -237,7 +235,7 @@ public void testCoordinatorRank() { expected.scores[0] = 9.5f; expected.scores[1] = 7.5f; expected.score = 0.6666667f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[0]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[0]); expected = new RRFRankDoc(3, 1, 2); expected.rank = 2; @@ -246,7 +244,7 @@ public void testCoordinatorRank() { expected.scores[0] = 10.0f; expected.scores[1] = 0.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[1]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[1]); expected = new RRFRankDoc(1, 1, 2); expected.rank = 3; @@ -255,7 +253,7 @@ public void testCoordinatorRank() { expected.scores[0] = 0.0f; expected.scores[1] = 8.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[2]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[2]); expected = new RRFRankDoc(2, 2, 2); expected.rank = 4; @@ -264,18 +262,18 @@ public void testCoordinatorRank() { expected.scores[0] = 8.5f; expected.scores[1] = 6.5f; expected.score = 0.4f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[3]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[3]); } public void testShardTieBreaker() { - RRFRankShardContext context = new RRFRankShardContext(null, 0, 10, 1); + RRFQueryPhaseRankShardContext context = new RRFQueryPhaseRankShardContext(null, 0, 10, 1); List topDocs = List.of( new TopDocs(null, new ScoreDoc[] { new ScoreDoc(1, 10.0f, -1), new ScoreDoc(2, 9.0f, -1) }), new TopDocs(null, new ScoreDoc[] { new ScoreDoc(2, 8.0f, -1), new ScoreDoc(1, 7.0f, -1) }) ); - RRFRankShardResult result = context.combine(topDocs); + RRFRankShardResult result = context.combineQueryPhaseResults(topDocs); assertEquals(2, result.queryCount); assertEquals(2, result.rrfRankDocs.length); @@ -302,7 +300,7 @@ public void testShardTieBreaker() { new TopDocs(null, new ScoreDoc[] { new ScoreDoc(4, 11.0f, -1), new ScoreDoc(3, 9.0f, -1), new ScoreDoc(2, 7.0f, -1) }) ); - result = context.combine(topDocs); + result = context.combineQueryPhaseResults(topDocs); assertEquals(2, result.queryCount); assertEquals(4, result.rrfRankDocs.length); @@ -347,7 +345,7 @@ public void testShardTieBreaker() { new TopDocs(null, new ScoreDoc[] { new ScoreDoc(2, 8.0f, -1), new ScoreDoc(4, 5.0f, -1) }) ); - result = context.combine(topDocs); + result = context.combineQueryPhaseResults(topDocs); assertEquals(2, result.queryCount); assertEquals(4, result.rrfRankDocs.length); @@ -389,7 +387,7 @@ public void testShardTieBreaker() { } public void testCoordinatorRankTieBreaker() { - RRFRankCoordinatorContext context = new RRFRankCoordinatorContext(4, 0, 5, 1); + RRFQueryPhaseRankCoordinatorContext context = new RRFQueryPhaseRankCoordinatorContext(4, 0, 5, 1); QuerySearchResult qsr0 = new QuerySearchResult(); qsr0.setShardIndex(1); @@ -410,10 +408,10 @@ public void testCoordinatorRankTieBreaker() { qsr1.setRankShardResult(new RRFRankShardResult(2, new RRFRankDoc[] { rd21 })); TopDocsStats tds = new TopDocsStats(0); - SortedTopDocs std = context.rank(List.of(qsr0, qsr1), tds); + ScoreDoc[] scoreDocs = context.rankQueryPhaseResults(List.of(qsr0, qsr1), tds); assertEquals(2, tds.fetchHits); - assertEquals(2, std.scoreDocs().length); + assertEquals(2, scoreDocs.length); RRFRankDoc expected = new RRFRankDoc(1, 1, 2); expected.rank = 1; @@ -422,7 +420,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 10.0f; expected.scores[1] = 7.0f; expected.score = 0.8333333730697632f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[0]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[0]); expected = new RRFRankDoc(1, 2, 2); expected.rank = 2; @@ -431,7 +429,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 9.0f; expected.scores[1] = 8.0f; expected.score = 0.8333333730697632f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[1]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[1]); qsr0 = new QuerySearchResult(); qsr0.setShardIndex(1); @@ -462,10 +460,10 @@ public void testCoordinatorRankTieBreaker() { qsr1.setRankShardResult(new RRFRankShardResult(2, new RRFRankDoc[] { rd21, rd22 })); tds = new TopDocsStats(0); - std = context.rank(List.of(qsr0, qsr1), tds); + scoreDocs = context.rankQueryPhaseResults(List.of(qsr0, qsr1), tds); assertEquals(4, tds.fetchHits); - assertEquals(4, std.scoreDocs().length); + assertEquals(4, scoreDocs.length); expected = new RRFRankDoc(2, 2, 2); expected.rank = 1; @@ -474,7 +472,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 9.0f; expected.scores[1] = 9.0f; expected.score = 0.5833333730697632f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[0]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[0]); expected = new RRFRankDoc(2, 1, 2); expected.rank = 2; @@ -483,7 +481,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 9.0f; expected.scores[1] = 7.0f; expected.score = 0.5833333730697632f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[1]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[1]); expected = new RRFRankDoc(1, 1, 2); expected.rank = 3; @@ -492,7 +490,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 10.0f; expected.scores[1] = 0.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[2]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[2]); expected = new RRFRankDoc(1, 2, 2); expected.rank = 4; @@ -501,7 +499,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 0.0f; expected.scores[1] = 11.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[3]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[3]); qsr0 = new QuerySearchResult(); qsr0.setShardIndex(1); @@ -532,10 +530,10 @@ public void testCoordinatorRankTieBreaker() { qsr1.setRankShardResult(new RRFRankShardResult(2, new RRFRankDoc[] { rd21, rd22 })); tds = new TopDocsStats(0); - std = context.rank(List.of(qsr0, qsr1), tds); + scoreDocs = context.rankQueryPhaseResults(List.of(qsr0, qsr1), tds); assertEquals(4, tds.fetchHits); - assertEquals(4, std.scoreDocs().length); + assertEquals(4, scoreDocs.length); expected = new RRFRankDoc(1, 1, 2); expected.rank = 1; @@ -544,7 +542,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 10.0f; expected.scores[1] = 0.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[0]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[0]); expected = new RRFRankDoc(2, 1, 2); expected.rank = 2; @@ -553,7 +551,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 0.0f; expected.scores[1] = 12.0f; expected.score = 0.5f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[1]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[1]); expected = new RRFRankDoc(1, 2, 2); expected.rank = 3; @@ -562,7 +560,7 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 3.0f; expected.scores[1] = 0.0f; expected.score = 0.3333333333333333f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[2]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[2]); expected = new RRFRankDoc(2, 2, 2); expected.rank = 4; @@ -571,6 +569,6 @@ public void testCoordinatorRankTieBreaker() { expected.scores[0] = 0.0f; expected.scores[1] = 5.0f; expected.score = 0.3333333333333333f; - assertRDEquals(expected, (RRFRankDoc) std.scoreDocs()[3]); + assertRDEquals(expected, (RRFRankDoc) scoreDocs[3]); } } From 180ae4e4c27ba0c1c62057c9a6db77517c5bce00 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Przemys=C5=82aw=20Witek?= Date: Wed, 24 Apr 2024 14:06:25 +0200 Subject: [PATCH 20/27] Use wait_for_status option in frequent_item_sets_agg tests (#107783) --- .../test/ml/frequent_item_sets_agg.yml | 54 ++++++++----------- 1 file changed, 23 insertions(+), 31 deletions(-) diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/frequent_item_sets_agg.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/frequent_item_sets_agg.yml index ab06e266f1ac..a240c58d9388 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/frequent_item_sets_agg.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/ml/frequent_item_sets_agg.yml @@ -7,9 +7,6 @@ setup: indices.create: index: store body: - settings: - number_of_shards: 1 - number_of_replicas: 0 mappings: properties: features: @@ -58,9 +55,6 @@ setup: indices.create: index: store-flattened body: - settings: - number_of_shards: 1 - number_of_replicas: 0 mappings: properties: data: @@ -96,7 +90,30 @@ setup: { "data": {"features": ["rwd", "turbocharger", "autopilot", "pink wheels"], "error_message": "engine overheated"}, "timestamp": 1022137244000 } { "index": {} } { "data": {"features": ["fwd", "tow-bar", "turbocharger", "pink wheels", "electric mirror", "heated steering wheel", "lane assist", "sunroof", "autopilot", "air condition" ], "error_message": "low tire pressure"}, "timestamp": 1020920854000 } + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.create: + index: unavailable-data + body: + mappings: + properties: + features: + type: keyword + error_message: + type: keyword + timestamp: + type: date + geo_point: + type: geo_point + histogram: + type: histogram + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + cluster.health: + wait_for_status: yellow --- "Test frequent item sets array fields": @@ -550,31 +567,6 @@ setup: --- "Test frequent items on empty index": - - skip: - features: headers - - - do: - headers: - Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser - indices.create: - index: unavailable-data - body: - settings: - number_of_shards: 1 - number_of_replicas: 0 - mappings: - properties: - features: - type: keyword - error_message: - type: keyword - timestamp: - type: date - geo_point: - type: geo_point - histogram: - type: histogram - - do: search: index: unavailable-data From 52af16adb13e14ea6f943b82fd3a87505231f8a5 Mon Sep 17 00:00:00 2001 From: Jedr Blaszyk Date: Wed, 24 Apr 2024 14:25:03 +0200 Subject: [PATCH 21/27] [ent-searc] Reorganize yamlTests into product specific folders (#107688) --- .../rest-api-spec/test/entsearch/{100_usage.yml => 20_usage.yml} | 0 .../10_behavioral_analytics_list.yml} | 0 .../20_behavioral_analytics_put.yml} | 0 .../30_behavioral_analytics_delete.yml} | 0 .../40_behavioral_analytics_event_post.yml} | 0 .../100_connector_update_error.yml} | 0 .../{300_connector_put.yml => connector/10_connector_put.yml} | 0 .../110_connector_update_name.yml} | 0 .../120_connector_update_service_type.yml} | 0 .../130_connector_update_index_name.yml} | 0 .../140_connector_update_native.yml} | 0 .../150_connector_update_status.yml} | 0 .../{305_connector_post.yml => connector/15_connector_post.yml} | 0 .../160_connector_update_api_key_id.yml} | 0 .../{310_connector_list.yml => connector/20_connector_list.yml} | 0 .../30_connector_delete.yml} | 0 .../40_connector_update_pipeline.yml} | 0 .../50_connector_update_scheduling.yml} | 0 .../60_connector_update_filtering.yml} | 0 .../70_connector_check_in.yml} | 0 .../80_connector_update_last_sync_stats.yml} | 0 .../90_connector_update_configuration.yml} | 0 .../secret/10_connector_secret_post.yml} | 0 .../secret/20_connector_secret_put.yml} | 0 .../secret/30_connector_secret_get.yml} | 0 .../secret/40_connector_secret_delete.yml} | 0 .../sync_job/10_connector_sync_job_post.yml} | 0 .../sync_job/20_connector_sync_job_delete.yml} | 0 .../sync_job/30_connector_sync_job_check_in.yml} | 0 .../sync_job/40_connector_sync_job_cancel.yml} | 0 .../sync_job/50_connector_sync_job_get.yml} | 0 .../sync_job/60_connector_sync_job_error.yml} | 0 .../sync_job/70_connector_sync_job_update_stats.yml} | 0 .../sync_job/80_connector_sync_job_list.yml} | 0 .../{200_query_ruleset_put.yml => rules/10_query_ruleset_put.yml} | 0 .../20_query_ruleset_list.yml} | 0 .../30_query_ruleset_delete.yml} | 0 .../{260_rule_query_search.yml => rules/40_rule_query_search.yml} | 0 .../5_query_rulesets_before_setup.yml} | 0 .../entsearch/{ => search}/15_search_application_before_setup.yml | 0 .../test/entsearch/{ => search}/20_search_application_put.yml | 0 .../test/entsearch/{ => search}/30_search_application_get.yml | 0 .../test/entsearch/{ => search}/40_search_application_delete.yml | 0 .../test/entsearch/{ => search}/50_search_application_list.yml | 0 .../entsearch/{ => search}/52_search_application_render_query.yml | 0 .../test/entsearch/{ => search}/55_search_application_search.yml | 0 .../{ => search}/56_search_application_search_with_apikey.yml | 0 47 files changed, 0 insertions(+), 0 deletions(-) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{100_usage.yml => 20_usage.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{60_behavioral_analytics_list.yml => analytics/10_behavioral_analytics_list.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{70_behavioral_analytics_put.yml => analytics/20_behavioral_analytics_put.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{80_behavioral_analytics_delete.yml => analytics/30_behavioral_analytics_delete.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{90_behavioral_analytics_event_post.yml => analytics/40_behavioral_analytics_event_post.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{335_connector_update_error.yml => connector/100_connector_update_error.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{300_connector_put.yml => connector/10_connector_put.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{336_connector_update_name.yml => connector/110_connector_update_name.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{337_connector_update_service_type.yml => connector/120_connector_update_service_type.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{338_connector_update_index_name.yml => connector/130_connector_update_index_name.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{339_connector_update_native.yml => connector/140_connector_update_native.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{340_connector_update_status.yml => connector/150_connector_update_status.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{305_connector_post.yml => connector/15_connector_post.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{341_connector_update_api_key_id.yml => connector/160_connector_update_api_key_id.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{310_connector_list.yml => connector/20_connector_list.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{320_connector_delete.yml => connector/30_connector_delete.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{330_connector_update_pipeline.yml => connector/40_connector_update_pipeline.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{331_connector_update_scheduling.yml => connector/50_connector_update_scheduling.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{332_connector_update_filtering.yml => connector/60_connector_update_filtering.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{333_connector_check_in.yml => connector/70_connector_check_in.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{334_connector_update_last_sync_stats.yml => connector/80_connector_update_last_sync_stats.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{335_connector_update_configuration.yml => connector/90_connector_update_configuration.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{500_connector_secret_post.yml => connector/secret/10_connector_secret_post.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{505_connector_secret_put.yml => connector/secret/20_connector_secret_put.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{510_connector_secret_get.yml => connector/secret/30_connector_secret_get.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{520_connector_secret_delete.yml => connector/secret/40_connector_secret_delete.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{400_connector_sync_job_post.yml => connector/sync_job/10_connector_sync_job_post.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{410_connector_sync_job_delete.yml => connector/sync_job/20_connector_sync_job_delete.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{420_connector_sync_job_check_in.yml => connector/sync_job/30_connector_sync_job_check_in.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{430_connector_sync_job_cancel.yml => connector/sync_job/40_connector_sync_job_cancel.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{440_connector_sync_job_get.yml => connector/sync_job/50_connector_sync_job_get.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{450_connector_sync_job_error.yml => connector/sync_job/60_connector_sync_job_error.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{460_connector_sync_job_update_stats.yml => connector/sync_job/70_connector_sync_job_update_stats.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{470_connector_sync_job_list.yml => connector/sync_job/80_connector_sync_job_list.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{200_query_ruleset_put.yml => rules/10_query_ruleset_put.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{225_query_ruleset_list.yml => rules/20_query_ruleset_list.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{250_query_ruleset_delete.yml => rules/30_query_ruleset_delete.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{260_rule_query_search.yml => rules/40_rule_query_search.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{199_query_rulesets_before_setup.yml => rules/5_query_rulesets_before_setup.yml} (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/15_search_application_before_setup.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/20_search_application_put.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/30_search_application_get.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/40_search_application_delete.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/50_search_application_list.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/52_search_application_render_query.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/55_search_application_search.yml (100%) rename x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/{ => search}/56_search_application_search_with_apikey.yml (100%) diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/100_usage.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/20_usage.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/100_usage.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/20_usage.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/60_behavioral_analytics_list.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/10_behavioral_analytics_list.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/60_behavioral_analytics_list.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/10_behavioral_analytics_list.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/70_behavioral_analytics_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/20_behavioral_analytics_put.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/70_behavioral_analytics_put.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/20_behavioral_analytics_put.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/80_behavioral_analytics_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/30_behavioral_analytics_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/80_behavioral_analytics_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/30_behavioral_analytics_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/90_behavioral_analytics_event_post.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/40_behavioral_analytics_event_post.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/90_behavioral_analytics_event_post.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/analytics/40_behavioral_analytics_event_post.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/335_connector_update_error.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/100_connector_update_error.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/335_connector_update_error.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/100_connector_update_error.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/300_connector_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/10_connector_put.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/300_connector_put.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/10_connector_put.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/336_connector_update_name.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/110_connector_update_name.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/336_connector_update_name.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/110_connector_update_name.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/337_connector_update_service_type.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/120_connector_update_service_type.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/337_connector_update_service_type.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/120_connector_update_service_type.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/338_connector_update_index_name.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/130_connector_update_index_name.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/338_connector_update_index_name.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/130_connector_update_index_name.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/339_connector_update_native.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/140_connector_update_native.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/339_connector_update_native.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/140_connector_update_native.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/340_connector_update_status.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/150_connector_update_status.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/340_connector_update_status.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/150_connector_update_status.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/305_connector_post.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/15_connector_post.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/305_connector_post.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/15_connector_post.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/341_connector_update_api_key_id.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/160_connector_update_api_key_id.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/341_connector_update_api_key_id.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/160_connector_update_api_key_id.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/310_connector_list.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/20_connector_list.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/310_connector_list.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/20_connector_list.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/320_connector_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/30_connector_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/320_connector_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/30_connector_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/330_connector_update_pipeline.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/40_connector_update_pipeline.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/330_connector_update_pipeline.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/40_connector_update_pipeline.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/331_connector_update_scheduling.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/50_connector_update_scheduling.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/331_connector_update_scheduling.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/50_connector_update_scheduling.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/332_connector_update_filtering.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/60_connector_update_filtering.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/332_connector_update_filtering.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/60_connector_update_filtering.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/333_connector_check_in.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/70_connector_check_in.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/333_connector_check_in.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/70_connector_check_in.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/334_connector_update_last_sync_stats.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/80_connector_update_last_sync_stats.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/334_connector_update_last_sync_stats.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/80_connector_update_last_sync_stats.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/335_connector_update_configuration.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/90_connector_update_configuration.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/335_connector_update_configuration.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/90_connector_update_configuration.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/500_connector_secret_post.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/10_connector_secret_post.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/500_connector_secret_post.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/10_connector_secret_post.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/505_connector_secret_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/20_connector_secret_put.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/505_connector_secret_put.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/20_connector_secret_put.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/510_connector_secret_get.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/30_connector_secret_get.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/510_connector_secret_get.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/30_connector_secret_get.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/520_connector_secret_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/40_connector_secret_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/520_connector_secret_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/secret/40_connector_secret_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/400_connector_sync_job_post.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/10_connector_sync_job_post.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/400_connector_sync_job_post.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/10_connector_sync_job_post.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/410_connector_sync_job_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/20_connector_sync_job_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/410_connector_sync_job_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/20_connector_sync_job_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/420_connector_sync_job_check_in.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/30_connector_sync_job_check_in.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/420_connector_sync_job_check_in.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/30_connector_sync_job_check_in.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/430_connector_sync_job_cancel.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/40_connector_sync_job_cancel.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/430_connector_sync_job_cancel.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/40_connector_sync_job_cancel.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/440_connector_sync_job_get.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/50_connector_sync_job_get.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/440_connector_sync_job_get.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/50_connector_sync_job_get.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/450_connector_sync_job_error.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/60_connector_sync_job_error.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/450_connector_sync_job_error.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/60_connector_sync_job_error.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/460_connector_sync_job_update_stats.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/70_connector_sync_job_update_stats.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/460_connector_sync_job_update_stats.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/70_connector_sync_job_update_stats.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/470_connector_sync_job_list.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/80_connector_sync_job_list.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/470_connector_sync_job_list.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/connector/sync_job/80_connector_sync_job_list.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/200_query_ruleset_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/10_query_ruleset_put.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/200_query_ruleset_put.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/10_query_ruleset_put.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/225_query_ruleset_list.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/20_query_ruleset_list.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/225_query_ruleset_list.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/20_query_ruleset_list.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/250_query_ruleset_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/30_query_ruleset_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/250_query_ruleset_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/30_query_ruleset_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/260_rule_query_search.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/40_rule_query_search.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/260_rule_query_search.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/40_rule_query_search.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/199_query_rulesets_before_setup.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/5_query_rulesets_before_setup.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/199_query_rulesets_before_setup.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/5_query_rulesets_before_setup.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/15_search_application_before_setup.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/15_search_application_before_setup.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/15_search_application_before_setup.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/15_search_application_before_setup.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/20_search_application_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/20_search_application_put.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/20_search_application_put.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/20_search_application_put.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/30_search_application_get.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/30_search_application_get.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/30_search_application_get.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/30_search_application_get.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/40_search_application_delete.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/40_search_application_delete.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/40_search_application_delete.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/40_search_application_delete.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/50_search_application_list.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/50_search_application_list.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/50_search_application_list.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/50_search_application_list.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/52_search_application_render_query.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/52_search_application_render_query.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/52_search_application_render_query.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/52_search_application_render_query.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/55_search_application_search.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/55_search_application_search.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/55_search_application_search.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/55_search_application_search.yml diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/56_search_application_search_with_apikey.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/56_search_application_search_with_apikey.yml similarity index 100% rename from x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/56_search_application_search_with_apikey.yml rename to x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/search/56_search_application_search_with_apikey.yml From 0f68c673f7bd4ddbbc3a0561c4205447b5c5267b Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 24 Apr 2024 08:30:36 -0400 Subject: [PATCH 22/27] ESQL: Add `BlockHash#lookup` (#107762) Adds a `lookup` method to `BlockHash` which finds keys that were already in the hash without modifying it and returns the "ordinal" that the `BlockHash` produced when that key had been called with `add`. For multi-column keys this can change the number of values pretty drastically. You get a combinatorial explosion of values. So if you have three columns with 2 values each the most values you can get is 2*2*2=8. If you have five columns with ten values each you can have 100,000 values in a single position! That's too many. Let's do an example! This one has a two row block containing three colunms. One row has two values in each column so it could produce at most 8 values. In this case one of the values is missing from the hash, so it only produces 7. Block: | a | b | c | | ----:| ----:| ----:| | 1 | 4 | 6 | | 1, 2 | 3, 4 | 5, 6 | BlockHash contents: | a | b | c | | -:| -:| -:| | 1 | 3 | 5 | | 1 | 3 | 6 | | 1 | 4 | 5 | | 1 | 4 | 6 | | 2 | 3 | 5 | | 2 | 3 | 6 | | 2 | 4 | 6 | Results: | ord | | -------------------:| | 3 | | 0, 1, 2, 3, 4, 5, 6 | The `add` method has a fairly fool-proof mechanism to work around this, it calls it's consumers with a callback that can split positions into multiple calls. It calls the callback in batches of like 16,000 positions at a time. And aggs uses the callback. So you can aggregate over five colunms with ten values each. It's slow, but the callbacks let us get through it. Unlike `add`, `lookup` can't use a callback. We're going to need it to return `Iterator` of `IntBlock`s containing ordinals. That's just how we're going to use it. That'd be ok, but we can't split a single position across multiple `Block`s. That's just not how `Block` works. So, instead, we fail the query if we produce more than 100,000 entries in a single position. We'd like to stop collecting and emit a warning, but that's a problem for another change. That's a single 400kb array which is quite big. Anyway! If we're not bumping into massive rows we emit `IntBlocks` targeting a particular size in memory. Likely we'll also want to plug in a target number of rows as well, but for now this'll do. --- .../core/ReleasableIterator.java | 49 +++ .../blockhash/BytesRefBlockHash.java | 38 +++ .../blockhash/DoubleBlockHash.java | 38 +++ .../aggregation/blockhash/IntBlockHash.java | 38 +++ .../aggregation/blockhash/LongBlockHash.java | 38 +++ .../elasticsearch/compute/data/IntBlock.java | 7 + .../compute/data/IntBlockBuilder.java | 5 + .../mvdedupe/MultivalueDedupeBytesRef.java | 6 + .../mvdedupe/MultivalueDedupeDouble.java | 5 + .../mvdedupe/MultivalueDedupeInt.java | 5 + .../mvdedupe/MultivalueDedupeLong.java | 5 + .../blockhash/AbstractAddBlock.java | 2 +- .../aggregation/blockhash/BlockHash.java | 16 + .../blockhash/BooleanBlockHash.java | 36 +++ .../blockhash/BytesRefLongBlockHash.java | 7 + .../blockhash/LongLongBlockHash.java | 7 + .../aggregation/blockhash/NullBlockHash.java | 12 + .../blockhash/PackedValuesBlockHash.java | 281 +++++++++++++----- .../blockhash/TimeSeriesBlockHash.java | 8 + .../aggregation/blockhash/X-BlockHash.java.st | 48 ++- .../compute/data/X-Block.java.st | 9 + .../compute/data/X-BlockBuilder.java.st | 6 + .../operator/mvdedupe/BatchEncoder.java | 9 +- .../mvdedupe/MultivalueDedupeBoolean.java | 6 + .../mvdedupe/X-MultivalueDedupe.java.st | 6 + .../blockhash/BlockHashRandomizedTests.java | 146 +++++++++ .../aggregation/blockhash/BlockHashTests.java | 76 +++-- 27 files changed, 808 insertions(+), 101 deletions(-) create mode 100644 libs/core/src/main/java/org/elasticsearch/core/ReleasableIterator.java diff --git a/libs/core/src/main/java/org/elasticsearch/core/ReleasableIterator.java b/libs/core/src/main/java/org/elasticsearch/core/ReleasableIterator.java new file mode 100644 index 000000000000..68a4a136c530 --- /dev/null +++ b/libs/core/src/main/java/org/elasticsearch/core/ReleasableIterator.java @@ -0,0 +1,49 @@ +/* + * 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.core; + +import java.util.Iterator; +import java.util.Objects; + +/** + * An {@link Iterator} with state that must be {@link #close() released}. + */ +public interface ReleasableIterator extends Releasable, Iterator { + /** + * Returns a single element iterator over the supplied value. + */ + static ReleasableIterator single(T element) { + return new ReleasableIterator<>() { + private T value = Objects.requireNonNull(element); + + @Override + public boolean hasNext() { + return value != null; + } + + @Override + public T next() { + final T res = value; + value = null; + return res; + } + + @Override + public void close() { + Releasables.close(value); + } + + @Override + public String toString() { + return "ReleasableIterator[" + value + "]"; + } + + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java index 1b4e0d8a8e4c..4c413ad54f2f 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java @@ -25,6 +25,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeBytesRef; +import org.elasticsearch.core.ReleasableIterator; import java.io.IOException; @@ -91,6 +92,43 @@ IntBlock add(BytesRefBlock block) { return result.ords(); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + + BytesRefBlock castBlock = (BytesRefBlock) block; + BytesRefVector vector = castBlock.asVector(); + // TODO honor targetBlockSize and chunk the pages if requested. + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup(BytesRefVector vector) { + BytesRef scratch = new BytesRef(); + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { + BytesRef v = vector.getBytesRef(i, scratch); + long found = hash.find(v); + if (found < 0) { + builder.appendNull(); + } else { + builder.appendInt(Math.toIntExact(hashOrdToGroupNullReserved(found))); + } + } + return builder.build(); + } + } + + private IntBlock lookup(BytesRefBlock block) { + return new MultivalueDedupeBytesRef(block).hashLookup(blockFactory, hash); + } + @Override public BytesRefBlock[] getKeys() { /* diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java index 857ab64bd915..bd9d752302ae 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.LongHash; @@ -21,6 +22,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeDouble; +import org.elasticsearch.core.ReleasableIterator; import java.util.BitSet; @@ -86,6 +88,42 @@ IntBlock add(DoubleBlock block) { return result.ords(); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + + DoubleBlock castBlock = (DoubleBlock) block; + DoubleVector vector = castBlock.asVector(); + // TODO honor targetBlockSize and chunk the pages if requested. + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup(DoubleVector vector) { + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { + long v = Double.doubleToLongBits(vector.getDouble(i)); + long found = hash.find(v); + if (found < 0) { + builder.appendNull(); + } else { + builder.appendInt(Math.toIntExact(hashOrdToGroupNullReserved(found))); + } + } + return builder.build(); + } + } + + private IntBlock lookup(DoubleBlock block) { + return new MultivalueDedupeDouble(block).hashLookup(blockFactory, hash); + } + @Override public DoubleBlock[] getKeys() { if (seenNull) { diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java index 2c4f9a1bb229..5b1b48bd270a 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.LongHash; @@ -19,6 +20,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeInt; +import org.elasticsearch.core.ReleasableIterator; import java.util.BitSet; @@ -84,6 +86,42 @@ IntBlock add(IntBlock block) { return result.ords(); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + + IntBlock castBlock = (IntBlock) block; + IntVector vector = castBlock.asVector(); + // TODO honor targetBlockSize and chunk the pages if requested. + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup(IntVector vector) { + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { + int v = vector.getInt(i); + long found = hash.find(v); + if (found < 0) { + builder.appendNull(); + } else { + builder.appendInt(Math.toIntExact(hashOrdToGroupNullReserved(found))); + } + } + return builder.build(); + } + } + + private IntBlock lookup(IntBlock block) { + return new MultivalueDedupeInt(block).hashLookup(blockFactory, hash); + } + @Override public IntBlock[] getKeys() { if (seenNull) { diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java index f5893eb977b4..074ccb2f7cd7 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.LongHash; @@ -21,6 +22,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeLong; +import org.elasticsearch.core.ReleasableIterator; import java.util.BitSet; @@ -86,6 +88,42 @@ IntBlock add(LongBlock block) { return result.ords(); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + + LongBlock castBlock = (LongBlock) block; + LongVector vector = castBlock.asVector(); + // TODO honor targetBlockSize and chunk the pages if requested. + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup(LongVector vector) { + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { + long v = vector.getLong(i); + long found = hash.find(v); + if (found < 0) { + builder.appendNull(); + } else { + builder.appendInt(Math.toIntExact(hashOrdToGroupNullReserved(found))); + } + } + return builder.build(); + } + } + + private IntBlock lookup(LongBlock block) { + return new MultivalueDedupeLong(block).hashLookup(blockFactory, hash); + } + @Override public LongBlock[] getKeys() { if (seenNull) { diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java index e9d606b51c6a..2747862d534b 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java @@ -223,6 +223,13 @@ sealed interface Builder extends Block.Builder, BlockLoader.IntBuilder permits I @Override Builder mvOrdering(Block.MvOrdering mvOrdering); + /** + * An estimate of the number of bytes the {@link IntBlock} created by + * {@link #build} will use. This may overestimate the size but shouldn't + * underestimate it. + */ + long estimatedBytes(); + @Override IntBlock build(); } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java index 85f943004de2..886bf98f4e04 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java @@ -182,4 +182,9 @@ public IntBlock build() { throw e; } } + + @Override + public long estimatedBytes() { + return estimatedBytes; + } } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBytesRef.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBytesRef.java index f4e966367ed6..c9043344c6aa 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBytesRef.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBytesRef.java @@ -250,6 +250,7 @@ public IntBlock hashLookup(BlockFactory blockFactory, BytesRefHash hash) { * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.BytesRefs(batchSize) { @Override protected void readNextBatch() { @@ -305,6 +306,11 @@ private int workSize() { } return size; } + + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeDouble.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeDouble.java index f2ed5fa8676d..4d383fe51cc7 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeDouble.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeDouble.java @@ -247,6 +247,7 @@ public IntBlock hashLookup(BlockFactory blockFactory, LongHash hash) { * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.Doubles(batchSize) { @Override protected void readNextBatch() { @@ -295,6 +296,10 @@ protected void readNextBatch() { } } + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeInt.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeInt.java index 82ed0dda927c..d60cdcdede17 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeInt.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeInt.java @@ -247,6 +247,7 @@ public IntBlock hashLookup(BlockFactory blockFactory, LongHash hash) { * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.Ints(batchSize) { @Override protected void readNextBatch() { @@ -295,6 +296,10 @@ protected void readNextBatch() { } } + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeLong.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeLong.java index 998d7300f0a9..00a608e9b68e 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeLong.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeLong.java @@ -248,6 +248,7 @@ public IntBlock hashLookup(BlockFactory blockFactory, LongHash hash) { * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.Longs(batchSize) { @Override protected void readNextBatch() { @@ -296,6 +297,10 @@ protected void readNextBatch() { } } + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractAddBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractAddBlock.java index defa7879479a..a5997bbb7f48 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractAddBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractAddBlock.java @@ -61,7 +61,7 @@ private void rollover(int position) { } @Override - public final void close() { + public void close() { ords.close(); } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java index 7c36cc6087fd..1e7ecebc16a6 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.BytesRefHash; @@ -17,10 +18,13 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.ReleasableIterator; +import java.util.Iterator; import java.util.List; /** @@ -46,6 +50,18 @@ public abstract sealed class BlockHash implements Releasable, SeenGroupIds // */ public abstract void add(Page page, GroupingAggregatorFunction.AddInput addInput); + /** + * Lookup all values for the "group by" columns in the page to the hash and return an + * {@link Iterator} of the values. The sum of {@link IntBlock#getPositionCount} for + * all blocks returned by the iterator will equal {@link Page#getPositionCount} but + * will "target" a size of {@code targetBlockSize}. + *

+ * The returned {@link ReleasableIterator} may retain a reference to {@link Block}s + * inside the {@link Page}. Close it to release those references. + *

+ */ + public abstract ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize); + /** * Returns a {@link Block} that contains all the keys that are inserted by {@link #add}. */ diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java index 5858e4e0b88c..17aa5afbe3ad 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; @@ -17,6 +18,7 @@ import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeBoolean; +import org.elasticsearch.core.ReleasableIterator; import static org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeBoolean.FALSE_ORD; import static org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeBoolean.NULL_ORD; @@ -72,6 +74,40 @@ private IntBlock add(BooleanBlock block) { return new MultivalueDedupeBoolean(block).hash(blockFactory, everSeen); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + BooleanBlock castBlock = page.getBlock(channel); + BooleanVector vector = castBlock.asVector(); + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup(BooleanVector vector) { + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { + boolean v = vector.getBoolean(i); + int ord = v ? TRUE_ORD : FALSE_ORD; + if (everSeen[ord]) { + builder.appendInt(ord); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + } + + private IntBlock lookup(BooleanBlock block) { + return new MultivalueDedupeBoolean(block).hash(blockFactory, new boolean[TRUE_ORD + 1]); + } + @Override public BooleanBlock[] getKeys() { try (BooleanBlock.Builder builder = blockFactory.newBooleanBlockBuilder(everSeen.length)) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java index 616b3be4bcee..a1414c57247c 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java @@ -8,6 +8,7 @@ package org.elasticsearch.compute.aggregation.blockhash; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.LongLongHash; @@ -23,6 +24,7 @@ import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.IntLongBlockAdd; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; /** @@ -104,6 +106,11 @@ public IntVector add(IntVector bytesHashes, LongVector longsVector) { return blockFactory.newIntArrayVector(ords, positions); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + throw new UnsupportedOperationException("TODO"); + } + @Override public Block[] getKeys() { int positions = (int) finalHash.size(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java index 4ec5581236c5..11423539db39 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.LongLongHash; @@ -20,6 +21,7 @@ import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.LongLongBlockAdd; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; /** @@ -71,6 +73,11 @@ IntVector add(LongVector vector1, LongVector vector2) { } } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + throw new UnsupportedOperationException("TODO"); + } + @Override public Block[] getKeys() { int positions = (int) hash.size(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/NullBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/NullBlockHash.java index 601d75d83200..e61d9640c64f 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/NullBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/NullBlockHash.java @@ -7,14 +7,17 @@ package org.elasticsearch.compute.aggregation.blockhash; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.ReleasableIterator; /** * Maps a {@link BooleanBlock} column to group ids. Assigns group @@ -42,6 +45,15 @@ public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { } } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + Block block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + throw new IllegalArgumentException("can't use NullBlockHash for non-null blocks"); + } + @Override public Block[] getKeys() { return new Block[] { blockFactory.newConstantNullBlock(seenNull ? 1 : 0) }; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java index e84acc26598b..85c535faf318 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java @@ -9,6 +9,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; @@ -18,10 +19,12 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.BatchEncoder; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; import java.util.Arrays; @@ -29,7 +32,7 @@ /** * Maps any number of columns to a group ids with every unique combination resulting - * in a unique group id. Works by uniqing the values of each column and concatenating + * in a unique group id. Works by unique-ing the values of each column and concatenating * the combinatorial explosion of all values into a byte array and then hashing each * byte array. If the values are *
{@code
@@ -48,9 +51,15 @@
  *     3, 2, 4
  *     3, 3, 5
  * }
+ *

+ * The iteration order in the above is how we do it - it's as though it's + * nested {@code for} loops with the first column being the outer-most loop + * and the last column being the inner-most loop. See {@link Group} for more. + *

*/ final class PackedValuesBlockHash extends BlockHash { static final int DEFAULT_BATCH_SIZE = Math.toIntExact(ByteSizeValue.ofKb(10).getBytes()); + private static final long MAX_LOOKUP = 100_000; private final int emitBatchSize; private final BytesRefHash bytesRefHash; @@ -64,6 +73,7 @@ final class PackedValuesBlockHash extends BlockHash { this.emitBatchSize = emitBatchSize; this.bytesRefHash = new BytesRefHash(1, blockFactory.bigArrays()); this.nullTrackingBytes = (groups.length + 7) / 8; + bytes.grow(nullTrackingBytes); } @Override @@ -77,12 +87,23 @@ void add(Page page, GroupingAggregatorFunction.AddInput addInput, int batchSize) } } + /** + * The on-heap representation of a {@code for} loop for each group key. + */ private static class Group { final GroupSpec spec; BatchEncoder encoder; int positionOffset; int valueOffset; - int loopedIndex; + /** + * The number of values we've written for this group. Think of it as + * the loop variable in a {@code for} loop. + */ + int writtenValues; + /** + * The number of values of this group at this position. Think of it as + * the maximum value in a {@code for} loop. + */ int valueCount; int bytesStart; @@ -97,10 +118,7 @@ class AddWork extends AbstractAddBlock { AddWork(Page page, GroupingAggregatorFunction.AddInput addInput, int batchSize) { super(blockFactory, emitBatchSize, addInput); - for (Group group : groups) { - group.encoder = MultivalueDedupe.batchEncoder(page.getBlock(group.spec.channel()), batchSize, true); - } - bytes.grow(nullTrackingBytes); + initializeGroupsForPage(page, batchSize); this.positionCount = page.getPositionCount(); } @@ -111,21 +129,7 @@ class AddWork extends AbstractAddBlock { */ void add() { for (position = 0; position < positionCount; position++) { - // Make sure all encoders have encoded the current position and the offsets are queued to it's start - boolean singleEntry = true; - for (Group g : groups) { - var encoder = g.encoder; - g.positionOffset++; - while (g.positionOffset >= encoder.positionCount()) { - encoder.encodeNextBatch(); - g.positionOffset = 0; - g.valueOffset = 0; - } - g.valueCount = encoder.valueCount(g.positionOffset); - singleEntry &= (g.valueCount == 1); - } - Arrays.fill(bytes.bytes(), 0, nullTrackingBytes, (byte) 0); - bytes.setLength(nullTrackingBytes); + boolean singleEntry = startPosition(); if (singleEntry) { addSingleEntry(); } else { @@ -136,57 +140,211 @@ void add() { } private void addSingleEntry() { - for (int g = 0; g < groups.length; g++) { - Group group = groups[g]; - if (group.encoder.read(group.valueOffset++, bytes) == 0) { - int nullByte = g / 8; - int nullShift = g % 8; - bytes.bytes()[nullByte] |= (byte) (1 << nullShift); - } - } - int ord = Math.toIntExact(hashOrdToGroup(bytesRefHash.add(bytes.get()))); - ords.appendInt(ord); + fillBytesSv(); + ords.appendInt(Math.toIntExact(hashOrdToGroup(bytesRefHash.add(bytes.get())))); addedValue(position); } private void addMultipleEntries() { ords.beginPositionEntry(); int g = 0; - outer: for (;;) { - for (; g < groups.length; g++) { - Group group = groups[g]; - group.bytesStart = bytes.length(); - if (group.encoder.read(group.valueOffset + group.loopedIndex, bytes) == 0) { - assert group.valueCount == 1 : "null value in non-singleton list"; - int nullByte = g / 8; - int nullShift = g % 8; - bytes.bytes()[nullByte] |= (byte) (1 << nullShift); - } - ++group.loopedIndex; - } + do { + fillBytesMv(g); + // emit ords - int ord = Math.toIntExact(hashOrdToGroup(bytesRefHash.add(bytes.get()))); - ords.appendInt(ord); + ords.appendInt(Math.toIntExact(hashOrdToGroup(bytesRefHash.add(bytes.get())))); addedValueInMultivaluePosition(position); - // rewind - Group group = groups[--g]; - bytes.setLength(group.bytesStart); - while (group.loopedIndex == group.valueCount) { - group.loopedIndex = 0; - if (g == 0) { - break outer; + g = rewindKeys(); + } while (g >= 0); + ords.endPositionEntry(); + for (Group group : groups) { + group.valueOffset += group.valueCount; + } + } + + @Override + public void close() { + Releasables.closeExpectNoException( + super::close, + Releasables.wrap(() -> Iterators.map(Iterators.forArray(groups), g -> g.encoder)) + ); + } + } + + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + return new LookupWork(page, targetBlockSize.getBytes(), DEFAULT_BATCH_SIZE); + } + + class LookupWork implements ReleasableIterator { + private final long targetBytesSize; + private final int positionCount; + private int position; + + LookupWork(Page page, long targetBytesSize, int batchSize) { + this.positionCount = page.getPositionCount(); + this.targetBytesSize = targetBytesSize; + initializeGroupsForPage(page, batchSize); + } + + @Override + public boolean hasNext() { + return position < positionCount; + } + + @Override + public IntBlock next() { + int size = Math.toIntExact(Math.min(Integer.MAX_VALUE, targetBytesSize / Integer.BYTES / 2)); + try (IntBlock.Builder ords = blockFactory.newIntBlockBuilder(size)) { + while (position < positionCount && ords.estimatedBytes() < targetBytesSize) { + boolean singleEntry = startPosition(); + if (singleEntry) { + lookupSingleEntry(ords); } else { - group = groups[--g]; - bytes.setLength(group.bytesStart); + lookupMultipleEntries(ords); } + position++; } + return ords.build(); + } + } + + private void lookupSingleEntry(IntBlock.Builder ords) { + fillBytesSv(); + long found = bytesRefHash.find(bytes.get()); + if (found < 0) { + ords.appendNull(); + } else { + ords.appendInt(Math.toIntExact(found)); + } + } + + private void lookupMultipleEntries(IntBlock.Builder ords) { + long firstFound = -1; + boolean began = false; + int g = 0; + int count = 0; + do { + fillBytesMv(g); + + // emit ords + long found = bytesRefHash.find(bytes.get()); + if (found >= 0) { + if (firstFound < 0) { + firstFound = found; + } else { + if (began == false) { + began = true; + ords.beginPositionEntry(); + ords.appendInt(Math.toIntExact(firstFound)); + count++; + } + ords.appendInt(Math.toIntExact(found)); + count++; + if (count > MAX_LOOKUP) { + // TODO replace this with a warning and break + throw new IllegalArgumentException("Found a single entry with " + count + " entries"); + } + } + } + g = rewindKeys(); + } while (g >= 0); + if (firstFound < 0) { + ords.appendNull(); + } else if (began) { + ords.endPositionEntry(); + } else { + // Only found one value + ords.appendInt(Math.toIntExact(hashOrdToGroup(firstFound))); } - ords.endPositionEntry(); for (Group group : groups) { group.valueOffset += group.valueCount; } } + + @Override + public void close() { + Releasables.closeExpectNoException(Releasables.wrap(() -> Iterators.map(Iterators.forArray(groups), g -> g.encoder))); + } + } + + private void initializeGroupsForPage(Page page, int batchSize) { + for (Group group : groups) { + Block b = page.getBlock(group.spec.channel()); + group.encoder = MultivalueDedupe.batchEncoder(b, batchSize, true); + } + } + + /** + * Correctly position all {@link #groups}, clear the {@link #bytes}, + * and position it past the null tracking bytes. Call this before + * encoding a new position. + * @return true if this position has only a single ordinal + */ + private boolean startPosition() { + boolean singleEntry = true; + for (Group g : groups) { + /* + * Make sure all encoders have encoded the current position and the + * offsets are queued to its start. + */ + var encoder = g.encoder; + g.positionOffset++; + while (g.positionOffset >= encoder.positionCount()) { + encoder.encodeNextBatch(); + g.positionOffset = 0; + g.valueOffset = 0; + } + g.valueCount = encoder.valueCount(g.positionOffset); + singleEntry &= (g.valueCount == 1); + } + Arrays.fill(bytes.bytes(), 0, nullTrackingBytes, (byte) 0); + bytes.setLength(nullTrackingBytes); + return singleEntry; + } + + private void fillBytesSv() { + for (int g = 0; g < groups.length; g++) { + Group group = groups[g]; + assert group.writtenValues == 0; + assert group.valueCount == 1; + if (group.encoder.read(group.valueOffset++, bytes) == 0) { + int nullByte = g / 8; + int nullShift = g % 8; + bytes.bytes()[nullByte] |= (byte) (1 << nullShift); + } + } + } + + private void fillBytesMv(int startingGroup) { + for (int g = startingGroup; g < groups.length; g++) { + Group group = groups[g]; + group.bytesStart = bytes.length(); + if (group.encoder.read(group.valueOffset + group.writtenValues, bytes) == 0) { + assert group.valueCount == 1 : "null value in non-singleton list"; + int nullByte = g / 8; + int nullShift = g % 8; + bytes.bytes()[nullByte] |= (byte) (1 << nullShift); + } + ++group.writtenValues; + } + } + + private int rewindKeys() { + int g = groups.length - 1; + Group group = groups[g]; + bytes.setLength(group.bytesStart); + while (group.writtenValues == group.valueCount) { + group.writtenValues = 0; + if (g == 0) { + return -1; + } else { + group = groups[--g]; + bytes.setLength(group.bytesStart); + } + } + return g; } @Override @@ -227,18 +385,7 @@ public Block[] getKeys() { if (offset > 0) { readKeys(decoders, builders, nulls, values, offset); } - - Block[] keyBlocks = new Block[groups.length]; - try { - for (int g = 0; g < keyBlocks.length; g++) { - keyBlocks[g] = builders[g].build(); - } - } finally { - if (keyBlocks[keyBlocks.length - 1] == null) { - Releasables.closeExpectNoException(keyBlocks); - } - } - return keyBlocks; + return Block.Builder.buildAll(builders); } finally { Releasables.closeExpectNoException(builders); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/TimeSeriesBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/TimeSeriesBlockHash.java index a3d2bcae73df..09b1022200b6 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/TimeSeriesBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/TimeSeriesBlockHash.java @@ -8,6 +8,7 @@ package org.elasticsearch.compute.aggregation.blockhash; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.BytesRefHash; @@ -17,11 +18,13 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BytesRefBlock; import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; import java.util.Objects; @@ -81,6 +84,11 @@ public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { } } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + throw new UnsupportedOperationException("TODO"); + } + @Override public Block[] getKeys() { int positions = (int) intervalHash.size(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/X-BlockHash.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/X-BlockHash.java.st index 3314783d857e..1e4c5af134aa 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/X-BlockHash.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/X-BlockHash.java.st @@ -11,8 +11,8 @@ $if(BytesRef)$ import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.unit.ByteSizeValue; $endif$ +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; $if(BytesRef)$ @@ -50,6 +50,7 @@ $endif$ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupe$Type$; +import org.elasticsearch.core.ReleasableIterator; $if(BytesRef)$ import java.io.IOException; @@ -129,6 +130,51 @@ $endif$ return result.ords(); } + @Override + public ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + var block = page.getBlock(channel); + if (block.areAllValuesNull()) { + return ReleasableIterator.single(blockFactory.newConstantIntVector(0, block.getPositionCount()).asBlock()); + } + + $Type$Block castBlock = ($Type$Block) block; + $Type$Vector vector = castBlock.asVector(); + // TODO honor targetBlockSize and chunk the pages if requested. + if (vector == null) { + return ReleasableIterator.single(lookup(castBlock)); + } + return ReleasableIterator.single(lookup(vector)); + } + + private IntBlock lookup($Type$Vector vector) { +$if(BytesRef)$ + BytesRef scratch = new BytesRef(); +$endif$ + int positions = vector.getPositionCount(); + try (var builder = blockFactory.newIntBlockBuilder(positions)) { + for (int i = 0; i < positions; i++) { +$if(double)$ + long v = Double.doubleToLongBits(vector.getDouble(i)); +$elseif(BytesRef)$ + BytesRef v = vector.getBytesRef(i, scratch); +$else$ + $type$ v = vector.get$Type$(i); +$endif$ + long found = hash.find(v); + if (found < 0) { + builder.appendNull(); + } else { + builder.appendInt(Math.toIntExact(hashOrdToGroupNullReserved(found))); + } + } + return builder.build(); + } + } + + private IntBlock lookup($Type$Block block) { + return new MultivalueDedupe$Type$(block).hashLookup(blockFactory, hash); + } + @Override public $Type$Block[] getKeys() { $if(BytesRef)$ diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st index 331a5713fa3d..b82061b85760 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st @@ -277,6 +277,15 @@ $endif$ @Override Builder mvOrdering(Block.MvOrdering mvOrdering); +$if(int)$ + /** + * An estimate of the number of bytes the {@link IntBlock} created by + * {@link #build} will use. This may overestimate the size but shouldn't + * underestimate it. + */ + long estimatedBytes(); + +$endif$ @Override $Type$Block build(); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st index fab3be0be423..347f37cd7828 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st @@ -295,5 +295,11 @@ $if(BytesRef)$ public void extraClose() { Releasables.closeExpectNoException(values); } +$elseif(int)$ + + @Override + public long estimatedBytes() { + return estimatedBytes; + } $endif$ } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/BatchEncoder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/BatchEncoder.java index 0aa5a21bad58..8c584f441f64 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/BatchEncoder.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/BatchEncoder.java @@ -19,12 +19,13 @@ import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.core.Releasable; import java.lang.invoke.MethodHandles; import java.lang.invoke.VarHandle; import java.nio.ByteOrder; -public abstract class BatchEncoder implements Accountable { +public abstract class BatchEncoder implements Releasable, Accountable { /** * Checks if an offset is {@code null}. */ @@ -265,6 +266,7 @@ protected abstract static class DirectEncoder extends BatchEncoder { DirectEncoder(Block block) { this.block = block; + block.incRef(); } @Override @@ -300,6 +302,11 @@ public int read(int index, BytesRefBuilder dst) { public final long ramBytesUsed() { return BASE_RAM_USAGE; } + + @Override + public void close() { + block.decRef(); + } } private static final VarHandle intHandle = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.nativeOrder()); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBoolean.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBoolean.java index db0360b2281e..b78efd5c870b 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBoolean.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeBoolean.java @@ -125,6 +125,7 @@ public IntBlock hash(BlockFactory blockFactory, boolean[] everSeen) { * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.Booleans(Math.max(2, batchSize)) { @Override protected void readNextBatch() { @@ -151,6 +152,11 @@ protected void readNextBatch() { } } } + + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/X-MultivalueDedupe.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/X-MultivalueDedupe.java.st index 06cf85bf7f00..954ee890fd8a 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/X-MultivalueDedupe.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/mvdedupe/X-MultivalueDedupe.java.st @@ -304,6 +304,7 @@ $endif$ * things like hashing many fields together. */ public BatchEncoder batchEncoder(int batchSize) { + block.incRef(); return new BatchEncoder.$Type$s(batchSize) { @Override protected void readNextBatch() { @@ -374,6 +375,11 @@ $if(BytesRef)$ return size; } $endif$ + + @Override + public void close() { + block.decRef(); + } }; } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java index ee1cc8009747..73863bec7bf8 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java @@ -10,6 +10,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.unit.ByteSizeValue; @@ -19,18 +20,26 @@ import org.elasticsearch.compute.data.BasicBlockTests; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BlockTestUtils; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.MockBlockFactory; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.mvdedupe.MultivalueDedupeTests; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.indices.CrankyCircuitBreakerService; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ListMatcher; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.NavigableSet; import java.util.Set; import java.util.TreeSet; @@ -38,7 +47,9 @@ import static org.elasticsearch.test.ListMatcher.matchesList; import static org.elasticsearch.test.MapMatcher.assertMap; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.in; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -132,6 +143,11 @@ private void test(MockBlockFactory blockFactory) { || types.equals(List.of(ElementType.LONG, ElementType.BYTES_REF)) || types.equals(List.of(ElementType.BYTES_REF, ElementType.LONG))) ); + /* + * Expected ordinals for checking lookup. Skipped if we have more than 5 groups because + * it'd be too expensive to calculate. + */ + Map, Set> expectedOrds = groups > 5 ? null : new HashMap<>(); for (int p = 0; p < pageCount; p++) { for (int g = 0; g < blocks.length; g++) { @@ -155,6 +171,9 @@ private void test(MockBlockFactory blockFactory) { assertThat(ordsAndKeys.ords().getTotalValueCount(), lessThanOrEqualTo(emitBatchSize)); } batchCount[0]++; + if (expectedOrds != null) { + recordExpectedOrds(expectedOrds, blocks, ordsAndKeys); + } }, blocks); if (usingSingle) { assertThat(batchCount[0], equalTo(1)); @@ -187,6 +206,10 @@ private void test(MockBlockFactory blockFactory) { } assertMap(keyList, keyMatcher); } + + if (blockHash instanceof LongLongBlockHash == false && blockHash instanceof BytesRefLongBlockHash == false) { + assertLookup(blockFactory, expectedOrds, types, blockHash, oracle); + } } finally { Releasables.closeExpectNoException(keyBlocks); blockFactory.ensureAllBlocksAreReleased(); @@ -205,6 +228,113 @@ private BlockHash newBlockHash(BlockFactory blockFactory, int emitBatchSize, Lis : BlockHash.build(specs, blockFactory, emitBatchSize, true); } + private static final int LOOKUP_POSITIONS = 1_000; + + private void assertLookup( + BlockFactory blockFactory, + Map, Set> expectedOrds, + List types, + BlockHash blockHash, + Oracle oracle + ) { + Block.Builder[] builders = new Block.Builder[types.size()]; + try { + for (int b = 0; b < builders.length; b++) { + builders[b] = types.get(b).newBlockBuilder(LOOKUP_POSITIONS, blockFactory); + } + for (int p = 0; p < LOOKUP_POSITIONS; p++) { + /* + * Pick a random key, about half the time one that's present. + * Note: if the universe of keys is small the randomKey method + * is quite likely to spit out a key in the oracle. That's fine + * so long as we have tests with a large universe too. + */ + List key = randomBoolean() ? randomKey(types) : randomFrom(oracle.keys); + for (int b = 0; b < builders.length; b++) { + BlockTestUtils.append(builders[b], key.get(b)); + } + } + Block[] keyBlocks = Block.Builder.buildAll(builders); + try { + for (Block block : keyBlocks) { + assertThat(block.getPositionCount(), equalTo(LOOKUP_POSITIONS)); + } + try (ReleasableIterator lookup = blockHash.lookup(new Page(keyBlocks), ByteSizeValue.ofKb(between(1, 100)))) { + int positionOffset = 0; + while (lookup.hasNext()) { + try (IntBlock ords = lookup.next()) { + for (int p = 0; p < ords.getPositionCount(); p++) { + List key = readKey(keyBlocks, positionOffset + p); + if (oracle.keys.contains(key) == false) { + assertTrue(ords.isNull(p)); + continue; + } + assertThat(ords.getValueCount(p), equalTo(1)); + if (expectedOrds != null) { + assertThat(ords.getInt(ords.getFirstValueIndex(p)), in(expectedOrds.get(key))); + } + } + positionOffset += ords.getPositionCount(); + } + } + assertThat(positionOffset, equalTo(LOOKUP_POSITIONS)); + } + } finally { + Releasables.closeExpectNoException(keyBlocks); + } + + } finally { + Releasables.closeExpectNoException(builders); + } + } + + private static List readKey(Block[] keyBlocks, int position) { + List key = new ArrayList<>(keyBlocks.length); + for (Block block : keyBlocks) { + assertThat(block.getValueCount(position), lessThan(2)); + List v = BasicBlockTests.valuesAtPositions(block, position, position + 1).get(0); + key.add(v == null ? null : v.get(0)); + } + return key; + } + + private void recordExpectedOrds( + Map, Set> expectedOrds, + Block[] keyBlocks, + BlockHashTests.OrdsAndKeys ordsAndKeys + ) { + long start = System.nanoTime(); + for (int p = 0; p < ordsAndKeys.ords().getPositionCount(); p++) { + for (List key : readKeys(keyBlocks, p + ordsAndKeys.positionOffset())) { + Set ords = expectedOrds.computeIfAbsent(key, k -> new TreeSet<>()); + int firstOrd = ordsAndKeys.ords().getFirstValueIndex(p); + int endOrd = ordsAndKeys.ords().getValueCount(p) + firstOrd; + for (int i = firstOrd; i < endOrd; i++) { + ords.add(ordsAndKeys.ords().getInt(i)); + } + } + } + logger.info("finished collecting ords {} {}", expectedOrds.size(), TimeValue.timeValueNanos(System.nanoTime() - start)); + } + + private static List> readKeys(Block[] keyBlocks, int position) { + List> keys = new ArrayList<>(); + keys.add(List.of()); + for (Block block : keyBlocks) { + List values = BasicBlockTests.valuesAtPositions(block, position, position + 1).get(0); + List> newKeys = new ArrayList<>(); + for (Object v : values == null ? Collections.singletonList(null) : values) { + for (List k : keys) { + List newKey = new ArrayList<>(k); + newKey.add(v); + newKeys.add(newKey); + } + } + keys = newKeys; + } + return keys.stream().distinct().toList(); + } + private static class KeyComparator implements Comparator> { @Override public int compare(List lhs, List rhs) { @@ -275,4 +405,20 @@ static CircuitBreakerService mockBreakerService(CircuitBreaker breaker) { when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(breaker); return breakerService; } + + private static List randomKey(List types) { + return types.stream().map(BlockHashRandomizedTests::randomKeyElement).toList(); + } + + private static Object randomKeyElement(ElementType type) { + return switch (type) { + case INT -> randomInt(); + case LONG -> randomLong(); + case DOUBLE -> randomDouble(); + case BYTES_REF -> new BytesRef(randomAlphaOfLength(5)); + case BOOLEAN -> randomBoolean(); + case NULL -> null; + default -> throw new IllegalArgumentException("unsupported element type [" + type + "]"); + }; + } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java index c1609697f004..cf43df98e262 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.compute.data.MockBlockFactory; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.data.TestBlockFactory; +import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.test.ESTestCase; @@ -1111,11 +1112,7 @@ record OrdsAndKeys(String description, int positionOffset, IntBlock ords, Block[ * more than one block of group ids this will fail. */ private void hash(Consumer callback, Block.Builder... values) { - Block[] blocks = new Block[values.length]; - for (int i = 0; i < blocks.length; i++) { - blocks[i] = values[i].build(); - } - hash(callback, blocks); + hash(callback, Block.Builder.buildAll(values)); } /** @@ -1124,39 +1121,44 @@ private void hash(Consumer callback, Block.Builder... values) { */ private void hash(Consumer callback, Block... values) { boolean[] called = new boolean[] { false }; - hash(ordsAndKeys -> { - if (called[0]) { - throw new IllegalStateException("hash produced more than one block"); - } - called[0] = true; - callback.accept(ordsAndKeys); - }, 16 * 1024, values); + try (BlockHash hash = buildBlockHash(16 * 1024, values)) { + hash(true, hash, ordsAndKeys -> { + if (called[0]) { + throw new IllegalStateException("hash produced more than one block"); + } + called[0] = true; + callback.accept(ordsAndKeys); + if (hash instanceof LongLongBlockHash == false && hash instanceof BytesRefLongBlockHash == false) { + try (ReleasableIterator lookup = hash.lookup(new Page(values), ByteSizeValue.ofKb(between(1, 100)))) { + assertThat(lookup.hasNext(), equalTo(true)); + try (IntBlock ords = lookup.next()) { + assertThat(ords, equalTo(ordsAndKeys.ords)); + } + } + } + }, values); + } finally { + Releasables.close(values); + } } private void hash(Consumer callback, int emitBatchSize, Block.Builder... values) { - Block[] blocks = new Block[values.length]; - for (int i = 0; i < blocks.length; i++) { - blocks[i] = values[i].build(); + Block[] blocks = Block.Builder.buildAll(values); + try (BlockHash hash = buildBlockHash(emitBatchSize, blocks)) { + hash(true, hash, callback, blocks); + } finally { + Releasables.closeExpectNoException(blocks); } - hash(callback, emitBatchSize, blocks); } - private void hash(Consumer callback, int emitBatchSize, Block... values) { - try { - List specs = new ArrayList<>(values.length); - for (int c = 0; c < values.length; c++) { - specs.add(new BlockHash.GroupSpec(c, values[c].elementType())); - } - try ( - BlockHash blockHash = forcePackedHash - ? new PackedValuesBlockHash(specs, blockFactory, emitBatchSize) - : BlockHash.build(specs, blockFactory, emitBatchSize, true) - ) { - hash(true, blockHash, callback, values); - } - } finally { - Releasables.closeExpectNoException(values); + private BlockHash buildBlockHash(int emitBatchSize, Block... values) { + List specs = new ArrayList<>(values.length); + for (int c = 0; c < values.length; c++) { + specs.add(new BlockHash.GroupSpec(c, values[c].elementType())); } + return forcePackedHash + ? new PackedValuesBlockHash(specs, blockFactory, emitBatchSize) + : BlockHash.build(specs, blockFactory, emitBatchSize, true); } static void hash(boolean collectKeys, BlockHash blockHash, Consumer callback, Block... values) { @@ -1200,6 +1202,18 @@ public void add(int positionOffset, IntVector groupIds) { add(positionOffset, groupIds.asBlock()); } }); + if (blockHash instanceof LongLongBlockHash == false && blockHash instanceof BytesRefLongBlockHash == false) { + Block[] keys = blockHash.getKeys(); + try (ReleasableIterator lookup = blockHash.lookup(new Page(keys), ByteSizeValue.ofKb(between(1, 100)))) { + while (lookup.hasNext()) { + try (IntBlock ords = lookup.next()) { + assertThat(ords.nullValuesCount(), equalTo(0)); + } + } + } finally { + Releasables.closeExpectNoException(keys); + } + } } private void assertOrds(IntBlock ordsBlock, Integer... expectedOrds) { From 3fc83a069cd11ffb3f2f2eecd528bcf0e1ef6ba4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 24 Apr 2024 14:31:18 +0200 Subject: [PATCH 23/27] Remove redundant mutex field from SparseFileTracker (#107778) This thing is only there to guard the `ranges` field. For a 1TB cache, this results in at least a waste of 1M in heap and adds more needless indirection. --- .../blobcache/common/SparseFileTracker.java | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/common/SparseFileTracker.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/common/SparseFileTracker.java index e9be9577063c..17e48c91a4ea 100644 --- a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/common/SparseFileTracker.java +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/common/SparseFileTracker.java @@ -39,8 +39,6 @@ public class SparseFileTracker { */ private volatile long complete = 0L; - private final Object mutex = new Object(); - private final String description; private final long length; @@ -79,7 +77,7 @@ public SparseFileTracker(String description, long length, SortedSet r private long addInitialRanges(long length, SortedSet ranges) { long initialLength = 0; - synchronized (mutex) { + synchronized (this.ranges) { Range previous = null; for (ByteRange next : ranges) { if (next.isEmpty()) { @@ -108,7 +106,7 @@ public long getLength() { public SortedSet getCompletedRanges() { SortedSet completedRanges = null; - synchronized (mutex) { + synchronized (ranges) { assert invariant(); for (Range range : ranges) { if (range.isPending()) { @@ -136,7 +134,7 @@ public long getInitialLength() { * @return the sum of the length of the ranges */ private long computeLengthOfRanges() { - assert Thread.holdsLock(mutex) : "sum of length of the ranges must be computed under mutex"; + assert Thread.holdsLock(ranges) : "sum of length of the ranges must be computed under mutex"; return ranges.stream().mapToLong(range -> range.end - range.start).sum(); } @@ -188,7 +186,7 @@ private List doWaitForRange(ByteRange range, ByteRange subRange, ActionList final List gaps = new ArrayList<>(); final List pendingRanges = new ArrayList<>(); final Range targetRange = new Range(range); - synchronized (mutex) { + synchronized (ranges) { determineStartingRange(range, pendingRanges, targetRange); while (targetRange.start < range.end()) { @@ -287,7 +285,7 @@ public boolean waitForRangeIfPending(final ByteRange range, final ActionListener final List pendingRanges = new ArrayList<>(); final Range targetRange = new Range(range); - synchronized (mutex) { + synchronized (ranges) { determineStartingRange(range, pendingRanges, targetRange); while (targetRange.start < range.end()) { @@ -347,7 +345,7 @@ private ActionListener wrapWithAssertions(ActionListener listener) { if (Assertions.ENABLED) { return ActionListener.runAfter( listener, - () -> { assert Thread.holdsLock(mutex) == false : "mutex unexpectedly held in listener"; } + () -> { assert Thread.holdsLock(ranges) == false : "mutex unexpectedly held in listener"; } ); } else { return listener; @@ -364,7 +362,7 @@ private ActionListener wrapWithAssertions(ActionListener listener) { */ @Nullable public ByteRange getAbsentRangeWithin(ByteRange range) { - synchronized (mutex) { + synchronized (ranges) { final long start = range.start(); // Find the first absent byte in the range @@ -408,7 +406,7 @@ public ByteRange getAbsentRangeWithin(ByteRange range) { } private boolean assertPendingRangeExists(Range range) { - assert Thread.holdsLock(mutex); + assert Thread.holdsLock(ranges); final SortedSet existingRanges = ranges.tailSet(range); assert existingRanges.isEmpty() == false; final Range existingRange = existingRanges.first(); @@ -418,7 +416,7 @@ private boolean assertPendingRangeExists(Range range) { } private void onGapSuccess(final Range gapRange) { - synchronized (mutex) { + synchronized (ranges) { assert invariant(); assert assertPendingRangeExists(gapRange); ranges.remove(gapRange); @@ -463,7 +461,7 @@ private void onGapSuccess(final Range gapRange) { } private void maybeUpdateCompletePointer(Range gapRange) { - assert Thread.holdsLock(mutex); + assert Thread.holdsLock(ranges); if (gapRange.start == 0) { assert complete <= gapRange.end; complete = gapRange.end; @@ -471,7 +469,7 @@ private void maybeUpdateCompletePointer(Range gapRange) { } private boolean assertGapRangePending(Range gapRange) { - synchronized (mutex) { + synchronized (ranges) { assert invariant(); assert assertPendingRangeExists(gapRange); } @@ -479,7 +477,7 @@ private boolean assertGapRangePending(Range gapRange) { } private void onGapFailure(final Range gapRange, Exception e) { - synchronized (mutex) { + synchronized (ranges) { assert invariant(); assert assertPendingRangeExists(gapRange); final boolean removed = ranges.remove(gapRange); @@ -491,7 +489,7 @@ private void onGapFailure(final Range gapRange, Exception e) { } private boolean invariant() { - assert Thread.holdsLock(mutex); + assert Thread.holdsLock(ranges); long lengthOfRanges = 0L; Range previousRange = null; for (final Range range : ranges) { From c008039c5ec0f85d7339e58dde40df83d46dc8d4 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Wed, 24 Apr 2024 14:36:17 +0200 Subject: [PATCH 24/27] Use LogDocMergePolicy in DiversifiedSamplerTests#testDiversifiedSampler (#107826) Similar to tother cases, the addition of a new merge policy that reverse the order of the documents in lucene causes this test to fail in edge cases. To avoid randomisation we hardcode the merge policy to LogDocMergePolicy. --- .../bucket/sampler/DiversifiedSamplerTests.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerTests.java index 96ad3cd5afb2..072453e129be 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedSamplerTests.java @@ -16,9 +16,13 @@ import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LogDocMergePolicy; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.lucene.search.function.FieldValueFactorFunction; import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery; @@ -85,7 +89,9 @@ private void writeBooks(RandomIndexWriter iw) throws IOException { public void testDiversifiedSampler() throws Exception { Directory directory = newDirectory(); - RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory); + IndexWriterConfig iwc = LuceneTestCase.newIndexWriterConfig(random(), new MockAnalyzer(random())); + iwc.setMergePolicy(new LogDocMergePolicy()); + RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory, iwc); MappedFieldType genreFieldType = new KeywordFieldMapper.KeywordFieldType("genre"); writeBooks(indexWriter); indexWriter.close(); From 972060f1afe0d926405184895dbd15f84529d27d Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Wed, 24 Apr 2024 09:03:54 -0400 Subject: [PATCH 25/27] NoSuchRemoteClusterException should not be thrown when a remote is configured (#107435) When a remote cluster has been configured by an admin, but is not connected, clusters marked with skip_unavailable=true will have NoSuchRemoteClusterException thrown, which is confusing. This fix changes that for one of the connection lookup pathways. Fixes #107381 --- docs/changelog/107435.yaml | 6 ++++++ .../transport/ActionTransportException.java | 2 +- .../transport/RemoteClusterAwareClient.java | 2 +- .../transport/RemoteClusterService.java | 2 +- .../transport/RemoteConnectionManager.java | 2 +- .../action/search/TransportSearchActionTests.java | 2 +- .../transport/RemoteClusterClientTests.java | 4 ++-- .../transport/RemoteClusterConnectionTests.java | 4 ++-- .../transport/RemoteConnectionManagerTests.java | 15 +++++++++++++++ 9 files changed, 30 insertions(+), 9 deletions(-) create mode 100644 docs/changelog/107435.yaml diff --git a/docs/changelog/107435.yaml b/docs/changelog/107435.yaml new file mode 100644 index 000000000000..ae5d2215419c --- /dev/null +++ b/docs/changelog/107435.yaml @@ -0,0 +1,6 @@ +pr: 107435 +summary: '`NoSuchRemoteClusterException` should not be thrown when a remote is configured' +area: Network +type: bug +issues: + - 107381 diff --git a/server/src/main/java/org/elasticsearch/transport/ActionTransportException.java b/server/src/main/java/org/elasticsearch/transport/ActionTransportException.java index bba4a2a6b86b..5676e64c670c 100644 --- a/server/src/main/java/org/elasticsearch/transport/ActionTransportException.java +++ b/server/src/main/java/org/elasticsearch/transport/ActionTransportException.java @@ -65,6 +65,6 @@ private static String buildMessage(String name, InetSocketAddress address, Strin if (msg != null) { sb.append(" ").append(msg); } - return sb.toString(); + return sb.toString().trim(); } } diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java index 66d9032d2666..83c022a8c95e 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAwareClient.java @@ -47,7 +47,7 @@ public void } else { connection = remoteClusterService.getConnection(clusterAlias); } - } catch (NoSuchRemoteClusterException e) { + } catch (ConnectTransportException e) { if (ensureConnected == false) { // trigger another connection attempt, but don't wait for it to complete remoteClusterService.ensureConnected(clusterAlias, ActionListener.noop()); diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java index d452a78431f7..6060e1fed139 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java @@ -260,7 +260,7 @@ public void maybeEnsureConnectedAndGetConnection( (l, nullValue) -> ActionListener.completeWith(l, () -> { try { return getConnection(clusterAlias); - } catch (NoSuchRemoteClusterException e) { + } catch (ConnectTransportException e) { if (ensureConnected == false) { // trigger another connection attempt, but don't wait for it to complete ensureConnected(clusterAlias, ActionListener.noop()); diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteConnectionManager.java b/server/src/main/java/org/elasticsearch/transport/RemoteConnectionManager.java index 3b531d54fb03..dfaf3a6fa5ca 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteConnectionManager.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteConnectionManager.java @@ -151,7 +151,7 @@ public Transport.Connection getAnyRemoteConnection() { // Ignore. We will try the next one until all are exhausted. } } - throw new NoSuchRemoteClusterException(clusterAlias); + throw new ConnectTransportException(null, "Unable to connect to [" + clusterAlias + "]"); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java index d943fc8354c0..2613ce7e5a65 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -938,9 +938,9 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti if (randomBoolean()) { RemoteClusterServiceTests.updateSkipUnavailable(remoteClusterService, "remote" + i, true); } - } } + // put the following in assert busy as connections are lazily reestablished assertBusy(() -> { SearchRequest searchRequest = new SearchRequest(); diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterClientTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterClientTests.java index 1b37dcf18fff..bfd626dd3d15 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterClientTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterClientTests.java @@ -267,7 +267,7 @@ public void testQuicklySkipUnavailableClusters() throws Exception { // check that we quickly fail expectThrows( - NoSuchRemoteClusterException.class, + ConnectTransportException.class, () -> PlainActionFuture.get( f -> client.execute(ClusterStateAction.REMOTE_TYPE, new ClusterStateRequest(), f) ) @@ -282,7 +282,7 @@ public void testQuicklySkipUnavailableClusters() throws Exception { PlainActionFuture.get( f -> client.execute(ClusterStateAction.REMOTE_TYPE, new ClusterStateRequest(), f) ); - } catch (NoSuchRemoteClusterException e) { + } catch (ConnectTransportException e) { // keep retrying on this exception, the goal is to check that we eventually reconnect throw new AssertionError(e); } diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java index 2ebd51b2d7f4..8297070ed3d5 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java @@ -835,7 +835,7 @@ public void testConnectedNodesConcurrentAccess() throws IOException, Interrupted try { Transport.Connection lowLevelConnection = connection.getConnection(); assertNotNull(lowLevelConnection); - } catch (NoSuchRemoteClusterException e) { + } catch (ConnectTransportException e) { // ignore, this is an expected exception } } @@ -855,7 +855,7 @@ public void testConnectedNodesConcurrentAccess() throws IOException, Interrupted DiscoveryNode node = randomFrom(discoverableNodes); try { connection.getConnectionManager().getConnection(node); - } catch (NoSuchRemoteClusterException e) { + } catch (ConnectTransportException e) { // Ignore } } diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteConnectionManagerTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteConnectionManagerTests.java index d8ddd7c356b3..667277f9ea19 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteConnectionManagerTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteConnectionManagerTests.java @@ -101,6 +101,21 @@ public void testGetConnection() { assertThat(proxyNodes, containsInAnyOrder("node-2")); } + public void testDisconnectedException() { + assertEquals( + "Unable to connect to [remote-cluster]", + expectThrows(ConnectTransportException.class, remoteConnectionManager::getAnyRemoteConnection).getMessage() + ); + + assertEquals( + "Unable to connect to [remote-cluster]", + expectThrows( + ConnectTransportException.class, + () -> remoteConnectionManager.getConnection(DiscoveryNodeUtils.create("node-1", address)) + ).getMessage() + ); + } + public void testResolveRemoteClusterAlias() throws ExecutionException, InterruptedException { DiscoveryNode remoteNode1 = DiscoveryNodeUtils.create("remote-node-1", address); PlainActionFuture future = new PlainActionFuture<>(); From 05d4bc3f15199ba63c7ed6c286d2a295d3eafcd8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Wed, 24 Apr 2024 15:28:51 +0200 Subject: [PATCH 26/27] Mute testCreateAndDeletePipelineWithInferenceProcessor (#107777) (#107838) Related: #107777 --- .../elasticsearch/xpack/ml/integration/InferenceProcessorIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceProcessorIT.java b/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceProcessorIT.java index b62b0e170585..a8b017462889 100644 --- a/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceProcessorIT.java +++ b/x-pack/plugin/ml/qa/single-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/InferenceProcessorIT.java @@ -40,6 +40,7 @@ private void putModelAlias(String modelAlias, String newModel) throws IOExceptio } @SuppressWarnings("unchecked") + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/107777") public void testCreateAndDeletePipelineWithInferenceProcessor() throws Exception { putRegressionModel(MODEL_ID); String pipelineId = "regression-model-pipeline"; From fee9097b009f781df19c94297785dadba193305d Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 24 Apr 2024 14:29:29 +0100 Subject: [PATCH 27/27] Strengthen locking in `FsBlobContainer` register impl (#107830) Expands the JVM-wide mutex to prevent all concurrent operations on file-based registers, but then introduces an artificial mechanism for emulating write/write contention within a single JVM. --- .../common/blobstore/BlobContainer.java | 2 +- .../common/blobstore/fs/FsBlobContainer.java | 132 +++++++++++------- .../blobstore/fs/FsBlobContainerTests.java | 76 ++++++++++ .../testkit/RepositoryAnalysisFailureIT.java | 11 ++ .../ContendedRegisterAnalyzeAction.java | 8 +- 5 files changed, 178 insertions(+), 51 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java index ae48de05a620..4bcf3a9855a3 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/BlobContainer.java @@ -259,7 +259,7 @@ default void compareAndSetRegister( * @param purpose The purpose of the operation * @param key key of the value to get * @param listener a listener, completed with the value read from the register or {@code OptionalBytesReference#MISSING} if the value - * could not be read due to concurrent activity. + * could not be read due to concurrent activity (which should not happen). */ default void getRegister(OperationPurpose purpose, String key, ActionListener listener) { compareAndExchangeRegister(purpose, key, BytesArray.EMPTY, BytesArray.EMPTY, listener); diff --git a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java index aea24b7020a0..453cc6e3e199 100644 --- a/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java +++ b/server/src/main/java/org/elasticsearch/common/blobstore/fs/FsBlobContainer.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Strings; @@ -60,6 +61,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; import static java.util.Collections.unmodifiableMap; @@ -392,7 +394,15 @@ private static OutputStream blobOutputStream(Path file) throws IOException { } @Override - @SuppressForbidden(reason = "write to channel that we have open for locking purposes already directly") + public void getRegister(OperationPurpose purpose, String key, ActionListener listener) { + // no lock to acquire here, we are emulating the lack of read/read and read/write contention in cloud repositories + ActionListener.completeWith( + listener, + () -> doUncontendedCompareAndExchangeRegister(path.resolve(key), BytesArray.EMPTY, BytesArray.EMPTY) + ); + } + + @Override public void compareAndExchangeRegister( OperationPurpose purpose, String key, @@ -400,66 +410,92 @@ public void compareAndExchangeRegister( BytesReference updated, ActionListener listener ) { - ActionListener.completeWith(listener, () -> { - BlobContainerUtils.ensureValidRegisterContent(updated); - try (LockedFileChannel lockedFileChannel = LockedFileChannel.open(path.resolve(key))) { - final FileChannel fileChannel = lockedFileChannel.fileChannel(); - final ByteBuffer readBuf = ByteBuffer.allocate(BlobContainerUtils.MAX_REGISTER_CONTENT_LENGTH); - while (readBuf.remaining() > 0) { - if (fileChannel.read(readBuf) == -1) { - break; - } - } - final var found = new BytesArray(readBuf.array(), readBuf.arrayOffset(), readBuf.position()); - readBuf.clear(); - if (fileChannel.read(readBuf) != -1) { - throw new IllegalStateException( - "register contains more than [" + BlobContainerUtils.MAX_REGISTER_CONTENT_LENGTH + "] bytes" - ); + ActionListener.completeWith(listener, () -> doCompareAndExchangeRegister(path.resolve(key), expected, updated)); + } + + private static final KeyedLock writeMutexes = new KeyedLock<>(); + + private static OptionalBytesReference doCompareAndExchangeRegister(Path registerPath, BytesReference expected, BytesReference updated) + throws IOException { + // Emulate write/write contention as might happen in cloud repositories, at least for the case where the writers are all in this + // JVM (e.g. for an ESIntegTestCase). + try (var mutex = writeMutexes.tryAcquire(registerPath)) { + return mutex == null + ? OptionalBytesReference.MISSING + : doUncontendedCompareAndExchangeRegister(registerPath, expected, updated); + } + } + + @SuppressForbidden(reason = "write to channel that we have open for locking purposes already directly") + private static OptionalBytesReference doUncontendedCompareAndExchangeRegister( + Path registerPath, + BytesReference expected, + BytesReference updated + ) throws IOException { + BlobContainerUtils.ensureValidRegisterContent(updated); + try (LockedFileChannel lockedFileChannel = LockedFileChannel.open(registerPath)) { + final FileChannel fileChannel = lockedFileChannel.fileChannel(); + final ByteBuffer readBuf = ByteBuffer.allocate(BlobContainerUtils.MAX_REGISTER_CONTENT_LENGTH); + while (readBuf.remaining() > 0) { + if (fileChannel.read(readBuf) == -1) { + break; } + } + final var found = new BytesArray(readBuf.array(), readBuf.arrayOffset(), readBuf.position()); + readBuf.clear(); + if (fileChannel.read(readBuf) != -1) { + throw new IllegalStateException( + "register contains more than [" + BlobContainerUtils.MAX_REGISTER_CONTENT_LENGTH + "] bytes" + ); + } - if (expected.equals(found)) { - var pageStart = 0L; - final var iterator = updated.iterator(); - BytesRef page; - while ((page = iterator.next()) != null) { - final var writeBuf = ByteBuffer.wrap(page.bytes, page.offset, page.length); - while (writeBuf.remaining() > 0) { - fileChannel.write(writeBuf, pageStart + writeBuf.position()); - } - pageStart += page.length; + if (expected.equals(found)) { + var pageStart = 0L; + final var iterator = updated.iterator(); + BytesRef page; + while ((page = iterator.next()) != null) { + final var writeBuf = ByteBuffer.wrap(page.bytes, page.offset, page.length); + while (writeBuf.remaining() > 0) { + fileChannel.write(writeBuf, pageStart + writeBuf.position()); } - fileChannel.force(true); + pageStart += page.length; } - return OptionalBytesReference.of(found); - } catch (OverlappingFileLockException e) { - return OptionalBytesReference.MISSING; + fileChannel.force(true); } - }); + return OptionalBytesReference.of(found); + } catch (OverlappingFileLockException e) { + assert false : e; // should be impossible, we protect against all concurrent operations within this JVM + return OptionalBytesReference.MISSING; + } } private record LockedFileChannel(FileChannel fileChannel, Closeable fileLock) implements Closeable { // Avoid concurrently opening/closing locked files, because this can trip an assertion within the JDK (see #93955 for details). // Perhaps it would work with finer-grained locks too, but we don't currently need to be fancy here. - private static final Object mutex = new Object(); + // + // Also, avoid concurrent operations on FsBlobContainer registers within a single JVM with a simple blocking lock, to avoid + // OverlappingFileLockException. FileChannel#lock blocks on concurrent operations on the file in a different process. This emulates + // the lack of read/read and read/write contention that can happen on a cloud repository register. + private static final ReentrantLock mutex = new ReentrantLock(); static LockedFileChannel open(Path path) throws IOException { - synchronized (mutex) { - List resources = new ArrayList<>(2); - try { - final FileChannel fileChannel = openOrCreateAtomic(path); - resources.add(fileChannel); + List resources = new ArrayList<>(3); + try { + mutex.lock(); + resources.add(mutex::unlock); - final Closeable fileLock = fileChannel.lock()::close; - resources.add(fileLock); + final FileChannel fileChannel = openOrCreateAtomic(path); + resources.add(fileChannel); - final var result = new LockedFileChannel(fileChannel, fileLock); - resources.clear(); - return result; - } finally { - IOUtils.closeWhileHandlingException(resources); - } + final Closeable fileLock = fileChannel.lock()::close; + resources.add(fileLock); + + final var result = new LockedFileChannel(fileChannel, fileLock); + resources.clear(); + return result; + } finally { + IOUtils.closeWhileHandlingException(resources); } } @@ -476,9 +512,7 @@ private static FileChannel openOrCreateAtomic(Path path) throws IOException { @Override public void close() throws IOException { - synchronized (mutex) { - IOUtils.close(fileLock, fileChannel); - } + IOUtils.close(fileLock, fileChannel, mutex::unlock); } } } diff --git a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobContainerTests.java b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobContainerTests.java index 16213f1f761e..b4ddc02aeb2d 100644 --- a/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobContainerTests.java +++ b/server/src/test/java/org/elasticsearch/common/blobstore/fs/FsBlobContainerTests.java @@ -42,7 +42,9 @@ import java.util.List; import java.util.Locale; import java.util.Set; +import java.util.concurrent.CyclicBarrier; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -52,6 +54,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.oneOf; import static org.hamcrest.Matchers.startsWith; @LuceneTestCase.SuppressFileSystems("*") // we do our own mocking @@ -238,6 +241,79 @@ public void testCompareAndExchange() throws Exception { ); } + public void testRegisterContention() throws Exception { + final Path path = PathUtils.get(createTempDir().toString()); + final FsBlobContainer container = new FsBlobContainer( + new FsBlobStore(randomIntBetween(1, 8) * 1024, path, false), + BlobPath.EMPTY, + path + ); + + final String contendedKey = randomAlphaOfLength(10); + final String uncontendedKey = randomAlphaOfLength(10); + + final var startValue = new BytesArray(randomByteArrayOfLength(8)); + final var finalValue = randomValueOtherThan(startValue, () -> new BytesArray(randomByteArrayOfLength(8))); + + final var p = randomPurpose(); + assertTrue(PlainActionFuture.get(l -> container.compareAndSetRegister(p, contendedKey, BytesArray.EMPTY, startValue, l))); + assertTrue(PlainActionFuture.get(l -> container.compareAndSetRegister(p, uncontendedKey, BytesArray.EMPTY, startValue, l))); + + final var threads = new Thread[between(2, 5)]; + final var startBarrier = new CyclicBarrier(threads.length + 1); + final var casSucceeded = new AtomicBoolean(); + for (int i = 0; i < threads.length; i++) { + threads[i] = new Thread( + i == 0 + // first thread does an uncontended write, which must succeed + ? () -> { + safeAwait(startBarrier); + final OptionalBytesReference result = PlainActionFuture.get( + l -> container.compareAndExchangeRegister(p, uncontendedKey, startValue, finalValue, l) + ); + // NB calling .bytesReference() asserts that the result is present, there was no contention + assertEquals(startValue, result.bytesReference()); + } + // other threads try and do contended writes, which may fail and need retrying + : () -> { + safeAwait(startBarrier); + while (casSucceeded.get() == false) { + final OptionalBytesReference result = PlainActionFuture.get( + l -> container.compareAndExchangeRegister(p, contendedKey, startValue, finalValue, l) + ); + if (result.isPresent() && result.bytesReference().equals(startValue)) { + casSucceeded.set(true); + } + } + }, + "write-thread-" + i + ); + threads[i].start(); + } + + safeAwait(startBarrier); + while (casSucceeded.get() == false) { + for (var key : new String[] { contendedKey, uncontendedKey }) { + // NB calling .bytesReference() asserts that the read did not experience contention + assertThat( + PlainActionFuture.get(l -> container.getRegister(p, key, l)).bytesReference(), + oneOf(startValue, finalValue) + ); + } + } + + for (Thread thread : threads) { + thread.join(); + } + + for (var key : new String[] { contendedKey, uncontendedKey }) { + assertEquals( + finalValue, + PlainActionFuture.get(l -> container.getRegister(p, key, l)).bytesReference() + ); + } + } + public void testAtomicWriteMetadataWithoutAtomicOverwrite() throws IOException { this.fileSystem = new FilterFileSystemProvider("nooverwritefs://", fileSystem) { @Override diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java index 025516f8529a..7715b9e8d42b 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/testkit/RepositoryAnalysisFailureIT.java @@ -712,6 +712,17 @@ public Map listBlobsByPrefix(OperationPurpose purpose, Str return blobMetadataByName; } + @Override + public void getRegister(OperationPurpose purpose, String key, ActionListener listener) { + assertPurpose(purpose); + final var register = registers.get(key); + if (register == null) { + listener.onResponse(OptionalBytesReference.EMPTY); + } else { + listener.onResponse(OptionalBytesReference.of(register.get())); + } + } + @Override public void compareAndExchangeRegister( OperationPurpose purpose, diff --git a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java index 8058b270d310..40cb4a45a033 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java +++ b/x-pack/plugin/snapshot-repo-test-kit/src/main/java/org/elasticsearch/repositories/blobstore/testkit/ContendedRegisterAnalyzeAction.java @@ -156,7 +156,13 @@ public void onFailure(Exception e) { }; if (request.getInitialRead() > request.getRequestCount()) { - blobContainer.getRegister(OperationPurpose.REPOSITORY_ANALYSIS, registerName, initialValueListener); + blobContainer.getRegister(OperationPurpose.REPOSITORY_ANALYSIS, registerName, initialValueListener.delegateFailure((l, r) -> { + if (r.isPresent()) { + l.onResponse(r); + } else { + l.onFailure(new IllegalStateException("register read failed due to contention")); + } + })); } else { blobContainer.compareAndExchangeRegister( OperationPurpose.REPOSITORY_ANALYSIS,