From 501f78701d03760852b2aae2194027ea17b3e44d Mon Sep 17 00:00:00 2001 From: Pat Whelan Date: Mon, 10 Jun 2024 07:47:36 -0400 Subject: [PATCH 01/31] [Transform] Reset max page size to settings value (#109449) When a circuit breaker exception causes the transform checkpoint to fail, the transform will reduce the max page size to the value specified by the circuit breaker. Transform will retry the checkpoint until the search succeeds. When the search succeeds, the Transform will now reset the max search size to the value specified in the Transform settings rather than the now-deprecated value in the Latest/Pivot config. Fix #109308 --- docs/changelog/109449.yaml | 6 + .../transforms/TransformIndexer.java | 14 +- .../transforms/TransformIndexerTests.java | 169 +++++++++++++++++- 3 files changed, 183 insertions(+), 6 deletions(-) create mode 100644 docs/changelog/109449.yaml diff --git a/docs/changelog/109449.yaml b/docs/changelog/109449.yaml new file mode 100644 index 0000000000000..90cb908227f1b --- /dev/null +++ b/docs/changelog/109449.yaml @@ -0,0 +1,6 @@ +pr: 109449 +summary: Reset max page size to settings value +area: Transform +type: bug +issues: + - 109308 diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformIndexer.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformIndexer.java index 201f20ac1eb09..712a95ded2076 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformIndexer.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/TransformIndexer.java @@ -45,6 +45,7 @@ import org.elasticsearch.xpack.core.transform.transforms.TransformState; import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState; import org.elasticsearch.xpack.core.transform.utils.ExceptionsHelper; +import org.elasticsearch.xpack.transform.Transform; import org.elasticsearch.xpack.transform.TransformServices; import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider; import org.elasticsearch.xpack.transform.notifications.TransformAuditor; @@ -570,9 +571,7 @@ private void executeRetentionPolicy(ActionListener listener) { private void finalizeCheckpoint(ActionListener listener) { try { // reset the page size, so we do not memorize a low page size forever - if (function != null) { - context.setPageSize(function.getInitialPageSize()); - } + resetPageSize(); // reset the changed bucket to free memory if (changeCollector != null) { changeCollector.clear(); @@ -1234,12 +1233,17 @@ private RunState determineRunStateAtStart() { private void configurePageSize(Integer newPageSize) { initialConfiguredPageSize = newPageSize; + resetPageSize(); + } - // if the user explicitly set a page size, take it from the config, otherwise let the function decide + private void resetPageSize() { if (initialConfiguredPageSize != null && initialConfiguredPageSize > 0) { context.setPageSize(initialConfiguredPageSize); - } else { + } else if (function != null) { context.setPageSize(function.getInitialPageSize()); + } else { + // we should never be in a state where both initialConfiguredPageSize and function are null, but just in case... + context.setPageSize(Transform.DEFAULT_INITIAL_MAX_PAGE_SEARCH_SIZE); } } diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java index abad10b148f21..1c268174f5be5 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java @@ -19,6 +19,8 @@ import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.reindex.BulkByScrollResponse; @@ -35,6 +37,7 @@ import org.elasticsearch.xpack.core.indexing.IndexerState; import org.elasticsearch.xpack.core.indexing.IterationResult; import org.elasticsearch.xpack.core.transform.action.ValidateTransformAction; +import org.elasticsearch.xpack.core.transform.transforms.SettingsConfig; import org.elasticsearch.xpack.core.transform.transforms.TimeRetentionPolicyConfigTests; import org.elasticsearch.xpack.core.transform.transforms.TimeSyncConfig; import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint; @@ -43,6 +46,7 @@ import org.elasticsearch.xpack.core.transform.transforms.TransformIndexerStats; import org.elasticsearch.xpack.core.transform.transforms.TransformState; import org.elasticsearch.xpack.core.transform.transforms.TransformTaskState; +import org.elasticsearch.xpack.transform.Transform; import org.elasticsearch.xpack.transform.TransformNode; import org.elasticsearch.xpack.transform.TransformServices; import org.elasticsearch.xpack.transform.checkpoint.CheckpointProvider; @@ -59,7 +63,9 @@ import java.time.Clock; import java.util.Collections; import java.util.Map; +import java.util.concurrent.BlockingDeque; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -107,9 +113,13 @@ class MockedTransformIndexer extends TransformIndexer { private CountDownLatch searchLatch; private CountDownLatch doProcessLatch; private CountDownLatch doSaveStateLatch; + private CountDownLatch afterFinishOrFailureLatch; private AtomicBoolean saveStateInProgress = new AtomicBoolean(false); + private BlockingDeque searchExceptions = new LinkedBlockingDeque<>(); + private BlockingDeque runBeforeOnFinish = new LinkedBlockingDeque<>(); + // how many loops to execute until reporting done private int numberOfLoops; @@ -211,7 +221,11 @@ protected void doNextSearch(long waitTimeInNanos, ActionListener throw new IllegalStateException(e); } } - threadPool.generic().execute(() -> nextPhase.onResponse(ONE_HIT_SEARCH_RESPONSE)); + if (searchExceptions.isEmpty() == false) { + nextPhase.onFailure(searchExceptions.poll()); + } else { + threadPool.generic().execute(() -> nextPhase.onResponse(ONE_HIT_SEARCH_RESPONSE)); + } } @Override @@ -261,6 +275,22 @@ void doMaybeCreateDestIndex(Map deducedDestIndexMappings, Action listener.onResponse(null); } + @Override + protected void onFinish(ActionListener listener) { + while (runBeforeOnFinish.isEmpty() == false) { + runBeforeOnFinish.poll().run(); + } + super.onFinish(listener); + } + + @Override + protected void afterFinishOrFailure() { + super.afterFinishOrFailure(); + if (afterFinishOrFailureLatch != null) { + afterFinishOrFailureLatch.countDown(); + } + } + public boolean waitingForNextSearch() { return super.getScheduledNextSearch() != null; } @@ -278,6 +308,14 @@ void persistState(TransformState state, ActionListener listener) { void validate(ActionListener listener) { listener.onResponse(null); } + + public void addAfterFinishOrFailureLatch() { + afterFinishOrFailureLatch = new CountDownLatch(1); + } + + public void waitForAfterFinishOrFailureLatch(long timeout, TimeUnit unit) throws InterruptedException { + assertTrue(afterFinishOrFailureLatch.await(timeout, unit)); + } } @Before @@ -439,6 +477,135 @@ public void testInterActionWhileIndexerShutsdown() throws Exception { assertBusy(() -> assertEquals(IndexerState.STOPPED, indexer.getState()), 5, TimeUnit.SECONDS); } + public void testMaxPageSearchSizeIsResetToDefaultValue() throws Exception { + TransformConfig config = new TransformConfig( + randomAlphaOfLength(10), + randomSourceConfig(), + randomDestConfig(), + null, + new TimeSyncConfig("timestamp", TimeValue.timeValueSeconds(1)), + null, + randomPivotConfig(), + null, + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000), + null, + null, + null, + null, + null + ); + AtomicReference state = new AtomicReference<>(IndexerState.STARTED); + + TransformContext context = new TransformContext(TransformTaskState.STARTED, "", 0, mock(TransformContext.Listener.class)); + final MockedTransformIndexer indexer = createMockIndexer( + 1, + config, + state, + null, + threadPool, + auditor, + new TransformIndexerStats(), + context + ); + + // add latches + CountDownLatch searchLatch = indexer.createAwaitForSearchLatch(1); + indexer.addAfterFinishOrFailureLatch(); + + indexer.start(); + assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); + assertEquals(indexer.getState(), IndexerState.INDEXING); + + // set circuit breaker to 50% + indexer.searchExceptions.offer(new CircuitBreakingException("hello", 2, 1, CircuitBreaker.Durability.TRANSIENT)); + indexer.runBeforeOnFinish.offer(() -> { + assertEquals(Math.round(Transform.DEFAULT_INITIAL_MAX_PAGE_SEARCH_SIZE / 2.0), context.getPageSize()); + }); + assertFalse(indexer.runBeforeOnFinish.isEmpty()); + + // run and wait + searchLatch.countDown(); + indexer.waitForAfterFinishOrFailureLatch(5, TimeUnit.SECONDS); + + // rerun, don't throw an exception this time + searchLatch = indexer.createAwaitForSearchLatch(1); + indexer.addAfterFinishOrFailureLatch(); + assertBusy(() -> assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis()))); + searchLatch.countDown(); + indexer.waitForAfterFinishOrFailureLatch(5, TimeUnit.SECONDS); + + // verify that we checked the pageSize decreased + assertTrue(indexer.runBeforeOnFinish.isEmpty()); + // verify that the pageSize reset + assertEquals(Transform.DEFAULT_INITIAL_MAX_PAGE_SEARCH_SIZE.intValue(), context.getPageSize()); + } + + public void testMaxPageSearchSizeIsResetToConfiguredValue() throws Exception { + TransformConfig config = new TransformConfig( + randomAlphaOfLength(10), + randomSourceConfig(), + randomDestConfig(), + null, + new TimeSyncConfig("timestamp", TimeValue.timeValueSeconds(1)), + null, + randomPivotConfig(), + null, + randomBoolean() ? null : randomAlphaOfLengthBetween(1, 1000), + null, + null, + null, + null, + null + ); + AtomicReference state = new AtomicReference<>(IndexerState.STARTED); + + TransformContext context = new TransformContext(TransformTaskState.STARTED, "", 0, mock(TransformContext.Listener.class)); + final MockedTransformIndexer indexer = createMockIndexer( + 1, + config, + state, + null, + threadPool, + auditor, + new TransformIndexerStats(), + context + ); + + // add latches + CountDownLatch searchLatch = indexer.createAwaitForSearchLatch(1); + indexer.addAfterFinishOrFailureLatch(); + + indexer.start(); + assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis())); + assertEquals(indexer.getState(), IndexerState.INDEXING); + + var configuredMaxPageSearchSize = 20_000; + indexer.applyNewSettings( + new SettingsConfig.Builder(SettingsConfig.EMPTY).setMaxPageSearchSize(configuredMaxPageSearchSize).build() + ); + + // set circuit breaker to 50% + indexer.searchExceptions.offer(new CircuitBreakingException("hello", 2, 1, CircuitBreaker.Durability.TRANSIENT)); + indexer.runBeforeOnFinish.offer(() -> { assertEquals(Math.round(configuredMaxPageSearchSize / 2.0), context.getPageSize()); }); + assertFalse(indexer.runBeforeOnFinish.isEmpty()); + + // run and wait + searchLatch.countDown(); + indexer.waitForAfterFinishOrFailureLatch(5, TimeUnit.SECONDS); + + // rerun, don't throw an exception this time + searchLatch = indexer.createAwaitForSearchLatch(1); + indexer.addAfterFinishOrFailureLatch(); + assertBusy(() -> assertTrue(indexer.maybeTriggerAsyncJob(System.currentTimeMillis()))); + searchLatch.countDown(); + indexer.waitForAfterFinishOrFailureLatch(5, TimeUnit.SECONDS); + + // verify that we checked the pageSize decreased + assertTrue(indexer.runBeforeOnFinish.isEmpty()); + // verify that the pageSize reset + assertEquals(configuredMaxPageSearchSize, context.getPageSize()); + } + private MockedTransformIndexer createMockIndexer( int numberOfLoops, TransformConfig config, From 28faacd8690ed89fe7088cd63454100745d14941 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Przemys=C5=82aw=20Witek?= Date: Mon, 10 Jun 2024 14:03:49 +0200 Subject: [PATCH 02/31] [Transform] Introduce `_transform/_node_stats` API (#107279) --- docs/changelog/107279.yaml | 5 + .../api/transform.get_node_stats.json | 23 +++ .../action/GetTransformNodeStatsAction.java | 149 ++++++++++++++++++ .../transforms/TransformSchedulerStats.java | 40 +++++ ...odeStatsActionNodesStatsResponseTests.java | 64 ++++++++ .../xpack/security/operator/Constants.java | 1 + .../common/TransformCommonRestTestCase.java | 9 ++ .../transform/integration/TransformIT.java | 5 + .../integration/TransformNodeStatsIT.java | 100 ++++++++++++ .../xpack/transform/Transform.java | 7 +- .../TransportGetTransformNodeStatsAction.java | 84 ++++++++++ .../RestGetTransformNodeStatsAction.java | 42 +++++ .../TransformScheduledTaskQueue.java | 9 ++ .../scheduling/TransformScheduler.java | 19 +++ .../TransformScheduledTaskQueueTests.java | 12 ++ .../scheduling/TransformSchedulerTests.java | 13 ++ 16 files changed, 581 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/107279.yaml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/api/transform.get_node_stats.json create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/transforms/TransformSchedulerStats.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsActionNodesStatsResponseTests.java create mode 100644 x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformNodeStatsIT.java create mode 100644 x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetTransformNodeStatsAction.java create mode 100644 x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/rest/action/RestGetTransformNodeStatsAction.java diff --git a/docs/changelog/107279.yaml b/docs/changelog/107279.yaml new file mode 100644 index 0000000000000..a2940ecc9ba2d --- /dev/null +++ b/docs/changelog/107279.yaml @@ -0,0 +1,5 @@ +pr: 107279 +summary: Introduce _transform/_node_stats API +area: Transform +type: feature +issues: [] diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/transform.get_node_stats.json b/rest-api-spec/src/main/resources/rest-api-spec/api/transform.get_node_stats.json new file mode 100644 index 0000000000000..ca3fde65f6363 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/transform.get_node_stats.json @@ -0,0 +1,23 @@ +{ + "transform.get_node_stats":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/get-transform-node-stats.html", + "description":"Retrieves transform usage information for transform nodes." + }, + "stability":"stable", + "visibility":"public", + "headers":{ + "accept": [ "application/json"] + }, + "url":{ + "paths":[ + { + "path":"/_transform/_node_stats", + "methods":[ + "GET" + ] + } + ] + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsAction.java new file mode 100644 index 0000000000000..6cadefbe206f0 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsAction.java @@ -0,0 +1,149 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.transform.action; + +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.support.TransportAction; +import org.elasticsearch.action.support.nodes.BaseNodeResponse; +import org.elasticsearch.action.support.nodes.BaseNodesRequest; +import org.elasticsearch.action.support.nodes.BaseNodesResponse; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.core.transform.transforms.TransformSchedulerStats; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.xpack.core.transform.transforms.TransformSchedulerStats.REGISTERED_TRANSFORM_COUNT_FIELD_NAME; + +public class GetTransformNodeStatsAction extends ActionType { + + public static final GetTransformNodeStatsAction INSTANCE = new GetTransformNodeStatsAction(); + public static final String NAME = "cluster:admin/transform/node_stats"; + + private static final String SCHEDULER_STATS_FIELD_NAME = "scheduler"; + + private GetTransformNodeStatsAction() { + super(NAME); + } + + public static class NodesStatsRequest extends BaseNodesRequest { + + public NodesStatsRequest() { + super(Strings.EMPTY_ARRAY); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + TransportAction.localOnly(); + } + } + + public static class NodesStatsResponse extends BaseNodesResponse implements ToXContentObject { + + private static final String TOTAL_FIELD_NAME = "total"; + + public int getTotalRegisteredTransformCount() { + int totalRegisteredTransformCount = 0; + for (var nodeResponse : getNodes()) { + totalRegisteredTransformCount += nodeResponse.schedulerStats().registeredTransformCount(); + } + return totalRegisteredTransformCount; + } + + public NodesStatsResponse(ClusterName clusterName, List nodes, List failures) { + super(clusterName, nodes, failures); + } + + public RestStatus status() { + return this.hasFailures() ? RestStatus.INTERNAL_SERVER_ERROR : RestStatus.OK; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + for (var nodeEntry : getNodesMap().entrySet()) { + String nodeName = nodeEntry.getKey(); + NodeStatsResponse nodeResponse = nodeEntry.getValue(); + builder.field(nodeName); + nodeResponse.toXContent(builder, params); + } + builder.startObject(TOTAL_FIELD_NAME); + builder.startObject(SCHEDULER_STATS_FIELD_NAME); + builder.field(REGISTERED_TRANSFORM_COUNT_FIELD_NAME, getTotalRegisteredTransformCount()); + builder.endObject(); + builder.endObject(); + return builder.endObject(); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return TransportAction.localOnly(); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + TransportAction.localOnly(); + } + } + + public static class NodeStatsRequest extends TransportRequest { + + public NodeStatsRequest() {} + + public NodeStatsRequest(StreamInput in) throws IOException { + super(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + } + } + + public static class NodeStatsResponse extends BaseNodeResponse implements ToXContentObject { + + private final TransformSchedulerStats schedulerStats; + + public NodeStatsResponse(DiscoveryNode node, TransformSchedulerStats schedulerStats) { + super(node); + this.schedulerStats = schedulerStats; + } + + public NodeStatsResponse(StreamInput in) throws IOException { + super(in); + this.schedulerStats = in.readOptionalWriteable(TransformSchedulerStats::new); + } + + TransformSchedulerStats schedulerStats() { + return schedulerStats; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalWriteable(schedulerStats); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(SCHEDULER_STATS_FIELD_NAME, schedulerStats); + return builder.endObject(); + } + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/transforms/TransformSchedulerStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/transforms/TransformSchedulerStats.java new file mode 100644 index 0000000000000..ab6e9d587cb8d --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/transform/transforms/TransformSchedulerStats.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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.transform.transforms; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; + +public record TransformSchedulerStats(int registeredTransformCount, String peekTransformName) implements ToXContent, Writeable { + + public static final String REGISTERED_TRANSFORM_COUNT_FIELD_NAME = "registered_transform_count"; + public static final String PEEK_TRANSFORM_FIELD_NAME = "peek_transform"; + + public TransformSchedulerStats(StreamInput in) throws IOException { + this(in.readVInt(), in.readOptionalString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(this.registeredTransformCount); + out.writeOptionalString(this.peekTransformName); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(REGISTERED_TRANSFORM_COUNT_FIELD_NAME, this.registeredTransformCount); + builder.field(PEEK_TRANSFORM_FIELD_NAME, this.peekTransformName); + return builder.endObject(); + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsActionNodesStatsResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsActionNodesStatsResponseTests.java new file mode 100644 index 0000000000000..b50895659ddfd --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/transform/action/GetTransformNodeStatsActionNodesStatsResponseTests.java @@ -0,0 +1,64 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.transform.action; + +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeUtils; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodeStatsResponse; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodesStatsResponse; +import org.elasticsearch.xpack.core.transform.transforms.TransformSchedulerStats; + +import java.util.List; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class GetTransformNodeStatsActionNodesStatsResponseTests extends ESTestCase { + + private static final ClusterName CLUSTER_NAME = new ClusterName("my-cluster"); + + public void testEmptyResponse() { + var nodesStatsResponse = new NodesStatsResponse(CLUSTER_NAME, List.of(), List.of()); + assertThat(nodesStatsResponse.getNodes(), is(empty())); + assertThat(nodesStatsResponse.failures(), is(empty())); + assertThat(nodesStatsResponse.getTotalRegisteredTransformCount(), is(equalTo(0))); + } + + public void testResponse() { + var nodeA = new NodeStatsResponse(createNode("node-A"), new TransformSchedulerStats(7, null)); + var nodeB = new NodeStatsResponse(createNode("node-B"), new TransformSchedulerStats(0, null)); + var nodeC = new NodeStatsResponse(createNode("node-C"), new TransformSchedulerStats(4, null)); + + var nodesStatsResponse = new NodesStatsResponse(CLUSTER_NAME, List.of(nodeA, nodeB, nodeC), List.of()); + assertThat(nodesStatsResponse.getNodes(), containsInAnyOrder(nodeA, nodeB, nodeC)); + assertThat(nodesStatsResponse.failures(), is(empty())); + assertThat(nodesStatsResponse.getTotalRegisteredTransformCount(), is(equalTo(11))); + } + + public void testResponseWithFailure() { + var nodeA = new NodeStatsResponse(createNode("node-A"), new TransformSchedulerStats(7, null)); + var nodeB = new NodeStatsResponse(createNode("node-B"), new TransformSchedulerStats(0, null)); + var nodeC = new FailedNodeException("node-C", "node C failed", null); + + var nodesStatsResponse = new NodesStatsResponse(CLUSTER_NAME, List.of(nodeA, nodeB), List.of(nodeC)); + assertThat(nodesStatsResponse.getNodes(), containsInAnyOrder(nodeA, nodeB)); + assertThat(nodesStatsResponse.failures(), contains(nodeC)); + assertThat(nodesStatsResponse.getTotalRegisteredTransformCount(), is(equalTo(7))); + } + + private static DiscoveryNode createNode(String name) { + return DiscoveryNodeUtils.builder(UUIDs.randomBase64UUID(random())).name(name).build(); + } +} diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index 5561e14da980a..33503bc558795 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -99,6 +99,7 @@ public class Constants { "cluster:admin/features/reset", "cluster:admin/tasks/cancel", "cluster:admin/transform/delete", + "cluster:admin/transform/node_stats", "cluster:admin/transform/preview", "cluster:admin/transform/put", "cluster:admin/transform/reset", diff --git a/x-pack/plugin/transform/qa/common/src/main/java/org/elasticsearch/xpack/transform/integration/common/TransformCommonRestTestCase.java b/x-pack/plugin/transform/qa/common/src/main/java/org/elasticsearch/xpack/transform/integration/common/TransformCommonRestTestCase.java index 98cf817d6c018..97d38807f5c17 100644 --- a/x-pack/plugin/transform/qa/common/src/main/java/org/elasticsearch/xpack/transform/integration/common/TransformCommonRestTestCase.java +++ b/x-pack/plugin/transform/qa/common/src/main/java/org/elasticsearch/xpack/transform/integration/common/TransformCommonRestTestCase.java @@ -77,6 +77,15 @@ protected List getTransformTasksFromClusterState(String transformId) thr return tasks.stream().map(t -> (String) t.get("id")).filter(transformId::equals).toList(); } + protected Response getNodeStats() throws IOException { + return adminClient().performRequest(new Request("GET", "/_transform/_node_stats")); + } + + protected int getTotalRegisteredTransformCount() throws IOException { + Response response = getNodeStats(); + return (int) XContentMapValues.extractValue(entityAsMap(response), "total", "scheduler", "registered_transform_count"); + } + @SuppressWarnings("unchecked") protected void logAudits() throws Exception { logger.info("writing audit messages to the log"); diff --git a/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformIT.java b/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformIT.java index 4db0d0d8baaf1..ab478dc16f224 100644 --- a/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformIT.java +++ b/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformIT.java @@ -245,16 +245,19 @@ public void testTransformLifecycleInALoop() throws Exception { putTransform(transformId, config, RequestOptions.DEFAULT); assertThat(getTransformTasks(), is(empty())); assertThat(getTransformTasksFromClusterState(transformId), is(empty())); + assertThat("Node stats were: " + entityAsMap(getNodeStats()), getTotalRegisteredTransformCount(), is(equalTo(0))); startTransform(transformId, RequestOptions.DEFAULT); // There is 1 transform task after start. assertThat(getTransformTasks(), hasSize(1)); assertThat(getTransformTasksFromClusterState(transformId), hasSize(1)); + assertThat("Node stats were: " + entityAsMap(getNodeStats()), getTotalRegisteredTransformCount(), is(equalTo(1))); Thread.sleep(sleepAfterStartMillis); // There should still be 1 transform task as the transform is continuous. assertThat(getTransformTasks(), hasSize(1)); assertThat(getTransformTasksFromClusterState(transformId), hasSize(1)); + assertThat("Node stats were: " + entityAsMap(getNodeStats()), getTotalRegisteredTransformCount(), is(equalTo(1))); // Stop the transform with force set randomly. stopTransform(transformId, true, null, false, force); @@ -268,6 +271,7 @@ public void testTransformLifecycleInALoop() throws Exception { } // After the transform is stopped, there should be no transform task left in the cluster state. assertThat(getTransformTasksFromClusterState(transformId), is(empty())); + assertThat("Node stats were: " + entityAsMap(getNodeStats()), getTotalRegisteredTransformCount(), is(equalTo(0))); // Delete the transform deleteTransform(transformId); @@ -586,6 +590,7 @@ public void testContinuousTransformRethrottle() throws Exception { deleteTransform(config.getId()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/109101") public void testStartTransform_GivenTimeout_Returns408() throws Exception { String indexName = "start-transform-timeout-index"; String transformId = "start-transform-timeout"; diff --git a/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformNodeStatsIT.java b/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformNodeStatsIT.java new file mode 100644 index 0000000000000..04483873a2aff --- /dev/null +++ b/x-pack/plugin/transform/qa/multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/transform/integration/TransformNodeStatsIT.java @@ -0,0 +1,100 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.transform.integration; + +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.xpack.core.transform.transforms.QueryConfig; +import org.elasticsearch.xpack.core.transform.transforms.TimeSyncConfig; +import org.elasticsearch.xpack.core.transform.transforms.pivot.TermsGroupSource; +import org.junit.After; + +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.is; + +public class TransformNodeStatsIT extends TransformRestTestCase { + + private static final int NUM_USERS = 28; + + static Integer getUserIdForRow(int row) { + return row % NUM_USERS; + } + + static String getDateStringForRow(int row) { + int day = (11 + (row / 100)) % 28; + int hour = 10 + (row % 13); + int min = 10 + (row % 49); + int sec = 10 + (row % 49); + return "2017-01-" + (day < 10 ? "0" + day : day) + "T" + hour + ":" + min + ":" + sec + "Z"; + } + + @After + public void cleanTransforms() throws Exception { + cleanUp(); + } + + @SuppressWarnings("unchecked") + public void testTransformNodeStats() throws Exception { + var transformId = "transform-node-stats"; + createTransform("basic-stats-reviews", transformId); + + var nodesInfo = getNodesInfo(adminClient()); + assertThat("Nodes were: " + nodesInfo, nodesInfo.size(), is(equalTo(3))); + + var response = entityAsMap(getNodeStats()); + assertThat(response, hasKey("total")); + assertThat( + "Response was: " + response, + (int) XContentMapValues.extractValue(response, "total", "scheduler", "registered_transform_count"), + is(equalTo(1)) + ); + for (String nodeId : nodesInfo.keySet()) { + assertThat(response, hasKey(nodeId)); + assertThat( + "Response was: " + response, + (int) XContentMapValues.extractValue(response, nodeId, "scheduler", "registered_transform_count"), + is(greaterThanOrEqualTo(0)) + ); + } + } + + private void createTransform(String indexName, String transformId) throws Exception { + createReviewsIndex(indexName, 100, NUM_USERS, TransformNodeStatsIT::getUserIdForRow, TransformNodeStatsIT::getDateStringForRow); + + var groups = Map.of( + "by-day", + createDateHistogramGroupSourceWithCalendarInterval("timestamp", DateHistogramInterval.DAY, null), + "by-user", + new TermsGroupSource("user_id", null, false), + "by-business", + new TermsGroupSource("business_id", null, false) + ); + + var aggs = AggregatorFactories.builder() + .addAggregator(AggregationBuilders.avg("review_score").field("stars")) + .addAggregator(AggregationBuilders.max("timestamp").field("timestamp")); + + var config = createTransformConfigBuilder(transformId, "reviews-by-user-business-day", QueryConfig.matchAll(), indexName) + .setPivotConfig(createPivotConfig(groups, aggs)) + .setSyncConfig(new TimeSyncConfig("timestamp", null)) + .build(); + + putTransform(transformId, Strings.toString(config), RequestOptions.DEFAULT); + startTransform(config.getId(), RequestOptions.DEFAULT); + + waitUntilCheckpoint(config.getId(), 1L); + } +} diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java index e788a85562e8e..ab4652c562e22 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/Transform.java @@ -58,6 +58,7 @@ import org.elasticsearch.xpack.core.transform.action.GetCheckpointAction; import org.elasticsearch.xpack.core.transform.action.GetCheckpointNodeAction; import org.elasticsearch.xpack.core.transform.action.GetTransformAction; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction; import org.elasticsearch.xpack.core.transform.action.GetTransformStatsAction; import org.elasticsearch.xpack.core.transform.action.PreviewTransformAction; import org.elasticsearch.xpack.core.transform.action.PutTransformAction; @@ -74,6 +75,7 @@ import org.elasticsearch.xpack.transform.action.TransportGetCheckpointAction; import org.elasticsearch.xpack.transform.action.TransportGetCheckpointNodeAction; import org.elasticsearch.xpack.transform.action.TransportGetTransformAction; +import org.elasticsearch.xpack.transform.action.TransportGetTransformNodeStatsAction; import org.elasticsearch.xpack.transform.action.TransportGetTransformStatsAction; import org.elasticsearch.xpack.transform.action.TransportPreviewTransformAction; import org.elasticsearch.xpack.transform.action.TransportPutTransformAction; @@ -93,6 +95,7 @@ import org.elasticsearch.xpack.transform.rest.action.RestCatTransformAction; import org.elasticsearch.xpack.transform.rest.action.RestDeleteTransformAction; import org.elasticsearch.xpack.transform.rest.action.RestGetTransformAction; +import org.elasticsearch.xpack.transform.rest.action.RestGetTransformNodeStatsAction; import org.elasticsearch.xpack.transform.rest.action.RestGetTransformStatsAction; import org.elasticsearch.xpack.transform.rest.action.RestPreviewTransformAction; import org.elasticsearch.xpack.transform.rest.action.RestPutTransformAction; @@ -191,7 +194,8 @@ public List getRestHandlers( new RestCatTransformAction(), new RestUpgradeTransformsAction(), new RestResetTransformAction(), - new RestScheduleNowTransformAction() + new RestScheduleNowTransformAction(), + new RestGetTransformNodeStatsAction() ); } @@ -211,6 +215,7 @@ public List getRestHandlers( new ActionHandler<>(UpgradeTransformsAction.INSTANCE, TransportUpgradeTransformsAction.class), new ActionHandler<>(ResetTransformAction.INSTANCE, TransportResetTransformAction.class), new ActionHandler<>(ScheduleNowTransformAction.INSTANCE, TransportScheduleNowTransformAction.class), + new ActionHandler<>(GetTransformNodeStatsAction.INSTANCE, TransportGetTransformNodeStatsAction.class), // internal, no rest endpoint new ActionHandler<>(ValidateTransformAction.INSTANCE, TransportValidateTransformAction.class), diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetTransformNodeStatsAction.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetTransformNodeStatsAction.java new file mode 100644 index 0000000000000..bbe8f6ea05b4c --- /dev/null +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetTransformNodeStatsAction.java @@ -0,0 +1,84 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.transform.action; + +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.nodes.TransportNodesAction; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodeStatsRequest; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodeStatsResponse; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodesStatsRequest; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodesStatsResponse; +import org.elasticsearch.xpack.transform.TransformServices; +import org.elasticsearch.xpack.transform.transforms.scheduling.TransformScheduler; + +import java.io.IOException; +import java.util.List; + +/** + * {@link TransportGetTransformNodeStatsAction} class fetches transform-related metrics from all the nodes and aggregates these metrics. + */ +public class TransportGetTransformNodeStatsAction extends TransportNodesAction< + NodesStatsRequest, + NodesStatsResponse, + NodeStatsRequest, + NodeStatsResponse> { + + private final TransportService transportService; + private final TransformScheduler scheduler; + + @Inject + public TransportGetTransformNodeStatsAction( + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + ActionFilters actionFilters, + TransformServices transformServices + ) { + super( + GetTransformNodeStatsAction.NAME, + clusterService, + transportService, + actionFilters, + NodeStatsRequest::new, + threadPool.executor(ThreadPool.Names.MANAGEMENT) + ); + this.transportService = transportService; + this.scheduler = transformServices.scheduler(); + } + + @Override + protected NodesStatsResponse newResponse(NodesStatsRequest request, List nodes, List failures) { + return new NodesStatsResponse(clusterService.getClusterName(), nodes, failures); + } + + @Override + protected NodeStatsRequest newNodeRequest(NodesStatsRequest request) { + return new NodeStatsRequest(); + } + + @Override + protected NodeStatsResponse newNodeResponse(StreamInput in, DiscoveryNode node) throws IOException { + return new NodeStatsResponse(in); + } + + @Override + protected NodeStatsResponse nodeOperation(NodeStatsRequest request, Task task) { + var localNode = transportService.getLocalNode(); + var schedulerStats = scheduler.getStats(); + return new NodeStatsResponse(localNode, schedulerStats); + } +} diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/rest/action/RestGetTransformNodeStatsAction.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/rest/action/RestGetTransformNodeStatsAction.java new file mode 100644 index 0000000000000..30d3b6dbdcaae --- /dev/null +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/rest/action/RestGetTransformNodeStatsAction.java @@ -0,0 +1,42 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.transform.rest.action; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.core.transform.TransformField; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction; +import org.elasticsearch.xpack.core.transform.action.GetTransformNodeStatsAction.NodesStatsRequest; + +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.GET; + +@ServerlessScope(Scope.PUBLIC) +public class RestGetTransformNodeStatsAction extends BaseRestHandler { + + @Override + public List routes() { + return List.of(new Route(GET, TransformField.REST_BASE_PATH_TRANSFORMS + "_node_stats")); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) { + NodesStatsRequest request = new NodesStatsRequest(); + return channel -> client.execute(GetTransformNodeStatsAction.INSTANCE, request, new RestToXContentListener<>(channel)); + } + + @Override + public String getName() { + return "transform_get_transform_node_stats_action"; + } +} diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueue.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueue.java index e11da6af1c285..cd3630a095ed1 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueue.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueue.java @@ -108,6 +108,15 @@ public synchronized TransformScheduledTask remove(String transformId) { return task; } + /** + * Returns the current queue size. + * + * @return the current queue size + */ + public synchronized int size() { + return tasks.size(); + } + // Visible for testing /** * @return the set of all the transform ids diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduler.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduler.java index a02f2aac956e2..9c7afa38a5c59 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduler.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduler.java @@ -13,6 +13,7 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.threadpool.Scheduler; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.transform.transforms.TransformSchedulerStats; import org.elasticsearch.xpack.core.transform.transforms.TransformTaskParams; import org.elasticsearch.xpack.transform.Transform; @@ -21,6 +22,7 @@ import java.time.Instant; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.core.Strings.format; @@ -270,6 +272,23 @@ public void deregisterTransform(String transformId) { scheduledTasks.remove(transformId); } + public TransformSchedulerStats getStats() { + return new TransformSchedulerStats( + scheduledTasks.size(), + Optional.ofNullable(scheduledTasks.first()).map(TransformScheduledTask::getTransformId).orElse(null) + ); + } + + // Visible for testing + /** + * Returns the number of transforms currently in the queue. + * + * @return number of transforms currently in the queue + */ + int getRegisteredTransformCount() { + return scheduledTasks.size(); + } + // Visible for testing /** * @return queue current contents diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueueTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueueTests.java index 5030d42f9c17c..6c032e752613b 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueueTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformScheduledTaskQueueTests.java @@ -54,6 +54,7 @@ public void testEmptyQueue() { public void testNonEmptyQueue() { queue.add(createTask("task-1", 5)); assertThat(queue.first(), is(notNullValue())); + assertThat(queue.size(), is(equalTo(1))); } public void testAddAndRemove() { @@ -63,6 +64,7 @@ public void testAddAndRemove() { assertThat(queue.first(), is(notNullValue())); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1", "task-2", "task-3")); assertThat(queue.first(), is(equalTo(createTask("task-2", 1)))); + assertThat(queue.size(), is(equalTo(3))); queue.remove("task-1"); queue.remove("task-2"); @@ -86,6 +88,7 @@ public void testConcurrentAddAndRemove() throws Exception { } assertThat(queue.first(), is(notNullValue())); assertThat(queue.getTransformIds(), hasSize(100)); + assertThat(queue.size(), is(equalTo(100))); { Set removedTaskIds = new HashSet<>(); @@ -107,11 +110,13 @@ public void testConcurrentAddAndRemove() throws Exception { public void testAddNoOp() { queue.add(createTask("task-1", 5)); assertThat(queue.first(), is(equalTo(createTask("task-1", 5)))); + assertThat(queue.size(), is(equalTo(1))); // Try adding a task with a duplicate key queue.add(createTask("task-1", 6)); // Verify that the add operation had no effect assertThat(queue.first(), is(equalTo(createTask("task-1", 5)))); + assertThat(queue.size(), is(equalTo(1))); } public void testRemoveNoOp() { @@ -121,6 +126,7 @@ public void testRemoveNoOp() { assertThat(queue.first(), is(notNullValue())); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1")); assertThat(queue.first(), is(equalTo(createTask("task-1", 5)))); + assertThat(queue.size(), is(equalTo(1))); } public void testUpdateNoOp() { @@ -130,6 +136,7 @@ public void testUpdateNoOp() { assertThat(queue.first(), is(notNullValue())); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1")); assertThat(queue.first(), is(equalTo(createTask("task-1", 5)))); + assertThat(queue.size(), is(equalTo(1))); } public void testUpdateModifiesId() { @@ -154,6 +161,7 @@ public void testRemoveAll() { containsInAnyOrder("task-1", "task-2", "task-3", "task-4", "task-5", "task-6", "task-7", "task-8", "task-9") ); assertThat(queue.first(), is(equalTo(createTask("task-7", 0)))); + assertThat(queue.size(), is(equalTo(9))); List tasksByPriority = new ArrayList<>(); while (queue.first() != null) { @@ -184,15 +192,18 @@ public void testUpdatePriority() { queue.add(createTask("task-3", 9)); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1", "task-2", "task-3")); assertThat(queue.first(), is(equalTo(createTask("task-2", 1)))); + assertThat(queue.size(), is(equalTo(3))); queue.update("task-3", task -> createTask(task.getTransformId(), -999)); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1", "task-2", "task-3")); assertThat(queue.first(), is(equalTo(createTask("task-3", -999)))); + assertThat(queue.size(), is(equalTo(3))); queue.update("task-1", task -> createTask(task.getTransformId(), 0)); queue.remove("task-3"); assertThat(queue.getTransformIds(), containsInAnyOrder("task-1", "task-2")); assertThat(queue.first(), is(equalTo(createTask("task-1", 0)))); + assertThat(queue.size(), is(equalTo(2))); } private static TransformScheduledTask createTask(String transformId, long nextScheduledTimeMillis) { @@ -213,5 +224,6 @@ private static void failUnexpectedCall(Event event) { private void assertThatQueueIsEmpty() { assertThat(queue.first(), is(nullValue())); assertThat(queue.getTransformIds(), is(empty())); + assertThat(queue.size(), is(equalTo(0))); } } diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformSchedulerTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformSchedulerTests.java index 8d3220a5b4de3..06fdfd7b538b1 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformSchedulerTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/scheduling/TransformSchedulerTests.java @@ -80,7 +80,9 @@ private void testScheduling(int frequencySeconds, int minFreqencySeconds) { TransformScheduler.Listener listener = events::add; TransformScheduler transformScheduler = new TransformScheduler(clock, threadPool, SETTINGS, minFrequency); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); transformScheduler.registerTransform(transformTaskParams, listener); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(1))); assertThat( transformScheduler.getTransformScheduledTasks(), contains(new TransformScheduledTask(transformId, fiveSeconds, 0L, 0, 5000, listener)) @@ -125,6 +127,7 @@ private void testScheduling(int frequencySeconds, int minFreqencySeconds) { assertThat(events.get(2), is(equalTo(new TransformScheduler.Event(transformId, 10005, 10010)))); transformScheduler.deregisterTransform(transformId); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); assertThat(transformScheduler.getTransformScheduledTasks(), is(empty())); transformScheduler.stop(); @@ -139,7 +142,9 @@ public void testSchedulingWithFailures() { TransformScheduler.Listener listener = events::add; TransformScheduler transformScheduler = new TransformScheduler(clock, threadPool, SETTINGS, TimeValue.ZERO); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); transformScheduler.registerTransform(transformTaskParams, listener); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(1))); assertThat( transformScheduler.getTransformScheduledTasks(), contains(new TransformScheduledTask(transformId, frequency, 0L, 0, 60 * 60 * 1000, listener)) @@ -177,6 +182,7 @@ public void testSchedulingWithFailures() { ); transformScheduler.deregisterTransform(transformId); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); assertThat(transformScheduler.getTransformScheduledTasks(), is(empty())); transformScheduler.stop(); @@ -191,7 +197,9 @@ public void testScheduleNow() { TransformScheduler.Listener listener = events::add; TransformScheduler transformScheduler = new TransformScheduler(clock, threadPool, SETTINGS, TimeValue.ZERO); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); transformScheduler.registerTransform(transformTaskParams, listener); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(1))); assertThat( transformScheduler.getTransformScheduledTasks(), contains(new TransformScheduledTask(transformId, frequency, 0L, 0, 60 * 60 * 1000, listener)) @@ -226,6 +234,7 @@ public void testScheduleNow() { assertThat(events.get(2), is(equalTo(new TransformScheduler.Event(transformId, 31 * 60 * 1000, 31 * 60 * 1000)))); transformScheduler.deregisterTransform(transformId); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); assertThat(transformScheduler.getTransformScheduledTasks(), is(empty())); transformScheduler.stop(); @@ -402,9 +411,11 @@ public void testRegisterMultipleTransforms() { TransformScheduler.Listener listener = events::add; TransformScheduler transformScheduler = new TransformScheduler(clock, threadPool, SETTINGS, TimeValue.ZERO); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); transformScheduler.registerTransform(transformTaskParams1, listener); transformScheduler.registerTransform(transformTaskParams2, listener); transformScheduler.registerTransform(transformTaskParams3, listener); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(3))); assertThat( transformScheduler.getTransformScheduledTasks(), contains( @@ -432,9 +443,11 @@ public void testMultipleTransformsEligibleForProcessingAtOnce() { TransformScheduler.Listener listener = events::add; TransformScheduler transformScheduler = new TransformScheduler(clock, threadPool, SETTINGS, TimeValue.ZERO); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(0))); transformScheduler.registerTransform(transformTaskParams1, listener); transformScheduler.registerTransform(transformTaskParams2, listener); transformScheduler.registerTransform(transformTaskParams3, listener); + assertThat(transformScheduler.getRegisteredTransformCount(), is(equalTo(3))); assertThat( transformScheduler.getTransformScheduledTasks(), contains( From 02439e321d00eb904c4e1413c49abf6ea631d55e Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Mon, 10 Jun 2024 14:26:43 +0200 Subject: [PATCH 03/31] Update Gradle gradleEnterprise plugin (develocity now) (#109443) * Update Gradle gradleEnterprise plugin (develocity now) * Fix imports in build scan script * Fix build scan api usage * Dependency cleanup and API fix * Fix API update for BuildResult in Build scans * Fix buildkite buildscan annotations based on gradle failures --- .ci/init.gradle | 4 +- .../groovy/elasticsearch.build-scan.gradle | 295 +++++++++--------- gradle/build.versions.toml | 2 +- gradle/verification-metadata.xml | 6 +- plugins/examples/settings.gradle | 2 +- settings.gradle | 2 +- 6 files changed, 161 insertions(+), 150 deletions(-) diff --git a/.ci/init.gradle b/.ci/init.gradle index 4b2cbd1907ca0..3e1f23804cf98 100644 --- a/.ci/init.gradle +++ b/.ci/init.gradle @@ -91,8 +91,8 @@ if (USE_ARTIFACTORY) { } gradle.settingsEvaluated { settings -> - settings.pluginManager.withPlugin("com.gradle.enterprise") { - settings.gradleEnterprise { + settings.pluginManager.withPlugin("com.gradle.develocity") { + settings.develocity { server = 'https://gradle-enterprise.elastic.co' } } diff --git a/build-tools-internal/src/main/groovy/elasticsearch.build-scan.gradle b/build-tools-internal/src/main/groovy/elasticsearch.build-scan.gradle index 67123119f7cd9..f0a7b1a6d0b1c 100644 --- a/build-tools-internal/src/main/groovy/elasticsearch.build-scan.gradle +++ b/build-tools-internal/src/main/groovy/elasticsearch.build-scan.gradle @@ -12,168 +12,179 @@ import java.time.LocalDateTime; import org.elasticsearch.gradle.Architecture import org.elasticsearch.gradle.OS import org.elasticsearch.gradle.internal.info.BuildParams -import org.gradle.initialization.BuildRequestMetaData -buildScan { - URL jenkinsUrl = System.getenv('JENKINS_URL') ? new URL(System.getenv('JENKINS_URL')) : null - String buildKiteUrl = System.getenv('BUILDKITE_BUILD_URL') ? System.getenv('BUILDKITE_BUILD_URL') : null +import java.lang.management.ManagementFactory +import java.time.LocalDateTime - // Automatically publish scans from Elasticsearch CI - if (jenkinsUrl?.host?.endsWith('elastic.co') || jenkinsUrl?.host?.endsWith('elastic.dev') || System.getenv('BUILDKITE') == 'true') { - publishAlways() - buildScan.server = 'https://gradle-enterprise.elastic.co' - } +develocity { - background { - tag OS.current().name() - tag Architecture.current().name() + buildScan { + URL jenkinsUrl = System.getenv('JENKINS_URL') ? new URL(System.getenv('JENKINS_URL')) : null + String buildKiteUrl = System.getenv('BUILDKITE_BUILD_URL') ? System.getenv('BUILDKITE_BUILD_URL') : null - // Tag if this build is run in FIPS mode - if (BuildParams.inFipsJvm) { - tag 'FIPS' + // Automatically publish scans from Elasticsearch CI + if (jenkinsUrl?.host?.endsWith('elastic.co') || jenkinsUrl?.host?.endsWith('elastic.dev') || System.getenv('BUILDKITE') == 'true') { + publishing.onlyIf { true } + server = 'https://gradle-enterprise.elastic.co' } - // Jenkins-specific build scan metadata - if (jenkinsUrl) { - // Disable async upload in CI to ensure scan upload completes before CI agent is terminated - uploadInBackground = false - - String buildNumber = System.getenv('BUILD_NUMBER') - String buildUrl = System.getenv('BUILD_URL') - String jobName = System.getenv('JOB_NAME') - String nodeName = System.getenv('NODE_NAME') - String jobBranch = System.getenv('ghprbTargetBranch') ?: System.getenv('JOB_BRANCH') - - // Link to Jenkins worker logs and system metrics - if (nodeName) { - link 'System logs', "https://ci-stats.elastic.co/app/infra#/logs?&logFilter=(expression:'host.name:${nodeName}',kind:kuery)" - buildFinished { - link 'System metrics', "https://ci-stats.elastic.co/app/metrics/detail/host/${nodeName}" - } + + background { + tag OS.current().name() + tag Architecture.current().name() + + // Tag if this build is run in FIPS mode + if (BuildParams.inFipsJvm) { + tag 'FIPS' } - // Parse job name in the case of matrix builds - // Matrix job names come in the form of "base-job-name/matrix_param1=value1,matrix_param2=value2" - def splitJobName = jobName.split('/') - if (splitJobName.length > 1 && splitJobName.last() ==~ /^([a-zA-Z0-9_\-]+=[a-zA-Z0-9_\-&\.]+,?)+$/) { - def baseJobName = splitJobName.dropRight(1).join('/') - tag baseJobName - tag splitJobName.last() - value 'Job Name', baseJobName - def matrixParams = splitJobName.last().split(',') - matrixParams.collect { it.split('=') }.each { param -> - value "MATRIX_${param[0].toUpperCase()}", param[1] + // Jenkins-specific build scan metadata + if (jenkinsUrl) { + // Disable async upload in CI to ensure scan upload completes before CI agent is terminated + uploadInBackground = false + + String buildNumber = System.getenv('BUILD_NUMBER') + String buildUrl = System.getenv('BUILD_URL') + String jobName = System.getenv('JOB_NAME') + String nodeName = System.getenv('NODE_NAME') + String jobBranch = System.getenv('ghprbTargetBranch') ?: System.getenv('JOB_BRANCH') + + // Link to Jenkins worker logs and system metrics + if (nodeName) { + link 'System logs', "https://ci-stats.elastic.co/app/infra#/logs?&logFilter=(expression:'host.name:${nodeName}',kind:kuery)" + buildFinished { + link 'System metrics', "https://ci-stats.elastic.co/app/metrics/detail/host/${nodeName}" + } } - } else { - tag jobName - value 'Job Name', jobName - } - tag 'CI' - link 'CI Build', buildUrl - link 'GCP Upload', "https://console.cloud.google.com/storage/browser/_details/elasticsearch-ci-artifacts/jobs/${URLEncoder.encode(jobName, "UTF-8")}/build/${buildNumber}.tar.bz2" - value 'Job Number', buildNumber - if (jobBranch) { - tag jobBranch - value 'Git Branch', jobBranch - } + // Parse job name in the case of matrix builds + // Matrix job names come in the form of "base-job-name/matrix_param1=value1,matrix_param2=value2" + def splitJobName = jobName.split('/') + if (splitJobName.length > 1 && splitJobName.last() ==~ /^([a-zA-Z0-9_\-]+=[a-zA-Z0-9_\-&\.]+,?)+$/) { + def baseJobName = splitJobName.dropRight(1).join('/') + tag baseJobName + tag splitJobName.last() + value 'Job Name', baseJobName + def matrixParams = splitJobName.last().split(',') + matrixParams.collect { it.split('=') }.each { param -> + value "MATRIX_${param[0].toUpperCase()}", param[1] + } + } else { + tag jobName + value 'Job Name', jobName + } - System.getenv().getOrDefault('NODE_LABELS', '').split(' ').each { - value 'Jenkins Worker Label', it - } + tag 'CI' + link 'CI Build', buildUrl + link 'GCP Upload', + "https://console.cloud.google.com/storage/browser/_details/elasticsearch-ci-artifacts/jobs/${URLEncoder.encode(jobName, "UTF-8")}/build/${buildNumber}.tar.bz2" + value 'Job Number', buildNumber + if (jobBranch) { + tag jobBranch + value 'Git Branch', jobBranch + } - // Add SCM information - def isPrBuild = System.getenv('ROOT_BUILD_CAUSE_GHPRBCAUSE') != null - if (isPrBuild) { - value 'Git Commit ID', System.getenv('ghprbActualCommit') - tag "pr/${System.getenv('ghprbPullId')}" - tag 'pull-request' - link 'Source', "https://github.com/elastic/elasticsearch/tree/${System.getenv('ghprbActualCommit')}" - link 'Pull Request', System.getenv('ghprbPullLink') - } else { - value 'Git Commit ID', BuildParams.gitRevision - link 'Source', "https://github.com/elastic/elasticsearch/tree/${BuildParams.gitRevision}" - } - } else if (buildKiteUrl) { //Buildkite-specific build scan metadata - // Disable async upload in CI to ensure scan upload completes before CI agent is terminated - uploadInBackground = false - - def branch = System.getenv('BUILDKITE_PULL_REQUEST_BASE_BRANCH') ?: System.getenv('BUILDKITE_BRANCH') - def repoMatcher = System.getenv('BUILDKITE_REPO') =~ /(https:\/\/github\.com\/|git@github\.com:)(\S+)\.git/ - def repository = repoMatcher.matches() ? repoMatcher.group(2) : "" - def jobLabel = System.getenv('BUILDKITE_LABEL') ?: '' - def jobName = safeName(jobLabel) - - tag 'CI' - link 'CI Build', "${buildKiteUrl}#${System.getenv('BUILDKITE_JOB_ID')}" - value 'Job Number', System.getenv('BUILDKITE_BUILD_NUMBER') - value 'Build ID', System.getenv('BUILDKITE_BUILD_ID') - value 'Job ID', System.getenv('BUILDKITE_JOB_ID') - - value 'Pipeline', System.getenv('BUILDKITE_PIPELINE_SLUG') - tag System.getenv('BUILDKITE_PIPELINE_SLUG') - - value 'Job Name', jobName - tag jobName - if (jobLabel.contains("/")) { - jobLabel.split("/").collect {safeName(it) }.each {matrix -> - tag matrix + System.getenv().getOrDefault('NODE_LABELS', '').split(' ').each { + value 'Jenkins Worker Label', it } - } - def uptime = ManagementFactory.getRuntimeMXBean().getUptime() / 1000; - def metricsStartTime = LocalDateTime.now().minusSeconds(uptime.longValue()).minusMinutes(15).toString() - def metricsEndTime = LocalDateTime.now().plusMinutes(15).toString() + // Add SCM information + def isPrBuild = System.getenv('ROOT_BUILD_CAUSE_GHPRBCAUSE') != null + if (isPrBuild) { + value 'Git Commit ID', System.getenv('ghprbActualCommit') + tag "pr/${System.getenv('ghprbPullId')}" + tag 'pull-request' + link 'Source', "https://github.com/elastic/elasticsearch/tree/${System.getenv('ghprbActualCommit')}" + link 'Pull Request', System.getenv('ghprbPullLink') + } else { + value 'Git Commit ID', BuildParams.gitRevision + link 'Source', "https://github.com/elastic/elasticsearch/tree/${BuildParams.gitRevision}" + } + } else if (buildKiteUrl) { //Buildkite-specific build scan metadata + // Disable async upload in CI to ensure scan upload completes before CI agent is terminated + uploadInBackground = false - link 'Agent Metrics', "https://es-buildkite-agents.elastic.dev/app/metrics/detail/host/${System.getenv('BUILDKITE_AGENT_NAME')}?_a=(time:(from:%27${metricsStartTime}Z%27,interval:%3E%3D1m,to:%27${metricsEndTime}Z%27))" - link 'Agent Logs', "https://es-buildkite-agents.elastic.dev/app/logs/stream?logFilter=(filters:!(),query:(language:kuery,query:%27host.name:%20${System.getenv('BUILDKITE_AGENT_NAME')}%27),timeRange:(from:%27${metricsStartTime}Z%27,to:%27${metricsEndTime}Z%27))" + def branch = System.getenv('BUILDKITE_PULL_REQUEST_BASE_BRANCH') ?: System.getenv('BUILDKITE_BRANCH') + def repoMatcher = System.getenv('BUILDKITE_REPO') =~ /(https:\/\/github\.com\/|git@github\.com:)(\S+)\.git/ + def repository = repoMatcher.matches() ? repoMatcher.group(2) : "" + def jobLabel = System.getenv('BUILDKITE_LABEL') ?: '' + def jobName = safeName(jobLabel) - if (branch) { - tag branch - value 'Git Branch', branch - } + tag 'CI' + link 'CI Build', "${buildKiteUrl}#${System.getenv('BUILDKITE_JOB_ID')}" + value 'Job Number', System.getenv('BUILDKITE_BUILD_NUMBER') + value 'Build ID', System.getenv('BUILDKITE_BUILD_ID') + value 'Job ID', System.getenv('BUILDKITE_JOB_ID') - // Add SCM information - def prId = System.getenv('BUILDKITE_PULL_REQUEST') - if (prId != 'false') { - def prBaseUrl = (System.getenv('BUILDKITE_PULL_REQUEST_REPO') - ".git").replaceFirst("git://", "https://") - value 'Git Commit ID', System.getenv('BUILDKITE_COMMIT') - tag "pr/${prId}" - tag 'pull-request' - link 'Source', "${prBaseUrl}/tree/${System.getenv('BUILDKITE_COMMIT')}" - link 'Pull Request', "https://github.com/${repository}/pull/${prId}" - } else { - value 'Git Commit ID', BuildParams.gitRevision - link 'Source', "https://github.com/${repository}/tree/${BuildParams.gitRevision}" - } + value 'Pipeline', System.getenv('BUILDKITE_PIPELINE_SLUG') + tag System.getenv('BUILDKITE_PIPELINE_SLUG') - buildFinished { result -> - - buildScanPublished { scan -> - // Attach build scan link as build metadata - // See: https://buildkite.com/docs/pipelines/build-meta-data - new ProcessBuilder('buildkite-agent', 'meta-data', 'set', "build-scan-${System.getenv('BUILDKITE_JOB_ID')}", "${scan.buildScanUri}") - .start() - .waitFor() - - // Add a build annotation - // See: https://buildkite.com/docs/agent/v3/cli-annotate - def body = """
${System.getenv('BUILDKITE_LABEL')} :gradle: ${result.failure ? 'failed' : 'successful'} build: gradle ${gradle.startParameter.taskNames.join(' ')}
""" - def process = [ - 'buildkite-agent', - 'annotate', - '--context', - result.failure ? 'gradle-build-scans-failed' : 'gradle-build-scans', - '--append', - '--style', - result.failure ? 'error' : 'info' - ].execute() - process.withWriter { it.write(body) } // passing the body in as an argument has issues on Windows, so let's use stdin of the process instead - process.waitFor() + value 'Job Name', jobName + tag jobName + if (jobLabel.contains("/")) { + jobLabel.split("/").collect { safeName(it) }.each { matrix -> + tag matrix + } } + + def uptime = ManagementFactory.getRuntimeMXBean().getUptime() / 1000; + def metricsStartTime = LocalDateTime.now().minusSeconds(uptime.longValue()).minusMinutes(15).toString() + def metricsEndTime = LocalDateTime.now().plusMinutes(15).toString() + + link 'Agent Metrics', + "https://es-buildkite-agents.elastic.dev/app/metrics/detail/host/${System.getenv('BUILDKITE_AGENT_NAME')}?_a=(time:(from:%27${metricsStartTime}Z%27,interval:%3E%3D1m,to:%27${metricsEndTime}Z%27))" + link 'Agent Logs', + "https://es-buildkite-agents.elastic.dev/app/logs/stream?logFilter=(filters:!(),query:(language:kuery,query:%27host.name:%20${System.getenv('BUILDKITE_AGENT_NAME')}%27),timeRange:(from:%27${metricsStartTime}Z%27,to:%27${metricsEndTime}Z%27))" + + if (branch) { + tag branch + value 'Git Branch', branch + } + + // Add SCM information + def prId = System.getenv('BUILDKITE_PULL_REQUEST') + if (prId != 'false') { + def prBaseUrl = (System.getenv('BUILDKITE_PULL_REQUEST_REPO') - ".git").replaceFirst("git://", "https://") + value 'Git Commit ID', System.getenv('BUILDKITE_COMMIT') + tag "pr/${prId}" + tag 'pull-request' + link 'Source', "${prBaseUrl}/tree/${System.getenv('BUILDKITE_COMMIT')}" + link 'Pull Request', "https://github.com/${repository}/pull/${prId}" + } else { + value 'Git Commit ID', BuildParams.gitRevision + link 'Source', "https://github.com/${repository}/tree/${BuildParams.gitRevision}" + } + + buildFinished { result -> + + buildScanPublished { scan + -> + // Attach build scan link as build metadata + // See: https://buildkite.com/docs/pipelines/build-meta-data + new ProcessBuilder('buildkite-agent', 'meta-data', 'set', "build-scan-${System.getenv('BUILDKITE_JOB_ID')}", "${scan.buildScanUri}") + .start() + .waitFor() + + // Add a build annotation + // See: https://buildkite.com/docs/agent/v3/cli-annotate + def body = """
${System.getenv('BUILDKITE_LABEL')} :gradle: ${result.failures ? 'failed' : 'successful'} build: gradle ${gradle.startParameter.taskNames.join(' ')}
""" + def process = [ + 'buildkite-agent', + 'annotate', + '--context', + result.failures ? 'gradle-build-scans-failed' : 'gradle-build-scans', + '--append', + '--style', + result.failures ? 'error' : 'info' + ].execute() + process.withWriter { it.write(body) } + // passing the body in as an argument has issues on Windows, so let's use stdin of the process instead + process.waitFor() + } + } + } else { + tag 'LOCAL' } - } else { - tag 'LOCAL' } } } diff --git a/gradle/build.versions.toml b/gradle/build.versions.toml index ba81673120569..5a32d2e0a58cd 100644 --- a/gradle/build.versions.toml +++ b/gradle/build.versions.toml @@ -17,7 +17,7 @@ commons-codec = "commons-codec:commons-codec:1.11" commmons-io = "commons-io:commons-io:2.2" docker-compose = "com.avast.gradle:gradle-docker-compose-plugin:0.17.5" forbiddenApis = "de.thetaphi:forbiddenapis:3.6" -gradle-enterprise = "com.gradle:gradle-enterprise-gradle-plugin:3.16.2" +gradle-enterprise = "com.gradle:develocity-gradle-plugin:3.17.4" hamcrest = "org.hamcrest:hamcrest:2.1" httpcore = "org.apache.httpcomponents:httpcore:4.4.12" httpclient = "org.apache.httpcomponents:httpclient:4.5.14" diff --git a/gradle/verification-metadata.xml b/gradle/verification-metadata.xml index 532112d0138d3..29e0afe51364f 100644 --- a/gradle/verification-metadata.xml +++ b/gradle/verification-metadata.xml @@ -741,9 +741,9 @@ - - - + + + diff --git a/plugins/examples/settings.gradle b/plugins/examples/settings.gradle index af2596fdbafe3..09abbfa6b5863 100644 --- a/plugins/examples/settings.gradle +++ b/plugins/examples/settings.gradle @@ -7,7 +7,7 @@ */ plugins { - id "com.gradle.enterprise" version "3.16.2" + id "com.gradle.develocity" version "3.17.4" } // Include all subdirectories as example projects diff --git a/settings.gradle b/settings.gradle index ef758a7205cd0..a75c660016599 100644 --- a/settings.gradle +++ b/settings.gradle @@ -17,7 +17,7 @@ pluginManagement { } plugins { - id "com.gradle.enterprise" version "3.16.2" + id "com.gradle.develocity" version "3.17.4" id 'elasticsearch.java-toolchain' } From 335c03afa9a2d05c63a174b18aa65294e6330a9d Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Mon, 10 Jun 2024 08:56:40 -0400 Subject: [PATCH 04/31] Handle unmatching remote cluster wildcards properly for IndicesRequest.SingleIndexNoWildcards requests (#109185) When security is enabled and a IndicesRequest.SingleIndexNoWildcards request uses a wildcard for remote clusters that does not match (for example "*:myindex" when there are no configured remotes), RBACEngine#authorizeIndexAction throws an assert error because it is given an empty list of indices. This fix checks the IndicesRequest.SingleIndexNoWildcards special case in IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards and if splitLocalAndRemoteIndexNames filters out all indexes due to a non-matching remote cluster wildcard, that code now just throws the NoSuchRemoteClusterException (which is what RBACEngine does when asserts are disabled). This handles painless/execute requests to remote clusters. Added note to painless/execute API docs about wildcards not being allowed --- docs/changelog/109185.yaml | 6 ++ .../painless-execute-script.asciidoc | 4 + .../authz/IndicesAndAliasesResolver.java | 14 ++- .../authz/IndicesAndAliasesResolverTests.java | 98 +++++++++++++++++++ 4 files changed, 121 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/109185.yaml diff --git a/docs/changelog/109185.yaml b/docs/changelog/109185.yaml new file mode 100644 index 0000000000000..4da72c4b20ffb --- /dev/null +++ b/docs/changelog/109185.yaml @@ -0,0 +1,6 @@ +pr: 109185 +summary: Handle unmatching remote cluster wildcards properly for `IndicesRequest.SingleIndexNoWildcards` + requests +area: Authorization +type: bug +issues: [] diff --git a/docs/painless/painless-guide/painless-execute-script.asciidoc b/docs/painless/painless-guide/painless-execute-script.asciidoc index 1c7ec61ffa75b..4417daeb63efa 100644 --- a/docs/painless/painless-guide/painless-execute-script.asciidoc +++ b/docs/painless/painless-guide/painless-execute-script.asciidoc @@ -123,6 +123,10 @@ alias. For example, `remote1:my_index` indicates that you want to execute the painless script against the "my_index" index on the "remote1" cluster. This request will be forwarded to the "remote1" cluster if you have {ref}/remote-clusters-connect.html[configured a connection] to that remote cluster. + +NOTE: Wildcards are not accepted in the index expression for this endpoint. The +expression `*:myindex` will return the error "No such remote cluster" and the +expression `logs*` or `remote1:logs*` will return the error "index not found". ==== `params`:: (`Map`, read-only) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index bf1bf7b7d3cee..42a1d89a9aa00 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -27,6 +27,7 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteConnectionStrategy; import org.elasticsearch.transport.TransportRequest; @@ -164,6 +165,18 @@ ResolvedIndices resolveIndicesAndAliasesWithoutWildcards(String action, IndicesR final ResolvedIndices split; if (indicesRequest instanceof IndicesRequest.SingleIndexNoWildcards single && single.allowsRemoteIndices()) { split = remoteClusterResolver.splitLocalAndRemoteIndexNames(indicesRequest.indices()); + // all indices can come back empty when the remote index expression included a cluster alias with a wildcard + // and no remote clusters are configured that match it + if (split.getLocal().isEmpty() && split.getRemote().isEmpty()) { + for (String indexExpression : indices) { + String[] clusterAndIndex = indexExpression.split(":", 2); + if (clusterAndIndex.length == 2) { + if (clusterAndIndex[0].contains("*")) { + throw new NoSuchRemoteClusterException(clusterAndIndex[0]); + } + } + } + } } else { split = new ResolvedIndices(Arrays.asList(indicesRequest.indices()), List.of()); } @@ -473,5 +486,4 @@ ResolvedIndices splitLocalAndRemoteIndexNames(String... indices) { return new ResolvedIndices(local == null ? List.of() : local, remote); } } - } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index be0516ab180c9..82ac95a21086d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -58,6 +58,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.xpack.core.graph.action.GraphExploreAction; import org.elasticsearch.xpack.core.security.authc.Authentication.RealmRef; @@ -104,6 +105,7 @@ import static org.elasticsearch.xpack.core.security.test.TestRestrictedIndices.RESTRICTED_INDICES; import static org.elasticsearch.xpack.security.authz.AuthorizedIndicesTests.getRequestInfo; import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_MAIN_ALIAS; +import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.arrayContaining; @@ -455,6 +457,102 @@ public void testAllIsNotAllowedInShardLevelRequests() { ); } + public void testResolveIndicesAndAliasesWithoutWildcardsWithSingleIndexNoWildcardsRequest() { + // test 1: matching local index + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "index10" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(0)); + assertThat(resolvedIndices.getLocal().size(), equalTo(1)); + assertThat(resolvedIndices.getLocal().get(0), equalTo("index10")); + } + + // test 2: matching remote index + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "remote:indexName" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(1)); + assertThat(resolvedIndices.getRemote().get(0), equalTo("remote:indexName")); + assertThat(resolvedIndices.getLocal().size(), equalTo(0)); + } + + // test 3: missing local index + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "zzz_no_such_index_zzz" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(0)); + assertThat(resolvedIndices.getLocal().size(), equalTo(1)); + assertThat(resolvedIndices.getLocal().get(0), equalTo("zzz_no_such_index_zzz")); + } + + // test 4: missing remote index + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "remote:zzz_no_such_index_zzz" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(1)); + assertThat(resolvedIndices.getRemote().get(0), equalTo("remote:zzz_no_such_index_zzz")); + assertThat(resolvedIndices.getLocal().size(), equalTo(0)); + } + + // test 5: both local and remote indexes + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "index10", "remote:indexName" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(1)); + assertThat(resolvedIndices.getRemote().get(0), equalTo("remote:indexName")); + assertThat(resolvedIndices.getLocal().size(), equalTo(1)); + assertThat(resolvedIndices.getLocal().get(0), equalTo("index10")); + } + + // test 6: remote cluster name with wildcards that does not match any configured remotes + { + NoSuchRemoteClusterException exception = expectThrows( + NoSuchRemoteClusterException.class, + () -> defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "x*x:test" }) + ) + ); + assertThat(exception.getMessage(), containsString("no such remote cluster: [x*x]")); + } + + // test 7: mix and test 2 and test 6 - should not result in exception (wildcard without matches has no effect) + { + ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + createSingleIndexNoWildcardsRequest(new String[] { "x*x:test", "remote:indexName" }) + ); + assertThat(resolvedIndices.getRemote().size(), equalTo(1)); + assertThat(resolvedIndices.getRemote().get(0), equalTo("remote:indexName")); + assertThat(resolvedIndices.getLocal().size(), equalTo(0)); + } + } + + private static IndicesRequest.SingleIndexNoWildcards createSingleIndexNoWildcardsRequest(String[] indexExpression) { + IndicesRequest.SingleIndexNoWildcards singleIndexNoWildcardsRequest = new IndicesRequest.SingleIndexNoWildcards() { + @Override + public String[] indices() { + return indexExpression; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.DEFAULT; + } + }; + return singleIndexNoWildcardsRequest; + } + public void testExplicitDashIndices() { SearchRequest request = new SearchRequest("-index10", "-index20"); List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) From eedc2b9354b556799073de655e3904d599017aaa Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Mon, 10 Jun 2024 06:24:05 -0700 Subject: [PATCH 05/31] Fix typo in TSDB documentation (#109504) --- docs/reference/data-streams/tsds.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/data-streams/tsds.asciidoc b/docs/reference/data-streams/tsds.asciidoc index e943a25f1fb05..2e81e5b7e3848 100644 --- a/docs/reference/data-streams/tsds.asciidoc +++ b/docs/reference/data-streams/tsds.asciidoc @@ -285,7 +285,7 @@ field values that are older than 2 hours (the `index.look_back_time` default). A TSDS is designed to ingest current metrics data. When the TSDS is first created the initial backing index has: -* an `index.time_series.start_time` value set to `now - index.look_ahead_time` +* an `index.time_series.start_time` value set to `now - index.look_back_time` * an `index.time_series.end_time` value set to `now + index.look_ahead_time` Only data that falls inside that range can be indexed. From bf19072247c61c05ff7f307be57c5cea31df182c Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 10 Jun 2024 09:50:28 -0400 Subject: [PATCH 06/31] [ESQL] move optimizer rules to a package (#109408) This moves (most of) the Optimizer Rules from being public static sub-classes of OptimizerRules to being public top level classes in the optimizer.rules package. The main benefit of this is to reduce git conflicts when touching rules, since they will no longer be all in the same file. It also improves readability, as short files are easier to read in general. I would like to move the tests for these rules into class-specific test files, but there's some more general test refactoring that I'd like to get done first, so that'll be in a follow-up PR. --------- Co-authored-by: Elastic Machine --- .../esql/optimizer/LogicalPlanOptimizer.java | 9 +- .../xpack/esql/optimizer/OptimizerRules.java | 458 ------------------ .../BooleanFunctionEqualsElimination.java | 47 ++ .../rules/CombineDisjunctionsToIn.java | 101 ++++ .../esql/optimizer/rules/PropagateEquals.java | 348 +++++++++++++ .../esql/optimizer/OptimizerRulesTests.java | 73 +-- 6 files changed, 540 insertions(+), 496 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsElimination.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctionsToIn.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEquals.java diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java index 951c97d7e69f5..4e2cb2c8223e6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java @@ -59,8 +59,11 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.optimizer.rules.BooleanFunctionEqualsElimination; +import org.elasticsearch.xpack.esql.optimizer.rules.CombineDisjunctionsToIn; import org.elasticsearch.xpack.esql.optimizer.rules.ConstantFolding; import org.elasticsearch.xpack.esql.optimizer.rules.LiteralsOnTheRight; +import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEquals; import org.elasticsearch.xpack.esql.optimizer.rules.PruneLiteralsInOrderBy; import org.elasticsearch.xpack.esql.optimizer.rules.SetAsOptimized; import org.elasticsearch.xpack.esql.optimizer.rules.SimplifyComparisonsArithmetics; @@ -159,10 +162,10 @@ protected static Batch operators() { new BooleanSimplification(), new LiteralsOnTheRight(), // needs to occur before BinaryComparison combinations (see class) - new org.elasticsearch.xpack.esql.optimizer.OptimizerRules.PropagateEquals(), + new PropagateEquals(), new PropagateNullable(), - new org.elasticsearch.xpack.esql.optimizer.OptimizerRules.BooleanFunctionEqualsElimination(), - new org.elasticsearch.xpack.esql.optimizer.OptimizerRules.CombineDisjunctionsToIn(), + new BooleanFunctionEqualsElimination(), + new CombineDisjunctionsToIn(), new SimplifyComparisonsArithmetics(EsqlDataTypes::areCompatible), // prune/elimination new PruneFilters(), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java index 415de38e7a173..4c5d9efb449f7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRules.java @@ -9,29 +9,9 @@ import org.elasticsearch.xpack.esql.core.common.Failures; import org.elasticsearch.xpack.esql.core.expression.AttributeSet; -import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.Expressions; -import org.elasticsearch.xpack.esql.core.expression.Literal; -import org.elasticsearch.xpack.esql.core.expression.function.Function; -import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates; -import org.elasticsearch.xpack.esql.core.expression.predicate.Range; -import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And; -import org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic; -import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not; -import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or; -import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison; import org.elasticsearch.xpack.esql.core.plan.QueryPlan; import org.elasticsearch.xpack.esql.core.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.core.type.DataType; -import org.elasticsearch.xpack.esql.core.util.CollectionUtils; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual; -import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; -import org.elasticsearch.xpack.esql.optimizer.rules.LiteralsOnTheRight; import org.elasticsearch.xpack.esql.plan.logical.Aggregate; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.EsRelation; @@ -56,21 +36,7 @@ import org.elasticsearch.xpack.esql.plan.physical.RowExec; import org.elasticsearch.xpack.esql.plan.physical.ShowExec; -import java.time.ZoneId; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; - import static org.elasticsearch.xpack.esql.core.common.Failure.fail; -import static org.elasticsearch.xpack.esql.core.expression.Literal.FALSE; -import static org.elasticsearch.xpack.esql.core.expression.Literal.TRUE; -import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.combineOr; -import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.splitOr; class OptimizerRules { @@ -184,428 +150,4 @@ protected AttributeSet references(PhysicalPlan plan) { } } - /** - * Combine disjunctions on the same field into an In expression. - * This rule looks for both simple equalities: - * 1. a == 1 OR a == 2 becomes a IN (1, 2) - * and combinations of In - * 2. a == 1 OR a IN (2) becomes a IN (1, 2) - * 3. a IN (1) OR a IN (2) becomes a IN (1, 2) - * - * This rule does NOT check for type compatibility as that phase has been - * already be verified in the analyzer. - */ - public static class CombineDisjunctionsToIn extends org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule< - Or> { - CombineDisjunctionsToIn() { - super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.UP); - } - - protected In createIn(Expression key, List values, ZoneId zoneId) { - return new In(key.source(), key, values); - } - - protected Equals createEquals(Expression k, Set v, ZoneId finalZoneId) { - return new Equals(k.source(), k, v.iterator().next(), finalZoneId); - } - - @Override - protected Expression rule(Or or) { - Expression e = or; - // look only at equals and In - List exps = splitOr(e); - - Map> found = new LinkedHashMap<>(); - ZoneId zoneId = null; - List ors = new LinkedList<>(); - - for (Expression exp : exps) { - if (exp instanceof Equals eq) { - // consider only equals against foldables - if (eq.right().foldable()) { - found.computeIfAbsent(eq.left(), k -> new LinkedHashSet<>()).add(eq.right()); - } else { - ors.add(exp); - } - if (zoneId == null) { - zoneId = eq.zoneId(); - } - } else if (exp instanceof In in) { - found.computeIfAbsent(in.value(), k -> new LinkedHashSet<>()).addAll(in.list()); - if (zoneId == null) { - zoneId = in.zoneId(); - } - } else { - ors.add(exp); - } - } - - if (found.isEmpty() == false) { - // combine equals alongside the existing ors - final ZoneId finalZoneId = zoneId; - found.forEach( - (k, v) -> { ors.add(v.size() == 1 ? createEquals(k, v, finalZoneId) : createIn(k, new ArrayList<>(v), finalZoneId)); } - ); - - // TODO: this makes a QL `or`, not an ESQL `or` - Expression combineOr = combineOr(ors); - // check the result semantically since the result might different in order - // but be actually the same which can trigger a loop - // e.g. a == 1 OR a == 2 OR null --> null OR a in (1,2) --> literalsOnTheRight --> cycle - if (e.semanticEquals(combineOr) == false) { - e = combineOr; - } - } - - return e; - } - } - - /** - * This rule must always be placed after {@link LiteralsOnTheRight} - * since it looks at TRUE/FALSE literals' existence on the right hand-side of the {@link Equals}/{@link NotEquals} expressions. - */ - public static final class BooleanFunctionEqualsElimination extends - org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule { - - BooleanFunctionEqualsElimination() { - super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.UP); - } - - @Override - protected Expression rule(BinaryComparison bc) { - if ((bc instanceof Equals || bc instanceof NotEquals) && bc.left() instanceof Function) { - // for expression "==" or "!=" TRUE/FALSE, return the expression itself or its negated variant - - // TODO: Replace use of QL Not with ESQL Not - if (TRUE.equals(bc.right())) { - return bc instanceof Equals ? bc.left() : new Not(bc.left().source(), bc.left()); - } - if (FALSE.equals(bc.right())) { - return bc instanceof Equals ? new Not(bc.left().source(), bc.left()) : bc.left(); - } - } - - return bc; - } - } - - /** - * Propagate Equals to eliminate conjuncted Ranges or BinaryComparisons. - * When encountering a different Equals, non-containing {@link Range} or {@link BinaryComparison}, the conjunction becomes false. - * When encountering a containing {@link Range}, {@link BinaryComparison} or {@link NotEquals}, these get eliminated by the equality. - * - */ - public static final class PropagateEquals extends org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule< - BinaryLogic> { - - PropagateEquals() { - super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.DOWN); - } - - public Expression rule(BinaryLogic e) { - if (e instanceof And) { - return propagate((And) e); - } else if (e instanceof Or) { - return propagate((Or) e); - } - return e; - } - - // combine conjunction - private static Expression propagate(And and) { - List ranges = new ArrayList<>(); - // Only equalities, not-equalities and inequalities with a foldable .right are extracted separately; - // the others go into the general 'exps'. - // TODO: In 105217, this should change to EsqlBinaryComparison, but it doesn't exist in this branch yet - List equals = new ArrayList<>(); - List notEquals = new ArrayList<>(); - List inequalities = new ArrayList<>(); - List exps = new ArrayList<>(); - - boolean changed = false; - - for (Expression ex : Predicates.splitAnd(and)) { - if (ex instanceof Range) { - ranges.add((Range) ex); - } else if (ex instanceof Equals otherEq) { - // equals on different values evaluate to FALSE - // ignore date/time fields as equality comparison might actually be a range check - if (otherEq.right().foldable() && DataType.isDateTime(otherEq.left().dataType()) == false) { - for (BinaryComparison eq : equals) { - if (otherEq.left().semanticEquals(eq.left())) { - Integer comp = BinaryComparison.compare(eq.right().fold(), otherEq.right().fold()); - if (comp != null) { - // var cannot be equal to two different values at the same time - if (comp != 0) { - return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); - } - } - } - } - equals.add(otherEq); - } else { - exps.add(otherEq); - } - } else if (ex instanceof GreaterThan - || ex instanceof GreaterThanOrEqual - || ex instanceof LessThan - || ex instanceof LessThanOrEqual) { - BinaryComparison bc = (BinaryComparison) ex; - if (bc.right().foldable()) { - inequalities.add(bc); - } else { - exps.add(ex); - } - } else if (ex instanceof NotEquals otherNotEq) { - if (otherNotEq.right().foldable()) { - notEquals.add(otherNotEq); - } else { - exps.add(ex); - } - } else { - exps.add(ex); - } - } - - // check - for (BinaryComparison eq : equals) { - Object eqValue = eq.right().fold(); - - for (Iterator iterator = ranges.iterator(); iterator.hasNext();) { - Range range = iterator.next(); - - if (range.value().semanticEquals(eq.left())) { - // if equals is outside the interval, evaluate the whole expression to FALSE - if (range.lower().foldable()) { - Integer compare = BinaryComparison.compare(range.lower().fold(), eqValue); - if (compare != null && ( - // eq outside the lower boundary - compare > 0 || - // eq matches the boundary but should not be included - (compare == 0 && range.includeLower() == false))) { - return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); - } - } - if (range.upper().foldable()) { - Integer compare = BinaryComparison.compare(range.upper().fold(), eqValue); - if (compare != null && ( - // eq outside the upper boundary - compare < 0 || - // eq matches the boundary but should not be included - (compare == 0 && range.includeUpper() == false))) { - return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); - } - } - - // it's in the range and thus, remove it - iterator.remove(); - changed = true; - } - } - - // evaluate all NotEquals against the Equal - for (Iterator iter = notEquals.iterator(); iter.hasNext();) { - NotEquals neq = iter.next(); - if (eq.left().semanticEquals(neq.left())) { - Integer comp = BinaryComparison.compare(eqValue, neq.right().fold()); - if (comp != null) { - if (comp == 0) { // clashing and conflicting: a = 1 AND a != 1 - return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); - } else { // clashing and redundant: a = 1 AND a != 2 - iter.remove(); - changed = true; - } - } - } - } - - // evaluate all inequalities against the Equal - for (Iterator iter = inequalities.iterator(); iter.hasNext();) { - BinaryComparison bc = iter.next(); - if (eq.left().semanticEquals(bc.left())) { - Integer compare = BinaryComparison.compare(eqValue, bc.right().fold()); - if (compare != null) { - if (bc instanceof LessThan || bc instanceof LessThanOrEqual) { // a = 2 AND a />= ? - if ((compare == 0 && bc instanceof GreaterThan) || // a = 2 AND a > 2 - compare < 0) { // a = 2 AND a >/>= 3 - return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); - } - } - - iter.remove(); - changed = true; - } - } - } - } - - return changed ? Predicates.combineAnd(CollectionUtils.combine(exps, equals, notEquals, inequalities, ranges)) : and; - } - - // combine disjunction: - // a = 2 OR a > 3 -> nop; a = 2 OR a > 1 -> a > 1 - // a = 2 OR a < 3 -> a < 3; a = 2 OR a < 1 -> nop - // a = 2 OR 3 < a < 5 -> nop; a = 2 OR 1 < a < 3 -> 1 < a < 3; a = 2 OR 0 < a < 1 -> nop - // a = 2 OR a != 2 -> TRUE; a = 2 OR a = 5 -> nop; a = 2 OR a != 5 -> a != 5 - private static Expression propagate(Or or) { - List exps = new ArrayList<>(); - List equals = new ArrayList<>(); // foldable right term Equals - List notEquals = new ArrayList<>(); // foldable right term NotEquals - List ranges = new ArrayList<>(); - List inequalities = new ArrayList<>(); // foldable right term (=limit) BinaryComparision - - // split expressions by type - for (Expression ex : Predicates.splitOr(or)) { - if (ex instanceof Equals eq) { - if (eq.right().foldable()) { - equals.add(eq); - } else { - exps.add(ex); - } - } else if (ex instanceof NotEquals neq) { - if (neq.right().foldable()) { - notEquals.add(neq); - } else { - exps.add(ex); - } - } else if (ex instanceof Range) { - ranges.add((Range) ex); - } else if (ex instanceof BinaryComparison bc) { - if (bc.right().foldable()) { - inequalities.add(bc); - } else { - exps.add(ex); - } - } else { - exps.add(ex); - } - } - - boolean updated = false; // has the expression been modified? - - // evaluate the impact of each Equal over the different types of Expressions - for (Iterator iterEq = equals.iterator(); iterEq.hasNext();) { - Equals eq = iterEq.next(); - Object eqValue = eq.right().fold(); - boolean removeEquals = false; - - // Equals OR NotEquals - for (NotEquals neq : notEquals) { - if (eq.left().semanticEquals(neq.left())) { // a = 2 OR a != ? -> ... - Integer comp = BinaryComparison.compare(eqValue, neq.right().fold()); - if (comp != null) { - if (comp == 0) { // a = 2 OR a != 2 -> TRUE - return TRUE; - } else { // a = 2 OR a != 5 -> a != 5 - removeEquals = true; - break; - } - } - } - } - if (removeEquals) { - iterEq.remove(); - updated = true; - continue; - } - - // Equals OR Range - for (int i = 0; i < ranges.size(); i++) { // might modify list, so use index loop - Range range = ranges.get(i); - if (eq.left().semanticEquals(range.value())) { - Integer lowerComp = range.lower().foldable() ? BinaryComparison.compare(eqValue, range.lower().fold()) : null; - Integer upperComp = range.upper().foldable() ? BinaryComparison.compare(eqValue, range.upper().fold()) : null; - - if (lowerComp != null && lowerComp == 0) { - if (range.includeLower() == false) { // a = 2 OR 2 < a < ? -> 2 <= a < ? - ranges.set( - i, - new Range( - range.source(), - range.value(), - range.lower(), - true, - range.upper(), - range.includeUpper(), - range.zoneId() - ) - ); - } // else : a = 2 OR 2 <= a < ? -> 2 <= a < ? - removeEquals = true; // update range with lower equality instead or simply superfluous - break; - } else if (upperComp != null && upperComp == 0) { - if (range.includeUpper() == false) { // a = 2 OR ? < a < 2 -> ? < a <= 2 - ranges.set( - i, - new Range( - range.source(), - range.value(), - range.lower(), - range.includeLower(), - range.upper(), - true, - range.zoneId() - ) - ); - } // else : a = 2 OR ? < a <= 2 -> ? < a <= 2 - removeEquals = true; // update range with upper equality instead - break; - } else if (lowerComp != null && upperComp != null) { - if (0 < lowerComp && upperComp < 0) { // a = 2 OR 1 < a < 3 - removeEquals = true; // equality is superfluous - break; - } - } - } - } - if (removeEquals) { - iterEq.remove(); - updated = true; - continue; - } - - // Equals OR Inequality - for (int i = 0; i < inequalities.size(); i++) { - BinaryComparison bc = inequalities.get(i); - if (eq.left().semanticEquals(bc.left())) { - Integer comp = BinaryComparison.compare(eqValue, bc.right().fold()); - if (comp != null) { - if (bc instanceof GreaterThan || bc instanceof GreaterThanOrEqual) { - if (comp < 0) { // a = 1 OR a > 2 -> nop - continue; - } else if (comp == 0 && bc instanceof GreaterThan) { // a = 2 OR a > 2 -> a >= 2 - inequalities.set(i, new GreaterThanOrEqual(bc.source(), bc.left(), bc.right(), bc.zoneId())); - } // else (0 < comp || bc instanceof GreaterThanOrEqual) : - // a = 3 OR a > 2 -> a > 2; a = 2 OR a => 2 -> a => 2 - - removeEquals = true; // update range with equality instead or simply superfluous - break; - } else if (bc instanceof LessThan || bc instanceof LessThanOrEqual) { - if (comp > 0) { // a = 2 OR a < 1 -> nop - continue; - } - if (comp == 0 && bc instanceof LessThan) { // a = 2 OR a < 2 -> a <= 2 - inequalities.set(i, new LessThanOrEqual(bc.source(), bc.left(), bc.right(), bc.zoneId())); - } // else (comp < 0 || bc instanceof LessThanOrEqual) : a = 2 OR a < 3 -> a < 3; a = 2 OR a <= 2 -> a <= 2 - removeEquals = true; // update range with equality instead or simply superfluous - break; - } - } - } - } - if (removeEquals) { - iterEq.remove(); - updated = true; - } - } - - return updated ? Predicates.combineOr(CollectionUtils.combine(exps, equals, notEquals, inequalities, ranges)) : or; - } - } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsElimination.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsElimination.java new file mode 100644 index 0000000000000..cf62f9219f3c8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/BooleanFunctionEqualsElimination.java @@ -0,0 +1,47 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.optimizer.rules; + +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.function.Function; +import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not; +import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; + +import static org.elasticsearch.xpack.esql.core.expression.Literal.FALSE; +import static org.elasticsearch.xpack.esql.core.expression.Literal.TRUE; + +/** + * This rule must always be placed after {@link LiteralsOnTheRight} + * since it looks at TRUE/FALSE literals' existence on the right hand-side of the {@link Equals}/{@link NotEquals} expressions. + */ +public final class BooleanFunctionEqualsElimination extends + org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule { + + public BooleanFunctionEqualsElimination() { + super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.UP); + } + + @Override + public Expression rule(BinaryComparison bc) { + if ((bc instanceof Equals || bc instanceof NotEquals) && bc.left() instanceof Function) { + // for expression "==" or "!=" TRUE/FALSE, return the expression itself or its negated variant + + // TODO: Replace use of QL Not with ESQL Not + if (TRUE.equals(bc.right())) { + return bc instanceof Equals ? bc.left() : new Not(bc.left().source(), bc.left()); + } + if (FALSE.equals(bc.right())) { + return bc instanceof Equals ? new Not(bc.left().source(), bc.left()) : bc.left(); + } + } + + return bc; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctionsToIn.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctionsToIn.java new file mode 100644 index 0000000000000..5cc3184d9ea70 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/CombineDisjunctionsToIn.java @@ -0,0 +1,101 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.optimizer.rules; + +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; + +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.combineOr; +import static org.elasticsearch.xpack.esql.core.expression.predicate.Predicates.splitOr; + +/** + * Combine disjunctions on the same field into an In expression. + * This rule looks for both simple equalities: + * 1. a == 1 OR a == 2 becomes a IN (1, 2) + * and combinations of In + * 2. a == 1 OR a IN (2) becomes a IN (1, 2) + * 3. a IN (1) OR a IN (2) becomes a IN (1, 2) + *

+ * This rule does NOT check for type compatibility as that phase has been + * already be verified in the analyzer. + */ +public class CombineDisjunctionsToIn extends org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule { + public CombineDisjunctionsToIn() { + super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.UP); + } + + protected In createIn(Expression key, List values, ZoneId zoneId) { + return new In(key.source(), key, values); + } + + protected Equals createEquals(Expression k, Set v, ZoneId finalZoneId) { + return new Equals(k.source(), k, v.iterator().next(), finalZoneId); + } + + @Override + public Expression rule(Or or) { + Expression e = or; + // look only at equals and In + List exps = splitOr(e); + + Map> found = new LinkedHashMap<>(); + ZoneId zoneId = null; + List ors = new LinkedList<>(); + + for (Expression exp : exps) { + if (exp instanceof Equals eq) { + // consider only equals against foldables + if (eq.right().foldable()) { + found.computeIfAbsent(eq.left(), k -> new LinkedHashSet<>()).add(eq.right()); + } else { + ors.add(exp); + } + if (zoneId == null) { + zoneId = eq.zoneId(); + } + } else if (exp instanceof In in) { + found.computeIfAbsent(in.value(), k -> new LinkedHashSet<>()).addAll(in.list()); + if (zoneId == null) { + zoneId = in.zoneId(); + } + } else { + ors.add(exp); + } + } + + if (found.isEmpty() == false) { + // combine equals alongside the existing ors + final ZoneId finalZoneId = zoneId; + found.forEach( + (k, v) -> { ors.add(v.size() == 1 ? createEquals(k, v, finalZoneId) : createIn(k, new ArrayList<>(v), finalZoneId)); } + ); + + // TODO: this makes a QL `or`, not an ESQL `or` + Expression combineOr = combineOr(ors); + // check the result semantically since the result might different in order + // but be actually the same which can trigger a loop + // e.g. a == 1 OR a == 2 OR null --> null OR a in (1,2) --> literalsOnTheRight --> cycle + if (e.semanticEquals(combineOr) == false) { + e = combineOr; + } + } + + return e; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEquals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEquals.java new file mode 100644 index 0000000000000..5f08363abdbaf --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/PropagateEquals.java @@ -0,0 +1,348 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.optimizer.rules; + +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.Literal; +import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates; +import org.elasticsearch.xpack.esql.core.expression.predicate.Range; +import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And; +import org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic; +import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or; +import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.core.util.CollectionUtils; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.elasticsearch.xpack.esql.core.expression.Literal.TRUE; + +/** + * Propagate Equals to eliminate conjuncted Ranges or BinaryComparisons. + * When encountering a different Equals, non-containing {@link Range} or {@link BinaryComparison}, the conjunction becomes false. + * When encountering a containing {@link Range}, {@link BinaryComparison} or {@link NotEquals}, these get eliminated by the equality. + */ +public final class PropagateEquals extends org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.OptimizerExpressionRule { + + public PropagateEquals() { + super(org.elasticsearch.xpack.esql.core.optimizer.OptimizerRules.TransformDirection.DOWN); + } + + public Expression rule(BinaryLogic e) { + if (e instanceof And) { + return propagate((And) e); + } else if (e instanceof Or) { + return propagate((Or) e); + } + return e; + } + + // combine conjunction + private static Expression propagate(And and) { + List ranges = new ArrayList<>(); + // Only equalities, not-equalities and inequalities with a foldable .right are extracted separately; + // the others go into the general 'exps'. + // TODO: In 105217, this should change to EsqlBinaryComparison, but it doesn't exist in this branch yet + List equals = new ArrayList<>(); + List notEquals = new ArrayList<>(); + List inequalities = new ArrayList<>(); + List exps = new ArrayList<>(); + + boolean changed = false; + + for (Expression ex : Predicates.splitAnd(and)) { + if (ex instanceof Range) { + ranges.add((Range) ex); + } else if (ex instanceof Equals otherEq) { + // equals on different values evaluate to FALSE + // ignore date/time fields as equality comparison might actually be a range check + if (otherEq.right().foldable() && DataType.isDateTime(otherEq.left().dataType()) == false) { + for (BinaryComparison eq : equals) { + if (otherEq.left().semanticEquals(eq.left())) { + Integer comp = BinaryComparison.compare(eq.right().fold(), otherEq.right().fold()); + if (comp != null) { + // var cannot be equal to two different values at the same time + if (comp != 0) { + return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); + } + } + } + } + equals.add(otherEq); + } else { + exps.add(otherEq); + } + } else if (ex instanceof GreaterThan + || ex instanceof GreaterThanOrEqual + || ex instanceof LessThan + || ex instanceof LessThanOrEqual) { + BinaryComparison bc = (BinaryComparison) ex; + if (bc.right().foldable()) { + inequalities.add(bc); + } else { + exps.add(ex); + } + } else if (ex instanceof NotEquals otherNotEq) { + if (otherNotEq.right().foldable()) { + notEquals.add(otherNotEq); + } else { + exps.add(ex); + } + } else { + exps.add(ex); + } + } + + // check + for (BinaryComparison eq : equals) { + Object eqValue = eq.right().fold(); + + for (Iterator iterator = ranges.iterator(); iterator.hasNext();) { + Range range = iterator.next(); + + if (range.value().semanticEquals(eq.left())) { + // if equals is outside the interval, evaluate the whole expression to FALSE + if (range.lower().foldable()) { + Integer compare = BinaryComparison.compare(range.lower().fold(), eqValue); + if (compare != null && ( + // eq outside the lower boundary + compare > 0 || + // eq matches the boundary but should not be included + (compare == 0 && range.includeLower() == false))) { + return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); + } + } + if (range.upper().foldable()) { + Integer compare = BinaryComparison.compare(range.upper().fold(), eqValue); + if (compare != null && ( + // eq outside the upper boundary + compare < 0 || + // eq matches the boundary but should not be included + (compare == 0 && range.includeUpper() == false))) { + return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); + } + } + + // it's in the range and thus, remove it + iterator.remove(); + changed = true; + } + } + + // evaluate all NotEquals against the Equal + for (Iterator iter = notEquals.iterator(); iter.hasNext();) { + NotEquals neq = iter.next(); + if (eq.left().semanticEquals(neq.left())) { + Integer comp = BinaryComparison.compare(eqValue, neq.right().fold()); + if (comp != null) { + if (comp == 0) { // clashing and conflicting: a = 1 AND a != 1 + return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); + } else { // clashing and redundant: a = 1 AND a != 2 + iter.remove(); + changed = true; + } + } + } + } + + // evaluate all inequalities against the Equal + for (Iterator iter = inequalities.iterator(); iter.hasNext();) { + BinaryComparison bc = iter.next(); + if (eq.left().semanticEquals(bc.left())) { + Integer compare = BinaryComparison.compare(eqValue, bc.right().fold()); + if (compare != null) { + if (bc instanceof LessThan || bc instanceof LessThanOrEqual) { // a = 2 AND a />= ? + if ((compare == 0 && bc instanceof GreaterThan) || // a = 2 AND a > 2 + compare < 0) { // a = 2 AND a >/>= 3 + return new Literal(and.source(), Boolean.FALSE, DataType.BOOLEAN); + } + } + + iter.remove(); + changed = true; + } + } + } + } + + return changed ? Predicates.combineAnd(CollectionUtils.combine(exps, equals, notEquals, inequalities, ranges)) : and; + } + + // combine disjunction: + // a = 2 OR a > 3 -> nop; a = 2 OR a > 1 -> a > 1 + // a = 2 OR a < 3 -> a < 3; a = 2 OR a < 1 -> nop + // a = 2 OR 3 < a < 5 -> nop; a = 2 OR 1 < a < 3 -> 1 < a < 3; a = 2 OR 0 < a < 1 -> nop + // a = 2 OR a != 2 -> TRUE; a = 2 OR a = 5 -> nop; a = 2 OR a != 5 -> a != 5 + private static Expression propagate(Or or) { + List exps = new ArrayList<>(); + List equals = new ArrayList<>(); // foldable right term Equals + List notEquals = new ArrayList<>(); // foldable right term NotEquals + List ranges = new ArrayList<>(); + List inequalities = new ArrayList<>(); // foldable right term (=limit) BinaryComparision + + // split expressions by type + for (Expression ex : Predicates.splitOr(or)) { + if (ex instanceof Equals eq) { + if (eq.right().foldable()) { + equals.add(eq); + } else { + exps.add(ex); + } + } else if (ex instanceof NotEquals neq) { + if (neq.right().foldable()) { + notEquals.add(neq); + } else { + exps.add(ex); + } + } else if (ex instanceof Range) { + ranges.add((Range) ex); + } else if (ex instanceof BinaryComparison bc) { + if (bc.right().foldable()) { + inequalities.add(bc); + } else { + exps.add(ex); + } + } else { + exps.add(ex); + } + } + + boolean updated = false; // has the expression been modified? + + // evaluate the impact of each Equal over the different types of Expressions + for (Iterator iterEq = equals.iterator(); iterEq.hasNext();) { + Equals eq = iterEq.next(); + Object eqValue = eq.right().fold(); + boolean removeEquals = false; + + // Equals OR NotEquals + for (NotEquals neq : notEquals) { + if (eq.left().semanticEquals(neq.left())) { // a = 2 OR a != ? -> ... + Integer comp = BinaryComparison.compare(eqValue, neq.right().fold()); + if (comp != null) { + if (comp == 0) { // a = 2 OR a != 2 -> TRUE + return TRUE; + } else { // a = 2 OR a != 5 -> a != 5 + removeEquals = true; + break; + } + } + } + } + if (removeEquals) { + iterEq.remove(); + updated = true; + continue; + } + + // Equals OR Range + for (int i = 0; i < ranges.size(); i++) { // might modify list, so use index loop + Range range = ranges.get(i); + if (eq.left().semanticEquals(range.value())) { + Integer lowerComp = range.lower().foldable() ? BinaryComparison.compare(eqValue, range.lower().fold()) : null; + Integer upperComp = range.upper().foldable() ? BinaryComparison.compare(eqValue, range.upper().fold()) : null; + + if (lowerComp != null && lowerComp == 0) { + if (range.includeLower() == false) { // a = 2 OR 2 < a < ? -> 2 <= a < ? + ranges.set( + i, + new Range( + range.source(), + range.value(), + range.lower(), + true, + range.upper(), + range.includeUpper(), + range.zoneId() + ) + ); + } // else : a = 2 OR 2 <= a < ? -> 2 <= a < ? + removeEquals = true; // update range with lower equality instead or simply superfluous + break; + } else if (upperComp != null && upperComp == 0) { + if (range.includeUpper() == false) { // a = 2 OR ? < a < 2 -> ? < a <= 2 + ranges.set( + i, + new Range( + range.source(), + range.value(), + range.lower(), + range.includeLower(), + range.upper(), + true, + range.zoneId() + ) + ); + } // else : a = 2 OR ? < a <= 2 -> ? < a <= 2 + removeEquals = true; // update range with upper equality instead + break; + } else if (lowerComp != null && upperComp != null) { + if (0 < lowerComp && upperComp < 0) { // a = 2 OR 1 < a < 3 + removeEquals = true; // equality is superfluous + break; + } + } + } + } + if (removeEquals) { + iterEq.remove(); + updated = true; + continue; + } + + // Equals OR Inequality + for (int i = 0; i < inequalities.size(); i++) { + BinaryComparison bc = inequalities.get(i); + if (eq.left().semanticEquals(bc.left())) { + Integer comp = BinaryComparison.compare(eqValue, bc.right().fold()); + if (comp != null) { + if (bc instanceof GreaterThan || bc instanceof GreaterThanOrEqual) { + if (comp < 0) { // a = 1 OR a > 2 -> nop + continue; + } else if (comp == 0 && bc instanceof GreaterThan) { // a = 2 OR a > 2 -> a >= 2 + inequalities.set(i, new GreaterThanOrEqual(bc.source(), bc.left(), bc.right(), bc.zoneId())); + } // else (0 < comp || bc instanceof GreaterThanOrEqual) : + // a = 3 OR a > 2 -> a > 2; a = 2 OR a => 2 -> a => 2 + + removeEquals = true; // update range with equality instead or simply superfluous + break; + } else if (bc instanceof LessThan || bc instanceof LessThanOrEqual) { + if (comp > 0) { // a = 2 OR a < 1 -> nop + continue; + } + if (comp == 0 && bc instanceof LessThan) { // a = 2 OR a < 2 -> a <= 2 + inequalities.set(i, new LessThanOrEqual(bc.source(), bc.left(), bc.right(), bc.zoneId())); + } // else (comp < 0 || bc instanceof LessThanOrEqual) : a = 2 OR a < 3 -> a < 3; a = 2 OR a <= 2 -> a <= 2 + removeEquals = true; // update range with equality instead or simply superfluous + break; + } + } + } + } + if (removeEquals) { + iterEq.remove(); + updated = true; + } + } + + return updated ? Predicates.combineOr(CollectionUtils.combine(exps, equals, notEquals, inequalities, ranges)) : or; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java index fb4f35b7173cd..a829808ee040c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/OptimizerRulesTests.java @@ -50,8 +50,11 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer.ReplaceRegexMatch; +import org.elasticsearch.xpack.esql.optimizer.rules.BooleanFunctionEqualsElimination; +import org.elasticsearch.xpack.esql.optimizer.rules.CombineDisjunctionsToIn; import org.elasticsearch.xpack.esql.optimizer.rules.ConstantFolding; import org.elasticsearch.xpack.esql.optimizer.rules.LiteralsOnTheRight; +import org.elasticsearch.xpack.esql.optimizer.rules.PropagateEquals; import java.util.List; @@ -189,7 +192,7 @@ public void testTwoEqualsWithOr() { FieldAttribute fa = getFieldAttribute(); Or or = new Or(EMPTY, equalsOf(fa, ONE), equalsOf(fa, TWO)); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(In.class, e.getClass()); In in = (In) e; assertEquals(fa, in.value()); @@ -200,7 +203,7 @@ public void testTwoEqualsWithSameValue() { FieldAttribute fa = getFieldAttribute(); Or or = new Or(EMPTY, equalsOf(fa, ONE), equalsOf(fa, ONE)); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(Equals.class, e.getClass()); Equals eq = (Equals) e; assertEquals(fa, eq.left()); @@ -211,7 +214,7 @@ public void testOneEqualsOneIn() { FieldAttribute fa = getFieldAttribute(); Or or = new Or(EMPTY, equalsOf(fa, ONE), new In(EMPTY, fa, List.of(TWO))); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(In.class, e.getClass()); In in = (In) e; assertEquals(fa, in.value()); @@ -222,7 +225,7 @@ public void testOneEqualsOneInWithSameValue() { FieldAttribute fa = getFieldAttribute(); Or or = new Or(EMPTY, equalsOf(fa, ONE), new In(EMPTY, fa, asList(ONE, TWO))); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(In.class, e.getClass()); In in = (In) e; assertEquals(fa, in.value()); @@ -234,7 +237,7 @@ public void testSingleValueInToEquals() { Equals equals = equalsOf(fa, ONE); Or or = new Or(EMPTY, equals, new In(EMPTY, fa, List.of(ONE))); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(equals, e); } @@ -243,7 +246,7 @@ public void testEqualsBehindAnd() { And and = new And(EMPTY, equalsOf(fa, ONE), equalsOf(fa, TWO)); Filter dummy = new Filter(EMPTY, relation(), and); - LogicalPlan transformed = new OptimizerRules.CombineDisjunctionsToIn().apply(dummy); + LogicalPlan transformed = new CombineDisjunctionsToIn().apply(dummy); assertSame(dummy, transformed); assertEquals(and, ((Filter) transformed).condition()); } @@ -253,7 +256,7 @@ public void testTwoEqualsDifferentFields() { FieldAttribute fieldTwo = TestUtils.getFieldAttribute("TWO"); Or or = new Or(EMPTY, equalsOf(fieldOne, ONE), equalsOf(fieldTwo, TWO)); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(or); + Expression e = new CombineDisjunctionsToIn().rule(or); assertEquals(or, e); } @@ -262,7 +265,7 @@ public void testMultipleIn() { Or firstOr = new Or(EMPTY, new In(EMPTY, fa, List.of(ONE)), new In(EMPTY, fa, List.of(TWO))); Or secondOr = new Or(EMPTY, firstOr, new In(EMPTY, fa, List.of(THREE))); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(secondOr); + Expression e = new CombineDisjunctionsToIn().rule(secondOr); assertEquals(In.class, e.getClass()); In in = (In) e; assertEquals(fa, in.value()); @@ -274,7 +277,7 @@ public void testOrWithNonCombinableExpressions() { Or firstOr = new Or(EMPTY, new In(EMPTY, fa, List.of(ONE)), lessThanOf(fa, TWO)); Or secondOr = new Or(EMPTY, firstOr, new In(EMPTY, fa, List.of(THREE))); - Expression e = new OptimizerRules.CombineDisjunctionsToIn().rule(secondOr); + Expression e = new CombineDisjunctionsToIn().rule(secondOr); assertEquals(Or.class, e.getClass()); Or or = (Or) e; assertEquals(or.left(), firstOr.right()); @@ -286,7 +289,7 @@ public void testOrWithNonCombinableExpressions() { // Test BooleanFunctionEqualsElimination public void testBoolEqualsSimplificationOnExpressions() { - OptimizerRules.BooleanFunctionEqualsElimination s = new OptimizerRules.BooleanFunctionEqualsElimination(); + BooleanFunctionEqualsElimination s = new BooleanFunctionEqualsElimination(); Expression exp = new GreaterThan(EMPTY, getFieldAttribute(), new Literal(EMPTY, 0, DataType.INTEGER), null); assertEquals(exp, s.rule(new Equals(EMPTY, exp, TRUE))); @@ -295,7 +298,7 @@ public void testBoolEqualsSimplificationOnExpressions() { } public void testBoolEqualsSimplificationOnFields() { - OptimizerRules.BooleanFunctionEqualsElimination s = new OptimizerRules.BooleanFunctionEqualsElimination(); + BooleanFunctionEqualsElimination s = new BooleanFunctionEqualsElimination(); FieldAttribute field = getFieldAttribute(); @@ -323,7 +326,7 @@ public void testDualEqualsConjunction() { Equals eq1 = equalsOf(fa, ONE); Equals eq2 = equalsOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq1, eq2)); assertEquals(FALSE, exp); } @@ -334,7 +337,7 @@ public void testEliminateRangeByEqualsOutsideInterval() { Equals eq1 = equalsOf(fa, new Literal(EMPTY, 10, DataType.INTEGER)); Range r = rangeOf(fa, ONE, false, new Literal(EMPTY, 10, DataType.INTEGER), false); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq1, r)); assertEquals(FALSE, exp); } @@ -345,7 +348,7 @@ public void testPropagateEquals_VarNeq3AndVarEq3() { NotEquals neq = notEqualsOf(fa, THREE); Equals eq = equalsOf(fa, THREE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, neq, eq)); assertEquals(FALSE, exp); } @@ -356,7 +359,7 @@ public void testPropagateEquals_VarNeq4AndVarEq3() { NotEquals neq = notEqualsOf(fa, FOUR); Equals eq = equalsOf(fa, THREE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, neq, eq)); assertEquals(Equals.class, exp.getClass()); assertEquals(eq, exp); @@ -368,7 +371,7 @@ public void testPropagateEquals_VarEq2AndVarLt2() { Equals eq = equalsOf(fa, TWO); LessThan lt = lessThanOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, lt)); assertEquals(FALSE, exp); } @@ -379,7 +382,7 @@ public void testPropagateEquals_VarEq2AndVarLte2() { Equals eq = equalsOf(fa, TWO); LessThanOrEqual lt = lessThanOrEqualOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, lt)); assertEquals(eq, exp); } @@ -390,7 +393,7 @@ public void testPropagateEquals_VarEq2AndVarLte1() { Equals eq = equalsOf(fa, TWO); LessThanOrEqual lt = lessThanOrEqualOf(fa, ONE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, lt)); assertEquals(FALSE, exp); } @@ -401,7 +404,7 @@ public void testPropagateEquals_VarEq2AndVarGt2() { Equals eq = equalsOf(fa, TWO); GreaterThan gt = greaterThanOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, gt)); assertEquals(FALSE, exp); } @@ -412,7 +415,7 @@ public void testPropagateEquals_VarEq2AndVarGte2() { Equals eq = equalsOf(fa, TWO); GreaterThanOrEqual gte = greaterThanOrEqualOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, gte)); assertEquals(eq, exp); } @@ -423,7 +426,7 @@ public void testPropagateEquals_VarEq2AndVarLt3() { Equals eq = equalsOf(fa, TWO); GreaterThan gt = greaterThanOf(fa, THREE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq, gt)); assertEquals(FALSE, exp); } @@ -436,7 +439,7 @@ public void testPropagateEquals_VarEq2AndVarLt3AndVarGt1AndVarNeq4() { GreaterThan gt = greaterThanOf(fa, ONE); NotEquals neq = notEqualsOf(fa, FOUR); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression and = Predicates.combineAnd(asList(eq, lt, gt, neq)); Expression exp = rule.rule((And) and); assertEquals(eq, exp); @@ -450,7 +453,7 @@ public void testPropagateEquals_VarEq2AndVarRangeGt1Lt3AndVarGt0AndVarNeq4() { GreaterThan gt = greaterThanOf(fa, new Literal(EMPTY, 0, DataType.INTEGER)); NotEquals neq = notEqualsOf(fa, FOUR); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression and = Predicates.combineAnd(asList(eq, range, gt, neq)); Expression exp = rule.rule((And) and); assertEquals(eq, exp); @@ -462,7 +465,7 @@ public void testPropagateEquals_VarEq2OrVarGt1() { Equals eq = equalsOf(fa, TWO); GreaterThan gt = greaterThanOf(fa, ONE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, gt)); assertEquals(gt, exp); } @@ -473,7 +476,7 @@ public void testPropagateEquals_VarEq2OrVarGte2() { Equals eq = equalsOf(fa, TWO); GreaterThan gt = greaterThanOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, gt)); assertEquals(GreaterThanOrEqual.class, exp.getClass()); GreaterThanOrEqual gte = (GreaterThanOrEqual) exp; @@ -486,7 +489,7 @@ public void testPropagateEquals_VarEq2OrVarLt3() { Equals eq = equalsOf(fa, TWO); LessThan lt = lessThanOf(fa, THREE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, lt)); assertEquals(lt, exp); } @@ -497,7 +500,7 @@ public void testPropagateEquals_VarEq3OrVarLt3() { Equals eq = equalsOf(fa, THREE); LessThan lt = lessThanOf(fa, THREE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, lt)); assertEquals(LessThanOrEqual.class, exp.getClass()); LessThanOrEqual lte = (LessThanOrEqual) exp; @@ -510,7 +513,7 @@ public void testPropagateEquals_VarEq2OrVarRangeGt1Lt3() { Equals eq = equalsOf(fa, TWO); Range range = rangeOf(fa, ONE, false, THREE, false); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, range)); assertEquals(range, exp); } @@ -521,7 +524,7 @@ public void testPropagateEquals_VarEq2OrVarRangeGt2Lt3() { Equals eq = equalsOf(fa, TWO); Range range = rangeOf(fa, TWO, false, THREE, false); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, range)); assertEquals(Range.class, exp.getClass()); Range r = (Range) exp; @@ -537,7 +540,7 @@ public void testPropagateEquals_VarEq3OrVarRangeGt2Lt3() { Equals eq = equalsOf(fa, THREE); Range range = rangeOf(fa, TWO, false, THREE, false); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, range)); assertEquals(Range.class, exp.getClass()); Range r = (Range) exp; @@ -553,7 +556,7 @@ public void testPropagateEquals_VarEq2OrVarNeq2() { Equals eq = equalsOf(fa, TWO); NotEquals neq = notEqualsOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, neq)); assertEquals(TRUE, exp); } @@ -564,7 +567,7 @@ public void testPropagateEquals_VarEq2OrVarNeq5() { Equals eq = equalsOf(fa, TWO); NotEquals neq = notEqualsOf(fa, FIVE); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new Or(EMPTY, eq, neq)); assertEquals(NotEquals.class, exp.getClass()); NotEquals ne = (NotEquals) exp; @@ -579,7 +582,7 @@ public void testPropagateEquals_VarEq2OrVarRangeGt3Lt4OrVarGt2OrVarNe2() { GreaterThan gt = greaterThanOf(fa, TWO); NotEquals neq = notEqualsOf(fa, TWO); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule((Or) Predicates.combineOr(asList(eq, range, neq, gt))); assertEquals(TRUE, exp); } @@ -591,7 +594,7 @@ public void testPropagateEquals_ignoreDateTimeFields() { Equals eq2 = equalsOf(fa, TWO); And and = new And(EMPTY, eq1, eq2); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(and); assertEquals(and, exp); } @@ -602,7 +605,7 @@ public void testEliminateRangeByEqualsInInterval() { Equals eq1 = equalsOf(fa, ONE); Range r = rangeOf(fa, ONE, true, new Literal(EMPTY, 10, DataType.INTEGER), false); - OptimizerRules.PropagateEquals rule = new OptimizerRules.PropagateEquals(); + PropagateEquals rule = new PropagateEquals(); Expression exp = rule.rule(new And(EMPTY, eq1, r)); assertEquals(eq1, exp); } From 50592ec2f5b6abdde012a74d2a588b2a0decc339 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 10 Jun 2024 14:53:28 +0100 Subject: [PATCH 07/31] Add `@UpdateForV9` to `IndexSettingDeprecatedInV7AndRemovedInV8` (#109531) This setting property is not applicable in v9, but we'll need an equivalent one for the 8-to-9 transition. This commit adds the `@UpdateForV9` annotation as a reminder. --- .../src/main/java/org/elasticsearch/common/settings/Setting.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/common/settings/Setting.java b/server/src/main/java/org/elasticsearch/common/settings/Setting.java index 71882e0c68191..e96de685381eb 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Setting.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Setting.java @@ -149,6 +149,7 @@ public enum Property { * Indicates that this index-level setting was deprecated in {@link Version#V_7_17_0} and is * forbidden in indices created from {@link Version#V_8_0_0} onwards. */ + @UpdateForV9 // introduce IndexSettingDeprecatedInV8AndRemovedInV9 to replace this constant IndexSettingDeprecatedInV7AndRemovedInV8, /** From 7cba6c8c166def80134b2f215eea8f49b31698b7 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Mon, 10 Jun 2024 15:57:56 +0200 Subject: [PATCH 08/31] Docs: Fix available update by query operations (#109486) --- docs/reference/docs/update-by-query.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index bc63fa4e33d01..d470080fc602f 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -431,7 +431,7 @@ The update by query operation skips updating the document and increments the `n Set `ctx.op = "delete"` if your script decides that the document should be deleted. The update by query operation deletes the document and increments the `deleted` counter. -Update by query only supports `update`, `noop`, and `delete`. +Update by query only supports `index`, `noop`, and `delete`. Setting `ctx.op` to anything else is an error. Setting any other field in `ctx` is an error. This API only enables you to modify the source of matching documents, you cannot move them. From beec35813c4cabf462611dd88c236bebdc0f85f2 Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Mon, 10 Jun 2024 07:53:15 -0700 Subject: [PATCH 09/31] Remove explicit references to SyntheticSourceMode.FALLBACK (#109503) --- .../index/mapper/extras/SearchAsYouTypeFieldMapper.java | 5 ----- .../index/mapper/extras/TokenCountFieldMapper.java | 5 ----- .../index/mapper/GeoShapeWithDocValuesFieldMapper.java | 5 ----- .../xpack/spatial/index/mapper/PointFieldMapper.java | 5 ----- .../xpack/spatial/index/mapper/ShapeFieldMapper.java | 5 ----- 5 files changed, 25 deletions(-) diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/SearchAsYouTypeFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/SearchAsYouTypeFieldMapper.java index 878e22312d643..a5e011d5772f0 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/SearchAsYouTypeFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/SearchAsYouTypeFieldMapper.java @@ -762,11 +762,6 @@ public Iterator sourcePathUsedBy() { return subfieldsAndMultifieldsIterator(); } - @Override - protected SyntheticSourceMode syntheticSourceMode() { - return SyntheticSourceMode.FALLBACK; - } - /** * An analyzer wrapper to add a shingle token filter, an edge ngram token filter or both to its wrapped analyzer. When adding an edge * ngrams token filter, it also adds a {@link TrailingShingleTokenFilter} to add extra position increments at the end of the stream diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/TokenCountFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/TokenCountFieldMapper.java index c538c7641a015..831306a8e8594 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/TokenCountFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/TokenCountFieldMapper.java @@ -215,9 +215,4 @@ protected String contentType() { public FieldMapper.Builder getMergeBuilder() { return new Builder(simpleName()).init(this); } - - @Override - protected SyntheticSourceMode syntheticSourceMode() { - return SyntheticSourceMode.FALLBACK; - } } diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeWithDocValuesFieldMapper.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeWithDocValuesFieldMapper.java index 44cfa99c003d7..a8f437f476ada 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeWithDocValuesFieldMapper.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/GeoShapeWithDocValuesFieldMapper.java @@ -482,11 +482,6 @@ protected void checkIncomingMergeType(FieldMapper mergeWith) { super.checkIncomingMergeType(mergeWith); } - @Override - protected SyntheticSourceMode syntheticSourceMode() { - return SyntheticSourceMode.FALLBACK; - } - public static class GeoShapeDocValuesField extends AbstractScriptFieldFactory implements Field, diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/PointFieldMapper.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/PointFieldMapper.java index 2a2a6ad164677..1657a3bf7fbce 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/PointFieldMapper.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/PointFieldMapper.java @@ -180,11 +180,6 @@ public FieldMapper.Builder getMergeBuilder() { return new Builder(simpleName(), builder.ignoreMalformed.getDefaultValue().value()).init(this); } - @Override - protected SyntheticSourceMode syntheticSourceMode() { - return SyntheticSourceMode.FALLBACK; - } - public static class PointFieldType extends AbstractPointFieldType implements ShapeQueryable { private final ShapeQueryPointProcessor queryProcessor; diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/ShapeFieldMapper.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/ShapeFieldMapper.java index 4d6a909f96c3f..83e434f829591 100644 --- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/ShapeFieldMapper.java +++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/index/mapper/ShapeFieldMapper.java @@ -249,11 +249,6 @@ public ShapeFieldType fieldType() { return (ShapeFieldType) super.fieldType(); } - @Override - protected SyntheticSourceMode syntheticSourceMode() { - return SyntheticSourceMode.FALLBACK; - } - public static class CartesianShapeDocValuesField extends AbstractScriptFieldFactory implements Field, From a9bc30d66e03c076a3f0909e745818d76d240455 Mon Sep 17 00:00:00 2001 From: Jean-Fabrice Bobo <2949987+jeanfabrice@users.noreply.github.com> Date: Mon, 10 Jun 2024 17:51:55 +0200 Subject: [PATCH 10/31] FIx misleading repository-s3 type (#109347) in 8.x, `repository-s3` type has been replaced by `s3` type. Fixing remaining reference to `repository-s3` in the documentation. --- docs/reference/snapshot-restore/repository-s3.asciidoc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/reference/snapshot-restore/repository-s3.asciidoc b/docs/reference/snapshot-restore/repository-s3.asciidoc index d0a1d36e55608..d757a74110ca9 100644 --- a/docs/reference/snapshot-restore/repository-s3.asciidoc +++ b/docs/reference/snapshot-restore/repository-s3.asciidoc @@ -539,7 +539,7 @@ VPC's internet gateway and not be bandwidth limited by the VPC's NAT instance. ==== S3-compatible services There are a number of storage systems that provide an S3-compatible API, and -the `repository-s3` type allows you to use these systems in place of AWS S3. +the `s3` repository type allows you to use these systems in place of AWS S3. To do so, you should set the `s3.client.CLIENT_NAME.endpoint` setting to the system's endpoint. This setting accepts IP addresses and hostnames and may include a port. For example, the endpoint may be `172.17.0.2` or @@ -552,7 +552,7 @@ you wish to use unsecured HTTP communication instead of HTTPS, set `s3.client.CLIENT_NAME.protocol` to `http`. https://minio.io[MinIO] is an example of a storage system that provides an -S3-compatible API. The `repository-s3` type allows {es} to work with +S3-compatible API. The `s3` repository type allows {es} to work with MinIO-backed repositories as well as repositories stored on AWS S3. Other S3-compatible storage systems may also work with {es}, but these are not covered by the {es} test suite. @@ -562,7 +562,7 @@ which claim to offer an S3-compatible API despite failing to emulate S3's behaviour in full. If you are using such a system for your snapshots, consider using a <> based on a standardized protocol such as NFS to access your storage system instead. -The `repository-s3` type requires full compatibility with S3. In particular it +The `s3` repository type requires full compatibility with S3. In particular it must support the same set of API endpoints, with the same parameters, return the same errors in case of failures, and offer consistency and performance at least as good as S3 even when accessed concurrently by multiple nodes. You will From 6dbcd7908a19ee776b98b8ffb97130ee8b21b04f Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Mon, 10 Jun 2024 10:52:21 -0500 Subject: [PATCH 11/31] Fix test by removing unecessary mTLS (#109324) fixes: #108774 --- muted-tests.yml | 3 --- .../xpack/core/ssl/SSLConfigurationReloaderTests.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 5b9478f1e452e..d82c823f664be 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -5,9 +5,6 @@ tests: - class: "org.elasticsearch.cluster.coordination.CoordinatorVotingConfigurationTests" issue: "https://github.com/elastic/elasticsearch/issues/108729" method: "testClusterUUIDLogging" -- class: "org.elasticsearch.xpack.core.ssl.SSLConfigurationReloaderTests" - issue: "https://github.com/elastic/elasticsearch/issues/108774" - method: "testReloadingKeyStore" - class: "org.elasticsearch.xpack.esql.ccq.MultiClusterSpecIT" issue: "https://github.com/elastic/elasticsearch/issues/108808" method: "test {k8s-metrics.MetricsWithAggs}" diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java index 02f9a58d7947d..7b19d53663a08 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java @@ -130,7 +130,7 @@ public void testReloadingKeyStore() throws Exception { // Load HTTPClient only once. Client uses the same store as a truststore try (CloseableHttpClient client = getSSLClient(keystorePath, "testnode")) { final Consumer keyMaterialPreChecks = (context) -> { - try (MockWebServer server = new MockWebServer(context, true)) { + try (MockWebServer server = new MockWebServer(context, false)) { server.enqueue(new MockResponse().setResponseCode(200).setBody("body")); server.start(); privilegedConnect(() -> client.execute(new HttpGet("https://localhost:" + server.getPort())).close()); From 832029a1b547c984afe8834770f974dc2041d1ed Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 10 Jun 2024 17:53:41 +0200 Subject: [PATCH 12/31] Use BytesRef instead of Term on o.e.i.engine.Engine.Operation (#109508) No need to have the `Term` objects here. Creating a `Term` forces copying the id bytes (in addition to the cost of creating the object) and we do not need a term here to begin with. This saves GBs of allocations when running the http_logs indexing benchmark and sets up further simplifications around the id value. --- .../uid/PerThreadIDVersionAndSeqNoLookup.java | 13 ++-- .../lucene/uid/VersionsAndSeqNoResolver.java | 38 +++++------- .../elasticsearch/index/engine/Engine.java | 44 +++++-------- .../index/engine/InternalEngine.java | 46 ++++++-------- .../org/elasticsearch/index/mapper/Uid.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 10 +-- .../common/lucene/uid/VersionLookupTests.java | 16 ++--- .../common/lucene/uid/VersionsTests.java | 46 ++++++-------- .../elasticsearch/index/IndexModuleTests.java | 3 +- .../index/IndexingSlowLogTests.java | 5 +- .../index/engine/InternalEngineTests.java | 61 ++++++++++++------- .../engine/LuceneChangesSnapshotTests.java | 5 +- .../index/engine/NoOpEngineTests.java | 3 +- .../index/engine/ReadOnlyEngineTests.java | 3 +- .../shard/IndexingOperationListenerTests.java | 5 +- .../index/shard/RefreshListenersTests.java | 6 +- .../index/translog/TranslogTests.java | 6 +- .../index/engine/EngineTestCase.java | 12 ++-- .../index/engine/FollowingEngineTests.java | 4 +- 19 files changed, 144 insertions(+), 184 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java index 52ddaa9a87589..43924eb86f12f 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/PerThreadIDVersionAndSeqNoLookup.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndSeqNo; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; +import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.VersionFieldMapper; @@ -45,8 +46,6 @@ final class PerThreadIDVersionAndSeqNoLookup { // TODO: do we really need to store all this stuff? some if it might not speed up anything. // we keep it around for now, to reduce the amount of e.g. hash lookups by field and stuff - /** terms enum for uid field */ - final String uidField; private final TermsEnum termsEnum; /** Reused for iteration (when the term exists) */ @@ -62,10 +61,8 @@ final class PerThreadIDVersionAndSeqNoLookup { /** * Initialize lookup for the provided segment */ - PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField, boolean trackReaderKey, boolean loadTimestampRange) - throws IOException { - this.uidField = uidField; - final Terms terms = reader.terms(uidField); + PerThreadIDVersionAndSeqNoLookup(LeafReader reader, boolean trackReaderKey, boolean loadTimestampRange) throws IOException { + final Terms terms = reader.terms(IdFieldMapper.NAME); if (terms == null) { // If a segment contains only no-ops, it does not have _uid but has both _soft_deletes and _tombstone fields. final NumericDocValues softDeletesDV = reader.getNumericDocValues(Lucene.SOFT_DELETES_FIELD); @@ -107,8 +104,8 @@ final class PerThreadIDVersionAndSeqNoLookup { } } - PerThreadIDVersionAndSeqNoLookup(LeafReader reader, String uidField, boolean loadTimestampRange) throws IOException { - this(reader, uidField, true, loadTimestampRange); + PerThreadIDVersionAndSeqNoLookup(LeafReader reader, boolean loadTimestampRange) throws IOException { + this(reader, true, loadTimestampRange); } /** Return null if id is not found. diff --git a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java index 56c0869992bac..1743343b44bf4 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/uid/VersionsAndSeqNoResolver.java @@ -11,7 +11,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.Term; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CloseableThreadLocal; import org.elasticsearch.common.util.ByteUtils; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Base64; import java.util.List; -import java.util.Objects; import java.util.concurrent.ConcurrentMap; /** Utility class to resolve the Lucene doc ID, version, seqNo and primaryTerms for a given uid. */ @@ -37,8 +36,7 @@ public final class VersionsAndSeqNoResolver { } }; - private static PerThreadIDVersionAndSeqNoLookup[] getLookupState(IndexReader reader, String uidField, boolean loadTimestampRange) - throws IOException { + private static PerThreadIDVersionAndSeqNoLookup[] getLookupState(IndexReader reader, boolean loadTimestampRange) throws IOException { // We cache on the top level // This means cache entries have a shorter lifetime, maybe as low as 1s with the // default refresh interval and a steady indexing rate, but on the other hand it @@ -63,7 +61,7 @@ private static PerThreadIDVersionAndSeqNoLookup[] getLookupState(IndexReader rea if (lookupState == null) { lookupState = new PerThreadIDVersionAndSeqNoLookup[reader.leaves().size()]; for (LeafReaderContext leaf : reader.leaves()) { - lookupState[leaf.ord] = new PerThreadIDVersionAndSeqNoLookup(leaf.reader(), uidField, loadTimestampRange); + lookupState[leaf.ord] = new PerThreadIDVersionAndSeqNoLookup(leaf.reader(), loadTimestampRange); } ctl.set(lookupState); } else { @@ -87,12 +85,6 @@ private static PerThreadIDVersionAndSeqNoLookup[] getLookupState(IndexReader rea throw new AssertionError("Mismatched numbers of leaves: " + lookupState.length + " != " + reader.leaves().size()); } - if (lookupState.length > 0 && Objects.equals(lookupState[0].uidField, uidField) == false) { - throw new AssertionError( - "Index does not consistently use the same uid field: [" + uidField + "] != [" + lookupState[0].uidField + "]" - ); - } - return lookupState; } @@ -136,15 +128,15 @@ public static class DocIdAndSeqNo { *

  • a doc ID and a version otherwise * */ - public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, Term term, boolean loadSeqNo) throws IOException { - PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, term.field(), false); + public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, BytesRef term, boolean loadSeqNo) throws IOException { + PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, false); List leaves = reader.leaves(); // iterate backwards to optimize for the frequently updated documents // which are likely to be in the last segments for (int i = leaves.size() - 1; i >= 0; i--) { final LeafReaderContext leaf = leaves.get(i); PerThreadIDVersionAndSeqNoLookup lookup = lookups[leaf.ord]; - DocIdAndVersion result = lookup.lookupVersion(term.bytes(), loadSeqNo, leaf); + DocIdAndVersion result = lookup.lookupVersion(term, loadSeqNo, leaf); if (result != null) { return result; } @@ -168,7 +160,7 @@ public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, * returning null if no document was found for the specified id * @throws IOException In case of an i/o related failure */ - public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, Term uid, String id, boolean loadSeqNo) + public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, BytesRef uid, String id, boolean loadSeqNo) throws IOException { byte[] idAsBytes = Base64.getUrlDecoder().decode(id); assert idAsBytes.length == 20; @@ -176,7 +168,7 @@ public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, // @timestamp) long timestamp = ByteUtils.readLongBE(idAsBytes, 12); - PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, uid.field(), true); + PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, true); List leaves = reader.leaves(); // iterate in default order, the segments should be sorted by DataStream#TIMESERIES_LEAF_READERS_SORTER long prevMaxTimestamp = Long.MAX_VALUE; @@ -190,7 +182,7 @@ public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, if (timestamp > lookup.maxTimestamp) { return null; } - DocIdAndVersion result = lookup.lookupVersion(uid.bytes(), loadSeqNo, leaf); + DocIdAndVersion result = lookup.lookupVersion(uid, loadSeqNo, leaf); if (result != null) { return result; } @@ -199,12 +191,12 @@ public static DocIdAndVersion timeSeriesLoadDocIdAndVersion(IndexReader reader, return null; } - public static DocIdAndVersion loadDocIdAndVersionUncached(IndexReader reader, Term term, boolean loadSeqNo) throws IOException { + public static DocIdAndVersion loadDocIdAndVersionUncached(IndexReader reader, BytesRef term, boolean loadSeqNo) throws IOException { List leaves = reader.leaves(); for (int i = leaves.size() - 1; i >= 0; i--) { final LeafReaderContext leaf = leaves.get(i); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(leaf.reader(), term.field(), false, false); - DocIdAndVersion result = lookup.lookupVersion(term.bytes(), loadSeqNo, leaf); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(leaf.reader(), false, false); + DocIdAndVersion result = lookup.lookupVersion(term, loadSeqNo, leaf); if (result != null) { return result; } @@ -216,15 +208,15 @@ public static DocIdAndVersion loadDocIdAndVersionUncached(IndexReader reader, Te * Loads the internal docId and sequence number of the latest copy for a given uid from the provided reader. * The result is either null or the live and latest version of the given uid. */ - public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, Term term) throws IOException { - final PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, term.field(), false); + public static DocIdAndSeqNo loadDocIdAndSeqNo(IndexReader reader, BytesRef term) throws IOException { + final PerThreadIDVersionAndSeqNoLookup[] lookups = getLookupState(reader, false); final List leaves = reader.leaves(); // iterate backwards to optimize for the frequently updated documents // which are likely to be in the last segments for (int i = leaves.size() - 1; i >= 0; i--) { final LeafReaderContext leaf = leaves.get(i); final PerThreadIDVersionAndSeqNoLookup lookup = lookups[leaf.ord]; - final DocIdAndSeqNo result = lookup.lookupSeqNo(term.bytes(), leaf); + final DocIdAndSeqNo result = lookup.lookupSeqNo(term, leaf); if (result != null) { return result; } diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index c219e16659c99..4f461a5d51c75 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -23,13 +23,13 @@ import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentReader; -import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; @@ -59,7 +59,6 @@ import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.DocumentParser; -import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.LuceneDocument; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.MappingLookup; @@ -1496,7 +1495,7 @@ public String getLowercase() { } } - private final Term uid; + private final BytesRef uid; private final long version; private final long seqNo; private final long primaryTerm; @@ -1504,7 +1503,7 @@ public String getLowercase() { private final Origin origin; private final long startTime; - public Operation(Term uid, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime) { + public Operation(BytesRef uid, long seqNo, long primaryTerm, long version, VersionType versionType, Origin origin, long startTime) { this.uid = uid; this.seqNo = seqNo; this.primaryTerm = primaryTerm; @@ -1534,7 +1533,7 @@ public Origin origin() { return this.origin; } - public Term uid() { + public BytesRef uid() { return this.uid; } @@ -1577,7 +1576,7 @@ public static class Index extends Operation { private final long ifPrimaryTerm; public Index( - Term uid, + BytesRef uid, ParsedDocument doc, long seqNo, long primaryTerm, @@ -1603,11 +1602,11 @@ public Index( this.ifPrimaryTerm = ifPrimaryTerm; } - public Index(Term uid, long primaryTerm, ParsedDocument doc) { + public Index(BytesRef uid, long primaryTerm, ParsedDocument doc) { this(uid, primaryTerm, doc, Versions.MATCH_ANY); } // TEST ONLY - Index(Term uid, long primaryTerm, ParsedDocument doc, long version) { + Index(BytesRef uid, long primaryTerm, ParsedDocument doc, long version) { this( uid, doc, @@ -1689,7 +1688,7 @@ public static class Delete extends Operation { public Delete( String id, - Term uid, + BytesRef uid, long seqNo, long primaryTerm, long version, @@ -1710,7 +1709,7 @@ public Delete( this.ifPrimaryTerm = ifPrimaryTerm; } - public Delete(String id, Term uid, long primaryTerm) { + public Delete(String id, BytesRef uid, long primaryTerm) { this( id, uid, @@ -1725,21 +1724,6 @@ public Delete(String id, Term uid, long primaryTerm) { ); } - public Delete(Delete template, VersionType versionType) { - this( - template.id(), - template.uid(), - template.seqNo(), - template.primaryTerm(), - template.version(), - versionType, - template.origin(), - template.startTime(), - UNASSIGNED_SEQ_NO, - 0 - ); - } - @Override public String id() { return this.id; @@ -1752,7 +1736,7 @@ public TYPE operationType() { @Override public int estimatedSizeInBytes() { - return (uid().field().length() + uid().text().length()) * 2 + 20; + return uid().length * 2 + 20; } public long getIfSeqNo() { @@ -1778,7 +1762,7 @@ public NoOp(final long seqNo, final long primaryTerm, final Origin origin, final } @Override - public Term uid() { + public BytesRef uid() { throw new UnsupportedOperationException(); } @@ -1811,7 +1795,7 @@ public int estimatedSizeInBytes() { public static class Get { private final boolean realtime; - private final Term uid; + private final BytesRef uid; private final String id; private final boolean readFromTranslog; private long version = Versions.MATCH_ANY; @@ -1822,7 +1806,7 @@ public static class Get { public Get(boolean realtime, boolean readFromTranslog, String id) { this.realtime = realtime; this.id = id; - this.uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); + this.uid = Uid.encodeId(id); this.readFromTranslog = readFromTranslog; } @@ -1834,7 +1818,7 @@ public String id() { return id; } - public Term uid() { + public BytesRef uid() { return uid; } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 58bffa0369f44..245cef2d97b24 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -838,7 +838,6 @@ public GetResult get( DocumentParser documentParser, Function searcherWrapper ) { - assert assertGetUsesIdField(get); try (var ignored = acquireEnsureOpenRef()) { if (get.realtime()) { var result = realtimeGetUnderLock(get, mappingLookup, documentParser, searcherWrapper, true); @@ -858,7 +857,6 @@ public GetResult getFromTranslog( DocumentParser documentParser, Function searcherWrapper ) { - assert assertGetUsesIdField(get); try (var ignored = acquireEnsureOpenRef()) { return realtimeGetUnderLock(get, mappingLookup, documentParser, searcherWrapper, false); } @@ -878,9 +876,9 @@ protected GetResult realtimeGetUnderLock( assert isDrainedForClose() == false; assert get.realtime(); final VersionValue versionValue; - try (Releasable ignore = versionMap.acquireLock(get.uid().bytes())) { + try (Releasable ignore = versionMap.acquireLock(get.uid())) { // we need to lock here to access the version map to do this truly in RT - versionValue = getVersionFromMap(get.uid().bytes()); + versionValue = getVersionFromMap(get.uid()); } try { boolean getFromSearcherIfNotInTranslog = getFromSearcher; @@ -982,7 +980,7 @@ private static OpVsLuceneDocStatus compareOpToVersionMapOnSeqNo(String id, long private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) throws IOException { assert op.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO : "resolving ops based on seq# but no seqNo is found"; final OpVsLuceneDocStatus status; - VersionValue versionValue = getVersionFromMap(op.uid().bytes()); + VersionValue versionValue = getVersionFromMap(op.uid()); assert incrementVersionLookup(); if (versionValue != null) { status = compareOpToVersionMapOnSeqNo(op.id(), op.seqNo(), op.primaryTerm(), versionValue); @@ -1010,7 +1008,7 @@ private OpVsLuceneDocStatus compareOpToLuceneDocBasedOnSeqNo(final Operation op) /** resolves the current version of the document, returning null if not found */ private VersionValue resolveDocVersion(final Operation op, boolean loadSeqNo) throws IOException { assert incrementVersionLookup(); // used for asserting in tests - VersionValue versionValue = getVersionFromMap(op.uid().bytes()); + VersionValue versionValue = getVersionFromMap(op.uid()); if (versionValue == null) { assert incrementIndexVersionLookup(); // used for asserting in tests final VersionsAndSeqNoResolver.DocIdAndVersion docIdAndVersion; @@ -1139,13 +1137,12 @@ long doGenerateSeqNoForOperation(final Operation operation) { @Override public IndexResult index(Index index) throws IOException { - assert Objects.equals(index.uid().field(), IdFieldMapper.NAME) : index.uid().field(); final boolean doThrottle = index.origin().isRecovery() == false; try (var ignored1 = acquireEnsureOpenRef()) { assert assertIncomingSequenceNumber(index.origin(), index.seqNo()); int reservedDocs = 0; try ( - Releasable ignored = versionMap.acquireLock(index.uid().bytes()); + Releasable ignored = versionMap.acquireLock(index.uid()); Releasable indexThrottle = doThrottle ? throttle.acquireThrottle() : () -> {} ) { lastWriteNanos = index.startTime(); @@ -1245,7 +1242,7 @@ public IndexResult index(Index index) throws IOException { if (plan.indexIntoLucene && indexResult.getResultType() == Result.Type.SUCCESS) { final Translog.Location translogLocation = trackTranslogLocation.get() ? indexResult.getTranslogLocation() : null; versionMap.maybePutIndexUnderLock( - index.uid().bytes(), + index.uid(), new IndexVersionValue(translogLocation, plan.versionForIndexing, index.seqNo(), index.primaryTerm()) ); } @@ -1569,7 +1566,7 @@ static IndexingStrategy failAsTooManyDocs(Exception e, String id) { private boolean assertDocDoesNotExist(final Index index, final boolean allowDeleted) throws IOException { // NOTE this uses direct access to the version map since we are in the assertion code where we maintain a secondary // map in the version map such that we don't need to refresh if we are unsafe; - final VersionValue versionValue = versionMap.getVersionForAssert(index.uid().bytes()); + final VersionValue versionValue = versionMap.getVersionForAssert(index.uid()); if (versionValue != null) { if (versionValue.isDelete() == false || allowDeleted == false) { throw new AssertionError("doc [" + index.id() + "] exists in version map (version " + versionValue + ")"); @@ -1577,7 +1574,7 @@ private boolean assertDocDoesNotExist(final Index index, final boolean allowDele } else { try (Searcher searcher = acquireSearcher("assert doc doesn't exist", SearcherScope.INTERNAL)) { searcher.setQueryCache(null); // so that it does not interfere with tests that check caching behavior - final long docsWithId = searcher.count(new TermQuery(index.uid())); + final long docsWithId = searcher.count(new TermQuery(new Term(IdFieldMapper.NAME, index.uid()))); if (docsWithId > 0) { throw new AssertionError("doc [" + index.id() + "] exists [" + docsWithId + "] times in index"); } @@ -1586,11 +1583,12 @@ private boolean assertDocDoesNotExist(final Index index, final boolean allowDele return true; } - private void updateDocs(final Term uid, final List docs, final IndexWriter indexWriter) throws IOException { + private void updateDocs(final BytesRef uid, final List docs, final IndexWriter indexWriter) throws IOException { + final Term uidTerm = new Term(IdFieldMapper.NAME, uid); if (docs.size() > 1) { - indexWriter.softUpdateDocuments(uid, docs, softDeletesField); + indexWriter.softUpdateDocuments(uidTerm, docs, softDeletesField); } else { - indexWriter.softUpdateDocument(uid, docs.get(0), softDeletesField); + indexWriter.softUpdateDocument(uidTerm, docs.get(0), softDeletesField); } numDocUpdates.inc(docs.size()); } @@ -1598,12 +1596,11 @@ private void updateDocs(final Term uid, final List docs, final I @Override public DeleteResult delete(Delete delete) throws IOException { versionMap.enforceSafeAccess(); - assert Objects.equals(delete.uid().field(), IdFieldMapper.NAME) : delete.uid().field(); assert assertIncomingSequenceNumber(delete.origin(), delete.seqNo()); final DeleteResult deleteResult; int reservedDocs = 0; // NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments: - try (var ignored = acquireEnsureOpenRef(); Releasable ignored2 = versionMap.acquireLock(delete.uid().bytes())) { + try (var ignored = acquireEnsureOpenRef(); Releasable ignored2 = versionMap.acquireLock(delete.uid())) { lastWriteNanos = delete.startTime(); final DeletionStrategy plan = deletionStrategyForOperation(delete); reservedDocs = plan.reservedDocs; @@ -1647,7 +1644,7 @@ public DeleteResult delete(Delete delete) throws IOException { if (plan.deleteFromLucene) { numDocDeletes.inc(); versionMap.putDeleteUnderLock( - delete.uid().bytes(), + delete.uid(), new DeleteVersionValue( plan.versionOfDeletion, delete.seqNo(), @@ -1812,7 +1809,7 @@ private DeleteResult deleteInLucene(Delete delete, DeletionStrategy plan) throws if (plan.addStaleOpToLucene || plan.currentlyDeleted) { indexWriter.addDocument(doc); } else { - indexWriter.softUpdateDocument(delete.uid(), doc, softDeletesField); + indexWriter.softUpdateDocument(new Term(IdFieldMapper.NAME, delete.uid()), doc, softDeletesField); } return new DeleteResult( plan.versionOfDeletion, @@ -3011,7 +3008,7 @@ protected final boolean hasBeenProcessedBefore(Operation op) { if (op.operationType() == Operation.TYPE.NO_OP) { assert noOpKeyedLock.isHeldByCurrentThread(op.seqNo()); } else { - assert versionMap.assertKeyedLockHeldByCurrentThread(op.uid().bytes()); + assert versionMap.assertKeyedLockHeldByCurrentThread(op.uid()); } } return localCheckpointTracker.hasProcessed(op.seqNo()); @@ -3266,11 +3263,11 @@ public void advanceMaxSeqNoOfUpdatesOrDeletes(long maxSeqNoOfUpdatesOnPrimary) { this.maxSeqNoOfUpdatesOrDeletes.accumulateAndGet(maxSeqNoOfUpdatesOnPrimary, Math::max); } - private boolean assertMaxSeqNoOfUpdatesIsAdvanced(Term id, long seqNo, boolean allowDeleted, boolean relaxIfGapInSeqNo) { + private boolean assertMaxSeqNoOfUpdatesIsAdvanced(BytesRef id, long seqNo, boolean allowDeleted, boolean relaxIfGapInSeqNo) { final long maxSeqNoOfUpdates = getMaxSeqNoOfUpdatesOrDeletes(); // We treat a delete on the tombstones on replicas as a regular document, then use updateDocument (not addDocument). if (allowDeleted) { - final VersionValue versionValue = versionMap.getVersionForAssert(id.bytes()); + final VersionValue versionValue = versionMap.getVersionForAssert(id); if (versionValue != null && versionValue.isDelete()) { return true; } @@ -3320,7 +3317,7 @@ private void restoreVersionMapAndCheckpointTracker(DirectoryReader directoryRead assert dv.isTombstone(docId); continue; } - final BytesRef uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)).bytes(); + final BytesRef uid = Uid.encodeId(id); try (Releasable ignored = versionMap.acquireLock(uid)) { final VersionValue curr = versionMap.getUnderLock(uid); if (curr == null || compareOpToVersionMapOnSeqNo(id, seqNo, primaryTerm, curr) == OpVsLuceneDocStatus.OP_NEWER) { @@ -3391,11 +3388,6 @@ public LiveVersionMap getLiveVersionMap() { return versionMap; } - private static boolean assertGetUsesIdField(Get get) { - assert Objects.equals(get.uid().field(), IdFieldMapper.NAME) : get.uid().field(); - return true; - } - protected long getPreCommitSegmentGeneration() { return preCommitSegmentGeneration.get(); } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/Uid.java b/server/src/main/java/org/elasticsearch/index/mapper/Uid.java index 05593ee3e99c8..84b3a5cb4e735 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/Uid.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/Uid.java @@ -112,7 +112,7 @@ private static BytesRef encodeBase64Id(String id) { } private static BytesRef encodeUtf8Id(String id) { - byte[] b = new byte[1 + UnicodeUtil.maxUTF8Length(id.length())]; + byte[] b = new byte[1 + UnicodeUtil.calcUTF16toUTF8Length(id, 0, id.length())]; // Prepend a byte that indicates that the content is an utf8 string b[0] = (byte) UTF8; int length = UnicodeUtil.UTF16toUTF8(id, 0, id.length(), b, 1); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 9c16f62335d56..39044720bea16 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -18,7 +18,6 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.Term; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; @@ -101,7 +100,6 @@ import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperMetrics; import org.elasticsearch.index.mapper.MapperService; @@ -1045,9 +1043,8 @@ public static Engine.Index prepareIndex( // whether mappings were provided or not. doc.addDynamicMappingsUpdate(mapping); } - Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(doc.id())); return new Engine.Index( - uid, + Uid.encodeId(doc.id()), doc, seqNo, primaryTerm, @@ -1210,7 +1207,7 @@ private Engine.DeleteResult applyDeleteOperation( final Engine.DeleteResult result; try { if (logger.isTraceEnabled()) { - logger.trace("delete [{}] (seq no [{}])", delete.uid().text(), delete.seqNo()); + logger.trace("delete [{}] (seq no [{}])", delete.uid(), delete.seqNo()); } result = engine.delete(delete); } catch (Exception e) { @@ -1235,8 +1232,7 @@ public static Engine.Delete prepareDelete( long ifPrimaryTerm ) { long startTime = System.nanoTime(); - final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); - return new Engine.Delete(id, uid, seqNo, primaryTerm, version, versionType, origin, startTime, ifSeqNo, ifPrimaryTerm); + return new Engine.Delete(id, Uid.encodeId(id), seqNo, primaryTerm, version, versionType, origin, startTime, ifSeqNo, ifPrimaryTerm); } public Engine.GetResult get(Engine.Get get) { diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java index b463a5ddf11a9..b05bdedef7b60 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionLookupTests.java @@ -55,7 +55,7 @@ public void testSimple() throws Exception { writer.addDocument(new Document()); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); // found doc DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); @@ -68,7 +68,7 @@ public void testSimple() throws Exception { reader.close(); reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); - lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); assertNull(lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment)); reader.close(); writer.close(); @@ -91,7 +91,7 @@ public void testTwoDocuments() throws Exception { writer.addDocument(new Document()); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); // return the last doc when there are duplicates DocIdAndVersion result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); @@ -102,7 +102,7 @@ public void testTwoDocuments() throws Exception { reader.close(); reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); - lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); result = lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment); assertNotNull(result); assertEquals(87, result.version); @@ -112,7 +112,7 @@ public void testTwoDocuments() throws Exception { reader.close(); reader = DirectoryReader.open(writer); segment = reader.leaves().get(0); - lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); assertNull(lookup.lookupVersion(new BytesRef("6"), randomBoolean(), segment)); reader.close(); writer.close(); @@ -139,12 +139,12 @@ public void testLoadTimestampRange() throws Exception { DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, true); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), true); assertTrue(lookup.loadedTimestampRange); assertEquals(lookup.minTimestamp, 1_000L); assertEquals(lookup.maxTimestamp, 1_000_000L); - lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, false); + lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), false); assertFalse(lookup.loadedTimestampRange); assertEquals(lookup.minTimestamp, 0L); assertEquals(lookup.maxTimestamp, Long.MAX_VALUE); @@ -160,7 +160,7 @@ public void testLoadTimestampRangeWithDeleteTombstone() throws Exception { writer.addDocument(ParsedDocument.deleteTombstone("_id").docs().get(0)); DirectoryReader reader = DirectoryReader.open(writer); LeafReaderContext segment = reader.leaves().get(0); - PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), IdFieldMapper.NAME, true); + PerThreadIDVersionAndSeqNoLookup lookup = new PerThreadIDVersionAndSeqNoLookup(segment.reader(), true); assertTrue(lookup.loadedTimestampRange); assertEquals(lookup.minTimestamp, 0L); assertEquals(lookup.maxTimestamp, Long.MAX_VALUE); diff --git a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java index 011a23ddb0512..cc1a677f2e2f9 100644 --- a/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java +++ b/server/src/test/java/org/elasticsearch/common/lucene/uid/VersionsTests.java @@ -57,7 +57,7 @@ public void testVersions() throws Exception { Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Lucene.STANDARD_ANALYZER)); DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()), nullValue()); Document doc = new Document(); doc.add(new StringField(IdFieldMapper.NAME, "1", Field.Store.YES)); @@ -66,7 +66,7 @@ public void testVersions() throws Exception { doc.add(new NumericDocValuesField(SeqNoFieldMapper.PRIMARY_TERM_NAME, randomLongBetween(1, Long.MAX_VALUE))); writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(1L)); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()).version, equalTo(1L)); doc = new Document(); Field uid = new StringField(IdFieldMapper.NAME, "1", Field.Store.YES); @@ -77,7 +77,7 @@ public void testVersions() throws Exception { doc.add(new NumericDocValuesField(SeqNoFieldMapper.PRIMARY_TERM_NAME, randomLongBetween(1, Long.MAX_VALUE))); writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(2L)); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()).version, equalTo(2L)); // test reuse of uid field doc = new Document(); @@ -89,11 +89,11 @@ public void testVersions() throws Exception { writer.updateDocument(new Term(IdFieldMapper.NAME, "1"), doc); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(3L)); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()).version, equalTo(3L)); writer.deleteDocuments(new Term(IdFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -121,18 +121,18 @@ public void testNestedDocuments() throws IOException { writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(5L)); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()).version, equalTo(5L)); version.setLongValue(6L); writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); version.setLongValue(7L); writer.updateDocuments(new Term(IdFieldMapper.NAME, "1"), docs); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()).version, equalTo(7L)); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()).version, equalTo(7L)); writer.deleteDocuments(new Term(IdFieldMapper.NAME, "1")); directoryReader = reopen(directoryReader); - assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), randomBoolean()), nullValue()); + assertThat(timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), randomBoolean()), nullValue()); directoryReader.close(); writer.close(); dir.close(); @@ -152,10 +152,10 @@ public void testCache() throws Exception { writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); // should increase cache size by 1 - assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); + assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new BytesRef("6"), randomBoolean()).version); assertEquals(size + 1, VersionsAndSeqNoResolver.lookupStates.size()); // should be cache hit - assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); + assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new BytesRef("6"), randomBoolean()).version); assertEquals(size + 1, VersionsAndSeqNoResolver.lookupStates.size()); reader.close(); @@ -178,11 +178,11 @@ public void testCacheFilterReader() throws Exception { doc.add(new NumericDocValuesField(SeqNoFieldMapper.PRIMARY_TERM_NAME, randomLongBetween(1, Long.MAX_VALUE))); writer.addDocument(doc); DirectoryReader reader = DirectoryReader.open(writer); - assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); + assertEquals(87, timeSeriesLoadDocIdAndVersion(reader, new BytesRef("6"), randomBoolean()).version); assertEquals(size + 1, VersionsAndSeqNoResolver.lookupStates.size()); // now wrap the reader DirectoryReader wrapped = ElasticsearchDirectoryReader.wrap(reader, new ShardId("bogus", "_na_", 5)); - assertEquals(87, timeSeriesLoadDocIdAndVersion(wrapped, new Term(IdFieldMapper.NAME, "6"), randomBoolean()).version); + assertEquals(87, timeSeriesLoadDocIdAndVersion(wrapped, new BytesRef("6"), randomBoolean()).version); // same size map: core cache key is shared assertEquals(size + 1, VersionsAndSeqNoResolver.lookupStates.size()); @@ -199,7 +199,7 @@ public void testTimeSeriesLoadDocIdAndVersion() throws Exception { DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "_na_", 1)); String id = createTSDBId(1000L); assertThat( - VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), id, randomBoolean()), + VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), id, randomBoolean()), nullValue() ); @@ -221,23 +221,11 @@ public void testTimeSeriesLoadDocIdAndVersion() throws Exception { directoryReader = reopen(directoryReader); id = createTSDBId(randomLongBetween(1000, 10000)); - assertThat( - VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), id, true), - notNullValue() - ); - assertThat( - VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "2"), id, true), - notNullValue() - ); + assertThat(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), id, true), notNullValue()); + assertThat(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("2"), id, true), notNullValue()); id = createTSDBId(randomBoolean() ? randomLongBetween(0, 999) : randomLongBetween(10001, Long.MAX_VALUE)); - assertThat( - VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "1"), id, true), - nullValue() - ); - assertThat( - VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new Term(IdFieldMapper.NAME, "2"), id, true), - nullValue() - ); + assertThat(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("1"), id, true), nullValue()); + assertThat(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(directoryReader, new BytesRef("2"), id, true), nullValue()); directoryReader.close(); writer.close(); diff --git a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java index c3c94c2730366..d753d268e45da 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -12,7 +12,6 @@ import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.FieldInvertState; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.Term; import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.TermStatistics; @@ -383,7 +382,7 @@ public Engine.Index preIndex(ShardId shardId, Engine.Index operation) { assertSame(listener, indexService.getIndexOperationListeners().get(1)); ParsedDocument doc = EngineTestCase.createParsedDoc("1", null); - Engine.Index index = new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), randomNonNegativeLong(), doc); + Engine.Index index = new Engine.Index(Uid.encodeId(doc.id()), randomNonNegativeLong(), doc); ShardId shardId = new ShardId(new Index("foo", "bar"), 0); for (IndexingOperationListener l : indexService.getIndexOperationListeners()) { l.preIndex(shardId, index); diff --git a/server/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java b/server/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java index be10fea3779b6..c743a83208a24 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexingSlowLogTests.java @@ -13,7 +13,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.core.LoggerContext; import org.apache.lucene.document.NumericDocValuesField; -import org.apache.lucene.index.Term; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; @@ -79,7 +78,7 @@ public void testLevelPrecedence() { IndexingSlowLog log = new IndexingSlowLog(settings, mock(SlowLogFieldProvider.class)); ParsedDocument doc = EngineTestCase.createParsedDoc("1", null); - Engine.Index index = new Engine.Index(new Term("_id", Uid.encodeId("doc_id")), randomNonNegativeLong(), doc); + Engine.Index index = new Engine.Index(Uid.encodeId("doc_id"), randomNonNegativeLong(), doc); Engine.IndexResult result = Mockito.mock(Engine.IndexResult.class);// (0, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, false); Mockito.when(result.getResultType()).thenReturn(Engine.Result.Type.SUCCESS); @@ -153,7 +152,7 @@ public void testTwoLoggersDifferentLevel() { IndexingSlowLog log2 = new IndexingSlowLog(index2Settings, mock(SlowLogFieldProvider.class)); ParsedDocument doc = EngineTestCase.createParsedDoc("1", null); - Engine.Index index = new Engine.Index(new Term("_id", Uid.encodeId("doc_id")), randomNonNegativeLong(), doc); + Engine.Index index = new Engine.Index(Uid.encodeId("doc_id"), randomNonNegativeLong(), doc); Engine.IndexResult result = Mockito.mock(Engine.IndexResult.class); Mockito.when(result.getResultType()).thenReturn(Engine.Result.Type.SUCCESS); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 175ce085994e5..00de132f9200e 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -1105,7 +1105,10 @@ public void testGetWithSearcherWrapper() throws Exception { new Engine.Get(true, true, "1"), mappingLookup, documentParser, - searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new TermQuery(newUid("1")))) + searcher -> SearcherHelper.wrapSearcher( + searcher, + reader -> new MatchingDirectoryReader(reader, new TermQuery(new Term(IdFieldMapper.NAME, Uid.encodeId("1")))) + ) ) ) { assertTrue(get.exists()); @@ -1119,7 +1122,10 @@ public void testGetWithSearcherWrapper() throws Exception { new Engine.Get(true, true, "1"), mappingLookup, documentParser, - searcher -> SearcherHelper.wrapSearcher(searcher, reader -> new MatchingDirectoryReader(reader, new TermQuery(newUid("2")))) + searcher -> SearcherHelper.wrapSearcher( + searcher, + reader -> new MatchingDirectoryReader(reader, new TermQuery(new Term(IdFieldMapper.NAME, Uid.encodeId("2")))) + ) ) ) { assertFalse(get.exists()); @@ -1604,7 +1610,7 @@ public void testLookupVersionWithPrunedAwayIds() throws IOException { writer.forceMerge(1); try (DirectoryReader reader = DirectoryReader.open(writer)) { assertEquals(1, reader.leaves().size()); - assertNull(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(reader, new Term(IdFieldMapper.NAME, "1"), false)); + assertNull(VersionsAndSeqNoResolver.timeSeriesLoadDocIdAndVersion(reader, new BytesRef("1"), false)); } } } @@ -1658,7 +1664,8 @@ public void testForceMergeWithSoftDeletesRetention() throws Exception { for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null); if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.id(), newUid(doc.id()), primaryTerm.get())); + String id = doc.id(); + engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(id), primaryTerm.get())); liveDocs.remove(doc.id()); } if (randomBoolean()) { @@ -1739,7 +1746,8 @@ public void testForceMergeWithSoftDeletesRetentionAndRecoverySource() throws Exc boolean useRecoverySource = randomBoolean() || omitSourceAllTheTime; ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), B_1, null, useRecoverySource); if (randomBoolean()) { - engine.delete(new Engine.Delete(doc.id(), newUid(doc.id()), primaryTerm.get())); + String id = doc.id(); + engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(id), primaryTerm.get())); liveDocs.remove(doc.id()); liveDocsWithSource.remove(doc.id()); } @@ -2373,7 +2381,7 @@ public void testVersioningPromotedReplica() throws IOException { final long finalReplicaSeqNo = lastReplicaOp.seqNo(); assertOpsOnReplica(replicaOps, replicaEngine, true, logger); final int opsOnPrimary = assertOpsOnPrimary(primaryOps, finalReplicaVersion, deletedOnReplica, replicaEngine); - final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, lastReplicaOp.uid().text())).v1(); + final long currentSeqNo = getSequenceID(replicaEngine, new Engine.Get(false, false, Term.toString(lastReplicaOp.uid()))).v1(); try (Engine.Searcher searcher = engine.acquireSearcher("test", Engine.SearcherScope.INTERNAL)) { final TotalHitCountCollector collector = new TotalHitCountCollector(); searcher.search(new MatchAllDocsQuery(), collector); @@ -2428,7 +2436,7 @@ class OpAndVersion { final AtomicInteger idGenerator = new AtomicInteger(); final Queue history = ConcurrentCollections.newQueue(); ParsedDocument doc = testParsedDocument("1", null, testDocument(), bytesArray(""), null); - final Term uidTerm = newUid(doc); + final BytesRef uidTerm = newUid(doc); engine.index(indexForDoc(doc)); for (int i = 0; i < thread.length; i++) { thread[i] = new Thread(() -> { @@ -2721,7 +2729,7 @@ public void testSeqNoAndCheckpoints() throws IOException, InterruptedException { id = randomFrom(indexedIds); final Engine.Delete delete = new Engine.Delete( id, - newUid(id), + Uid.encodeId(id), UNASSIGNED_SEQ_NO, primaryTerm.get(), rarely() ? 100 : Versions.MATCH_ANY, @@ -3063,7 +3071,7 @@ public void testEnableGcDeletes() throws Exception { engine.delete( new Engine.Delete( "2", - newUid("2"), + Uid.encodeId("2"), UNASSIGNED_SEQ_NO, 0, 10, @@ -3855,7 +3863,10 @@ public StoredValue storedValue() { try (InternalEngine engine = createEngine(indexWriterFactory, null, null, config)) { final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc)); - expectThrows(IllegalStateException.class, () -> engine.delete(new Engine.Delete("1", newUid("1"), primaryTerm.get()))); + expectThrows( + IllegalStateException.class, + () -> engine.delete(new Engine.Delete("1", Uid.encodeId("1"), primaryTerm.get())) + ); assertTrue(engine.isClosed.get()); assertSame(tragicException, engine.failedEngine.get()); } @@ -4730,7 +4741,10 @@ public void testLookupSeqNoByIdInLucene() throws Exception { assertThat(getDocIds(engine, true).stream().collect(Collectors.toMap(e -> e.id(), e -> e.seqNo())), equalTo(liveOps)); for (String id : latestOps.keySet()) { String msg = "latestOps=" + latestOps + " op=" + id; - DocIdAndSeqNo docIdAndSeqNo = VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.getIndexReader(), newUid(id)); + DocIdAndSeqNo docIdAndSeqNo = VersionsAndSeqNoResolver.loadDocIdAndSeqNo( + searcher.getIndexReader(), + Uid.encodeId(id) + ); if (liveOps.containsKey(id) == false) { assertNull(msg, docIdAndSeqNo); } else { @@ -4739,7 +4753,7 @@ public void testLookupSeqNoByIdInLucene() throws Exception { } } String notFoundId = randomValueOtherThanMany(liveOps::containsKey, () -> Long.toString(randomNonNegativeLong())); - assertNull(VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.getIndexReader(), newUid(notFoundId))); + assertNull(VersionsAndSeqNoResolver.loadDocIdAndSeqNo(searcher.getIndexReader(), Uid.encodeId(notFoundId))); } }; for (Engine.Operation op : operations) { @@ -4880,7 +4894,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); return testParsedDocument("1", null, document, B_1, null); }; - final Term uid = newUid("1"); + final BytesRef uid = Uid.encodeId("1"); final BiFunction searcherFactory = engine::acquireSearcher; for (int i = 0; i < numberOfOperations; i++) { if (randomBoolean()) { @@ -4970,12 +4984,12 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio */ public void testVersionConflictIgnoreDeletedDoc() throws IOException { ParsedDocument doc = testParsedDocument("1", null, testDocument(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); - engine.delete(new Engine.Delete("1", newUid("1"), 1)); + engine.delete(new Engine.Delete("1", Uid.encodeId("1"), 1)); for (long seqNo : new long[] { 0, 1, randomNonNegativeLong() }) { assertDeletedVersionConflict( engine.index( new Engine.Index( - newUid("1"), + Uid.encodeId("1"), doc, UNASSIGNED_SEQ_NO, 1, @@ -4996,7 +5010,7 @@ public void testVersionConflictIgnoreDeletedDoc() throws IOException { engine.delete( new Engine.Delete( "1", - newUid("1"), + Uid.encodeId("1"), UNASSIGNED_SEQ_NO, 1, Versions.MATCH_ANY, @@ -5539,7 +5553,7 @@ public void testSeqNoGenerator() throws IOException { ); final Engine.Index index = new Engine.Index( - new Term("_id", parsedDocument.id()), + BytesRef.deepCopyOf(new BytesRef(parsedDocument.id())), parsedDocument, UNASSIGNED_SEQ_NO, randomIntBetween(1, 8), @@ -5558,7 +5572,7 @@ public void testSeqNoGenerator() throws IOException { final Engine.Delete delete = new Engine.Delete( id, - new Term("_id", parsedDocument.id()), + BytesRef.deepCopyOf(new BytesRef(parsedDocument.id())), UNASSIGNED_SEQ_NO, randomIntBetween(1, 8), Versions.MATCH_ANY, @@ -6547,7 +6561,8 @@ public void testTrackMaxSeqNoOfUpdatesOrDeletesOnPrimary() throws Exception { ); } } else { - Engine.DeleteResult result = engine.delete(new Engine.Delete(doc.id(), newUid(doc.id()), primaryTerm.get())); + String id = doc.id(); + Engine.DeleteResult result = engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(id), primaryTerm.get())); liveDocIds.remove(doc.id()); assertThat( "delete operations on primary must advance max_seq_no_of_updates", @@ -6895,7 +6910,7 @@ public void testPruneAwayDeletedButRetainedIds() throws Exception { index(engine, i); } engine.forceMerge(true, 1, false, UUIDs.randomBase64UUID()); - engine.delete(new Engine.Delete("0", newUid("0"), primaryTerm.get())); + engine.delete(new Engine.Delete("0", Uid.encodeId("0"), primaryTerm.get())); engine.refresh("test"); // now we have 2 segments since we now added a tombstone plus the old segment with the delete try (Engine.Searcher searcher = engine.acquireSearcher("test")) { @@ -7106,8 +7121,8 @@ private void runTestDeleteFailure(final CheckedBiConsumer new IllegalArgumentException("fatal")); final IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> engine.delete(op)); @@ -7385,7 +7400,7 @@ public void testMaxDocsOnPrimary() throws Exception { if (randomBoolean()) { operations.add(indexForDoc(createParsedDoc(id, null))); } else { - operations.add(new Engine.Delete(id, newUid(id), primaryTerm.get())); + operations.add(new Engine.Delete(id, Uid.encodeId(id), primaryTerm.get())); } } for (int i = 0; i < numDocs; i++) { diff --git a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java index 8ed162f8cda81..6f568ecf347c4 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LuceneChangesSnapshotTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.SnapshotMatchers; import org.elasticsearch.index.translog.Translog; @@ -65,7 +66,7 @@ public void testBasics() throws Exception { if (randomBoolean()) { engine.index(indexForDoc(doc)); } else { - engine.delete(new Engine.Delete(doc.id(), newUid(doc.id()), primaryTerm.get())); + engine.delete(new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get())); } if (rarely()) { if (randomBoolean()) { @@ -270,7 +271,7 @@ public void testUpdateAndReadChangesConcurrently() throws Exception { if (randomBoolean()) { op = new Engine.Index(newUid(doc), primaryTerm.get(), doc); } else { - op = new Engine.Delete(doc.id(), newUid(doc.id()), primaryTerm.get()); + op = new Engine.Delete(doc.id(), Uid.encodeId(doc.id()), primaryTerm.get()); } } else { if (randomBoolean()) { diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java index 5efdd4c79940c..aa298955fc08f 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.DocsStats; @@ -117,7 +118,7 @@ public void testNoOpEngineStats() throws Exception { for (int i = 0; i < numDocs; i++) { if (randomBoolean()) { String delId = Integer.toString(i); - Engine.DeleteResult result = engine.delete(new Engine.Delete(delId, newUid(delId), primaryTerm.get())); + Engine.DeleteResult result = engine.delete(new Engine.Delete(delId, Uid.encodeId(delId), primaryTerm.get())); assertTrue(result.isFound()); engine.syncTranslog(); // advance persisted local checkpoint globalCheckpoint.set(engine.getPersistedLocalCheckpoint()); diff --git a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java index 5bb51f99dfb1c..cfc7e82fddab3 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/ReadOnlyEngineTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.store.Store; @@ -94,7 +95,7 @@ public void testReadOnlyEngine() throws Exception { for (int i = 0; i < numDocs; i++) { if (randomBoolean()) { String delId = Integer.toString(i); - engine.delete(new Engine.Delete(delId, newUid(delId), primaryTerm.get())); + engine.delete(new Engine.Delete(delId, Uid.encodeId(delId), primaryTerm.get())); } if (rarely()) { engine.flush(); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java index 4c05486f97990..99a85f7479dd5 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexingOperationListenerTests.java @@ -7,7 +7,6 @@ */ package org.elasticsearch.index.shard; -import org.apache.lucene.index.Term; import org.elasticsearch.index.Index; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineTestCase; @@ -127,8 +126,8 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { logger ); ParsedDocument doc = EngineTestCase.createParsedDoc("1", null); - Engine.Delete delete = new Engine.Delete("1", new Term("_id", Uid.encodeId(doc.id())), randomNonNegativeLong()); - Engine.Index index = new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), randomNonNegativeLong(), doc); + Engine.Delete delete = new Engine.Delete("1", Uid.encodeId(doc.id()), randomNonNegativeLong()); + Engine.Index index = new Engine.Index(Uid.encodeId(doc.id()), randomNonNegativeLong(), doc); compositeListener.postDelete( randomShardId, delete, diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index aaf0f40a73004..7f22c9f9ccc2a 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -13,10 +13,10 @@ import org.apache.lucene.document.StringField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.common.bytes.BytesArray; @@ -547,10 +547,10 @@ private Engine.IndexResult index(String id) throws IOException { } private Engine.IndexResult index(String id, String testFieldValue) throws IOException { - final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); + final BytesRef uid = Uid.encodeId(id); LuceneDocument document = new LuceneDocument(); document.add(new TextField("test", testFieldValue, Field.Store.YES)); - Field idField = new StringField(uid.field(), uid.bytes(), Field.Store.YES); + Field idField = new StringField(IdFieldMapper.NAME, uid, Field.Store.YES); Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY); SeqNoFieldMapper.SequenceIDFields seqID = SeqNoFieldMapper.SequenceIDFields.emptySeqID(); document.add(idField); diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index aa6ff7073a21d..6aaeabdc175da 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -16,7 +16,6 @@ import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.Term; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.ByteArrayDataOutput; import org.apache.lucene.store.DataOutput; @@ -25,6 +24,7 @@ import org.apache.lucene.tests.store.MockDirectoryWrapper; import org.apache.lucene.tests.util.LineFileDocs; import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -961,8 +961,8 @@ private void truncateTranslogs(Path directory) throws Exception { } } - private Term newUid(ParsedDocument doc) { - return new Term("_id", Uid.encodeId(doc.id())); + private static BytesRef newUid(ParsedDocument doc) { + return Uid.encodeId(doc.id()); } public void testVerifyTranslogIsNotDeleted() throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 52927432f5351..3a7a31e761e7f 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -914,12 +914,8 @@ protected static BytesArray bytesArray(String string) { return new BytesArray(string.getBytes(Charset.defaultCharset())); } - public static Term newUid(String id) { - return new Term("_id", Uid.encodeId(id)); - } - - public static Term newUid(ParsedDocument doc) { - return newUid(doc.id()); + public static BytesRef newUid(ParsedDocument doc) { + return Uid.encodeId(doc.id()); } protected Engine.Get newGet(boolean realtime, ParsedDocument doc) { @@ -950,7 +946,7 @@ protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { return new Engine.Delete( id, - newUid(id), + Uid.encodeId(id), seqNo, 1, version, @@ -987,7 +983,7 @@ public static List generateSingleDocHistory( ) { final int numOfOps = randomIntBetween(minOpCount, maxOpCount); final List ops = new ArrayList<>(); - final Term id = newUid(docId); + final BytesRef id = Uid.encodeId(docId); final int startWithSeqNo = 0; final String valuePrefix = (forReplica ? "r_" : "p_") + docId + "_"; final boolean incrementTermWhenIntroducingSeqNo = randomBoolean(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java index 15e358b68e648..7c9b1b5efbde2 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineTests.java @@ -7,9 +7,9 @@ package org.elasticsearch.xpack.ccr.index.engine; import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.CheckedBiConsumer; @@ -177,7 +177,7 @@ public void runDeleteTest( final String id = "id"; final Engine.Delete delete = new Engine.Delete( id, - new Term("_id", id), + BytesRef.deepCopyOf(new BytesRef(id)), seqNo, primaryTerm.get(), randomNonNegativeLong(), From f4d87e0f2570a6d80f58c90fbbce7e340c6d38f1 Mon Sep 17 00:00:00 2001 From: Karen Metts <35154725+karenzone@users.noreply.github.com> Date: Mon, 10 Jun 2024 12:24:23 -0400 Subject: [PATCH 13/31] [DOCS] Add note that Logstash sets up data streams (#109502) --- .../data-streams/set-up-a-data-stream.asciidoc | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/docs/reference/data-streams/set-up-a-data-stream.asciidoc b/docs/reference/data-streams/set-up-a-data-stream.asciidoc index 57388a1199f57..a8cbbeac06077 100644 --- a/docs/reference/data-streams/set-up-a-data-stream.asciidoc +++ b/docs/reference/data-streams/set-up-a-data-stream.asciidoc @@ -13,9 +13,16 @@ To set up a data stream, follow these steps: You can also <>. -IMPORTANT: If you use {fleet} or {agent}, skip this tutorial. {fleet} and -{agent} set up data streams for you. See {fleet}'s -{fleet-guide}/data-streams.html[data streams] documentation. +[IMPORTANT] +-- +If you use {fleet}, {agent}, or {ls}, skip this tutorial. +They all set up data streams for you. + +For {fleet} and {agent}, check out this {fleet-guide}/data-streams.html[data streams documentation]. +For {ls}, check out the +{logstash-ref}/plugins-outputs-elasticsearch.html#plugins-outputs-elasticsearch-data_stream[data streams settings] +for the `elasticsearch output` plugin. +-- [discrete] [[create-index-lifecycle-policy]] From 7e7f8a379a1dc2c302d71b7fa90f609250425527 Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Mon, 10 Jun 2024 18:39:02 +0200 Subject: [PATCH 14/31] Make dense vector field type updatable (#106591) --- docs/changelog/106591.yaml | 5 + .../upgrades/DenseVectorMappingUpdateIT.java | 160 ++ .../180_update_dense_vector_type.yml | 1363 +++++++++++++++++ .../vectors/DenseVectorFieldMapper.java | 48 +- .../vectors/DenseVectorFieldMapperTests.java | 102 +- .../index/mapper/MapperTestCase.java | 6 +- 6 files changed, 1677 insertions(+), 7 deletions(-) create mode 100644 docs/changelog/106591.yaml create mode 100644 qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DenseVectorMappingUpdateIT.java create mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml diff --git a/docs/changelog/106591.yaml b/docs/changelog/106591.yaml new file mode 100644 index 0000000000000..6a7814cb9cede --- /dev/null +++ b/docs/changelog/106591.yaml @@ -0,0 +1,5 @@ +pr: 106591 +summary: Make dense vector field type updatable +area: Search +type: enhancement +issues: [] diff --git a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DenseVectorMappingUpdateIT.java b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DenseVectorMappingUpdateIT.java new file mode 100644 index 0000000000000..99cbef6556932 --- /dev/null +++ b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DenseVectorMappingUpdateIT.java @@ -0,0 +1,160 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ +package org.elasticsearch.upgrades; + +import com.carrotsearch.randomizedtesting.annotations.Name; + +import org.apache.http.util.EntityUtils; +import org.elasticsearch.Version; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.common.Strings; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.function.Predicate; + +import static org.elasticsearch.rest.action.search.RestSearchAction.TOTAL_HITS_AS_INT_PARAM; + +/** + * This IT indexes some dense vector on an old node, then update its mapping and, once upgraded, checks that KNN search still works + * before and after further data indexing. + */ +public class DenseVectorMappingUpdateIT extends AbstractRollingUpgradeTestCase { + + private static final String BULK1 = """ + {"index": {"_id": "1"}} + {"embedding": [1, 1, 1, 1]} + {"index": {"_id": "2"}} + {"embedding": [1, 1, 1, 2]} + {"index": {"_id": "3"}} + {"embedding": [1, 1, 1, 3]} + {"index": {"_id": "4"}} + {"embedding": [1, 1, 1, 4]} + {"index": {"_id": "5"}} + {"embedding": [1, 1, 1, 5]} + {"index": {"_id": "6"}} + {"embedding": [1, 1, 1, 6]} + {"index": {"_id": "7"}} + {"embedding": [1, 1, 1, 7]} + {"index": {"_id": "8"}} + {"embedding": [1, 1, 1, 8]} + {"index": {"_id": "9"}} + {"embedding": [1, 1, 1, 9]} + {"index": {"_id": "10"}} + {"embedding": [1, 1, 1, 10]} + """; + + private static final String BULK2 = """ + {"index": {"_id": "11"}} + {"embedding": [1, 0, 1, 1]} + {"index": {"_id": "12"}} + {"embedding": [1, 2, 1, 1]} + {"index": {"_id": "13"}} + {"embedding": [1, 3, 1, 1]} + {"index": {"_id": "14"}} + {"embedding": [1, 4, 1, 1]} + {"index": {"_id": "15"}} + {"embedding": [1, 5, 1, 1]} + {"index": {"_id": "16"}} + {"embedding": [1, 6, 1, 1]} + {"index": {"_id": "17"}} + {"embedding": [1, 7, 1, 1]} + {"index": {"_id": "18"}} + {"embedding": [1, 8, 1, 1]} + {"index": {"_id": "19"}} + {"embedding": [1, 9, 1, 1]} + {"index": {"_id": "20"}} + {"embedding": [1, 10, 1, 1]} + """; + + public DenseVectorMappingUpdateIT(@Name("upgradedNodes") int upgradedNodes) { + super(upgradedNodes); + } + + public void testDenseVectorMappingUpdateOnOldCluster() throws IOException { + if (getOldClusterTestVersion().after(Version.V_8_7_0.toString())) { + String indexName = "test_index"; + if (isOldCluster()) { + Request createIndex = new Request("PUT", "/" + indexName); + XContentBuilder mappings = XContentBuilder.builder(XContentType.JSON.xContent()) + .startObject() + .startObject("mappings") + .startObject("properties") + .startObject("embedding") + .field("type", "dense_vector") + .field("dims", 4) + .startObject("index_options") + .field("type", "hnsw") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject(); + createIndex.setJsonEntity(Strings.toString(mappings)); + client().performRequest(createIndex); + Request index = new Request("POST", "/" + indexName + "/_bulk/"); + index.addParameter("refresh", "true"); + index.setJsonEntity(BULK1); + client().performRequest(index); + } + + int expectedCount = 10; + + assertCount("test_index", expectedCount); + + if (isUpgradedCluster() && clusterSupportsDenseVectorTypeUpdate()) { + Request updateMapping = new Request("PUT", "/" + indexName + "/_mapping"); + XContentBuilder mappings = XContentBuilder.builder(XContentType.JSON.xContent()) + .startObject() + .startObject("properties") + .startObject("embedding") + .field("type", "dense_vector") + .field("dims", 4) + .startObject("index_options") + .field("type", "int8_hnsw") + .endObject() + .endObject() + .endObject() + .endObject(); + updateMapping.setJsonEntity(Strings.toString(mappings)); + assertOK(client().performRequest(updateMapping)); + Request index = new Request("POST", "/" + indexName + "/_bulk/"); + index.addParameter("refresh", "true"); + index.setJsonEntity(BULK2); + assertOK(client().performRequest(index)); + expectedCount = 20; + assertCount("test_index", expectedCount); + } + } + } + + private void assertCount(String index, int count) throws IOException { + Request searchTestIndexRequest = new Request("POST", "/" + index + "/_search"); + searchTestIndexRequest.addParameter(TOTAL_HITS_AS_INT_PARAM, "true"); + searchTestIndexRequest.addParameter("filter_path", "hits.total"); + Response searchTestIndexResponse = client().performRequest(searchTestIndexRequest); + assertEquals( + "{\"hits\":{\"total\":" + count + "}}", + EntityUtils.toString(searchTestIndexResponse.getEntity(), StandardCharsets.UTF_8) + ); + } + + private boolean clusterSupportsDenseVectorTypeUpdate() throws IOException { + Map response = entityAsMap(client().performRequest(new Request("GET", "_nodes"))); + Map nodes = (Map) response.get("nodes"); + + Predicate> nodeSupportsBulkApi = n -> Version.fromString(n.get("version").toString()).onOrAfter(Version.V_8_14_0); + + return nodes.values().stream().map(o -> (Map) o).allMatch(nodeSupportsBulkApi); + } + +} diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml new file mode 100644 index 0000000000000..0780b789e92a1 --- /dev/null +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml @@ -0,0 +1,1363 @@ +setup: + - requires: + cluster_features: "gte_v8.15.0" + reason: 'updatable dense vector field types was added in 8.15' +--- +"Test create and update dense vector mapping with per-doc indexing and flush": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: flat } + + - do: + index: + index: test_index + id: "1" + body: + embedding: [ 1, 1, 1, 1 ] + - do: + index: + index: test_index + id: "2" + body: + embedding: [ 1, 1, 1, 2 ] + - do: + index: + index: test_index + id: "3" + body: + embedding: [ 1, 1, 1, 3 ] + - do: + index: + index: test_index + id: "4" + body: + embedding: [ 1, 1, 1, 4 ] + - do: + index: + index: test_index + id: "5" + body: + embedding: [ 1, 1, 1, 5 ] + + - do: + indices.flush: { } + + - do: + index: + index: test_index + id: "6" + body: + embedding: [ 1, 1, 1, 6 ] + - do: + index: + index: test_index + id: "7" + body: + embedding: [ 1, 1, 1, 7 ] + - do: + index: + index: test_index + id: "8" + body: + embedding: [ 1, 1, 1, 8 ] + - do: + index: + index: test_index + id: "9" + body: + embedding: [ 1, 1, 1, 9 ] + - do: + index: + index: test_index + id: "10" + body: + embedding: [ 1, 1, 1, 10 ] + + - do: + indices.flush: { } + + - do: + indices.refresh: {} + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [1, 1, 1, 1] + num_candidates: 10 + + - match: { hits.total.value: 10 } + - length: {hits.hits: 3} + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "3" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + + - do: + index: + index: test_index + id: "11" + body: + embedding: [ 2, 1, 1, 1 ] + - do: + index: + index: test_index + id: "12" + body: + embedding: [ 3, 1, 1, 2 ] + - do: + index: + index: test_index + id: "13" + body: + embedding: [ 4, 1, 1, 3 ] + - do: + index: + index: test_index + id: "14" + body: + embedding: [ 5, 1, 1, 4 ] + - do: + index: + index: test_index + id: "15" + body: + embedding: [ 6, 1, 1, 5 ] + + - do: + indices.flush: { } + + - do: + index: + index: test_index + id: "16" + body: + embedding: [ 7, 1, 1, 6 ] + - do: + index: + index: test_index + id: "17" + body: + embedding: [ 8, 1, 1, 7 ] + - do: + index: + index: test_index + id: "18" + body: + embedding: [ 9, 1, 1, 8 ] + - do: + index: + index: test_index + id: "19" + body: + embedding: [ 10, 1, 1, 9 ] + - do: + index: + index: test_index + id: "20" + body: + embedding: [ 1, 11, 1, 10 ] + + - do: + indices.flush: { } + + - do: + indices.refresh: {} + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 20 + + - match: { hits.total.value: 20 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "11" } + - match: { hits.hits.2._id: "2" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 3 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + index: + index: test_index + id: "21" + body: + embedding: [ 1, 1, 2, 1 ] + - do: + index: + index: test_index + id: "22" + body: + embedding: [ 1, 1, 3, 1 ] + - do: + index: + index: test_index + id: "23" + body: + embedding: [ 1, 1, 4, 1 ] + - do: + index: + index: test_index + id: "24" + body: + embedding: [ 1, 1, 5, 1 ] + - do: + index: + index: test_index + id: "25" + body: + embedding: [ 1, 1, 6, 1 ] + + - do: + indices.flush: { } + + - do: + index: + index: test_index + id: "26" + body: + embedding: [ 1, 1, 7, 1 ] + - do: + index: + index: test_index + id: "27" + body: + embedding: [ 1, 1, 8, 1 ] + - do: + index: + index: test_index + id: "28" + body: + embedding: [ 1, 1, 9, 1 ] + - do: + index: + index: test_index + id: "29" + body: + embedding: [ 1, 1, 10, 1 ] + - do: + index: + index: test_index + id: "30" + body: + embedding: [ 1, 1, 11, 1 ] + + - do: + indices.flush: { } + + - do: + indices.refresh: {} + + - do: + search: + index: test_index + body: + size: 4 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 30 + + - match: { hits.total.value: 30 } + - length: { hits.hits: 4 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "11" } + - match: { hits.hits.2._id: "2" } + - match: { hits.hits.3._id: "21" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + ef_construction: 200 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + + - do: + index: + index: test_index + id: "31" + body: + embedding: [ 1, 1, 1, 2 ] + - do: + index: + index: test_index + id: "32" + body: + embedding: [ 1, 1, 1, 3 ] + - do: + index: + index: test_index + id: "33" + body: + embedding: [ 1, 1, 1, 4 ] + - do: + index: + index: test_index + id: "34" + body: + embedding: [ 1, 1, 1, 5 ] + - do: + index: + index: test_index + id: "35" + body: + embedding: [ 1, 1, 1, 6 ] + + - do: + indices.flush: { } + + - do: + index: + index: test_index + id: "36" + body: + embedding: [ 1, 1, 1, 7 ] + - do: + index: + index: test_index + id: "37" + body: + embedding: [ 1, 1, 1, 8 ] + - do: + index: + index: test_index + id: "38" + body: + embedding: [ 1, 1, 1, 9 ] + - do: + index: + index: test_index + id: "39" + body: + embedding: [ 1, 1, 1, 10 ] + - do: + index: + index: test_index + id: "40" + body: + embedding: [ 1, 1, 1, 11 ] + + - do: + indices.flush: { } + + - do: + indices.refresh: {} + + - do: + search: + index: test_index + body: + size: 5 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 40 + + - match: { hits.total.value: 40 } + - length: { hits.hits: 5 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "11" } + - match: { hits.hits.2._id: "31" } + - match: { hits.hits.3._id: "2" } + - match: { hits.hits.4._id: "21" } + + +--- +"Test create and update dense vector mapping with bulk indexing": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: flat } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "1"}}' + - '{"embedding": [1, 1, 1, 1]}' + - '{"index": {"_id": "2"}}' + - '{"embedding": [1, 1, 1, 2]}' + - '{"index": {"_id": "3"}}' + - '{"embedding": [1, 1, 1, 3]}' + - '{"index": {"_id": "4"}}' + - '{"embedding": [1, 1, 1, 4]}' + - '{"index": {"_id": "5"}}' + - '{"embedding": [1, 1, 1, 5]}' + - '{"index": {"_id": "6"}}' + - '{"embedding": [1, 1, 1, 6]}' + - '{"index": {"_id": "7"}}' + - '{"embedding": [1, 1, 1, 7]}' + - '{"index": {"_id": "8"}}' + - '{"embedding": [1, 1, 1, 8]}' + - '{"index": {"_id": "9"}}' + - '{"embedding": [1, 1, 1, 9]}' + - '{"index": {"_id": "10"}}' + - '{"embedding": [1, 1, 1, 10]}' + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [1, 1, 1, 1] + num_candidates: 10 + + - match: { hits.total.value: 10 } + - length: {hits.hits: 3} + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "3" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "11"}}' + - '{"embedding": [2, 1, 1, 1]}' + - '{"index": {"_id": "12"}}' + - '{"embedding": [3, 1, 1, 2]}' + - '{"index": {"_id": "13"}}' + - '{"embedding": [4, 1, 1, 3]}' + - '{"index": {"_id": "14"}}' + - '{"embedding": [5, 1, 1, 4]}' + - '{"index": {"_id": "15"}}' + - '{"embedding": [6, 1, 1, 5]}' + - '{"index": {"_id": "16"}}' + - '{"embedding": [7, 1, 1, 6]}' + - '{"index": {"_id": "17"}}' + - '{"embedding": [8, 1, 1, 7]}' + - '{"index": {"_id": "18"}}' + - '{"embedding": [9, 1, 1, 8]}' + - '{"index": {"_id": "19"}}' + - '{"embedding": [10, 1, 1, 9]}' + - '{"index": {"_id": "20"}}' + - '{"embedding": [1, 11, 1, 10]}' + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 20 + + - match: { hits.total.value: 20 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "11" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 3 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "21"}}' + - '{"embedding": [1, 1, 2, 1]}' + - '{"index": {"_id": "22"}}' + - '{"embedding": [1, 1, 3, 1]}' + - '{"index": {"_id": "23"}}' + - '{"embedding": [1, 1, 4, 1]}' + - '{"index": {"_id": "24"}}' + - '{"embedding": [1, 1, 6, 1]}' + - '{"index": {"_id": "25"}}' + - '{"embedding": [1, 1, 7, 1]}' + - '{"index": {"_id": "26"}}' + - '{"embedding": [1, 1, 8, 1]}' + - '{"index": {"_id": "27"}}' + - '{"embedding": [1, 1, 9, 1]}' + - '{"index": {"_id": "28"}}' + - '{"embedding": [1, 1, 10, 1]}' + - '{"index": {"_id": "29"}}' + - '{"embedding": [1, 1, 11, 1]}' + - '{"index": {"_id": "30"}}' + - '{"embedding": [1, 1, 12, 1]}' + - do: + search: + index: test_index + body: + size: 4 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 30 + + - match: { hits.total.value: 30 } + - length: { hits.hits: 4 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "21" } + - match: { hits.hits.3._id: "11" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + ef_construction: 200 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "31"}}' + - '{"embedding": [1, 1, 1, 2]}' + - '{"index": {"_id": "32"}}' + - '{"embedding": [1, 1, 1, 3]}' + - '{"index": {"_id": "33"}}' + - '{"embedding": [1, 1, 1, 4]}' + - '{"index": {"_id": "34"}}' + - '{"embedding": [1, 1, 1, 5]}' + - '{"index": {"_id": "35"}}' + - '{"embedding": [1, 1, 1, 6]}' + - '{"index": {"_id": "36"}}' + - '{"embedding": [1, 1, 1, 7]}' + - '{"index": {"_id": "37"}}' + - '{"embedding": [1, 1, 1, 8]}' + - '{"index": {"_id": "38"}}' + - '{"embedding": [1, 1, 1, 9]}' + - '{"index": {"_id": "39"}}' + - '{"embedding": [1, 1, 1, 10]}' + - '{"index": {"_id": "40"}}' + - '{"embedding": [1, 1, 1, 11]}' + - do: + search: + index: test_index + body: + size: 5 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 40 + + - match: { hits.total.value: 40 } + - length: { hits.hits: 5 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "21" } + - match: { hits.hits.3._id: "31" } + - match: { hits.hits.4._id: "11" } + +--- +"Index, update and merge": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: flat } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "1"}}' + - '{"embedding": [1, 1, 1, 1]}' + - '{"index": {"_id": "2"}}' + - '{"embedding": [1, 1, 1, 2]}' + - '{"index": {"_id": "3"}}' + - '{"embedding": [1, 1, 1, 3]}' + - '{"index": {"_id": "4"}}' + - '{"embedding": [1, 1, 1, 4]}' + - '{"index": {"_id": "5"}}' + - '{"embedding": [1, 1, 1, 5]}' + - '{"index": {"_id": "6"}}' + - '{"embedding": [1, 1, 1, 6]}' + - '{"index": {"_id": "7"}}' + - '{"embedding": [1, 1, 1, 7]}' + - '{"index": {"_id": "8"}}' + - '{"embedding": [1, 1, 1, 8]}' + - '{"index": {"_id": "9"}}' + - '{"embedding": [1, 1, 1, 9]}' + - '{"index": {"_id": "10"}}' + - '{"embedding": [1, 1, 1, 10]}' + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 10 + + - match: { hits.total.value: 10 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "3" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + + - do: + indices.forcemerge: + index: test_index + max_num_segments: 1 + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 10 + + - match: { hits.total.value: 10 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "3" } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "11"}}' + - '{"embedding": [2, 1, 1, 1]}' + - '{"index": {"_id": "12"}}' + - '{"embedding": [3, 1, 1, 2]}' + - '{"index": {"_id": "13"}}' + - '{"embedding": [4, 1, 1, 3]}' + - '{"index": {"_id": "14"}}' + - '{"embedding": [5, 1, 1, 4]}' + - '{"index": {"_id": "15"}}' + - '{"embedding": [6, 1, 1, 5]}' + - '{"index": {"_id": "16"}}' + - '{"embedding": [7, 1, 1, 6]}' + - '{"index": {"_id": "17"}}' + - '{"embedding": [8, 1, 1, 7]}' + - '{"index": {"_id": "18"}}' + - '{"embedding": [9, 1, 1, 8]}' + - '{"index": {"_id": "19"}}' + - '{"embedding": [10, 1, 1, 9]}' + - '{"index": {"_id": "20"}}' + - '{"embedding": [1, 11, 1, 10]}' + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 20 + + - match: { hits.total.value: 20 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "11" } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 3 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + search: + index: test_index + body: + size: 3 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 20 + + - match: { hits.total.value: 20 } + - length: { hits.hits: 3 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "11" } + + - do: + bulk: + refresh: true + index: test_index + body: + - '{"index": {"_id": "21"}}' + - '{"embedding": [1, 1, 2, 1]}' + - '{"index": {"_id": "22"}}' + - '{"embedding": [1, 1, 3, 1]}' + - '{"index": {"_id": "23"}}' + - '{"embedding": [1, 1, 4, 1]}' + - '{"index": {"_id": "24"}}' + - '{"embedding": [1, 1, 6, 1]}' + - '{"index": {"_id": "25"}}' + - '{"embedding": [1, 1, 7, 1]}' + - '{"index": {"_id": "26"}}' + - '{"embedding": [1, 1, 8, 1]}' + - '{"index": {"_id": "27"}}' + - '{"embedding": [1, 1, 9, 1]}' + - '{"index": {"_id": "28"}}' + - '{"embedding": [1, 1, 10, 1]}' + - '{"index": {"_id": "29"}}' + - '{"embedding": [1, 1, 11, 1]}' + - '{"index": {"_id": "30"}}' + - '{"embedding": [1, 1, 12, 1]}' + - do: + search: + index: test_index + body: + size: 4 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 30 + + - match: { hits.total.value: 30 } + - length: { hits.hits: 4 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "21" } + - match: { hits.hits.3._id: "11" } + + - do: + indices.forcemerge: + index: test_index + max_num_segments: 1 + + - do: + search: + index: test_index + body: + size: 4 + query: + knn: + field: embedding + query_vector: [ 1, 1, 1, 1 ] + num_candidates: 30 + + - match: { hits.total.value: 30 } + - length: { hits.hits: 4 } + - match: { hits.hits.0._id: "1" } + - match: { hits.hits.1._id: "2" } + - match: { hits.hits.2._id: "21" } + - match: { hits.hits.3._id: "11" } + + +--- +"Disallowed dense vector update path hnsw --> flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + +--- +"Disallowed dense vector update path hnsw --> int8_flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + +--- +"Disallowed dense vector update path int8_hnsw --> flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + +--- +"Disallowed dense vector update path int8_hnsw --> int8_flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + +--- +"Disallowed dense vector update path int8_flat --> flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: flat + +--- +"Allowed dense vector updates on same type but different other index_options, hnsw": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 24 + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + - match: { test_index.mappings.properties.embedding.index_options.m: 24 } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 24 + ef_construction: 200 + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: hnsw } + - match: { test_index.mappings.properties.embedding.index_options.m: 24 } + - match: { test_index.mappings.properties.embedding.index_options.ef_construction: 200 } + + - do: + catch: /illegal_argument_exception/ + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: hnsw + m: 3 + ef_construction: 200 + +--- +"Allowed dense vector updates on same type but different other index_options, int8_hnsw": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + m: 32 + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + - match: { test_index.mappings.properties.embedding.index_options.m: 32 } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + m: 32 + ef_construction: 200 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + - match: { test_index.mappings.properties.embedding.index_options.m: 32 } + - match: { test_index.mappings.properties.embedding.index_options.ef_construction: 200 } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + m: 32 + ef_construction: 200 + confidence_interval: 0.3 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_hnsw } + - match: { test_index.mappings.properties.embedding.index_options.m: 32 } + - match: { test_index.mappings.properties.embedding.index_options.ef_construction: 200 } + - match: { test_index.mappings.properties.embedding.index_options.confidence_interval: 0.3 } + + - do: + catch: /illegal_argument_exception/ # fails because m = 10 is less than the current value of 20 + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + ef_construction: 200 + m: 10 + confidence_interval: 0.3 + + - do: + catch: /illegal_argument_exception/ # fails because m = 16 by default, which is less than the current value of 20 + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_hnsw + ef_construction: 200 + confidence_interval: 0.3 + +--- +"Allowed dense vector updates on same type but different other index_options, int8_flat": + - do: + indices.create: + index: test_index + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + + - do: + indices.put_mapping: + index: test_index + body: + properties: + embedding: + type: dense_vector + dims: 4 + index_options: + type: int8_flat + confidence_interval: 0.3 + + - do: + indices.get_mapping: + index: test_index + + - match: { test_index.mappings.properties.embedding.type: dense_vector } + - match: { test_index.mappings.properties.embedding.index_options.type: int8_flat } + - match: { test_index.mappings.properties.embedding.index_options.confidence_interval: 0.3 } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapper.java index c00f360e94c7e..97961228f7680 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapper.java @@ -154,7 +154,9 @@ public static class Builder extends FieldMapper.Builder { }, m -> toType(m).fieldType().dims, XContentBuilder::field, Object::toString).setSerializerCheck((id, ic, v) -> v != null) .setMergeValidator((previous, current, c) -> previous == null || Objects.equals(previous, current)); private final Parameter similarity; + private final Parameter indexOptions; + private final Parameter indexed; private final Parameter> meta = Parameter.metaParam(); @@ -179,7 +181,7 @@ public Builder(String name, IndexVersion indexVersionCreated) { ).acceptsNull().setSerializerCheck((id, ic, v) -> v != null); this.indexOptions = new Parameter<>( "index_options", - false, + true, () -> defaultInt8Hnsw && elementType.getValue() != ElementType.BYTE && this.indexed.getValue() ? new Int8HnswIndexOptions( Lucene99HnswVectorsFormat.DEFAULT_MAX_CONN, @@ -201,7 +203,9 @@ public Builder(String name, IndexVersion indexVersionCreated) { "[element_type] cannot be [" + elementType.getValue().toString() + "] when using index type [" + v.type + "]" ); } - }).acceptsNull(); + }) + .acceptsNull() + .setMergeValidator((previous, current, c) -> previous == null || current == null || previous.updatableTo(current)); if (defaultInt8Hnsw) { this.indexOptions.alwaysSerialize(); } @@ -865,6 +869,8 @@ private abstract static class IndexOptions implements ToXContent { boolean supportsElementType(ElementType elementType) { return true; } + + abstract boolean updatableTo(IndexOptions update); } private enum VectorIndexType { @@ -981,6 +987,13 @@ public int hashCode() { boolean supportsElementType(ElementType elementType) { return elementType != ElementType.BYTE; } + + @Override + boolean updatableTo(IndexOptions update) { + return update.type.equals(this.type) + || update.type.equals(VectorIndexType.HNSW.name) + || update.type.equals(VectorIndexType.INT8_HNSW.name); + } } private static class FlatIndexOptions extends IndexOptions { @@ -1002,6 +1015,11 @@ KnnVectorsFormat getVectorsFormat() { return new ES813FlatVectorFormat(); } + @Override + boolean updatableTo(IndexOptions update) { + return true; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -1074,6 +1092,21 @@ public String toString() { boolean supportsElementType(ElementType elementType) { return elementType != ElementType.BYTE; } + + @Override + boolean updatableTo(IndexOptions update) { + boolean updatable = update.type.equals(this.type); + if (updatable) { + Int8HnswIndexOptions int8HnswIndexOptions = (Int8HnswIndexOptions) update; + // fewer connections would break assumptions on max number of connections (based on largest previous graph) during merge + // quantization could not behave as expected with different confidence intervals (and quantiles) to be created + updatable = int8HnswIndexOptions.m >= this.m; + updatable &= confidenceInterval == null + || int8HnswIndexOptions.confidenceInterval != null + && confidenceInterval.equals(int8HnswIndexOptions.confidenceInterval); + } + return updatable; + } } private static class HnswIndexOptions extends IndexOptions { @@ -1091,6 +1124,17 @@ public KnnVectorsFormat getVectorsFormat() { return new Lucene99HnswVectorsFormat(m, efConstruction, 1, null); } + @Override + boolean updatableTo(IndexOptions update) { + boolean updatable = update.type.equals(this.type); + if (updatable) { + // fewer connections would break assumptions on max number of connections (based on largest previous graph) during merge + HnswIndexOptions hnswIndexOptions = (HnswIndexOptions) update; + updatable = hnswIndexOptions.m >= this.m; + } + return updatable || (update.type.equals(VectorIndexType.INT8_HNSW.name) && ((Int8HnswIndexOptions) update).m >= m); + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapperTests.java index 4072e0e95bfe7..8e3b4e7e2e06e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/vectors/DenseVectorFieldMapperTests.java @@ -158,18 +158,112 @@ protected void registerParameters(ParameterChecker checker) throws IOException { .field("element_type", "float") ) ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "flat") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_flat") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"int8_flat\"")) + ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "flat") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "hnsw") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"hnsw\"")) + ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "flat") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_hnsw") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"int8_hnsw\"")) + ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_flat") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "hnsw") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"hnsw\"")) + ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_flat") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_hnsw") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"int8_hnsw\"")) + ); + checker.registerUpdateCheck( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "hnsw") + .endObject(), + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "int8_hnsw") + .endObject(), + m -> assertTrue(m.toString().contains("\"type\":\"int8_hnsw\"")) + ); checker.registerConflictCheck( "index_options", - fieldMapping(b -> b.field("type", "dense_vector").field("dims", 4).field("index", true).field("similarity", "dot_product")), fieldMapping( b -> b.field("type", "dense_vector") .field("dims", 4) .field("index", true) - .field("similarity", "dot_product") .startObject("index_options") .field("type", "hnsw") - .field("m", 5) - .field("ef_construction", 80) + .endObject() + ), + fieldMapping( + b -> b.field("type", "dense_vector") + .field("dims", 4) + .field("index", true) + .startObject("index_options") + .field("type", "flat") .endObject() ) ); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java index b2c2afd9df429..a3f965d06a5c3 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java @@ -712,7 +712,11 @@ public void testUpdates() throws IOException { ); assertThat( e.getMessage(), - anyOf(containsString("Cannot update parameter [" + param + "]"), containsString("different [" + param + "]")) + anyOf( + containsString("Cannot update parameter [" + param + "]"), + containsString("different [" + param + "]"), + containsString("[" + param + "] cannot be ") + ) ); } assertParseMaximalWarnings(); From 7490f5fc1a8b8e43ec79d35d85291a63b51d2c89 Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Mon, 10 Jun 2024 18:48:32 +0200 Subject: [PATCH 15/31] Make ErrorReportingTestListener Gradle configuration cache compatible (#109415) and make it a bit simpler too --- .../test/ErrorReportingTestListener.java | 17 ++++++----------- .../internal/test/rest/RestTestBasePlugin.java | 4 ++-- 2 files changed, 8 insertions(+), 13 deletions(-) diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/ErrorReportingTestListener.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/ErrorReportingTestListener.java index e3149d63e5c5b..4361349392de3 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/ErrorReportingTestListener.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/ErrorReportingTestListener.java @@ -7,7 +7,6 @@ */ package org.elasticsearch.gradle.internal.test; -import org.elasticsearch.gradle.internal.ElasticsearchTestBasePlugin; import org.gradle.api.internal.tasks.testing.logging.FullExceptionFormatter; import org.gradle.api.internal.tasks.testing.logging.TestExceptionFormatter; import org.gradle.api.logging.Logger; @@ -39,21 +38,24 @@ public class ErrorReportingTestListener implements TestOutputListener, TestListener { private static final String REPRODUCE_WITH_PREFIX = "REPRODUCE WITH"; - private final Test testTask; private final TestExceptionFormatter formatter; private final File outputDirectory; private final Logger taskLogger; private Map eventWriters = new ConcurrentHashMap<>(); private Map> reproductionLines = new ConcurrentHashMap<>(); private Set failedTests = new LinkedHashSet<>(); + private boolean dumpOutputOnFailure = true; public ErrorReportingTestListener(Test testTask, File outputDirectory) { - this.testTask = testTask; this.formatter = new FullExceptionFormatter(testTask.getTestLogging()); this.taskLogger = testTask.getLogger(); this.outputDirectory = outputDirectory; } + public void setDumpOutputOnFailure(boolean dumpOutputOnFailure) { + this.dumpOutputOnFailure = dumpOutputOnFailure; + } + @Override public void onOutput(TestDescriptor testDescriptor, TestOutputEvent outputEvent) { TestDescriptor suite = testDescriptor.getParent(); @@ -83,7 +85,7 @@ public void afterSuite(final TestDescriptor suite, TestResult result) { Descriptor descriptor = Descriptor.of(suite); try { - if (isDumpOutputEnabled()) { + if (dumpOutputOnFailure) { // if the test suite failed, report all captured output if (result.getResultType().equals(TestResult.ResultType.FAILURE)) { EventWriter eventWriter = eventWriters.get(descriptor); @@ -256,11 +258,4 @@ public void close() throws IOException { outputFile.delete(); } } - - private boolean isDumpOutputEnabled() { - return (Boolean) testTask.getExtensions() - .getExtraProperties() - .getProperties() - .getOrDefault(ElasticsearchTestBasePlugin.DUMP_OUTPUT_ON_FAILURE_PROP_NAME, true); - } } diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/RestTestBasePlugin.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/RestTestBasePlugin.java index 6d43ad109c323..77af3445f530c 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/RestTestBasePlugin.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/test/rest/RestTestBasePlugin.java @@ -18,9 +18,9 @@ import org.elasticsearch.gradle.VersionProperties; import org.elasticsearch.gradle.distribution.ElasticsearchDistributionTypes; import org.elasticsearch.gradle.internal.ElasticsearchJavaPlugin; -import org.elasticsearch.gradle.internal.ElasticsearchTestBasePlugin; import org.elasticsearch.gradle.internal.InternalDistributionDownloadPlugin; import org.elasticsearch.gradle.internal.info.BuildParams; +import org.elasticsearch.gradle.internal.test.ErrorReportingTestListener; import org.elasticsearch.gradle.internal.test.HistoricalFeaturesMetadataPlugin; import org.elasticsearch.gradle.plugin.BasePluginBuildPlugin; import org.elasticsearch.gradle.plugin.PluginBuildPlugin; @@ -167,7 +167,7 @@ public void apply(Project project) { nonInputSystemProperties.systemProperty(TESTS_MAX_PARALLEL_FORKS_SYSPROP, () -> String.valueOf(task.getMaxParallelForks())); // Disable test failure reporting since this stuff is now captured in build scans - task.getExtensions().getExtraProperties().set(ElasticsearchTestBasePlugin.DUMP_OUTPUT_ON_FAILURE_PROP_NAME, false); + task.getExtensions().getByType(ErrorReportingTestListener.class).setDumpOutputOnFailure(false); // Disable the security manager and syscall filter since the test framework needs to fork processes task.systemProperty("tests.security.manager", "false"); From a9f31bd2aa6f47376f3be7694ec30cbb2855c34f Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Mon, 10 Jun 2024 10:26:31 -0700 Subject: [PATCH 16/31] Support synthetic source for date fields when ignore_malformed is used (#109410) --- docs/changelog/109410.yaml | 5 + docs/reference/mapping/types/date.asciidoc | 3 +- .../index/mapper/DateFieldMapper.java | 23 ++- .../index/mapper/DateFieldMapperTests.java | 76 ++++++---- .../index/mapper/MapperTestCase.java | 10 +- ...AggregateDoubleMetricFieldMapperTests.java | 142 +++++++----------- 6 files changed, 137 insertions(+), 122 deletions(-) create mode 100644 docs/changelog/109410.yaml diff --git a/docs/changelog/109410.yaml b/docs/changelog/109410.yaml new file mode 100644 index 0000000000000..e8c4dcdab42c6 --- /dev/null +++ b/docs/changelog/109410.yaml @@ -0,0 +1,5 @@ +pr: 109410 +summary: Support synthetic source for date fields when `ignore_malformed` is used +area: Mapping +type: enhancement +issues: [] diff --git a/docs/reference/mapping/types/date.asciidoc b/docs/reference/mapping/types/date.asciidoc index e3e800fa117ff..a29db79167d2e 100644 --- a/docs/reference/mapping/types/date.asciidoc +++ b/docs/reference/mapping/types/date.asciidoc @@ -242,8 +242,7 @@ of official GA features. `date` fields support <> in their default configuration. Synthetic `_source` cannot be used together with -<>, <> set to true -or with <> disabled. +<> or with <> disabled. Synthetic source always sorts `date` fields. For example: [source,console,id=synthetic-source-date-example] diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java index 63045623c503b..c817bed6e503e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java @@ -364,7 +364,16 @@ public DateFieldMapper build(MapperBuilderContext context) { && ignoreMalformed.isConfigured() == false) { ignoreMalformed.setValue(false); } - return new DateFieldMapper(name(), ft, multiFieldsBuilder.build(this, context), copyTo, nullTimestamp, resolution, this); + return new DateFieldMapper( + name(), + ft, + multiFieldsBuilder.build(this, context), + copyTo, + nullTimestamp, + resolution, + context.isSourceSynthetic(), + this + ); } } @@ -850,6 +859,7 @@ public DocValueFormat docValueFormat(@Nullable String format, ZoneId timeZone) { private final Long nullValue; private final String nullValueAsString; private final Resolution resolution; + private final boolean isSourceSynthetic; private final boolean ignoreMalformedByDefault; private final IndexVersion indexCreatedVersion; @@ -865,6 +875,7 @@ private DateFieldMapper( CopyTo copyTo, Long nullValue, Resolution resolution, + boolean isSourceSynthetic, Builder builder ) { super(simpleName, mappedFieldType, multiFields, copyTo, builder.script.get() != null, builder.onScriptError.get()); @@ -877,6 +888,7 @@ private DateFieldMapper( this.nullValueAsString = builder.nullValue.getValue(); this.nullValue = nullValue; this.resolution = resolution; + this.isSourceSynthetic = isSourceSynthetic; this.ignoreMalformedByDefault = builder.ignoreMalformed.getDefaultValue(); this.indexCreatedVersion = builder.indexCreatedVersion; this.script = builder.script.get(); @@ -915,6 +927,10 @@ protected void parseCreateField(DocumentParserContext context) throws IOExceptio } catch (IllegalArgumentException | ElasticsearchParseException | DateTimeException | ArithmeticException e) { if (ignoreMalformed) { context.addIgnoredField(mappedFieldType.name()); + if (isSourceSynthetic) { + // Save a copy of the field so synthetic source can load it + context.doc().add(IgnoreMalformedStoredValues.storedField(name(), context.parser())); + } return; } else { throw e; @@ -976,11 +992,6 @@ public SourceLoader.SyntheticFieldLoader syntheticFieldLoader() { "field [" + name() + "] of type [" + typeName() + "] doesn't support synthetic source because it doesn't have doc values" ); } - if (ignoreMalformed) { - throw new IllegalArgumentException( - "field [" + name() + "] of type [" + typeName() + "] doesn't support synthetic source because it ignores malformed dates" - ); - } if (copyTo.copyToFields().isEmpty() != true) { throw new IllegalArgumentException( "field [" + name() + "] of type [" + typeName() + "] doesn't support synthetic source because it declares copy_to" diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java index 9e9437aa6b9db..d9894df9104a1 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java @@ -15,7 +15,6 @@ import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Strings; -import org.elasticsearch.core.Tuple; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.IndexVersions; import org.elasticsearch.index.mapper.DateFieldMapper.DateFieldType; @@ -35,6 +34,7 @@ import java.util.Comparator; import java.util.List; import java.util.function.Function; +import java.util.stream.Stream; import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; import static org.hamcrest.Matchers.containsString; @@ -152,7 +152,13 @@ protected List exampleMalformedValues() { return List.of( exampleMalformedValue("2016-03-99").mapping(mappingWithFormat("strict_date_optional_time||epoch_millis")) .errorMatches("failed to parse date field [2016-03-99] with format [strict_date_optional_time||epoch_millis]"), - exampleMalformedValue("-522000000").mapping(mappingWithFormat("date_optional_time")).errorMatches("long overflow") + exampleMalformedValue("-522000000").mapping(mappingWithFormat("date_optional_time")).errorMatches("long overflow"), + exampleMalformedValue("2020").mapping(mappingWithFormat("strict_date")) + .errorMatches("failed to parse date field [2020] with format [strict_date]"), + exampleMalformedValue("hello world").mapping(mappingWithFormat("strict_date_optional_time")) + .errorMatches("failed to parse date field [hello world]"), + exampleMalformedValue("true").mapping(mappingWithFormat("strict_date_optional_time")) + .errorMatches("failed to parse date field [true]") ); } @@ -561,7 +567,6 @@ public void testScriptAndPrecludedParameters() { @Override protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) { - assumeFalse("synthetic _source for date and date_millis doesn't support ignore_malformed", ignoreMalformed); return new SyntheticSourceSupport() { private final DateFieldMapper.Resolution resolution = randomFrom(DateFieldMapper.Resolution.values()); private final Object nullValue = usually() @@ -577,36 +582,62 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) @Override public SyntheticSourceExample example(int maxValues) { if (randomBoolean()) { - Tuple v = generateValue(); + Value v = generateValue(); + if (v.malformedOutput != null) { + return new SyntheticSourceExample(v.input, v.malformedOutput, null, this::mapping); + } + return new SyntheticSourceExample( - v.v1(), - v.v2(), - resolution.convert(Instant.from(formatter.parse(v.v2()))), + v.input, + v.output, + resolution.convert(Instant.from(formatter.parse(v.output))), this::mapping ); } - List> values = randomList(1, maxValues, this::generateValue); - List in = values.stream().map(Tuple::v1).toList(); - List outList = values.stream() + + List values = randomList(1, maxValues, this::generateValue); + List in = values.stream().map(Value::input).toList(); + + List outputFromDocValues = values.stream() + .filter(v -> v.malformedOutput == null) .sorted( - Comparator.comparing(v -> Instant.from(formatter.parse(v.v1() == null ? nullValue.toString() : v.v1().toString()))) + Comparator.comparing( + v -> Instant.from(formatter.parse(v.input == null ? nullValue.toString() : v.input.toString())) + ) ) - .map(Tuple::v2) + .map(Value::output) .toList(); + + Stream malformedOutput = values.stream().filter(v -> v.malformedOutput != null).map(Value::malformedOutput); + + // Malformed values are always last in the implementation. + List outList = Stream.concat(outputFromDocValues.stream(), malformedOutput).toList(); Object out = outList.size() == 1 ? outList.get(0) : outList; - List outBlockList = outList.stream().map(v -> resolution.convert(Instant.from(formatter.parse(v)))).toList(); + List outBlockList = outputFromDocValues.stream() + .map(v -> resolution.convert(Instant.from(formatter.parse(v)))) + .toList(); Object outBlock = outBlockList.size() == 1 ? outBlockList.get(0) : outBlockList; return new SyntheticSourceExample(in, out, outBlock, this::mapping); } - private Tuple generateValue() { + private record Value(Object input, String output, Object malformedOutput) {} + + private Value generateValue() { if (nullValue != null && randomBoolean()) { - return Tuple.tuple(null, outValue(nullValue)); + return new Value(null, outValue(nullValue), null); + } + // Different malformed values are tested in #exampleMalformedValues(). + // Here we only verify behavior of arrays that contain malformed + // values since there are modifications specific to synthetic source. + if (ignoreMalformed && randomBoolean()) { + var malformedInput = randomAlphaOfLengthBetween(1, 10); + return new Value(malformedInput, null, malformedInput); } + Object in = randomValue(); String out = outValue(in); - return Tuple.tuple(in, out); + return new Value(in, out, null); } private Object randomValue() { @@ -637,6 +668,9 @@ private void mapping(XContentBuilder b) throws IOException { if (nullValue != null) { b.field("null_value", nullValue); } + if (ignoreMalformed) { + b.field("ignore_malformed", true); + } } @Override @@ -653,16 +687,6 @@ public List invalidExample() throws IOException { b -> b.field("type", fieldType).field("doc_values", false) ) ); - examples.add( - new SyntheticSourceInvalidExample( - equalTo( - "field [field] of type [" - + fieldType - + "] doesn't support synthetic source because it ignores malformed dates" - ), - b -> b.field("type", fieldType).field("ignore_malformed", true) - ) - ); } return examples; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java index a3f965d06a5c3..4b0f45fcc230f 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java @@ -1129,8 +1129,16 @@ public final void testSyntheticSource() throws IOException { public void testSyntheticSourceIgnoreMalformedExamples() throws IOException { assumeTrue("type doesn't support ignore_malformed", supportsIgnoreMalformed()); - CheckedConsumer mapping = syntheticSourceSupport(true).example(1).mapping(); + // We need to call this in order to hit the assumption inside so that + // it tells us when field supports ignore_malformed but doesn't support it together with synthetic source. + // E.g. `assumeFalse(ignoreMalformed)` + syntheticSourceSupport(true); + for (ExampleMalformedValue v : exampleMalformedValues()) { + CheckedConsumer mapping = b -> { + v.mapping.accept(b); + b.field("ignore_malformed", true); + }; assertSyntheticSource(new SyntheticSourceExample(v.value, v.value, v.value, mapping)); } } diff --git a/x-pack/plugin/mapper-aggregate-metric/src/test/java/org/elasticsearch/xpack/aggregatemetric/mapper/AggregateDoubleMetricFieldMapperTests.java b/x-pack/plugin/mapper-aggregate-metric/src/test/java/org/elasticsearch/xpack/aggregatemetric/mapper/AggregateDoubleMetricFieldMapperTests.java index 83e701486d93a..f46508093c4ec 100644 --- a/x-pack/plugin/mapper-aggregate-metric/src/test/java/org/elasticsearch/xpack/aggregatemetric/mapper/AggregateDoubleMetricFieldMapperTests.java +++ b/x-pack/plugin/mapper-aggregate-metric/src/test/java/org/elasticsearch/xpack/aggregatemetric/mapper/AggregateDoubleMetricFieldMapperTests.java @@ -18,7 +18,6 @@ import org.elasticsearch.index.mapper.MapperTestCase; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentFactory; import org.elasticsearch.xpack.aggregatemetric.AggregateMetricMapperPlugin; @@ -34,7 +33,6 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.function.Supplier; import static org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Names.IGNORE_MALFORMED; import static org.elasticsearch.xpack.aggregatemetric.mapper.AggregateDoubleMetricFieldMapper.Names.METRICS; @@ -149,27 +147,75 @@ protected boolean supportsIgnoreMalformed() { @Override protected List exampleMalformedValues() { + var min = randomDoubleBetween(-100, 100, false); + var max = randomDoubleBetween(min, 150, false); + var valueCount = randomIntBetween(1, Integer.MAX_VALUE); + + var randomString = randomAlphaOfLengthBetween(1, 10); + var randomLong = randomLong(); + var randomDouble = randomDouble(); + var randomBoolean = randomBoolean(); + return List.of( + // wrong input structure + exampleMalformedValue(b -> b.value(randomString)).errorMatches("Failed to parse object"), + exampleMalformedValue(b -> b.value(randomLong)).errorMatches("Failed to parse object"), + exampleMalformedValue(b -> b.value(randomDouble)).errorMatches("Failed to parse object"), + exampleMalformedValue(b -> b.value(randomBoolean)).errorMatches("Failed to parse object"), // no metrics exampleMalformedValue(b -> b.startObject().endObject()).errorMatches( "Aggregate metric field [field] must contain all metrics [min, max, value_count]" ), // unmapped metric exampleMalformedValue( - b -> b.startObject().field("min", -10.1).field("max", 50.0).field("value_count", 14).field("sum", 55).endObject() + b -> b.startObject() + .field("min", min) + .field("max", max) + .field("value_count", valueCount) + .field("sum", randomLong) + .endObject() ).errorMatches("Aggregate metric [sum] does not exist in the mapping of field [field]"), // missing metric - exampleMalformedValue(b -> b.startObject().field("min", -10.1).field("max", 50.0).endObject()).errorMatches( + exampleMalformedValue(b -> b.startObject().field("min", min).field("max", max).endObject()).errorMatches( "Aggregate metric field [field] must contain all metrics [min, max, value_count]" ), // invalid metric value - exampleMalformedValue(b -> b.startObject().field("min", "10.0").field("max", 50.0).field("value_count", 14).endObject()) + exampleMalformedValue(b -> b.startObject().field("min", "10.0").field("max", max).field("value_count", valueCount).endObject()) .errorMatches("Failed to parse object: expecting token of type [VALUE_NUMBER] but found [VALUE_STRING]"), + // Invalid metric value with additional data. + // `min` field triggers the error and all additional data should be preserved in synthetic source. + exampleMalformedValue( + b -> b.startObject() + .field("max", max) + .field("value_count", valueCount) + .field("min", "10.0") + .field("hello", randomString) + .startObject("object") + .field("hello", randomLong) + .endObject() + .array("list", randomString, randomString) + .endObject() + ).errorMatches("Failed to parse object: expecting token of type [VALUE_NUMBER] but found [VALUE_STRING]"), + // metric is an object + exampleMalformedValue( + b -> b.startObject() + .startObject("min") + .field("hello", "world") + .endObject() + .field("max", max) + .field("value_count", valueCount) + .endObject() + ).errorMatches("Failed to parse object: expecting token of type [VALUE_NUMBER] but found [START_OBJECT]"), + // metric is an array + exampleMalformedValue( + b -> b.startObject().array("min", "hello", "world").field("max", max).field("value_count", valueCount).endObject() + ).errorMatches("Failed to parse object: expecting token of type [VALUE_NUMBER] but found [START_ARRAY]"), // negative value count - exampleMalformedValue(b -> b.startObject().field("min", 10.0).field("max", 50.0).field("value_count", -14).endObject()) - .errorMatches("Aggregate metric [value_count] of field [field] cannot be a negative number"), + exampleMalformedValue( + b -> b.startObject().field("min", min).field("max", max).field("value_count", -1 * valueCount).endObject() + ).errorMatches("Aggregate metric [value_count] of field [field] cannot be a negative number"), // value count with decimal digits (whole numbers formatted as doubles are permitted, but non-whole numbers are not) - exampleMalformedValue(b -> b.startObject().field("min", 10.0).field("max", 50.0).field("value_count", 77.33).endObject()) + exampleMalformedValue(b -> b.startObject().field("min", min).field("max", max).field("value_count", 77.33).endObject()) .errorMatches("failed to parse [value_count] sub field: 77.33 cannot be converted to Integer without data loss") ); } @@ -472,18 +518,12 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) return new AggregateDoubleMetricSyntheticSourceSupport(ignoreMalformed); } - @Override - public void testSyntheticSourceIgnoreMalformedExamples() { - assumeTrue("Scenarios are covered in scope of syntheticSourceSupport", false); - } - @Override protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); } protected final class AggregateDoubleMetricSyntheticSourceSupport implements SyntheticSourceSupport { - private final boolean malformedExample; private final EnumSet storedMetrics; @@ -499,79 +539,7 @@ public SyntheticSourceExample example(int maxVals) { return new SyntheticSourceExample(value, value, this::mapping); } - private Object randomAggregateMetric() { - if (malformedExample && randomBoolean()) { - return malformedValue(); - } - - return validMetrics(); - } - - private Object malformedValue() { - List> choices = List.of( - () -> randomAlphaOfLength(3), - ESTestCase::randomInt, - ESTestCase::randomLong, - ESTestCase::randomFloat, - ESTestCase::randomDouble, - ESTestCase::randomBoolean, - // no metrics - Map::of, - // unmapped metric - () -> { - var metrics = validMetrics(); - metrics.put("hello", "world"); - return metrics; - }, - // missing metric - () -> { - var metrics = validMetrics(); - metrics.remove(storedMetrics.stream().findFirst().get().name()); - return metrics; - }, - // invalid metric value - () -> { - var metrics = validMetrics(); - metrics.put(storedMetrics.stream().findFirst().get().name(), "boom"); - return metrics; - }, - // metric is an object - () -> { - var metrics = validMetrics(); - metrics.put(storedMetrics.stream().findFirst().get().name(), Map.of("hello", "world")); - return metrics; - }, - // invalid metric value with additional data - () -> { - var metrics = validMetrics(); - metrics.put(storedMetrics.stream().findFirst().get().name(), "boom"); - metrics.put("hello", "world"); - metrics.put("object", Map.of("hello", "world")); - metrics.put("list", List.of("hello", "world")); - return metrics; - }, - // negative value count - () -> { - var metrics = validMetrics(); - if (storedMetrics.contains(Metric.value_count.name())) { - metrics.put(Metric.value_count.name(), -100); - } - return metrics; - }, - // value count with decimal digits (whole numbers formatted as doubles are permitted, but non-whole numbers are not) - () -> { - var metrics = validMetrics(); - if (storedMetrics.contains(Metric.value_count.name())) { - metrics.put(Metric.value_count.name(), 10.5); - } - return metrics; - } - ); - - return randomFrom(choices).get(); - } - - private Map validMetrics() { + private Map randomAggregateMetric() { Map value = new LinkedHashMap<>(storedMetrics.size()); for (Metric m : storedMetrics) { if (Metric.value_count == m) { From 540d2b10a354c3b45b727f3488724fa008c5d6cf Mon Sep 17 00:00:00 2001 From: Felix Barnsteiner Date: Mon, 10 Jun 2024 19:32:31 +0200 Subject: [PATCH 17/31] Add metrics@custom component template to metrics-*-* index template (#109540) This lets users customize the metrics data stream mappings, without having to override a managed component template that may get overridden. Fixes #109475 --- docs/changelog/109540.yaml | 6 + .../datastreams/AbstractDataStreamIT.java | 169 ++++++++++++++++++ .../datastreams/EcsLogsDataStreamIT.java | 4 +- .../datastreams/LogsDataStreamIT.java | 151 +--------------- .../datastreams/MetricsDataStreamIT.java | 101 +++++++++++ .../src/main/resources/metrics@template.json | 4 +- .../xpack/stack/StackTemplateRegistry.java | 2 +- 7 files changed, 287 insertions(+), 150 deletions(-) create mode 100644 docs/changelog/109540.yaml create mode 100644 modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/AbstractDataStreamIT.java create mode 100644 modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/MetricsDataStreamIT.java diff --git a/docs/changelog/109540.yaml b/docs/changelog/109540.yaml new file mode 100644 index 0000000000000..722c60a30fb97 --- /dev/null +++ b/docs/changelog/109540.yaml @@ -0,0 +1,6 @@ +pr: 109540 +summary: Add metrics@custom component template to metrics-*-* index template +area: Data streams +type: enhancement +issues: + - 109475 diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/AbstractDataStreamIT.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/AbstractDataStreamIT.java new file mode 100644 index 0000000000000..ca33f08324539 --- /dev/null +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/AbstractDataStreamIT.java @@ -0,0 +1,169 @@ +/* + * 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.datastreams; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.FeatureFlag; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * This base class provides the boilerplate to simplify the development of integration tests. + * Aside from providing useful helper methods and disabling unnecessary plugins, + * it waits until an {@linkplain #indexTemplateName() index template} is installed, which happens asynchronously in StackTemplateRegistry. + * This avoids race conditions leading to flaky tests by ensuring the template has been installed before executing the tests. + */ +public abstract class AbstractDataStreamIT extends ESRestTestCase { + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.DEFAULT) + .feature(FeatureFlag.FAILURE_STORE_ENABLED) + .setting("xpack.security.enabled", "false") + .setting("xpack.watcher.enabled", "false") + // Disable apm-data so the index templates it installs do not impact + // tests such as testIgnoreDynamicBeyondLimit. + .setting("xpack.apm_data.enabled", "false") + .build(); + protected RestClient client; + + static void waitForIndexTemplate(RestClient client, String indexTemplate) throws Exception { + assertBusy(() -> { + try { + Request request = new Request("GET", "_index_template/" + indexTemplate); + assertOK(client.performRequest(request)); + } catch (ResponseException e) { + fail(e.getMessage()); + } + }); + } + + static void createDataStream(RestClient client, String name) throws IOException { + Request request = new Request("PUT", "_data_stream/" + name); + assertOK(client.performRequest(request)); + } + + @SuppressWarnings("unchecked") + static String getWriteBackingIndex(RestClient client, String name) throws IOException { + Request request = new Request("GET", "_data_stream/" + name); + List dataStreams = (List) entityAsMap(client.performRequest(request)).get("data_streams"); + Map dataStream = (Map) dataStreams.get(0); + List> indices = (List>) dataStream.get("indices"); + return indices.get(0).get("index_name"); + } + + @SuppressWarnings("unchecked") + static Map getSettings(RestClient client, String indexName) throws IOException { + Request request = new Request("GET", "/" + indexName + "/_settings?flat_settings"); + return ((Map>) entityAsMap(client.performRequest(request)).get(indexName)).get("settings"); + } + + static void putMapping(RestClient client, String indexName) throws IOException { + Request request = new Request("PUT", "/" + indexName + "/_mapping"); + request.setJsonEntity(""" + { + "properties": { + "numeric_field": { + "type": "integer" + } + } + } + """); + assertOK(client.performRequest(request)); + } + + @SuppressWarnings("unchecked") + static Map getMappingProperties(RestClient client, String indexName) throws IOException { + Request request = new Request("GET", "/" + indexName + "/_mapping"); + Map map = (Map) entityAsMap(client.performRequest(request)).get(indexName); + Map mappings = (Map) map.get("mappings"); + return (Map) mappings.get("properties"); + } + + static void indexDoc(RestClient client, String dataStreamName, String doc) throws IOException { + Request request = new Request("POST", "/" + dataStreamName + "/_doc?refresh=true"); + request.setJsonEntity(doc); + assertOK(client.performRequest(request)); + } + + @SuppressWarnings("unchecked") + static List searchDocs(RestClient client, String dataStreamName, String query) throws IOException { + Request request = new Request("GET", "/" + dataStreamName + "/_search"); + request.setJsonEntity(query); + Map hits = (Map) entityAsMap(client.performRequest(request)).get("hits"); + return (List) hits.get("hits"); + } + + @SuppressWarnings("unchecked") + static Object getValueFromPath(Map map, List path) { + Map current = map; + for (int i = 0; i < path.size(); i++) { + Object value = current.get(path.get(i)); + if (i == path.size() - 1) { + return value; + } + if (value == null) { + throw new IllegalStateException("Path " + String.join(".", path) + " was not found in " + map); + } + if (value instanceof Map next) { + current = (Map) next; + } else { + throw new IllegalStateException( + "Failed to reach the end of the path " + + String.join(".", path) + + " last reachable field was " + + path.get(i) + + " in " + + map + ); + } + } + return current; + } + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected Settings restAdminSettings() { + if (super.restAdminSettings().keySet().contains(ThreadContext.PREFIX + ".Authorization")) { + return super.restAdminSettings(); + } else { + String token = basicAuthHeaderValue("admin", new SecureString("admin-password".toCharArray())); + return Settings.builder().put(super.restAdminSettings()).put(ThreadContext.PREFIX + ".Authorization", token).build(); + } + } + + @Before + public void setup() throws Exception { + client = client(); + AbstractDataStreamIT.waitForIndexTemplate(client, indexTemplateName()); + } + + protected abstract String indexTemplateName(); + + @After + public void cleanUp() throws IOException { + adminClient().performRequest(new Request("DELETE", "_data_stream/*")); + } +} diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/EcsLogsDataStreamIT.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/EcsLogsDataStreamIT.java index 5fe72c38078ee..e43b1e451c312 100644 --- a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/EcsLogsDataStreamIT.java +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/EcsLogsDataStreamIT.java @@ -26,7 +26,7 @@ import static org.elasticsearch.datastreams.LogsDataStreamIT.getWriteBackingIndex; import static org.elasticsearch.datastreams.LogsDataStreamIT.indexDoc; import static org.elasticsearch.datastreams.LogsDataStreamIT.searchDocs; -import static org.elasticsearch.datastreams.LogsDataStreamIT.waitForLogs; +import static org.elasticsearch.datastreams.LogsDataStreamIT.waitForIndexTemplate; import static org.hamcrest.Matchers.is; public class EcsLogsDataStreamIT extends DisabledSecurityDataStreamTestCase { @@ -38,7 +38,7 @@ public class EcsLogsDataStreamIT extends DisabledSecurityDataStreamTestCase { @Before public void setup() throws Exception { client = client(); - waitForLogs(client); + waitForIndexTemplate(client, "logs"); { Request request = new Request("PUT", "/_ingest/pipeline/logs@custom"); diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LogsDataStreamIT.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LogsDataStreamIT.java index c2a7a76ab751a..9ab32f29f4a79 100644 --- a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LogsDataStreamIT.java +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/LogsDataStreamIT.java @@ -9,20 +9,7 @@ package org.elasticsearch.datastreams; import org.elasticsearch.client.Request; -import org.elasticsearch.client.ResponseException; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.common.settings.SecureString; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.test.cluster.ElasticsearchCluster; -import org.elasticsearch.test.cluster.FeatureFlag; -import org.elasticsearch.test.cluster.local.distribution.DistributionType; -import org.elasticsearch.test.rest.ESRestTestCase; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; - -import java.io.IOException; + import java.util.List; import java.util.Map; @@ -35,46 +22,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; -public class LogsDataStreamIT extends ESRestTestCase { - - @ClassRule - public static ElasticsearchCluster cluster = ElasticsearchCluster.local() - .distribution(DistributionType.DEFAULT) - .feature(FeatureFlag.FAILURE_STORE_ENABLED) - .setting("xpack.security.enabled", "false") - .setting("xpack.watcher.enabled", "false") - // Disable apm-data so the index templates it installs do not impact - // tests such as testIgnoreDynamicBeyondLimit. - .setting("xpack.apm_data.enabled", "false") - .build(); - - @Override - protected String getTestRestCluster() { - return cluster.getHttpAddresses(); - } - - @Override - protected Settings restAdminSettings() { - if (super.restAdminSettings().keySet().contains(ThreadContext.PREFIX + ".Authorization")) { - return super.restAdminSettings(); - } else { - String token = basicAuthHeaderValue("admin", new SecureString("admin-password".toCharArray())); - return Settings.builder().put(super.restAdminSettings()).put(ThreadContext.PREFIX + ".Authorization", token).build(); - } - } - - private RestClient client; - - @Before - public void setup() throws Exception { - client = client(); - waitForLogs(client); - } - - @After - public void cleanUp() throws IOException { - adminClient().performRequest(new Request("DELETE", "_data_stream/*")); - } +public class LogsDataStreamIT extends AbstractDataStreamIT { @SuppressWarnings("unchecked") public void testDefaultLogsSettingAndMapping() throws Exception { @@ -791,97 +739,8 @@ public void testIgnoreDynamicBeyondLimit() throws Exception { assertThat(ignored.stream().filter(i -> i.startsWith("field") == false).toList(), empty()); } - static void waitForLogs(RestClient client) throws Exception { - assertBusy(() -> { - try { - Request request = new Request("GET", "_index_template/logs"); - assertOK(client.performRequest(request)); - } catch (ResponseException e) { - fail(e.getMessage()); - } - }); - } - - static void createDataStream(RestClient client, String name) throws IOException { - Request request = new Request("PUT", "_data_stream/" + name); - assertOK(client.performRequest(request)); - } - - @SuppressWarnings("unchecked") - static String getWriteBackingIndex(RestClient client, String name) throws IOException { - Request request = new Request("GET", "_data_stream/" + name); - List dataStreams = (List) entityAsMap(client.performRequest(request)).get("data_streams"); - Map dataStream = (Map) dataStreams.get(0); - List> indices = (List>) dataStream.get("indices"); - return indices.get(0).get("index_name"); - } - - @SuppressWarnings("unchecked") - static Map getSettings(RestClient client, String indexName) throws IOException { - Request request = new Request("GET", "/" + indexName + "/_settings?flat_settings"); - return ((Map>) entityAsMap(client.performRequest(request)).get(indexName)).get("settings"); - } - - static void putMapping(RestClient client, String indexName) throws IOException { - Request request = new Request("PUT", "/" + indexName + "/_mapping"); - request.setJsonEntity(""" - { - "properties": { - "numeric_field": { - "type": "integer" - } - } - } - """); - assertOK(client.performRequest(request)); - } - - @SuppressWarnings("unchecked") - static Map getMappingProperties(RestClient client, String indexName) throws IOException { - Request request = new Request("GET", "/" + indexName + "/_mapping"); - Map map = (Map) entityAsMap(client.performRequest(request)).get(indexName); - Map mappings = (Map) map.get("mappings"); - return (Map) mappings.get("properties"); - } - - static void indexDoc(RestClient client, String dataStreamName, String doc) throws IOException { - Request request = new Request("POST", "/" + dataStreamName + "/_doc?refresh=true"); - request.setJsonEntity(doc); - assertOK(client.performRequest(request)); - } - - @SuppressWarnings("unchecked") - static List searchDocs(RestClient client, String dataStreamName, String query) throws IOException { - Request request = new Request("GET", "/" + dataStreamName + "/_search"); - request.setJsonEntity(query); - Map hits = (Map) entityAsMap(client.performRequest(request)).get("hits"); - return (List) hits.get("hits"); - } - - @SuppressWarnings("unchecked") - static Object getValueFromPath(Map map, List path) { - Map current = map; - for (int i = 0; i < path.size(); i++) { - Object value = current.get(path.get(i)); - if (i == path.size() - 1) { - return value; - } - if (value == null) { - throw new IllegalStateException("Path " + String.join(".", path) + " was not found in " + map); - } - if (value instanceof Map next) { - current = (Map) next; - } else { - throw new IllegalStateException( - "Failed to reach the end of the path " - + String.join(".", path) - + " last reachable field was " - + path.get(i) - + " in " - + map - ); - } - } - return current; + @Override + protected String indexTemplateName() { + return "logs"; } } diff --git a/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/MetricsDataStreamIT.java b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/MetricsDataStreamIT.java new file mode 100644 index 0000000000000..6cc300378a312 --- /dev/null +++ b/modules/data-streams/src/javaRestTest/java/org/elasticsearch/datastreams/MetricsDataStreamIT.java @@ -0,0 +1,101 @@ +/* + * 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.datastreams; + +import org.elasticsearch.client.Request; + +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class MetricsDataStreamIT extends AbstractDataStreamIT { + + @SuppressWarnings("unchecked") + public void testCustomMapping() throws Exception { + { + Request request = new Request("POST", "/_component_template/metrics@custom"); + request.setJsonEntity(""" + { + "template": { + "settings": { + "index": { + "query": { + "default_field": ["custom-message"] + } + } + }, + "mappings": { + "properties": { + "numeric_field": { + "type": "integer" + }, + "socket": { + "properties": { + "ip": { + "type": "keyword" + } + } + } + } + } + } + } + """); + assertOK(client.performRequest(request)); + } + + String dataStreamName = "metrics-generic-default"; + createDataStream(client, dataStreamName); + String backingIndex = getWriteBackingIndex(client, dataStreamName); + + // Verify that the custom settings.index.query.default_field overrides the default query field - "message" + Map settings = getSettings(client, backingIndex); + assertThat(settings.get("index.query.default_field"), is(List.of("custom-message"))); + + // Verify that the new field from the custom component template is applied + putMapping(client, backingIndex); + Map mappingProperties = getMappingProperties(client, backingIndex); + assertThat(getValueFromPath(mappingProperties, List.of("numeric_field", "type")), equalTo("integer")); + assertThat(getValueFromPath(mappingProperties, List.of("socket", "properties", "ip", "type")), is("keyword")); + + // Insert valid doc and verify successful indexing + { + indexDoc(client, dataStreamName, """ + { + "@timestamp": "2024-06-10", + "test": "doc-with-ip", + "socket": { + "ip": "127.0.0.1" + } + } + """); + List results = searchDocs(client, dataStreamName, """ + { + "query": { + "term": { + "test": { + "value": "doc-with-ip" + } + } + }, + "fields": ["socket.ip"] + } + """); + Map fields = ((Map>) results.get(0)).get("_source"); + assertThat(fields.get("socket"), is(Map.of("ip", "127.0.0.1"))); + } + } + + @Override + protected String indexTemplateName() { + return "metrics"; + } +} diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json index 464df09ffe2ce..776ed88857db5 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json @@ -5,8 +5,10 @@ "composed_of": [ "metrics@mappings", "data-streams@mappings", - "metrics@settings" + "metrics@settings", + "metrics@custom" ], + "ignore_missing_component_templates": ["metrics@custom"], "allow_auto_create": true, "_meta": { "description": "default metrics template installed by x-pack", diff --git a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java index 30323a1d7d363..3cd551ca1f3d9 100644 --- a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java +++ b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java @@ -47,7 +47,7 @@ public class StackTemplateRegistry extends IndexTemplateRegistry { // The stack template registry version. This number must be incremented when we make changes // to built-in templates. - public static final int REGISTRY_VERSION = 10; + public static final int REGISTRY_VERSION = 11; public static final String TEMPLATE_VERSION_VARIABLE = "xpack.stack.template.version"; public static final Setting STACK_TEMPLATES_ENABLED = Setting.boolSetting( From ec0b573af66961e1990ec81268135a57f7d28aa1 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 10 Jun 2024 14:17:25 -0400 Subject: [PATCH 18/31] Add Create or update query rule API call (#109042) --- docs/changelog/109042.yaml | 5 + .../rest-api-spec/api/query_rule.put.json | 42 +++ .../org/elasticsearch/TransportVersions.java | 1 + x-pack/plugin/ent-search/qa/rest/build.gradle | 15 +- .../entsearch/rules/10_query_ruleset_put.yml | 29 +- .../entsearch/rules/20_query_ruleset_list.yml | 54 +++- .../rules/30_query_ruleset_delete.yml | 17 +- .../entsearch/rules/40_rule_query_search.yml | 17 ++ .../entsearch/rules/50_query_rule_put.yml | 266 ++++++++++++++++++ .../xpack/application/EnterpriseSearch.java | 7 +- .../xpack/application/rules/QueryRule.java | 56 +++- .../rules/QueryRulesIndexService.java | 109 ++++++- .../xpack/application/rules/QueryRuleset.java | 6 +- .../rules/action/PutQueryRuleAction.java | 198 +++++++++++++ .../rules/action/PutQueryRulesetAction.java | 9 + .../rules/action/RestPutQueryRuleAction.java | 55 ++++ .../action/TransportPutQueryRuleAction.java | 49 ++++ ...a => EnterpriseSearchModuleTestUtils.java} | 19 +- ...ectorActionRequestBWCSerializingTests.java | 4 +- .../syncjob/ConnectorSyncJobTestUtils.java | 4 +- ...cJobsActionRequestBWCSerializingTests.java | 4 +- .../rules/QueryRuleCriteriaTests.java | 4 +- .../application/rules/QueryRuleTests.java | 29 +- .../rules/QueryRulesIndexServiceTests.java | 22 +- .../application/rules/QueryRulesetTests.java | 4 +- .../rules/RuleQueryBuilderTests.java | 3 +- ...esetActionResponseBWCSerializingTests.java | 11 +- ...esetsActionRequestBWCSerializingTests.java | 4 +- ...setsActionResponseBWCSerializingTests.java | 4 +- ...yRuleActionRequestBWCSerializingTests.java | 62 ++++ ...eryRuleActionResponseSerializingTests.java | 38 +++ ...lesetActionRequestBWCSerializingTests.java | 13 +- .../action/RestPutQueryRuleActionTests.java | 66 +++++ .../SearchApplicationIndexServiceTests.java | 5 +- .../search/SearchApplicationTests.java | 3 +- ...tionActionResponseBWCSerializingTests.java | 4 +- ...ationActionRequestBWCSerializingTests.java | 4 +- ...tionActionResponseBWCSerializingTests.java | 4 +- ...ationActionRequestBWCSerializingTests.java | 4 +- ...ationSearchRequestBWCSerializingTests.java | 4 +- .../xpack/security/operator/Constants.java | 1 + 41 files changed, 1144 insertions(+), 111 deletions(-) create mode 100644 docs/changelog/109042.yaml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/api/query_rule.put.json create mode 100644 x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/50_query_rule_put.yml create mode 100644 x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleAction.java create mode 100644 x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleAction.java create mode 100644 x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/TransportPutQueryRuleAction.java rename x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/{search/SearchApplicationTestUtils.java => EnterpriseSearchModuleTestUtils.java} (85%) create mode 100644 x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionRequestBWCSerializingTests.java create mode 100644 x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionResponseSerializingTests.java create mode 100644 x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleActionTests.java diff --git a/docs/changelog/109042.yaml b/docs/changelog/109042.yaml new file mode 100644 index 0000000000000..5aa80db991c0d --- /dev/null +++ b/docs/changelog/109042.yaml @@ -0,0 +1,5 @@ +pr: 109042 +summary: Add Create or update query rule API call +area: Application +type: enhancement +issues: [ ] diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/query_rule.put.json b/rest-api-spec/src/main/resources/rest-api-spec/api/query_rule.put.json new file mode 100644 index 0000000000000..895f3654b1622 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/query_rule.put.json @@ -0,0 +1,42 @@ +{ + "query_rule.put": { + "documentation": { + "url": "https://www.elastic.co/guide/en/elasticsearch/reference/master/put-query-rule.html", + "description": "Creates or updates a query rule within a ruleset." + }, + "stability": "experimental", + "visibility": "public", + "headers": { + "accept": [ + "application/json" + ], + "content_type": [ + "application/json" + ] + }, + "url": { + "paths": [ + { + "path": "/_query_rules/{ruleset_id}/{rule_id}", + "methods": [ + "PUT" + ], + "parts": { + "ruleset_id": { + "type": "string", + "description": "The unique identifier of the ruleset this rule should be added to. The ruleset will be created if it does not exist." + }, + "rule_id": { + "type": "string", + "description": "The unique identifier of the rule to be created or updated." + } + } + } + ] + }, + "body": { + "description": "The query rule configuration, including the type of rule, the criteria to match the rule, and the action that should be taken if the rule matches.", + "required": true + } + } +} diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index e6b98971ff8cb..c2be2da12534b 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -187,6 +187,7 @@ static TransportVersion def(int id) { public static final TransportVersion RANK_FEATURE_PHASE_ADDED = def(8_678_00_0); public static final TransportVersion RANK_DOC_IN_SHARD_FETCH_REQUEST = def(8_679_00_0); public static final TransportVersion SECURITY_SETTINGS_REQUEST_TIMEOUTS = def(8_680_00_0); + public static final TransportVersion QUERY_RULE_CRUD_API_PUT = def(8_681_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/ent-search/qa/rest/build.gradle b/x-pack/plugin/ent-search/qa/rest/build.gradle index 37f1d8f13c850..c24b0ffd44c65 100644 --- a/x-pack/plugin/ent-search/qa/rest/build.gradle +++ b/x-pack/plugin/ent-search/qa/rest/build.gradle @@ -7,7 +7,20 @@ dependencies { restResources { restApi { - include '_common', 'bulk', 'cluster', 'connector', 'nodes', 'indices', 'index', 'query_ruleset', 'search_application', 'xpack', 'security', 'search', 'ml' + include '_common', + 'bulk', + 'cluster', + 'connector', + 'nodes', + 'indices', + 'index', + 'query_ruleset', + 'query_rule', + 'search_application', + 'xpack', + 'security', + 'search', + 'ml' } } diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/10_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 index 7868919dd6d1f..f3f37e41ec756 100644 --- a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/10_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 @@ -1,9 +1,20 @@ - setup: - requires: - cluster_features: ["gte_v8.10.0"] + cluster_features: [ "gte_v8.10.0" ] reason: Introduced in 8.10.0 +--- +teardown: + - do: + query_ruleset.delete: + ruleset_id: test-ruleset + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: test-query-ruleset-recreating + ignore: 404 + --- 'Create Query Ruleset': - do: @@ -16,7 +27,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -26,7 +37,7 @@ setup: criteria: - type: exact metadata: query_string - values: [kibana] + values: [ kibana ] actions: docs: - '_index': 'test-index1' @@ -47,7 +58,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -57,7 +68,7 @@ setup: criteria: - type: exact metadata: query_string - values: [kibana] + values: [ kibana ] actions: docs: - '_index': 'test-index1' @@ -77,7 +88,7 @@ setup: criteria: type: 'exact' metadata: 'query_string' - values: ['elastic'] + values: [ 'elastic' ] actions: ids: - 'id1' @@ -94,7 +105,7 @@ setup: criteria: type: 'exact' metadata: 'query_string' - values: ['elastic'] + values: [ 'elastic' ] actions: ids: - 'id2' @@ -118,7 +129,7 @@ setup: criteria: type: 'exact' metadata: 'query_string' - values: ['elastic'] + values: [ 'elastic' ] actions: ids: - 'id1' diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/20_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 index 0183dc8930d75..b30f1c2418f4f 100644 --- a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/20_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 @@ -1,6 +1,6 @@ setup: - requires: - cluster_features: ["gte_v8.10.0"] + cluster_features: [ "gte_v8.10.0" ] reason: Introduced in 8.10.0 - do: query_ruleset.put: @@ -12,7 +12,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -22,7 +22,7 @@ setup: criteria: - type: exact metadata: query_string - values: [kibana] + values: [ kibana ] actions: ids: - 'id3' @@ -38,7 +38,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -48,7 +48,7 @@ setup: criteria: - type: exact metadata: query_string - values: [kibana] + values: [ kibana ] actions: ids: - 'id3' @@ -58,7 +58,7 @@ setup: criteria: - type: exact metadata: query_string - values: [logstash] + values: [ logstash ] actions: ids: - 'id5' @@ -74,7 +74,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -84,7 +84,7 @@ setup: criteria: - type: exact metadata: query_string - values: [kibana] + values: [ kibana ] actions: ids: - 'id3' @@ -94,7 +94,7 @@ setup: criteria: - type: exact metadata: query_string - values: [logstash] + values: [ logstash ] actions: ids: - 'id5' @@ -104,11 +104,32 @@ setup: criteria: - type: exact metadata: query_string - values: [beats] + values: [ beats ] actions: ids: - 'id7' - 'id8' +--- +teardown: + - do: + query_ruleset.delete: + ruleset_id: test-query-ruleset-1 + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: test-query-ruleset-2 + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: test-query-ruleset-3 + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: a-test-query-ruleset-with-lots-of-criteria + ignore: 404 --- "List Query Rulesets": @@ -263,3 +284,16 @@ setup: prefix: 1 suffix: 1 always: 1 + +--- +'List Query Rulesets - Insufficient privilege': + - skip: + features: headers + + - do: + catch: forbidden + headers: { Authorization: "Basic ZW50c2VhcmNoLXVzZXI6ZW50c2VhcmNoLXVzZXItcGFzc3dvcmQ=" } # user + query_ruleset.list: { } + + - match: { error.type: 'security_exception' } + diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/30_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 index cfc847b33f665..81e3e6c8411f7 100644 --- a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/30_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 @@ -1,6 +1,6 @@ setup: - requires: - cluster_features: ["gte_v8.10.0"] + cluster_features: [ "gte_v8.10.0" ] reason: Introduced in 8.10.0 - do: query_ruleset.put: @@ -12,7 +12,7 @@ setup: criteria: - type: exact metadata: query_string - values: [elastic] + values: [ elastic ] actions: ids: - 'id1' @@ -37,3 +37,16 @@ setup: catch: "missing" query_ruleset.delete: ruleset_id: test-nonexistent-query-ruleset + +--- +'Delete Query Ruleset - Insufficient privilege': + - skip: + features: headers + + - do: + catch: forbidden + headers: { Authorization: "Basic ZW50c2VhcmNoLXVzZXI6ZW50c2VhcmNoLXVzZXItcGFzc3dvcmQ=" } # user + query_ruleset.delete: + ruleset_id: test-query-ruleset-to-delete + + - match: { error.type: 'security_exception' } diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/40_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 index 5b67f966bba68..bfd4c5e8a831e 100644 --- a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/40_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 @@ -98,6 +98,23 @@ setup: ids: - 'doc6' +--- +teardown: + - do: + query_ruleset.delete: + ruleset_id: test-ruleset + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: another-test-ruleset + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: combined-ruleset + ignore: 404 + --- "Perform a rule query specifying a ruleset that does not exist": - do: diff --git a/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/50_query_rule_put.yml b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/50_query_rule_put.yml new file mode 100644 index 0000000000000..64a933261af90 --- /dev/null +++ b/x-pack/plugin/ent-search/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/entsearch/rules/50_query_rule_put.yml @@ -0,0 +1,266 @@ +setup: + - requires: + cluster_features: [ "gte_v8.15.0" ] + reason: Introduced in 8.15.0 + + +--- +teardown: + - do: + query_ruleset.delete: + ruleset_id: test-ruleset + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: test-query-rule-recreating + ignore: 404 + + - do: + query_ruleset.delete: + ruleset_id: forbidden-query-ruleset + ignore: 404 + +--- +'Create query rule with existing ruleset respecting priority order': + # Start with 2 rules, one that specifies priority and one that does not (should go at the end) + - do: + query_ruleset.put: + ruleset_id: test-ruleset + body: + rules: + - rule_id: query-rule-id1 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ elastic ] + actions: + ids: + - 'id1' + - 'id2' + - rule_id: query-rule-id2 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ kibana ] + actions: + ids: + - 'id3' + - 'id4' + priority: 1 + + - match: { result: 'created' } + + - do: + query_ruleset.get: + ruleset_id: test-ruleset + + - match: { ruleset_id: test-ruleset } + - match: + rules: + - rule_id: query-rule-id2 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ kibana ] + actions: + ids: + - 'id3' + - 'id4' + priority: 1 + - rule_id: query-rule-id1 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ elastic ] + actions: + ids: + - 'id1' + - 'id2' + + # Next, add a rule with a priority 2 - this should go in the middle + - do: + query_rule.put: + ruleset_id: test-ruleset + rule_id: query-rule-id3 + body: + type: 'pinned' + criteria: + type: 'exact' + metadata: 'query_string' + values: [ 'logstash' ] + actions: + ids: + - 'id1' + priority: 2 + + - match: { result: 'created' } + + - do: + query_ruleset.get: + ruleset_id: test-ruleset + + - match: { ruleset_id: test-ruleset } + - match: + rules: + - rule_id: query-rule-id2 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ kibana ] + actions: + ids: + - 'id3' + - 'id4' + priority: 1 + - rule_id: query-rule-id3 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ logstash ] + actions: + ids: + - 'id1' + priority: 2 + - rule_id: query-rule-id1 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ elastic ] + actions: + ids: + - 'id1' + - 'id2' + + # Finally, add another single rule with no priority. This should be appended to the ruleset. + - do: + query_rule.put: + ruleset_id: test-ruleset + rule_id: query-rule-id4 + body: + type: 'pinned' + criteria: + type: 'exact' + metadata: 'query_string' + values: [ 'search' ] + actions: + ids: + - 'id2' + + - match: { result: 'created' } + + - do: + query_ruleset.get: + ruleset_id: test-ruleset + + - match: { ruleset_id: test-ruleset } + - match: + rules: + - rule_id: query-rule-id2 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ kibana ] + actions: + ids: + - 'id3' + - 'id4' + priority: 1 + - rule_id: query-rule-id3 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ logstash ] + actions: + ids: + - 'id1' + priority: 2 + - rule_id: query-rule-id1 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ elastic ] + actions: + ids: + - 'id1' + - 'id2' + - rule_id: query-rule-id4 + type: pinned + criteria: + - type: exact + metadata: query_string + values: [ search ] + actions: + ids: + - 'id2' + + +--- +'Create Query Rule - Resource already exists': + - do: + query_rule.put: + ruleset_id: test-query-rule-recreating + rule_id: abc + body: + type: 'pinned' + criteria: + type: 'exact' + metadata: 'query_string' + values: [ 'elastic' ] + actions: + ids: + - 'id1' + priority: 5 + + - match: { result: 'created' } + + - do: + query_rule.put: + ruleset_id: test-query-rule-recreating + rule_id: abc + body: + type: 'pinned' + criteria: + type: 'exact' + metadata: 'query_string' + values: [ 'elastic' ] + actions: + ids: + - 'id2' + priority: 3 + + - match: { result: 'updated' } + +--- +'Create Query Rule - Insufficient privilege': + - skip: + features: headers + + - do: + catch: forbidden + headers: { Authorization: "Basic ZW50c2VhcmNoLXVzZXI6ZW50c2VhcmNoLXVzZXItcGFzc3dvcmQ=" } # user + query_rule.put: + ruleset_id: forbidden-query-ruleset + rule_id: abc + body: + type: 'pinned' + criteria: + type: 'exact' + metadata: 'query_string' + values: [ 'elastic' ] + actions: + ids: + - 'id1' + - 'id2' + + - match: { error.type: 'security_exception' } diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/EnterpriseSearch.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/EnterpriseSearch.java index 871bf7fb122b9..9572eb599f2de 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/EnterpriseSearch.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/EnterpriseSearch.java @@ -151,14 +151,17 @@ import org.elasticsearch.xpack.application.rules.action.DeleteQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.GetQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.ListQueryRulesetsAction; +import org.elasticsearch.xpack.application.rules.action.PutQueryRuleAction; import org.elasticsearch.xpack.application.rules.action.PutQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.RestDeleteQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.RestGetQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.RestListQueryRulesetsAction; +import org.elasticsearch.xpack.application.rules.action.RestPutQueryRuleAction; import org.elasticsearch.xpack.application.rules.action.RestPutQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.TransportDeleteQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.TransportGetQueryRulesetAction; import org.elasticsearch.xpack.application.rules.action.TransportListQueryRulesetsAction; +import org.elasticsearch.xpack.application.rules.action.TransportPutQueryRuleAction; import org.elasticsearch.xpack.application.rules.action.TransportPutQueryRulesetAction; import org.elasticsearch.xpack.application.search.SearchApplicationIndexService; import org.elasticsearch.xpack.application.search.action.DeleteSearchApplicationAction; @@ -251,6 +254,7 @@ protected XPackLicenseState getLicenseState() { new ActionHandler<>(GetQueryRulesetAction.INSTANCE, TransportGetQueryRulesetAction.class), new ActionHandler<>(ListQueryRulesetsAction.INSTANCE, TransportListQueryRulesetsAction.class), new ActionHandler<>(PutQueryRulesetAction.INSTANCE, TransportPutQueryRulesetAction.class), + new ActionHandler<>(PutQueryRuleAction.INSTANCE, TransportPutQueryRuleAction.class), usageAction, infoAction @@ -354,7 +358,8 @@ public List getRestHandlers( new RestDeleteQueryRulesetAction(getLicenseState()), new RestGetQueryRulesetAction(getLicenseState()), new RestListQueryRulesetsAction(getLicenseState()), - new RestPutQueryRulesetAction(getLicenseState()) + new RestPutQueryRulesetAction(getLicenseState()), + new RestPutQueryRuleAction(getLicenseState()) ) ); diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRule.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRule.java index b9093a2597d7d..33fa74e5178cf 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRule.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRule.java @@ -8,12 +8,14 @@ package org.elasticsearch.xpack.application.rules; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.Nullable; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentObject; @@ -53,6 +55,10 @@ public class QueryRule implements Writeable, ToXContentObject { private final QueryRuleType type; private final List criteria; private final Map actions; + private final Integer priority; + + public static final int MIN_PRIORITY = 0; + public static final int MAX_PRIORITY = 1000000; public enum QueryRuleType { PINNED; @@ -79,11 +85,17 @@ public String toString() { * @param type The {@link QueryRuleType} of this rule * @param criteria The {@link QueryRuleCriteria} required for a query to match this rule * @param actions The actions that should be taken if this rule is matched, dependent on the type of rule + * @param priority If specified, assigns a priority to the rule. Rules with specified priorities are applied before + * rules without specified priorities, in ascending priority order. */ - public QueryRule(String id, QueryRuleType type, List criteria, Map actions) { - if (Strings.isNullOrEmpty(id)) { - throw new IllegalArgumentException("Query rule id cannot be null or blank"); - } + public QueryRule( + @Nullable String id, + QueryRuleType type, + List criteria, + Map actions, + @Nullable Integer priority + ) { + // Interstitial null state allowed during rule creation; validation occurs in CRUD API this.id = id; Objects.requireNonNull(type, "Query rule type cannot be null"); @@ -100,16 +112,27 @@ public QueryRule(String id, QueryRuleType type, List criteria throw new IllegalArgumentException("Query rule actions cannot be empty"); } this.actions = actions; + this.priority = priority; validate(); } + public QueryRule(String id, QueryRule other) { + this(id, other.type, other.criteria, other.actions, other.priority); + } + public QueryRule(StreamInput in) throws IOException { this.id = in.readString(); this.type = QueryRuleType.queryRuleType(in.readString()); this.criteria = in.readCollectionAsList(QueryRuleCriteria::new); this.actions = in.readGenericMap(); + if (in.getTransportVersion().onOrAfter(TransportVersions.QUERY_RULE_CRUD_API_PUT)) { + this.priority = in.readOptionalVInt(); + } else { + this.priority = null; + } + validate(); } @@ -126,6 +149,10 @@ private void validate() { } else { throw new IllegalArgumentException("Unsupported QueryRuleType: " + type); } + + if (priority != null && (priority < MIN_PRIORITY || priority > MAX_PRIORITY)) { + throw new IllegalArgumentException("Priority was " + priority + ", must be between " + MIN_PRIORITY + " and " + MAX_PRIORITY); + } } private void validatePinnedAction(Object action) { @@ -146,6 +173,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(type.toString()); out.writeCollection(criteria); out.writeGenericMap(actions); + if (out.getTransportVersion().onOrAfter(TransportVersions.QUERY_RULE_CRUD_API_PUT)) { + out.writeOptionalVInt(priority); + } } @SuppressWarnings("unchecked") @@ -157,7 +187,8 @@ public void writeTo(StreamOutput out) throws IOException { final QueryRuleType type = QueryRuleType.queryRuleType((String) params[1]); final List criteria = (List) params[2]; final Map actions = (Map) params[3]; - return new QueryRule(id, type, criteria, actions); + final Integer priority = (Integer) params[4]; + return new QueryRule(id, type, criteria, actions, priority); } ); @@ -165,12 +196,14 @@ public void writeTo(StreamOutput out) throws IOException { public static final ParseField TYPE_FIELD = new ParseField("type"); public static final ParseField CRITERIA_FIELD = new ParseField("criteria"); public static final ParseField ACTIONS_FIELD = new ParseField("actions"); + public static final ParseField PRIORITY_FIELD = new ParseField("priority"); static { PARSER.declareStringOrNull(optionalConstructorArg(), ID_FIELD); PARSER.declareString(constructorArg(), TYPE_FIELD); PARSER.declareObjectArray(constructorArg(), (p, c) -> QueryRuleCriteria.fromXContent(p), CRITERIA_FIELD); PARSER.declareObject(constructorArg(), (p, c) -> p.map(), ACTIONS_FIELD); + PARSER.declareInt(optionalConstructorArg(), PRIORITY_FIELD); } /** @@ -213,7 +246,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.xContentList(CRITERIA_FIELD.getPreferredName(), criteria); builder.field(ACTIONS_FIELD.getPreferredName()); builder.map(actions); - + if (priority != null) { + builder.field(PRIORITY_FIELD.getPreferredName(), priority); + } } builder.endObject(); return builder; @@ -255,6 +290,10 @@ public Map actions() { return actions; } + public Integer priority() { + return priority; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -263,12 +302,13 @@ public boolean equals(Object o) { return Objects.equals(id, queryRule.id) && type == queryRule.type && Objects.equals(criteria, queryRule.criteria) - && Objects.equals(actions, queryRule.actions); + && Objects.equals(actions, queryRule.actions) + && Objects.equals(priority, queryRule.priority); } @Override public int hashCode() { - return Objects.hash(id, type, criteria, actions); + return Objects.hash(id, type, criteria, actions, priority); } @Override diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java index 1e98755cc7acf..adcd5da988b82 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java @@ -23,6 +23,7 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.VersionId; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -37,17 +38,20 @@ import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.application.rules.action.PutQueryRuleAction; import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.EnumMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Function; import java.util.stream.Collectors; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; @@ -60,9 +64,9 @@ public class QueryRulesIndexService { private static final Logger logger = LogManager.getLogger(QueryRulesIndexService.class); public static final String QUERY_RULES_ALIAS_NAME = ".query-rules"; - public static final String QUERY_RULES_CONCRETE_INDEX_NAME = ".query-rules-1"; + public static final String QUERY_RULES_INDEX_PREFIX = ".query-rules-"; + public static final String QUERY_RULES_CONCRETE_INDEX_NAME = QUERY_RULES_INDEX_PREFIX + QueryRulesIndexMappingVersion.latest().id; public static final String QUERY_RULES_INDEX_NAME_PATTERN = ".query-rules-*"; - private static final int QUERY_RULES_INDEX_MAPPINGS_VERSION = 1; private final Client clientWithOrigin; private final ClusterSettings clusterSettings; @@ -77,16 +81,21 @@ public QueryRulesIndexService(Client client, ClusterSettings clusterSettings) { * @return The {@link SystemIndexDescriptor} for the {@link QueryRuleset} system index. */ public static SystemIndexDescriptor getSystemIndexDescriptor() { - return SystemIndexDescriptor.builder() - .setIndexPattern(QUERY_RULES_INDEX_NAME_PATTERN) - .setPrimaryIndex(QUERY_RULES_CONCRETE_INDEX_NAME) - .setDescription("Contains query ruleset configuration for query rules") - .setMappings(getIndexMappings()) - .setSettings(getIndexSettings()) - .setAliasName(QUERY_RULES_ALIAS_NAME) - .setVersionMetaKey("version") - .setOrigin(ENT_SEARCH_ORIGIN) - .setThreadPools(ExecutorNames.DEFAULT_SYSTEM_INDEX_THREAD_POOLS) + final Function systemIndexDescriptorBuilder = + mappingVersion -> SystemIndexDescriptor.builder() + .setIndexPattern(QUERY_RULES_INDEX_NAME_PATTERN) + .setPrimaryIndex(QUERY_RULES_CONCRETE_INDEX_NAME) + .setDescription("Contains query ruleset configuration for query rules") + .setMappings(getIndexMappings(mappingVersion)) + .setSettings(getIndexSettings()) + .setAliasName(QUERY_RULES_ALIAS_NAME) + .setIndexFormat(QueryRulesIndexMappingVersion.latest().id) + .setVersionMetaKey("version") + .setOrigin(ENT_SEARCH_ORIGIN) + .setThreadPools(ExecutorNames.DEFAULT_SYSTEM_INDEX_THREAD_POOLS); + + return systemIndexDescriptorBuilder.apply(QueryRulesIndexMappingVersion.latest()) + .setPriorSystemIndexDescriptors(List.of(systemIndexDescriptorBuilder.apply(QueryRulesIndexMappingVersion.INITIAL).build())) .build(); } @@ -96,18 +105,19 @@ private static Settings getIndexSettings() { .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 100) + .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), QueryRulesIndexMappingVersion.latest().id) .put("index.refresh_interval", "1s") .build(); } - private static XContentBuilder getIndexMappings() { + private static XContentBuilder getIndexMappings(QueryRulesIndexMappingVersion version) { try { final XContentBuilder builder = jsonBuilder(); builder.startObject(); { builder.startObject("_meta"); builder.field("version", Version.CURRENT.toString()); - builder.field(SystemIndexDescriptor.VERSION_META_KEY, QUERY_RULES_INDEX_MAPPINGS_VERSION); + builder.field(SystemIndexDescriptor.VERSION_META_KEY, version.id); builder.endObject(); builder.field("dynamic", "strict"); @@ -151,6 +161,12 @@ private static XContentBuilder getIndexMappings() { builder.field("type", "object"); builder.field("enabled", false); builder.endObject(); + + if (version.onOrAfter(QueryRulesIndexMappingVersion.ADD_PRIORITY)) { + builder.startObject(QueryRule.PRIORITY_FIELD.getPreferredName()); + builder.field("type", "integer"); + builder.endObject(); + } } builder.endObject(); builder.endObject(); @@ -191,7 +207,8 @@ public void onResponse(GetResponse getResponse) { (String) rule.get(QueryRule.ID_FIELD.getPreferredName()), QueryRuleType.queryRuleType((String) rule.get(QueryRule.TYPE_FIELD.getPreferredName())), parseCriteria((List>) rule.get(QueryRule.CRITERIA_FIELD.getPreferredName())), - (Map) rule.get(QueryRule.ACTIONS_FIELD.getPreferredName()) + (Map) rule.get(QueryRule.ACTIONS_FIELD.getPreferredName()), + (Integer) rule.get(QueryRule.PRIORITY_FIELD.getPreferredName()) ) ) .collect(Collectors.toList()); @@ -243,7 +260,45 @@ public void putQueryRuleset(QueryRuleset queryRuleset, ActionListener listener) { + getQueryRuleset(queryRulesetId, new ActionListener<>() { + @Override + public void onResponse(QueryRuleset queryRuleset) { + final List rules = new ArrayList<>(queryRuleset.rules()).stream() + .filter(rule -> rule.id().equals(queryRule.id()) == false) + .collect(Collectors.toList()); + rules.add(queryRule); + final boolean created = queryRuleset.rules().stream().noneMatch(rule -> rule.id().equals(queryRule.id())); + + putQueryRuleset(new QueryRuleset(queryRulesetId, rules), listener.delegateFailureAndWrap((delegate, docWriteResponse) -> { + DocWriteResponse.Result result = created ? DocWriteResponse.Result.CREATED : docWriteResponse.getResult(); + delegate.onResponse(new PutQueryRuleAction.Response(result)); + })); + } + @Override + public void onFailure(Exception e) { + if (e instanceof ResourceNotFoundException) { + putQueryRuleset( + new QueryRuleset(queryRulesetId, List.of(queryRule)), + listener.delegateFailureAndWrap((delegate, docWriteResponse) -> { + delegate.onResponse(new PutQueryRuleAction.Response(DocWriteResponse.Result.CREATED)); + }) + ); + return; + } + listener.onFailure(e); + } + }); } private void validateQueryRuleset(QueryRuleset queryRuleset) { @@ -355,4 +410,28 @@ private static QueryRulesetListItem hitToQueryRulesetListItem(SearchHit searchHi } public record QueryRulesetResult(List rulesets, long totalResults) {} + + public enum QueryRulesIndexMappingVersion implements VersionId { + INITIAL(1), + ADD_PRIORITY(2),; + + private static final QueryRulesIndexMappingVersion LATEST = Arrays.stream(values()) + .max(Comparator.comparingInt(v -> v.id)) + .orElseThrow(); + + private final int id; + + QueryRulesIndexMappingVersion(int id) { + this.id = id; + } + + @Override + public int id() { + return id; + } + + public static QueryRulesIndexMappingVersion latest() { + return LATEST; + } + } } diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRuleset.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRuleset.java index f58d01e7afe71..6ce93113cee0e 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRuleset.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRuleset.java @@ -23,8 +23,10 @@ import org.elasticsearch.xcontent.XContentType; import java.io.IOException; +import java.util.Comparator; import java.util.List; import java.util.Objects; +import java.util.stream.Collectors; import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -51,7 +53,9 @@ public QueryRuleset(String id, List rules) { if (rules.isEmpty()) { throw new IllegalArgumentException("rules cannot be empty"); } - this.rules = rules; + this.rules = rules.stream() + .sorted(Comparator.comparing(QueryRule::priority, Comparator.nullsLast(Comparator.naturalOrder()))) + .collect(Collectors.toList()); } public QueryRuleset(StreamInput in) throws IOException { diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleAction.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleAction.java new file mode 100644 index 0000000000000..75ab19ce8dffe --- /dev/null +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleAction.java @@ -0,0 +1,198 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.application.rules.QueryRule; + +import java.io.IOException; +import java.util.Objects; + +import static org.elasticsearch.action.ValidateActions.addValidationError; +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; + +public class PutQueryRuleAction { + + public static final String NAME = "cluster:admin/xpack/query_rule/put"; + public static final ActionType INSTANCE = new ActionType<>(NAME); + + private PutQueryRuleAction() {/* no instances */} + + public static class Request extends ActionRequest implements ToXContentObject { + + private final String queryRulesetId; + private final QueryRule queryRule; + private static final ParseField QUERY_RULESET_ID_FIELD = new ParseField("queryRulesetId"); + private static final ParseField QUERY_RULE_FIELD = new ParseField("queryRule"); + + public Request(StreamInput in) throws IOException { + super(in); + this.queryRulesetId = in.readString(); + this.queryRule = new QueryRule(in); + } + + public Request(String queryRulesetId, QueryRule queryRule) { + this.queryRulesetId = queryRulesetId; + this.queryRule = queryRule; + } + + public Request(String rulesetId, String ruleId, BytesReference content, XContentType contentType) { + this.queryRulesetId = rulesetId; + + QueryRule queryRule = QueryRule.fromXContentBytes(content, contentType); + if (queryRule.id() == null) { + this.queryRule = new QueryRule(ruleId, queryRule); + } else if (ruleId.equals(queryRule.id()) == false) { + throw new IllegalArgumentException("rule_id does not match the id in the query rule"); + } else { + this.queryRule = queryRule; + } + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + + if (Strings.isNullOrEmpty(queryRulesetId)) { + validationException = addValidationError("ruleset_id cannot be null or empty", validationException); + } + + if (Strings.isNullOrEmpty(queryRule.id())) { + validationException = addValidationError("rule_id cannot be null or empty", validationException); + } + + return validationException; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(queryRulesetId); + queryRule.writeTo(out); + } + + public String queryRulesetId() { + return queryRulesetId; + } + + public QueryRule queryRule() { + return queryRule; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(queryRulesetId, request.queryRulesetId) && Objects.equals(queryRule, request.queryRule); + } + + @Override + public int hashCode() { + return Objects.hash(queryRulesetId, queryRule); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(QUERY_RULESET_ID_FIELD.getPreferredName(), queryRulesetId); + builder.field(QUERY_RULE_FIELD.getPreferredName(), queryRule); + builder.endObject(); + return builder; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "put_query_rule_request", + p -> new Request((String) p[0], (QueryRule) p[1]) + ); + + static { + PARSER.declareString(constructorArg(), QUERY_RULESET_ID_FIELD); + PARSER.declareObject(constructorArg(), (p, c) -> QueryRule.fromXContent(p), QUERY_RULE_FIELD); + } + + public static PutQueryRuleAction.Request parse(XContentParser parser, String resourceName) { + return PARSER.apply(parser, resourceName); + } + + public static PutQueryRuleAction.Request fromXContent(String queryRulesetId, XContentParser parser) throws IOException { + return new PutQueryRuleAction.Request(queryRulesetId, QueryRule.fromXContent(parser)); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + } + + public static class Response extends ActionResponse implements ToXContentObject { + + final DocWriteResponse.Result result; + + public Response(StreamInput in) throws IOException { + super(in); + result = DocWriteResponse.Result.readFrom(in); + } + + public Response(DocWriteResponse.Result result) { + this.result = result; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + this.result.writeTo(out); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("result", this.result.getLowercase()); + builder.endObject(); + return builder; + } + + public RestStatus status() { + return switch (result) { + case CREATED -> RestStatus.CREATED; + case NOT_FOUND -> RestStatus.NOT_FOUND; + default -> RestStatus.OK; + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Response that = (Response) o; + return Objects.equals(result, that.result); + } + + @Override + public int hashCode() { + return Objects.hash(result); + } + + } + +} diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetAction.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetAction.java index 1a42d4c631a9b..842d5d5e0cee4 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetAction.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetAction.java @@ -69,6 +69,15 @@ public ActionRequestValidationException validate() { List rules = queryRuleset.rules(); if (rules == null || rules.isEmpty()) { validationException = addValidationError("rules cannot be null or empty", validationException); + } else { + for (QueryRule rule : rules) { + if (rule.id() == null) { + validationException = addValidationError( + "rule_id cannot be null or empty. rule: [" + rule + "]", + validationException + ); + } + } } return validationException; diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleAction.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleAction.java new file mode 100644 index 0000000000000..7abf156c9a144 --- /dev/null +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleAction.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.xpack.application.EnterpriseSearch; +import org.elasticsearch.xpack.application.EnterpriseSearchBaseRestHandler; +import org.elasticsearch.xpack.application.utils.LicenseUtils; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.PUT; + +@ServerlessScope(Scope.PUBLIC) +public class RestPutQueryRuleAction extends EnterpriseSearchBaseRestHandler { + public RestPutQueryRuleAction(XPackLicenseState licenseState) { + super(licenseState, LicenseUtils.Product.QUERY_RULES); + } + + @Override + public String getName() { + return "query_rule_put_action"; + } + + @Override + public List routes() { + return List.of(new Route(PUT, "/" + EnterpriseSearch.QUERY_RULES_API_ENDPOINT + "/{ruleset_id}/{rule_id}")); + } + + @Override + protected RestChannelConsumer innerPrepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + PutQueryRuleAction.Request request = new PutQueryRuleAction.Request( + restRequest.param("ruleset_id"), + restRequest.param("rule_id"), + restRequest.content(), + restRequest.getXContentType() + ); + return channel -> client.execute( + PutQueryRuleAction.INSTANCE, + request, + new RestToXContentListener<>(channel, PutQueryRuleAction.Response::status, r -> null) + ); + } +} diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/TransportPutQueryRuleAction.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/TransportPutQueryRuleAction.java new file mode 100644 index 0000000000000..69a568ff3b1a1 --- /dev/null +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/action/TransportPutQueryRuleAction.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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.application.rules.QueryRule; +import org.elasticsearch.xpack.application.rules.QueryRulesIndexService; + +public class TransportPutQueryRuleAction extends HandledTransportAction { + protected final QueryRulesIndexService systemIndexService; + + @Inject + public TransportPutQueryRuleAction( + TransportService transportService, + ClusterService clusterService, + ActionFilters actionFilters, + Client client + ) { + super( + PutQueryRuleAction.NAME, + transportService, + actionFilters, + PutQueryRuleAction.Request::new, + EsExecutors.DIRECT_EXECUTOR_SERVICE + ); + this.systemIndexService = new QueryRulesIndexService(client, clusterService.getClusterSettings()); + } + + @Override + protected void doExecute(Task task, PutQueryRuleAction.Request request, ActionListener listener) { + String queryRulesetId = request.queryRulesetId(); + QueryRule queryRule = request.queryRule(); + systemIndexService.putQueryRule(queryRulesetId, queryRule, ActionListener.wrap(listener::onResponse, listener::onFailure)); + + } +} diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTestUtils.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/EnterpriseSearchModuleTestUtils.java similarity index 85% rename from x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTestUtils.java rename to x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/EnterpriseSearchModuleTestUtils.java index 711051cbaffd0..06adb29e32691 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTestUtils.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/EnterpriseSearchModuleTestUtils.java @@ -5,7 +5,7 @@ * 2.0. */ -package org.elasticsearch.xpack.application.search; +package org.elasticsearch.xpack.application; import org.elasticsearch.core.Tuple; import org.elasticsearch.script.Script; @@ -15,6 +15,9 @@ import org.elasticsearch.xpack.application.rules.QueryRuleCriteria; import org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType; import org.elasticsearch.xpack.application.rules.QueryRuleset; +import org.elasticsearch.xpack.application.search.SearchApplication; +import org.elasticsearch.xpack.application.search.SearchApplicationTemplate; +import org.elasticsearch.xpack.application.search.TemplateParamValidator; import org.elasticsearch.xpack.core.action.util.PageParams; import java.util.ArrayList; @@ -33,12 +36,13 @@ import static org.elasticsearch.test.ESTestCase.randomList; import static org.elasticsearch.test.ESTestCase.randomLongBetween; import static org.elasticsearch.test.ESTestCase.randomMap; +import static org.elasticsearch.xpack.application.rules.QueryRule.MAX_PRIORITY; +import static org.elasticsearch.xpack.application.rules.QueryRule.MIN_PRIORITY; import static org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType.ALWAYS; -// TODO - move this one package up and rename to EnterpriseSearchModuleTestUtils -public final class SearchApplicationTestUtils { +public final class EnterpriseSearchModuleTestUtils { - private SearchApplicationTestUtils() { + private EnterpriseSearchModuleTestUtils() { throw new UnsupportedOperationException("Don't instantiate this class!"); } @@ -93,7 +97,12 @@ public static QueryRule randomQueryRule() { QueryRule.QueryRuleType type = randomFrom(QueryRule.QueryRuleType.values()); List criteria = List.of(randomQueryRuleCriteria()); Map actions = Map.of(randomFrom("ids", "docs"), List.of(randomAlphaOfLengthBetween(2, 10))); - return new QueryRule(id, type, criteria, actions); + Integer priority = randomQueryRulePriority(); + return new QueryRule(id, type, criteria, actions, priority); + } + + public static Integer randomQueryRulePriority() { + return randomBoolean() ? randomIntBetween(MIN_PRIORITY, MAX_PRIORITY) : null; } public static QueryRuleset randomQueryRuleset() { diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/action/ListConnectorActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/action/ListConnectorActionRequestBWCSerializingTests.java index 366001b6dd215..c71fbaf6716e4 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/action/ListConnectorActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/action/ListConnectorActionRequestBWCSerializingTests.java @@ -10,7 +10,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.core.action.util.PageParams; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; @@ -25,7 +25,7 @@ protected Writeable.Reader instanceReader() { @Override protected ListConnectorAction.Request createTestInstance() { - PageParams pageParams = SearchApplicationTestUtils.randomPageParams(); + PageParams pageParams = EnterpriseSearchModuleTestUtils.randomPageParams(); return new ListConnectorAction.Request( pageParams, List.of(generateRandomStringArray(10, 10, false)), diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/ConnectorSyncJobTestUtils.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/ConnectorSyncJobTestUtils.java index eb280334510cb..dcc6c9ba242d6 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/ConnectorSyncJobTestUtils.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/ConnectorSyncJobTestUtils.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Tuple; import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.connector.ConnectorTestUtils; import org.elasticsearch.xpack.application.connector.syncjob.action.CancelConnectorSyncJobAction; import org.elasticsearch.xpack.application.connector.syncjob.action.CheckInConnectorSyncJobAction; @@ -20,7 +21,6 @@ import org.elasticsearch.xpack.application.connector.syncjob.action.PostConnectorSyncJobAction; import org.elasticsearch.xpack.application.connector.syncjob.action.UpdateConnectorSyncJobErrorAction; import org.elasticsearch.xpack.application.connector.syncjob.action.UpdateConnectorSyncJobIngestionStatsAction; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import java.io.IOException; import java.time.Instant; @@ -188,7 +188,7 @@ public static GetConnectorSyncJobAction.Response getRandomGetConnectorSyncJobRes public static ListConnectorSyncJobsAction.Request getRandomListConnectorSyncJobsActionRequest() { return new ListConnectorSyncJobsAction.Request( - SearchApplicationTestUtils.randomPageParams(), + EnterpriseSearchModuleTestUtils.randomPageParams(), randomAlphaOfLength(10), ConnectorTestUtils.getRandomSyncStatus(), Collections.singletonList(ConnectorTestUtils.getRandomSyncJobType()) diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/action/ListConnectorSyncJobsActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/action/ListConnectorSyncJobsActionRequestBWCSerializingTests.java index 790f588e8937c..967994ebe57e0 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/action/ListConnectorSyncJobsActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/connector/syncjob/action/ListConnectorSyncJobsActionRequestBWCSerializingTests.java @@ -10,10 +10,10 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.connector.ConnectorSyncStatus; import org.elasticsearch.xpack.application.connector.ConnectorTestUtils; import org.elasticsearch.xpack.application.connector.syncjob.ConnectorSyncJobType; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.action.util.PageParams; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; @@ -29,7 +29,7 @@ protected Writeable.Reader instanceReader() @Override protected ListConnectorSyncJobsAction.Request createTestInstance() { - PageParams pageParams = SearchApplicationTestUtils.randomPageParams(); + PageParams pageParams = EnterpriseSearchModuleTestUtils.randomPageParams(); String connectorId = randomAlphaOfLength(10); ConnectorSyncStatus syncStatus = ConnectorTestUtils.getRandomSyncStatus(); ConnectorSyncJobType syncJobType = ConnectorTestUtils.getRandomSyncJobType(); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleCriteriaTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleCriteriaTests.java index 7b5fa7d053df8..881b77442daca 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleCriteriaTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleCriteriaTests.java @@ -17,7 +17,7 @@ import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.io.IOException; @@ -51,7 +51,7 @@ public void registerNamedObjects() { public final void testRandomSerialization() throws IOException { for (int runs = 0; runs < 10; runs++) { - QueryRuleCriteria testInstance = SearchApplicationTestUtils.randomQueryRuleCriteria(); + QueryRuleCriteria testInstance = EnterpriseSearchModuleTestUtils.randomQueryRuleCriteria(); assertTransportSerialization(testInstance); assertXContent(testInstance, randomBoolean()); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleTests.java index 5576ec71667f4..a48d6f45589e3 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRuleTests.java @@ -17,7 +17,7 @@ import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.io.IOException; @@ -28,6 +28,8 @@ import static java.util.Collections.emptyList; import static org.elasticsearch.common.xcontent.XContentHelper.toXContent; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent; +import static org.elasticsearch.xpack.application.rules.QueryRule.MAX_PRIORITY; +import static org.elasticsearch.xpack.application.rules.QueryRule.MIN_PRIORITY; import static org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType.EXACT; import static org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType.PREFIX; import static org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType.SUFFIX; @@ -46,7 +48,7 @@ public void registerNamedObjects() { public final void testRandomSerialization() throws IOException { for (int runs = 0; runs < 10; runs++) { - QueryRule testInstance = SearchApplicationTestUtils.randomQueryRule(); + QueryRule testInstance = EnterpriseSearchModuleTestUtils.randomQueryRule(); assertTransportSerialization(testInstance); assertXContent(testInstance, randomBoolean()); } @@ -62,7 +64,8 @@ public void testToXContent() throws IOException { ], "actions": { "ids": ["id1", "id2"] - } + }, + "priority": 5 }"""); QueryRule queryRule = QueryRule.fromXContentBytes(new BytesArray(content), XContentType.JSON); @@ -75,20 +78,6 @@ public void testToXContent() throws IOException { assertToXContentEquivalent(originalBytes, toXContent(parsed, XContentType.JSON, humanReadable), XContentType.JSON); } - public void testToXContentMissingQueryRuleId() throws IOException { - String content = XContentHelper.stripWhitespace(""" - { - "type": "pinned", - "criteria": [ - { "type": "exact", "metadata": "query_string", "values": ["foo", "bar"] } - ], - "actions": { - "ids": ["id1", "id2"] - } - }"""); - expectThrows(IllegalArgumentException.class, () -> QueryRule.fromXContentBytes(new BytesArray(content), XContentType.JSON)); - } - public void testToXContentEmptyCriteria() throws IOException { String content = XContentHelper.stripWhitespace(""" { @@ -170,7 +159,8 @@ public void testApplyRuleWithOneCriteria() { randomAlphaOfLength(10), QueryRule.QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query", List.of("elastic"))), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + randomBoolean() ? randomIntBetween(MIN_PRIORITY, MAX_PRIORITY) : null ); AppliedQueryRules appliedQueryRules = new AppliedQueryRules(); rule.applyRule(appliedQueryRules, Map.of("query", "elastic")); @@ -186,7 +176,8 @@ public void testApplyRuleWithMultipleCriteria() { randomAlphaOfLength(10), QueryRule.QueryRuleType.PINNED, List.of(new QueryRuleCriteria(PREFIX, "query", List.of("elastic")), new QueryRuleCriteria(SUFFIX, "query", List.of("search"))), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + randomBoolean() ? randomIntBetween(MIN_PRIORITY, MAX_PRIORITY) : null ); AppliedQueryRules appliedQueryRules = new AppliedQueryRules(); rule.applyRule(appliedQueryRules, Map.of("query", "elastic - you know, for search")); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java index 9ce62ee8d4c16..5eeca8465d394 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexServiceTests.java @@ -19,6 +19,7 @@ import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.util.ArrayList; @@ -74,7 +75,8 @@ public void testUpdateQueryRuleset() throws Exception { "my_rule1", QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("foo"))), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", Collections.singletonList(myQueryRule1)); DocWriteResponse resp = awaitPutQueryRuleset(myQueryRuleset); @@ -89,13 +91,15 @@ public void testUpdateQueryRuleset() throws Exception { "my_rule1", QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("foo"))), - Map.of("docs", List.of(Map.of("_index", "my_index1", "_id", "id1"), Map.of("_index", "my_index2", "_id", "id2"))) + Map.of("docs", List.of(Map.of("_index", "my_index1", "_id", "id1"), Map.of("_index", "my_index2", "_id", "id2"))), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ); final QueryRule myQueryRule2 = new QueryRule( "my_rule2", QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("bar"))), - Map.of("docs", List.of(Map.of("_index", "my_index1", "_id", "id3"), Map.of("_index", "my_index2", "_id", "id4"))) + Map.of("docs", List.of(Map.of("_index", "my_index1", "_id", "id3"), Map.of("_index", "my_index2", "_id", "id4"))), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", List.of(myQueryRule1, myQueryRule2)); DocWriteResponse newResp = awaitPutQueryRuleset(myQueryRuleset); @@ -116,7 +120,8 @@ public void testListQueryRulesets() throws Exception { new QueryRuleCriteria(EXACT, "query_string", List.of("foo" + i)), new QueryRuleCriteria(GTE, "query_string", List.of(i)) ), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ), new QueryRule( "my_rule_" + i + "_" + (i + 1), @@ -125,7 +130,8 @@ public void testListQueryRulesets() throws Exception { new QueryRuleCriteria(FUZZY, "query_string", List.of("bar" + i)), new QueryRuleCriteria(GTE, "user.age", List.of(i)) ), - Map.of("ids", List.of("id3", "id4")) + Map.of("ids", List.of("id3", "id4")), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ) ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset_" + i, rules); @@ -175,13 +181,15 @@ public void testDeleteQueryRuleset() throws Exception { "my_rule1", QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("foo"))), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ); final QueryRule myQueryRule2 = new QueryRule( "my_rule2", QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("bar"))), - Map.of("ids", List.of("id3", "id4")) + Map.of("ids", List.of("id3", "id4")), + EnterpriseSearchModuleTestUtils.randomQueryRulePriority() ); final QueryRuleset myQueryRuleset = new QueryRuleset("my_ruleset", List.of(myQueryRule1, myQueryRule2)); DocWriteResponse resp = awaitPutQueryRuleset(myQueryRuleset); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesetTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesetTests.java index 4799396ef5223..185e2429cf3c1 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesetTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/QueryRulesetTests.java @@ -17,7 +17,7 @@ import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.io.IOException; @@ -41,7 +41,7 @@ public void registerNamedObjects() { public final void testRandomSerialization() throws IOException { for (int runs = 0; runs < 10; runs++) { - QueryRuleset testInstance = SearchApplicationTestUtils.randomQueryRuleset(); + QueryRuleset testInstance = EnterpriseSearchModuleTestUtils.randomQueryRuleset(); assertTransportSerialization(testInstance); assertXContent(testInstance, randomBoolean()); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/RuleQueryBuilderTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/RuleQueryBuilderTests.java index 6979f00476cb2..bedd015406312 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/RuleQueryBuilderTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/RuleQueryBuilderTests.java @@ -161,7 +161,8 @@ protected Object simulateMethod(Method method, Object[] args) { "my_rule1", QueryRule.QueryRuleType.PINNED, List.of(new QueryRuleCriteria(EXACT, "query_string", List.of("elastic"))), - Map.of("ids", List.of("id1", "id2")) + Map.of("ids", List.of("id1", "id2")), + null ) ); QueryRuleset queryRuleset = new QueryRuleset(rulesetId, rules); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/GetQueryRulesetActionResponseBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/GetQueryRulesetActionResponseBWCSerializingTests.java index 4e2ce15c00350..4942f9fb076af 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/GetQueryRulesetActionResponseBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/GetQueryRulesetActionResponseBWCSerializingTests.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.application.rules.action; import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xpack.application.rules.QueryRule; @@ -19,8 +20,8 @@ import java.util.ArrayList; import java.util.List; +import static org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils.randomQueryRuleset; import static org.elasticsearch.xpack.application.rules.QueryRuleCriteria.CRITERIA_METADATA_VALUES_TRANSPORT_VERSION; -import static org.elasticsearch.xpack.application.search.SearchApplicationTestUtils.randomQueryRuleset; public class GetQueryRulesetActionResponseBWCSerializingTests extends AbstractBWCSerializationTestCase { public QueryRuleset queryRuleset; @@ -57,7 +58,13 @@ protected GetQueryRulesetAction.Response mutateInstanceForVersion(GetQueryRulese new QueryRuleCriteria(criteria.criteriaType(), criteria.criteriaMetadata(), criteria.criteriaValues().subList(0, 1)) ); } - rules.add(new QueryRule(rule.id(), rule.type(), newCriteria, rule.actions())); + rules.add(new QueryRule(rule.id(), rule.type(), newCriteria, rule.actions(), null)); + } + return new GetQueryRulesetAction.Response(new QueryRuleset(instance.queryRuleset().id(), rules)); + } else if (version.before(TransportVersions.QUERY_RULE_CRUD_API_PUT)) { + List rules = new ArrayList<>(); + for (QueryRule rule : instance.queryRuleset().rules()) { + rules.add(new QueryRule(rule.id(), rule.type(), rule.criteria(), rule.actions(), null)); } return new GetQueryRulesetAction.Response(new QueryRuleset(instance.queryRuleset().id(), rules)); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionRequestBWCSerializingTests.java index 92219f5f317d5..dfac7c57e01d3 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionRequestBWCSerializingTests.java @@ -10,7 +10,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.core.action.util.PageParams; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; @@ -26,7 +26,7 @@ protected Writeable.Reader instanceReader() { @Override protected ListQueryRulesetsAction.Request createTestInstance() { - PageParams pageParams = SearchApplicationTestUtils.randomPageParams(); + PageParams pageParams = EnterpriseSearchModuleTestUtils.randomPageParams(); return new ListQueryRulesetsAction.Request(pageParams); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionResponseBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionResponseBWCSerializingTests.java index 1613e31f94206..5ae0f51cb6112 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionResponseBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/ListQueryRulesetsActionResponseBWCSerializingTests.java @@ -9,10 +9,10 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.rules.QueryRuleCriteriaType; import org.elasticsearch.xpack.application.rules.QueryRuleset; import org.elasticsearch.xpack.application.rules.QueryRulesetListItem; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; import java.util.ArrayList; @@ -29,7 +29,7 @@ protected Writeable.Reader instanceReader() { private static ListQueryRulesetsAction.Response randomQueryRulesetListItem() { return new ListQueryRulesetsAction.Response(randomList(10, () -> { - QueryRuleset queryRuleset = SearchApplicationTestUtils.randomQueryRuleset(); + QueryRuleset queryRuleset = EnterpriseSearchModuleTestUtils.randomQueryRuleset(); Map criteriaTypeToCountMap = Map.of( randomFrom(QueryRuleCriteriaType.values()), randomIntBetween(0, 10) diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionRequestBWCSerializingTests.java new file mode 100644 index 0000000000000..a66d0c0aa5895 --- /dev/null +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionRequestBWCSerializingTests.java @@ -0,0 +1,62 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; +import org.elasticsearch.xpack.application.rules.QueryRule; +import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase.getAllBWCVersions; + +public class PutQueryRuleActionRequestBWCSerializingTests extends AbstractBWCSerializationTestCase { + + private String queryRuleId; + + @Override + protected Writeable.Reader instanceReader() { + return PutQueryRuleAction.Request::new; + } + + @Override + protected PutQueryRuleAction.Request createTestInstance() { + String queryRulesetId = randomAlphaOfLengthBetween(5, 10); + QueryRule queryRule = EnterpriseSearchModuleTestUtils.randomQueryRule(); + this.queryRuleId = queryRule.id(); + return new PutQueryRuleAction.Request(queryRulesetId, queryRule); + } + + @Override + protected PutQueryRuleAction.Request mutateInstance(PutQueryRuleAction.Request instance) { + return randomValueOtherThan(instance, this::createTestInstance); + } + + @Override + protected PutQueryRuleAction.Request doParseInstance(XContentParser parser) throws IOException { + return PutQueryRuleAction.Request.parse(parser, this.queryRuleId); + } + + @Override + protected PutQueryRuleAction.Request mutateInstanceForVersion(PutQueryRuleAction.Request instance, TransportVersion version) { + return new PutQueryRuleAction.Request(instance.queryRulesetId(), instance.queryRule()); + } + + @Override + protected List bwcVersions() { + return getAllBWCVersions().stream() + .filter(v -> v.onOrAfter(TransportVersions.QUERY_RULE_CRUD_API_PUT)) + .collect(Collectors.toList()); + } +} diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionResponseSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionResponseSerializingTests.java new file mode 100644 index 0000000000000..47be14761684d --- /dev/null +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRuleActionResponseSerializingTests.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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; + +import java.io.IOException; + +public class PutQueryRuleActionResponseSerializingTests extends AbstractBWCWireSerializationTestCase { + + @Override + protected Writeable.Reader instanceReader() { + return PutQueryRuleAction.Response::new; + } + + @Override + protected PutQueryRuleAction.Response createTestInstance() { + return new PutQueryRuleAction.Response(randomFrom(DocWriteResponse.Result.values())); + } + + @Override + protected PutQueryRuleAction.Response mutateInstance(PutQueryRuleAction.Response instance) throws IOException { + return randomValueOtherThan(instance, this::createTestInstance); + } + + @Override + protected PutQueryRuleAction.Response mutateInstanceForVersion(PutQueryRuleAction.Response instance, TransportVersion version) { + return instance; + } +} diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetActionRequestBWCSerializingTests.java index c6c463b677afa..83702b0b0672c 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/PutQueryRulesetActionRequestBWCSerializingTests.java @@ -8,12 +8,13 @@ package org.elasticsearch.xpack.application.rules.action; import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.rules.QueryRule; import org.elasticsearch.xpack.application.rules.QueryRuleCriteria; import org.elasticsearch.xpack.application.rules.QueryRuleset; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; import java.io.IOException; @@ -33,7 +34,7 @@ protected Writeable.Reader instanceReader() { @Override protected PutQueryRulesetAction.Request createTestInstance() { - this.queryRulesSet = SearchApplicationTestUtils.randomQueryRuleset(); + this.queryRulesSet = EnterpriseSearchModuleTestUtils.randomQueryRuleset(); return new PutQueryRulesetAction.Request(this.queryRulesSet); } @@ -59,7 +60,13 @@ protected PutQueryRulesetAction.Request mutateInstanceForVersion(PutQueryRuleset new QueryRuleCriteria(criteria.criteriaType(), criteria.criteriaMetadata(), criteria.criteriaValues().subList(0, 1)) ); } - rules.add(new QueryRule(rule.id(), rule.type(), newCriteria, rule.actions())); + rules.add(new QueryRule(rule.id(), rule.type(), newCriteria, rule.actions(), null)); + } + return new PutQueryRulesetAction.Request(new QueryRuleset(instance.queryRuleset().id(), rules)); + } else if (version.before(TransportVersions.QUERY_RULE_CRUD_API_PUT)) { + List rules = new ArrayList<>(); + for (QueryRule rule : instance.queryRuleset().rules()) { + rules.add(new QueryRule(rule.id(), rule.type(), rule.criteria(), rule.actions(), null)); } return new PutQueryRulesetAction.Request(new QueryRuleset(instance.queryRuleset().id(), rules)); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleActionTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleActionTests.java new file mode 100644 index 0000000000000..0aff0b804e538 --- /dev/null +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/rules/action/RestPutQueryRuleActionTests.java @@ -0,0 +1,66 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.application.rules.action; + +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.application.AbstractRestEnterpriseSearchActionTests; +import org.elasticsearch.xpack.application.EnterpriseSearchBaseRestHandler; +import org.elasticsearch.xpack.application.utils.LicenseUtils; + +import java.util.Map; + +public class RestPutQueryRuleActionTests extends AbstractRestEnterpriseSearchActionTests { + public void testWithNonCompliantLicense() throws Exception { + checkLicenseForRequest( + new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withMethod(RestRequest.Method.PUT) + .withParams(Map.of("ruleset_id", "ruleset-id", "rule_id", "rule-id")) + .withContent(new BytesArray(""" + { + "rule_id": "rule-id", + "type": "pinned", + "criteria": [ + { + "type": "exact", + "metadata": "query_string", + "values": ["elastic"] + } + ], + "actions": + { + "ids": [ + "id1", + "id2" + ] + } + } + """), XContentType.JSON) + .build(), + LicenseUtils.Product.QUERY_RULES + ); + } + + public void testInvalidRequestWithNonCompliantLicense() throws Exception { + checkLicenseForRequest( + new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withMethod(RestRequest.Method.PUT) + .withParams(Map.of("invalid_param_name", "invalid_value")) + .withContent(new BytesArray("{}"), XContentType.JSON) + .build(), + LicenseUtils.Product.QUERY_RULES + ); + } + + @Override + protected EnterpriseSearchBaseRestHandler getRestAction(XPackLicenseState licenseState) { + return new RestPutQueryRuleAction(licenseState); + } +} diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java index 7891f5773d1a8..6e9d33b45041b 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexServiceTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.util.ArrayList; @@ -131,7 +132,7 @@ public void testUpdateSearchApplication() throws Exception { new String[] { "index_1", "index_2" }, null, System.currentTimeMillis(), - SearchApplicationTestUtils.getRandomSearchApplicationTemplate() + EnterpriseSearchModuleTestUtils.getRandomSearchApplicationTemplate() ); DocWriteResponse resp = awaitPutSearchApplication(searchApp, false); assertThat(resp.status(), equalTo(RestStatus.CREATED)); @@ -146,7 +147,7 @@ public void testUpdateSearchApplication() throws Exception { new String[] { "index_3", "index_4" }, "my_search_app_analytics_collection", System.currentTimeMillis(), - SearchApplicationTestUtils.getRandomSearchApplicationTemplate() + EnterpriseSearchModuleTestUtils.getRandomSearchApplicationTemplate() ); DocWriteResponse newResp = awaitPutSearchApplication(searchApp, false); assertThat(newResp.status(), equalTo(RestStatus.OK)); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTests.java index 60b88476285df..67a5bd6800447 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/SearchApplicationTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.junit.Before; import java.io.IOException; @@ -46,7 +47,7 @@ public void registerNamedObjects() { public final void testRandomSerialization() throws IOException { for (int runs = 0; runs < 10; runs++) { - SearchApplication testInstance = SearchApplicationTestUtils.randomSearchApplication(); + SearchApplication testInstance = EnterpriseSearchModuleTestUtils.randomSearchApplication(); assertTransportSerialization(testInstance); assertXContent(testInstance, randomBoolean()); assertIndexSerialization(testInstance); diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/GetSearchApplicationActionResponseBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/GetSearchApplicationActionResponseBWCSerializingTests.java index bb3e36c95f0ab..11c28f062d272 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/GetSearchApplicationActionResponseBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/GetSearchApplicationActionResponseBWCSerializingTests.java @@ -10,8 +10,8 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.search.SearchApplication; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; import java.io.IOException; @@ -28,7 +28,7 @@ protected Writeable.Reader instanceReader() @Override protected GetSearchApplicationAction.Response createTestInstance() { - SearchApplication searchApp = SearchApplicationTestUtils.randomSearchApplication(); + SearchApplication searchApp = EnterpriseSearchModuleTestUtils.randomSearchApplication(); this.searchApplicationName = searchApp.name(); return new GetSearchApplicationAction.Response(searchApp); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionRequestBWCSerializingTests.java index 62678e073a633..ba7b07441d8b1 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionRequestBWCSerializingTests.java @@ -10,7 +10,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.core.action.util.PageParams; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; @@ -27,7 +27,7 @@ protected Writeable.Reader instanceReader() @Override protected ListSearchApplicationAction.Request createTestInstance() { - PageParams pageParams = SearchApplicationTestUtils.randomPageParams(); + PageParams pageParams = EnterpriseSearchModuleTestUtils.randomPageParams(); String query = randomFrom(new String[] { null, randomAlphaOfLengthBetween(1, 10) }); return new ListSearchApplicationAction.Request(query, pageParams); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionResponseBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionResponseBWCSerializingTests.java index 38b1b94064b96..2489e14913e72 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionResponseBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/ListSearchApplicationActionResponseBWCSerializingTests.java @@ -9,9 +9,9 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.search.SearchApplication; import org.elasticsearch.xpack.application.search.SearchApplicationListItem; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; public class ListSearchApplicationActionResponseBWCSerializingTests extends AbstractBWCWireSerializationTestCase< @@ -24,7 +24,7 @@ protected Writeable.Reader instanceReader( private static ListSearchApplicationAction.Response randomSearchApplicationListItem() { return new ListSearchApplicationAction.Response(randomList(10, () -> { - SearchApplication app = SearchApplicationTestUtils.randomSearchApplication(); + SearchApplication app = EnterpriseSearchModuleTestUtils.randomSearchApplication(); return new SearchApplicationListItem(app.name(), app.analyticsCollectionName(), app.updatedAtMillis()); }), randomLongBetween(0, 1000)); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/PutSearchApplicationActionRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/PutSearchApplicationActionRequestBWCSerializingTests.java index 0d79950d2081a..88b752c80c26a 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/PutSearchApplicationActionRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/PutSearchApplicationActionRequestBWCSerializingTests.java @@ -10,8 +10,8 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.application.search.SearchApplication; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; import java.io.IOException; @@ -28,7 +28,7 @@ protected Writeable.Reader instanceReader() @Override protected PutSearchApplicationAction.Request createTestInstance() { - SearchApplication searchApp = SearchApplicationTestUtils.randomSearchApplication(); + SearchApplication searchApp = EnterpriseSearchModuleTestUtils.randomSearchApplication(); this.searchApplicationName = searchApp.name(); return new PutSearchApplicationAction.Request(searchApp, randomBoolean()); } diff --git a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/SearchApplicationSearchRequestBWCSerializingTests.java b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/SearchApplicationSearchRequestBWCSerializingTests.java index a107d02cc2ab2..7c3b504655bf3 100644 --- a/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/SearchApplicationSearchRequestBWCSerializingTests.java +++ b/x-pack/plugin/ent-search/src/test/java/org/elasticsearch/xpack/application/search/action/SearchApplicationSearchRequestBWCSerializingTests.java @@ -10,7 +10,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; -import org.elasticsearch.xpack.application.search.SearchApplicationTestUtils; +import org.elasticsearch.xpack.application.EnterpriseSearchModuleTestUtils; import org.elasticsearch.xpack.core.ml.AbstractBWCSerializationTestCase; import java.io.IOException; @@ -26,7 +26,7 @@ protected Writeable.Reader instanceReader() { protected SearchApplicationSearchRequest createTestInstance() { return new SearchApplicationSearchRequest( randomAlphaOfLengthBetween(1, 10), - SearchApplicationTestUtils.randomSearchApplicationQueryParams() + EnterpriseSearchModuleTestUtils.randomSearchApplicationQueryParams() ); } diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index 33503bc558795..8885c62dad55a 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -232,6 +232,7 @@ public class Constants { "cluster:admin/xpack/ml/upgrade_mode", "cluster:admin/xpack/monitoring/bulk", "cluster:admin/xpack/monitoring/migrate/alerts", + "cluster:admin/xpack/query_rule/put", "cluster:admin/xpack/query_rules/delete", "cluster:admin/xpack/query_rules/get", "cluster:admin/xpack/query_rules/list", From e4c10d82bb5ec9be854f1068377722661e813968 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 10 Jun 2024 12:03:28 -0700 Subject: [PATCH 19/31] Fix no match scorer time series source (#109545) The returned scorer can be null when the weight matches no document. --- ...TimeSeriesSortedSourceOperatorFactory.java | 7 ++- .../TimeSeriesSortedSourceOperatorTests.java | 54 +++++++++++++++++++ 2 files changed, 59 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorFactory.java index 887761fbd5a8b..8b52aa84aef21 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorFactory.java @@ -14,6 +14,7 @@ import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.PriorityQueue; @@ -361,7 +362,8 @@ static class Leaf { this.createdThread = Thread.currentThread(); tsids = leaf.reader().getSortedDocValues("_tsid"); timestamps = leaf.reader().getSortedNumericDocValues("@timestamp"); - iterator = weight.scorer(leaf).iterator(); + final Scorer scorer = weight.scorer(leaf); + iterator = scorer != null ? scorer.iterator() : DocIdSetIterator.empty(); } boolean nextDoc() throws IOException { @@ -384,7 +386,8 @@ void reinitializeIfNeeded(Thread executingThread) throws IOException { if (executingThread != createdThread) { tsids = leaf.reader().getSortedDocValues("_tsid"); timestamps = leaf.reader().getSortedNumericDocValues("@timestamp"); - iterator = weight.scorer(leaf).iterator(); + final Scorer scorer = weight.scorer(leaf); + iterator = scorer != null ? scorer.iterator() : DocIdSetIterator.empty(); if (docID != -1) { iterator.advance(docID); } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorTests.java index 29e78f7abffde..17d302f198bff 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/TimeSeriesSortedSourceOperatorTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.document.DoubleDocValuesField; import org.apache.lucene.document.FloatDocValuesField; +import org.apache.lucene.document.LongField; import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.SortedDocValuesField; @@ -18,6 +19,7 @@ import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; @@ -59,6 +61,7 @@ import java.util.function.Consumer; import java.util.function.Function; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -208,6 +211,57 @@ record Doc(int host, long timestamp, long metric) {} assertThat(offset, equalTo(Math.min(limit, numDocs))); } + public void testMatchNone() throws Exception { + long t0 = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis("2024-01-01T00:00:00Z"); + Sort sort = new Sort( + new SortField(TimeSeriesIdFieldMapper.NAME, SortField.Type.STRING, false), + new SortedNumericSortField(DataStreamTimestampFieldMapper.DEFAULT_PATH, SortField.Type.LONG, true) + ); + try ( + var directory = newDirectory(); + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setIndexSort(sort).setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + int numDocs = between(1, 100); + long timestamp = t0; + int metrics = randomIntBetween(1, 3); + for (int i = 0; i < numDocs; i++) { + timestamp += between(1, 1000); + for (int j = 0; j < metrics; j++) { + String hostname = String.format(Locale.ROOT, "sensor-%02d", j); + writeTS(writer, timestamp, new Object[] { "sensor", hostname }, new Object[] { "voltage", j + 5 }); + } + } + try (var reader = writer.getReader()) { + var ctx = new LuceneSourceOperatorTests.MockShardContext(reader, 0); + Query query = randomFrom(LongField.newRangeQuery("@timestamp", 0, t0), new MatchNoDocsQuery()); + var timeSeriesFactory = TimeSeriesSortedSourceOperatorFactory.create( + Integer.MAX_VALUE, + randomIntBetween(1, 1024), + 1, + TimeValue.ZERO, + List.of(ctx), + unused -> query + ); + var driverContext = driverContext(); + List results = new ArrayList<>(); + OperatorTestCase.runDriver( + new Driver( + driverContext, + timeSeriesFactory.get(driverContext), + List.of(), + new TestResultPageSinkOperator(results::add), + () -> {} + ) + ); + assertThat(results, empty()); + } + } + } + @Override protected Operator.OperatorFactory simple() { return createTimeSeriesSourceOperator(directory, r -> this.reader = r, 1, 1, false, TimeValue.ZERO, writer -> { From 0be3c741df3d8e0f09f2a3e652527eb9df4133fb Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Mon, 10 Jun 2024 13:55:53 -0700 Subject: [PATCH 20/31] Guard file settings readiness on file settings support (#109500) Consistency of file settings is an important invariant. However, when upgrading from Elasticsearch versions before file settings existed, cluster state will not yet have the file settings metadata. If the first node upgraded is not the master node, new nodes will never become ready while they wait for file settings metadata to exist. This commit adds a node feature for file settings to guard waiting on file settings for readiness. Although file settings has existed since 8.4, the feature is not a historical feature because historical features are not applied to cluster state that readiness checks. In this case it is not needed since upgrading from 8.4+ will already contain file settings metadata. --- docs/changelog/109500.yaml | 5 ++ server/src/main/java/module-info.java | 3 +- .../readiness/ReadinessService.java | 19 ++++- .../service/FileSettingsFeatures.java | 26 +++++++ ...lasticsearch.features.FeatureSpecification | 1 + .../readiness/ReadinessServiceTests.java | 70 +++++++++++++------ 6 files changed, 99 insertions(+), 25 deletions(-) create mode 100644 docs/changelog/109500.yaml create mode 100644 server/src/main/java/org/elasticsearch/reservedstate/service/FileSettingsFeatures.java diff --git a/docs/changelog/109500.yaml b/docs/changelog/109500.yaml new file mode 100644 index 0000000000000..cfd6bc770d5d6 --- /dev/null +++ b/docs/changelog/109500.yaml @@ -0,0 +1,5 @@ +pr: 109500 +summary: Guard file settings readiness on file settings support +area: Infra/Settings +type: bug +issues: [] diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 2f08129b4080d..0d6bc6b29b2c2 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -430,7 +430,8 @@ org.elasticsearch.indices.IndicesFeatures, org.elasticsearch.action.admin.cluster.allocation.AllocationStatsFeatures, org.elasticsearch.index.mapper.MapperFeatures, - org.elasticsearch.search.retriever.RetrieversFeatures; + org.elasticsearch.search.retriever.RetrieversFeatures, + org.elasticsearch.reservedstate.service.FileSettingsFeatures; uses org.elasticsearch.plugins.internal.SettingsExtension; uses RestExtension; diff --git a/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java b/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java index 4aab146230f7c..a50929062d518 100644 --- a/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java +++ b/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java @@ -21,7 +21,9 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.env.Environment; +import org.elasticsearch.reservedstate.service.FileSettingsFeatures; import org.elasticsearch.reservedstate.service.FileSettingsService; import org.elasticsearch.shutdown.PluginShutdownService; import org.elasticsearch.transport.BindTransportException; @@ -277,7 +279,22 @@ private boolean isMasterElected(ClusterState clusterState) { // protected to allow mock service to override protected boolean areFileSettingsApplied(ClusterState clusterState) { ReservedStateMetadata fileSettingsMetadata = clusterState.metadata().reservedStateMetadata().get(FileSettingsService.NAMESPACE); - return fileSettingsMetadata != null && fileSettingsMetadata.version().equals(ReservedStateMetadata.NO_VERSION) == false; + if (fileSettingsMetadata == null) { + // In order to block readiness on file settings being applied, we need to know that the master node has written an initial + // version, or a marker that file settings don't exist. When upgrading from a version that did not have file settings, the + // current master node may not be the first node upgraded. To be safe, we wait to consider file settings application for + // readiness until the whole cluster supports file settings. Note that this only applies when no reserved state metadata + // exists, so either we are starting up a current cluster (and the feature will be found) or we are upgrading from + // a version before file settings existed (before 8.4). + return supportsFileSettings(clusterState) == false; + } else { + return fileSettingsMetadata.version().equals(ReservedStateMetadata.NO_VERSION) == false; + } + } + + @SuppressForbidden(reason = "need to check file settings support on exact cluster state") + private static boolean supportsFileSettings(ClusterState clusterState) { + return clusterState.clusterFeatures().clusterHasFeature(FileSettingsFeatures.FILE_SETTINGS_SUPPORTED); } private void setReady(boolean ready) { diff --git a/server/src/main/java/org/elasticsearch/reservedstate/service/FileSettingsFeatures.java b/server/src/main/java/org/elasticsearch/reservedstate/service/FileSettingsFeatures.java new file mode 100644 index 0000000000000..d707680b3e065 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/reservedstate/service/FileSettingsFeatures.java @@ -0,0 +1,26 @@ +/* + * 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.reservedstate.service; + +import org.elasticsearch.features.FeatureSpecification; +import org.elasticsearch.features.NodeFeature; + +import java.util.Set; + +public class FileSettingsFeatures implements FeatureSpecification { + + // Although file settings were supported starting in 8.4.0, this is really about whether file settings + // are used in readiness. + public static final NodeFeature FILE_SETTINGS_SUPPORTED = new NodeFeature("file_settings"); + + @Override + public Set getFeatures() { + return Set.of(FILE_SETTINGS_SUPPORTED); + } +} diff --git a/server/src/main/resources/META-INF/services/org.elasticsearch.features.FeatureSpecification b/server/src/main/resources/META-INF/services/org.elasticsearch.features.FeatureSpecification index a158f91903c70..d8a29a84ddbb7 100644 --- a/server/src/main/resources/META-INF/services/org.elasticsearch.features.FeatureSpecification +++ b/server/src/main/resources/META-INF/services/org.elasticsearch.features.FeatureSpecification @@ -15,3 +15,4 @@ org.elasticsearch.indices.IndicesFeatures org.elasticsearch.action.admin.cluster.allocation.AllocationStatsFeatures org.elasticsearch.index.mapper.MapperFeatures org.elasticsearch.search.retriever.RetrieversFeatures +org.elasticsearch.reservedstate.service.FileSettingsFeatures diff --git a/server/src/test/java/org/elasticsearch/readiness/ReadinessServiceTests.java b/server/src/test/java/org/elasticsearch/readiness/ReadinessServiceTests.java index 6bc58cc37a314..62443d6accb41 100644 --- a/server/src/test/java/org/elasticsearch/readiness/ReadinessServiceTests.java +++ b/server/src/test/java/org/elasticsearch/readiness/ReadinessServiceTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpStats; +import org.elasticsearch.reservedstate.service.FileSettingsFeatures; import org.elasticsearch.reservedstate.service.FileSettingsService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLog; @@ -46,6 +47,7 @@ import java.nio.channels.ServerSocketChannel; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import static org.elasticsearch.cluster.metadata.ReservedStateErrorMetadata.ErrorKind.TRANSIENT; @@ -57,6 +59,7 @@ public class ReadinessServiceTests extends ESTestCase implements ReadinessClient private ThreadPool threadpool; private Environment env; private FakeHttpTransport httpTransport; + private static final Set nodeFeatures = Set.of(FileSettingsFeatures.FILE_SETTINGS_SUPPORTED.id()); private static Metadata emptyReservedStateMetadata; static { @@ -205,21 +208,8 @@ public void testStatusChange() throws Exception { // initially the service isn't ready assertFalse(readinessService.ready()); - ClusterState emptyState = ClusterState.builder(new ClusterName("cluster")) - .nodes( - DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("node2", new TransportAddress(TransportAddress.META_ADDRESS, 9201))) - ) - .build(); - - ClusterState noFileSettingsState = ClusterState.builder(emptyState) - .nodes( - DiscoveryNodes.builder(emptyState.nodes()) - .add(httpTransport.node) - .masterNodeId(httpTransport.node.getId()) - .localNodeId(httpTransport.node.getId()) - ) - .build(); - ClusterChangedEvent event = new ClusterChangedEvent("test", noFileSettingsState, emptyState); + ClusterState noFileSettingsState = noFileSettingsState(); + ClusterChangedEvent event = new ClusterChangedEvent("test", noFileSettingsState, emptyState()); readinessService.clusterChanged(event); // sending a cluster state with active master should not yet bring up the service, file settings still are not applied @@ -306,14 +296,7 @@ public void testFileSettingsUpdateError() throws Exception { var fileSettingsState = new ReservedStateMetadata.Builder(FileSettingsService.NAMESPACE).version(21L) .errorMetadata(new ReservedStateErrorMetadata(22L, TRANSIENT, List.of("dummy error"))); - ClusterState state = ClusterState.builder(new ClusterName("cluster")) - .nodes( - DiscoveryNodes.builder() - .add(DiscoveryNodeUtils.create("node2", new TransportAddress(TransportAddress.META_ADDRESS, 9201))) - .add(httpTransport.node) - .masterNodeId(httpTransport.node.getId()) - .localNodeId(httpTransport.node.getId()) - ) + ClusterState state = ClusterState.builder(noFileSettingsState()) .metadata(new Metadata.Builder().put(fileSettingsState.build())) .build(); @@ -324,4 +307,45 @@ public void testFileSettingsUpdateError() throws Exception { readinessService.stop(); readinessService.close(); } + + public void testFileSettingsMixedCluster() throws Exception { + readinessService.start(); + + // initially the service isn't ready because initial cluster state has not been applied yet + assertFalse(readinessService.ready()); + + ClusterState noFileSettingsState = ClusterState.builder(noFileSettingsState()) + // the master node is upgraded to support file settings, but existing node2 is not + .nodeFeatures(Map.of(httpTransport.node.getId(), nodeFeatures)) + .build(); + ClusterChangedEvent event = new ClusterChangedEvent("test", noFileSettingsState, emptyState()); + readinessService.clusterChanged(event); + + // when upgrading from nodes before file settings exist, readiness should return true once a master is elected + assertTrue(readinessService.ready()); + + readinessService.stop(); + readinessService.close(); + } + + private ClusterState emptyState() { + return ClusterState.builder(new ClusterName("cluster")) + .nodes( + DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("node2", new TransportAddress(TransportAddress.META_ADDRESS, 9201))) + ) + .build(); + } + + private ClusterState noFileSettingsState() { + ClusterState emptyState = emptyState(); + return ClusterState.builder(emptyState) + .nodes( + DiscoveryNodes.builder(emptyState.nodes()) + .add(httpTransport.node) + .masterNodeId(httpTransport.node.getId()) + .localNodeId(httpTransport.node.getId()) + ) + .nodeFeatures(Map.of(httpTransport.node.getId(), nodeFeatures, "node2", nodeFeatures)) + .build(); + } } From f75afb00983349bb3019a62370b05dfa8e03da25 Mon Sep 17 00:00:00 2001 From: Ry Biesemeyer Date: Mon, 10 Jun 2024 15:12:38 -0700 Subject: [PATCH 21/31] Update year in NOTICE.txt (#109548) --- NOTICE.txt | 2 +- .../InternalDistributionArchiveCheckPluginFuncTest.groovy | 4 ++-- .../internal/InternalDistributionArchiveCheckPlugin.java | 2 +- distribution/packages/build.gradle | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/NOTICE.txt b/NOTICE.txt index c44f918942dce..9027c024fd87f 100644 --- a/NOTICE.txt +++ b/NOTICE.txt @@ -1,5 +1,5 @@ Elasticsearch -Copyright 2009-2021 Elasticsearch +Copyright 2009-2024 Elasticsearch This product includes software developed by The Apache Software Foundation (http://www.apache.org/). diff --git a/build-tools-internal/src/integTest/groovy/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPluginFuncTest.groovy b/build-tools-internal/src/integTest/groovy/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPluginFuncTest.groovy index 587343133b08e..860dc4e6f4d91 100644 --- a/build-tools-internal/src/integTest/groovy/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPluginFuncTest.groovy +++ b/build-tools-internal/src/integTest/groovy/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPluginFuncTest.groovy @@ -117,7 +117,7 @@ Copyright 2009-2018 Acme Coorp""" result.task(":darwin-tar:checkNotice").outcome == TaskOutcome.FAILED result.output.contains("> expected line [2] in " + "[./darwin-tar/build/tar-extracted/elasticsearch-${VersionProperties.getElasticsearch()}/NOTICE.txt] " + - "to be [Copyright 2009-2021 Elasticsearch] but was [Copyright 2009-2018 Acme Coorp]") + "to be [Copyright 2009-2024 Elasticsearch] but was [Copyright 2009-2018 Acme Coorp]") } def "fails on unexpected ml notice content"() { @@ -125,7 +125,7 @@ Copyright 2009-2018 Acme Coorp""" elasticLicense() elasticLicense(file("LICENSE.txt")) file("NOTICE.txt").text = """Elasticsearch -Copyright 2009-2021 Elasticsearch""" +Copyright 2009-2024 Elasticsearch""" file("ml/NOTICE.txt").text = "Boost Software License - Version 1.0 - August 17th, 2003" file('darwin-tar/build.gradle') << """ diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPlugin.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPlugin.java index 6fafe513662c5..94b1c70f29650 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPlugin.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/InternalDistributionArchiveCheckPlugin.java @@ -135,7 +135,7 @@ private TaskProvider registerCheckNoticeTask(Project project, TaskProvider task.doLast(new Action() { @Override public void execute(Task task) { - final List noticeLines = Arrays.asList("Elasticsearch", "Copyright 2009-2021 Elasticsearch"); + final List noticeLines = Arrays.asList("Elasticsearch", "Copyright 2009-2024 Elasticsearch"); final Path noticePath = checkExtraction.get() .getDestinationDir() .toPath() diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle index 6c31bc44017c3..2dfd24d97cbba 100644 --- a/distribution/packages/build.gradle +++ b/distribution/packages/build.gradle @@ -495,7 +495,7 @@ subprojects { (project.name.contains('deb') && dpkgExists.call(it)) || (project.name.contains('rpm') && rpmExists.call(it)) } doLast { - final List noticeLines = Arrays.asList("Elasticsearch", "Copyright 2009-2021 Elasticsearch") + final List noticeLines = Arrays.asList("Elasticsearch", "Copyright 2009-2024 Elasticsearch") final Path noticePath = packageExtractionDir.toPath().resolve("usr/share/elasticsearch/NOTICE.txt") assertLinesInFile(noticePath, noticeLines) } From 09fc32090cce4edf2f3b28e90c935002181cbc2a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 11 Jun 2024 10:17:48 +0200 Subject: [PATCH 22/31] Re-define index.mapper.dynamic setting in 8.x (#109341) Currently when upgrading a 7.x cluster to 8.x with `index.mapper.dynamic` index setting defined the following happens: - In case of a full cluster restart upgrade, then the index setting gets archived and after the upgrade the cluster is in a green health. - In case of a rolling cluster restart upgrade, then shards of indices with the index setting fail to allocate as nodes start with 8.x version. The result is that the cluster has a red health and the index setting isn't archived. Closing and opening the index should archive the index setting and allocate the shards. The change is about ensuring the same behavior happens when upgrading a cluster from 7.x to 8.x with indices that have the `index.mapper.dynamic` index setting defined. By re-defining the `index.mapper.dynamic `index setting with `IndexSettingDeprecatedInV7AndRemovedInV8` property, the index is allowed to exist in 7.x indices, but can't be defined in new indices after the upgrade. This way we don't have to rely on index archiving and upgrading via full cluster restart or rolling restart will yield the same outcome. Based on the test in #109301. Relates to #109160 and #96075 --- docs/changelog/109341.yaml | 5 ++ .../UpgradeWithOldIndexSettingsIT.java | 82 +++++++++++++++++++ .../UpgradeWithOldIndexSettingsIT.java | 34 ++++++++ .../common/settings/IndexScopedSettings.java | 1 + .../index/mapper/MapperService.java | 14 ++++ 5 files changed, 136 insertions(+) create mode 100644 docs/changelog/109341.yaml create mode 100644 qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java diff --git a/docs/changelog/109341.yaml b/docs/changelog/109341.yaml new file mode 100644 index 0000000000000..0c1eaa98a8aa2 --- /dev/null +++ b/docs/changelog/109341.yaml @@ -0,0 +1,5 @@ +pr: 109341 +summary: Re-define `index.mapper.dynamic` setting in 8.x for a better 7.x to 8.x upgrade if this setting is used. +area: Mapping +type: bug +issues: [] diff --git a/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java b/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java new file mode 100644 index 0000000000000..95178429317bf --- /dev/null +++ b/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java @@ -0,0 +1,82 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.upgrades; + +import com.carrotsearch.randomizedtesting.annotations.Name; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.FeatureFlag; +import org.elasticsearch.test.cluster.local.LocalClusterConfigProvider; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.junit.ClassRule; + +import java.io.IOException; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class UpgradeWithOldIndexSettingsIT extends ParameterizedFullClusterRestartTestCase { + + protected static LocalClusterConfigProvider clusterConfig = c -> {}; + + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.DEFAULT) + .version(getOldClusterTestVersion()) + .nodes(2) + .setting("xpack.security.enabled", "false") + .feature(FeatureFlag.FAILURE_STORE_ENABLED) + .apply(() -> clusterConfig) + .build(); + + @Override + protected ElasticsearchCluster getUpgradeCluster() { + return cluster; + } + + public UpgradeWithOldIndexSettingsIT(@Name("cluster") FullClusterRestartUpgradeStatus upgradeStatus) { + super(upgradeStatus); + } + + public void testMapperDynamicIndexSetting() throws IOException { + assumeTrue( + "Setting deprecated in 6.x, but remained in 7.x and is no longer defined in 8.x", + getOldClusterTestVersion().before("8.0.0") + ); + String indexName = "my-index"; + if (isRunningAgainstOldCluster()) { + createIndex(indexName); + + var request = new Request("PUT", "/my-index/_settings"); + request.setJsonEntity(org.elasticsearch.common.Strings.toString(Settings.builder().put("index.mapper.dynamic", true).build())); + request.setOptions( + expectWarnings( + "[index.mapper.dynamic] setting was deprecated in Elasticsearch and will be removed in a future release! " + + "See the breaking changes documentation for the next major version." + ) + ); + assertOK(client().performRequest(request)); + } else { + var indexSettings = getIndexSettings(indexName); + assertThat(XContentMapValues.extractValue(indexName + ".settings.index.mapper.dynamic", indexSettings), equalTo("true")); + ensureGreen(indexName); + // New indices can never define the index.mapper.dynamic setting. + Exception e = expectThrows( + ResponseException.class, + () -> createIndex("my-index2", Settings.builder().put("index.mapper.dynamic", true).build()) + ); + assertThat(e.getMessage(), containsString("unknown setting [index.mapper.dynamic]")); + } + } + +} diff --git a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java index 8dc3b43abf3e1..ba873ef6bbd7e 100644 --- a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java +++ b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/UpgradeWithOldIndexSettingsIT.java @@ -22,6 +22,8 @@ import java.util.Map; import static org.elasticsearch.rest.action.search.RestSearchAction.TOTAL_HITS_AS_INT_PARAM; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; public class UpgradeWithOldIndexSettingsIT extends AbstractRollingUpgradeTestCase { @@ -102,6 +104,38 @@ public void testOldIndexSettings() throws Exception { } } + public void testMapperDynamicIndexSetting() throws IOException { + assumeTrue( + "Setting deprecated in 6.x, but remained in 7.x and is no longer defined in 8.x", + getOldClusterTestVersion().before("8.0.0") + ); + String indexName = "my-index"; + if (isOldCluster()) { + createIndex(indexName); + Request request = new Request("PUT", "/" + indexName + "/_settings"); + request.setJsonEntity(org.elasticsearch.common.Strings.toString(Settings.builder().put("index.mapper.dynamic", true).build())); + request.setOptions( + expectWarnings( + "[index.mapper.dynamic] setting was deprecated in Elasticsearch and will be removed in a future release! " + + "See the breaking changes documentation for the next major version." + ) + ); + assertOK(client().performRequest(request)); + } else { + if (isUpgradedCluster()) { + var indexSettings = getIndexSettings(indexName); + assertThat(XContentMapValues.extractValue(indexName + ".settings.index.mapper.dynamic", indexSettings), equalTo("true")); + ensureGreen(indexName); + // New indices can never define the index.mapper.dynamic setting. + Exception e = expectThrows( + ResponseException.class, + () -> createIndex("my-index2", Settings.builder().put("index.mapper.dynamic", true).build()) + ); + assertThat(e.getMessage(), containsString("unknown setting [index.mapper.dynamic]")); + } + } + } + private void assertCount(String index, int countAtLeast) throws IOException { Request searchTestIndexRequest = new Request("POST", "/" + index + "/_search"); searchTestIndexRequest.addParameter(TOTAL_HITS_AS_INT_PARAM, "true"); diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 452fc14025e2e..f3eff9ae8838c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -158,6 +158,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { MapperService.INDEX_MAPPING_DEPTH_LIMIT_SETTING, MapperService.INDEX_MAPPING_DIMENSION_FIELDS_LIMIT_SETTING, MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, + MapperService.INDEX_MAPPER_DYNAMIC_SETTING, BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, IndexModule.INDEX_STORE_TYPE_SETTING, IndexModule.INDEX_STORE_PRE_LOAD_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index e5dc95ddbc2a0..3ac4c0b0e18e1 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.UpdateForV9; import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; @@ -147,6 +148,19 @@ public boolean isAutoUpdate() { Property.Dynamic, Property.IndexScope ); + /** + * Legacy index setting, kept for 7.x BWC compatibility. This setting has no effect in 8.x. Do not use. + * TODO: Remove in 9.0 + */ + @Deprecated + @UpdateForV9 + public static final Setting INDEX_MAPPER_DYNAMIC_SETTING = Setting.boolSetting( + "index.mapper.dynamic", + true, + Property.Dynamic, + Property.IndexScope, + Property.IndexSettingDeprecatedInV7AndRemovedInV8 + ); private final IndexAnalyzers indexAnalyzers; private final MappingParser mappingParser; From 62f7c0e88f3a9d12c72a7cb099328003366615e7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 11 Jun 2024 09:19:32 +0100 Subject: [PATCH 23/31] Avoid closing when partial snapshotting in `SnapshotStressTestsIT` (#109526) Closing an index while it's being partially-snapshotted is forbidden, but `SnapshotStressTestsIT#testRandomActivities` still sometimes attempts to do so which causes it to fail. This commit changes the behaviour to avoid doing these things to the same index at the same time. Closes #109138 --- .../snapshots/SnapshotStressTestsIT.java | 51 +++++++++++++++++-- 1 file changed, 46 insertions(+), 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java index 3f43da20fec3e..b759993be26df 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java @@ -81,7 +81,9 @@ import static org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.SNAPSHOT_ONLY_FORMAT_PARAMS; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.notNullValue; @LuceneTestCase.SuppressFileSystems(value = "HandleLimitFS") // we sometimes have >2048 open files @@ -468,17 +470,20 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio restoreSpecificIndicesTmp = true; continue; } - if (randomBoolean() && localReleasables.add(tryAcquireAllPermits(indices.get(indexName).permits)) != null) { + final var trackedIndex = indices.get(indexName); + if (randomBoolean() && localReleasables.add(tryAcquireAllPermits(trackedIndex.permits)) != null) { indicesToRestoreList.add(indexName); final int snapshotShardCount = snapshotInfo.indexSnapshotDetails().get(indexName).getShardCount(); - final int indexShardCount = indices.get(indexName).shardCount; - if (snapshotShardCount == indexShardCount && randomBoolean()) { + final int indexShardCount = trackedIndex.shardCount; + if (snapshotShardCount == indexShardCount + && randomBoolean() + && localReleasables.add(trackedIndex.tryAcquireClosingPermit()) != null) { indicesToCloseList.add(indexName); } else { indicesToDeleteList.add(indexName); - indices.get(indexName).shardCount = snapshotShardCount; + trackedIndex.shardCount = snapshotShardCount; } } else { restoreSpecificIndicesTmp = true; @@ -994,7 +999,9 @@ private void startPartialSnapshotter() { boolean snapshotSpecificIndicesTmp = randomBoolean(); final List targetIndexNames = new ArrayList<>(indices.size()); for (TrackedIndex trackedIndex : indices.values()) { - if (usually() && releasableAfterStart.add(tryAcquirePermit(trackedIndex.permits)) != null) { + if (usually() + && releasableAfterStart.add(tryAcquirePermit(trackedIndex.permits)) != null + && localReleasables.add(trackedIndex.tryAcquirePartialSnapshottingPermit()) != null) { targetIndexNames.add(trackedIndex.indexName); } else { snapshotSpecificIndicesTmp = true; @@ -1550,6 +1557,40 @@ private void scheduleIndexingAndPossibleDelete() { }); } + /** + * We must not close an index while it's being partially snapshotted; this counter tracks the number of ongoing + * close operations (positive) or partial snapshot operations (negative) in order to avoid them happening concurrently. + *

    + * This is only a problem for partial snapshots because we release the index permit once a partial snapshot has started. With + * non-partial snapshots we retain the index permit until it completes which blocks other operations. + */ + private final AtomicInteger closingOrPartialSnapshottingCount = new AtomicInteger(); + + private static boolean closingPermitAvailable(int value) { + return value >= 0 && value != Integer.MAX_VALUE; + } + + private static boolean partialSnapshottingPermitAvailable(int value) { + return value <= 0 && value != Integer.MIN_VALUE; + } + + Releasable tryAcquireClosingPermit() { + final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> closingPermitAvailable(c) ? c + 1 : c); + if (closingPermitAvailable(previous)) { + return () -> assertThat(closingOrPartialSnapshottingCount.getAndDecrement(), greaterThan(0)); + } else { + return null; + } + } + + Releasable tryAcquirePartialSnapshottingPermit() { + final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> partialSnapshottingPermitAvailable(c) ? c - 1 : c); + if (partialSnapshottingPermitAvailable(previous)) { + return () -> assertThat(closingOrPartialSnapshottingCount.getAndIncrement(), lessThan(0)); + } else { + return null; + } + } } } From d975997a3a1ed7013ac957159604378b6f994afe Mon Sep 17 00:00:00 2001 From: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> Date: Tue, 11 Jun 2024 10:33:25 +0200 Subject: [PATCH 24/31] Add semantic-text warning about inference endpoints removal (#109561) --- docs/reference/mapping/types/semantic-text.asciidoc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/reference/mapping/types/semantic-text.asciidoc b/docs/reference/mapping/types/semantic-text.asciidoc index 7fc1314ff888f..454eefd20b07f 100644 --- a/docs/reference/mapping/types/semantic-text.asciidoc +++ b/docs/reference/mapping/types/semantic-text.asciidoc @@ -52,6 +52,8 @@ Use the <> to create the endpoint. The `inference_id` will not be validated when the mapping is created, but when documents are ingested into the index. When the first document is indexed, the `inference_id` will be used to generate underlying indexing structures for the field. +WARNING: Removing an inference endpoint will cause ingestion of documents and semantic queries to fail on indices that define `semantic_text` fields with that inference endpoint as their `inference_id`. +Please check that inference endpoints are not used in `semantic_text` fields before removal. [discrete] [[auto-text-chunking]] From 0cd8e48cfb8202753ce0e4d165484ff601b6944f Mon Sep 17 00:00:00 2001 From: Artem Prigoda Date: Tue, 11 Jun 2024 10:44:22 +0200 Subject: [PATCH 25/31] Improve PrevalidateShardPathIT#testCheckShards (#109525) Add additional logging to log why `PrevalidateShardPathResponse` returns that shards is on the node while it's not in the cluster state. See #104807 --- .../cluster/PrevalidateShardPathIT.java | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/PrevalidateShardPathIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/PrevalidateShardPathIT.java index dd701244756cf..ea566c90ad769 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/PrevalidateShardPathIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/PrevalidateShardPathIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.test.ESIntegTestCase; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -70,6 +71,8 @@ public void testCheckShards() throws Exception { } // Check that after relocation the source node doesn't have the shard path String node3 = internalCluster().startDataOnlyNode(); + ensureStableCluster(4); + logger.info("Relocating shards from the node {}", node2); updateIndexSettings(Settings.builder().put("index.routing.allocation.exclude._name", node2), indexName); ensureGreen(indexName); assertBusy(() -> { @@ -82,13 +85,20 @@ public void testCheckShards() throws Exception { assertTrue("There should be no failures in the response", resp.failures().isEmpty()); Set node2ShardIds = resp2.getNodes().get(0).getShardIds(); if (node2ShardIds.size() > 0) { - for (var node2Shard : clusterService().state() + logger.info( + "Relocation source node {} should have no shards after the relocation, but still got {}", + node2Id, + node2ShardIds + ); + List node2Shards = clusterService().state() .routingTable() .allShards() .filter(s -> s.getIndexName().equals(indexName)) .filter(s -> node2ShardIds.contains(s.shardId())) .filter(s -> s.currentNodeId().equals(node2Id)) - .toList()) { + .toList(); + logger.info("Found {} shards on the relocation source node {} in the cluster state", node2Shards, node2Id); + for (var node2Shard : node2Shards) { var explanation = ClusterAllocationExplanationUtils.getClusterAllocationExplanation( client(), node2Shard.getIndexName(), @@ -109,6 +119,7 @@ public void testCheckShards() throws Exception { // If for whatever reason the removal is not triggered (e.g. not enough nodes reported that the shards are active) or it // temporarily failed to clean up the shard folder, we need to trigger another cluster state change for this removal to // finally succeed. + logger.info("Triggering an extra cluster state update"); updateIndexSettings( Settings.builder().put("index.routing.allocation.exclude.name", "non-existent" + randomAlphaOfLength(5)), indexName From 0480c1acba41cb0cd8fb22c384adfff0790a2f85 Mon Sep 17 00:00:00 2001 From: Liam Thompson <32779855+leemthompo@users.noreply.github.com> Date: Tue, 11 Jun 2024 11:24:52 +0200 Subject: [PATCH 26/31] [DOCS] Remove ESQL demo env link from 8.14+ (#109562) --- docs/reference/esql/esql-get-started.asciidoc | 8 ++++---- .../esql/esql-getting-started-sample-data.asciidoc | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/reference/esql/esql-get-started.asciidoc b/docs/reference/esql/esql-get-started.asciidoc index b7928898a3bbb..207794c064da4 100644 --- a/docs/reference/esql/esql-get-started.asciidoc +++ b/docs/reference/esql/esql-get-started.asciidoc @@ -15,10 +15,9 @@ This getting started is also available as an https://github.com/elastic/elastics [[esql-getting-started-prerequisites]] === Prerequisites -To follow along with the queries in this guide, you can either set up your own -deployment, or use Elastic's public {esql} demo environment. +To follow along with the queries in this guide, you'll need an {es} deployment with our sample data. -include::{es-ref-dir}/tab-widgets/esql/esql-getting-started-widget-sample-data.asciidoc[] +include::{es-ref-dir}/tab-widgets/esql/esql-getting-started-sample-data.asciidoc[tag=own-deployment] [discrete] [[esql-getting-started-running-queries]] @@ -269,7 +268,8 @@ Before you can use `ENRICH`, you first need to <> and <> an <>. -include::{es-ref-dir}/tab-widgets/esql/esql-getting-started-widget-enrich-policy.asciidoc[] +include::{es-ref-dir}/tab-widgets/esql/esql-getting-started-enrich-policy.asciidoc[tag=own-deployment] + After creating and executing a policy, you can use it with the `ENRICH` command: diff --git a/docs/reference/tab-widgets/esql/esql-getting-started-sample-data.asciidoc b/docs/reference/tab-widgets/esql/esql-getting-started-sample-data.asciidoc index d9b08b7281f77..97f4859e012af 100644 --- a/docs/reference/tab-widgets/esql/esql-getting-started-sample-data.asciidoc +++ b/docs/reference/tab-widgets/esql/esql-getting-started-sample-data.asciidoc @@ -1,6 +1,6 @@ // tag::own-deployment[] -First ingest some sample data. In {kib}, open the main menu and select *Dev +First, you'll need to ingest the sample data. In {kib}, open the main menu and select *Dev Tools*. Run the following two requests: [source,console] From 5f3f7db7b622c7e706cb9c5032f66515479bd547 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Tue, 11 Jun 2024 10:40:47 +0100 Subject: [PATCH 27/31] [ML] Fix IndexOutOfBoundsException during inference (#109533) The error was caused by a empty input field list created by the ELSER and Elasticsearch inference services. --- docs/changelog/109533.yaml | 5 ++ .../core/ml/inference/TrainedModelConfig.java | 2 - .../ElasticsearchInternalService.java | 5 +- .../services/elser/ElserInternalService.java | 5 +- .../ElasticsearchInternalServiceTests.java | 40 +++++++++++++ .../elser/ElserInternalServiceTests.java | 58 +++++++++++++++++++ .../TransportPutTrainedModelAction.java | 31 ++++++---- .../TransportPutTrainedModelActionTests.java | 22 +++---- 8 files changed, 139 insertions(+), 29 deletions(-) create mode 100644 docs/changelog/109533.yaml diff --git a/docs/changelog/109533.yaml b/docs/changelog/109533.yaml new file mode 100644 index 0000000000000..5720410e5f370 --- /dev/null +++ b/docs/changelog/109533.yaml @@ -0,0 +1,5 @@ +pr: 109533 +summary: Fix IndexOutOfBoundsException during inference +area: Machine Learning +type: bug +issues: [] diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/inference/TrainedModelConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/inference/TrainedModelConfig.java index 80fd28f3ab03e..24fc24e43226b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/inference/TrainedModelConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/inference/TrainedModelConfig.java @@ -636,8 +636,6 @@ public static class Builder { private InferenceConfig inferenceConfig; private TrainedModelLocation location; private ModelPackageConfig modelPackageConfig; - private Long perDeploymentMemoryBytes; - private Long perAllocationMemoryBytes; private String platformArchitecture; private TrainedModelPrefixStrings prefixStrings; diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java index 0052607ce325b..df546efd161eb 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java @@ -418,9 +418,8 @@ public void putModel(Model model, ActionListener listener) { return; } else if (model instanceof MultilingualE5SmallModel e5Model) { String modelId = e5Model.getServiceSettings().getModelId(); - var fieldNames = List.of(); - var input = new TrainedModelInput(fieldNames); - var config = TrainedModelConfig.builder().setInput(input).setModelId(modelId).build(); + var input = new TrainedModelInput(List.of("text_field")); // by convention text_field is used + var config = TrainedModelConfig.builder().setInput(input).setModelId(modelId).validate(true).build(); PutTrainedModelAction.Request putRequest = new PutTrainedModelAction.Request(config, false, true); executeAsyncWithOrigin( client, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java index 6e7c177861cdd..ed0f1cd93c83e 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalService.java @@ -362,9 +362,8 @@ public void putModel(Model model, ActionListener listener) { return; } else { String modelId = ((ElserInternalModel) model).getServiceSettings().getModelId(); - var fieldNames = List.of(); - var input = new TrainedModelInput(fieldNames); - var config = TrainedModelConfig.builder().setInput(input).setModelId(modelId).build(); + var input = new TrainedModelInput(List.of("text_field")); // by convention text_field is used + var config = TrainedModelConfig.builder().setInput(input).setModelId(modelId).validate(true).build(); PutTrainedModelAction.Request putRequest = new PutTrainedModelAction.Request(config, false, true); executeAsyncWithOrigin( client, diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java index 8f8c73eaab79a..e34ce410bbab3 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.xpack.core.ml.action.GetTrainedModelsAction; import org.elasticsearch.xpack.core.ml.action.InferModelAction; import org.elasticsearch.xpack.core.ml.action.InferTrainedModelDeploymentAction; +import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction; import org.elasticsearch.xpack.core.ml.inference.TrainedModelConfig; import org.elasticsearch.xpack.core.ml.inference.TrainedModelPrefixStrings; import org.elasticsearch.xpack.core.ml.inference.results.ErrorInferenceResults; @@ -44,6 +45,7 @@ import org.elasticsearch.xpack.inference.services.settings.InternalServiceSettings; import org.junit.After; import org.junit.Before; +import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import java.util.ArrayList; @@ -694,6 +696,44 @@ public void testBuildInferenceRequest() { assertEquals(chunk, request.isChunked()); } + @SuppressWarnings("unchecked") + public void testPutModel() { + var client = mock(Client.class); + ArgumentCaptor argument = ArgumentCaptor.forClass(PutTrainedModelAction.Request.class); + + doAnswer(invocation -> { + var listener = (ActionListener) invocation.getArguments()[2]; + listener.onResponse(new PutTrainedModelAction.Response(mock(TrainedModelConfig.class))); + return null; + }).when(client).execute(Mockito.same(PutTrainedModelAction.INSTANCE), argument.capture(), any()); + + when(client.threadPool()).thenReturn(threadPool); + + var service = createService(client); + + var model = new MultilingualE5SmallModel( + "my-e5", + TaskType.TEXT_EMBEDDING, + "e5", + new MultilingualE5SmallInternalServiceSettings(1, 1, ".multilingual-e5-small") + ); + + service.putModel(model, new ActionListener<>() { + @Override + public void onResponse(Boolean success) { + assertTrue(success); + } + + @Override + public void onFailure(Exception e) { + fail(e); + } + }); + + var putConfig = argument.getValue().getTrainedModelConfig(); + assertEquals("text_field", putConfig.getInput().getFieldNames().get(0)); + } + private ElasticsearchInternalService createService(Client client) { var context = new InferenceServiceExtension.InferenceServiceFactoryContext(client); return new ElasticsearchInternalService(context); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalServiceTests.java index 27db8143f0c83..31962e44851c0 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elser/ElserInternalServiceTests.java @@ -27,10 +27,16 @@ import org.elasticsearch.xpack.core.inference.results.InferenceChunkedSparseEmbeddingResults; import org.elasticsearch.xpack.core.ml.action.InferModelAction; import org.elasticsearch.xpack.core.ml.action.InferTrainedModelDeploymentAction; +import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction; +import org.elasticsearch.xpack.core.ml.inference.TrainedModelConfig; import org.elasticsearch.xpack.core.ml.inference.results.ErrorInferenceResults; import org.elasticsearch.xpack.core.ml.inference.results.InferenceChunkedTextExpansionResults; import org.elasticsearch.xpack.core.ml.inference.results.InferenceChunkedTextExpansionResultsTests; import org.elasticsearch.xpack.core.ml.inference.trainedmodel.TokenizationConfigUpdate; +import org.junit.After; +import org.junit.Before; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import java.util.ArrayList; import java.util.Collections; @@ -38,6 +44,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -53,6 +60,18 @@ public class ElserInternalServiceTests extends ESTestCase { + private static ThreadPool threadPool; + + @Before + public void setUpThreadPool() { + threadPool = new TestThreadPool("test"); + } + + @After + public void shutdownThreadPool() { + TestThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + public static Model randomModelConfig(String inferenceEntityId, TaskType taskType) { return switch (taskType) { case SPARSE_EMBEDDING -> new ElserInternalModel( @@ -451,6 +470,45 @@ public void testChunkInferSetsTokenization() { } } + @SuppressWarnings("unchecked") + public void testPutModel() { + var client = mock(Client.class); + ArgumentCaptor argument = ArgumentCaptor.forClass(PutTrainedModelAction.Request.class); + + doAnswer(invocation -> { + var listener = (ActionListener) invocation.getArguments()[2]; + listener.onResponse(new PutTrainedModelAction.Response(mock(TrainedModelConfig.class))); + return null; + }).when(client).execute(Mockito.same(PutTrainedModelAction.INSTANCE), argument.capture(), any()); + + when(client.threadPool()).thenReturn(threadPool); + + var service = createService(client); + + var model = new ElserInternalModel( + "my-elser", + TaskType.SPARSE_EMBEDDING, + "elser", + new ElserInternalServiceSettings(1, 1, ".elser_model_2"), + ElserMlNodeTaskSettings.DEFAULT + ); + + service.putModel(model, new ActionListener<>() { + @Override + public void onResponse(Boolean success) { + assertTrue(success); + } + + @Override + public void onFailure(Exception e) { + fail(e); + } + }); + + var putConfig = argument.getValue().getTrainedModelConfig(); + assertEquals("text_field", putConfig.getInput().getFieldNames().get(0)); + } + private ElserInternalService createService(Client client) { var context = new InferenceServiceExtension.InferenceServiceFactoryContext(client); return new ElserInternalService(context); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java index 87fc956c224cc..c4cefc1750c35 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelAction.java @@ -323,12 +323,12 @@ protected void masterOperation( } }, finalResponseListener::onFailure); - checkForExistingTask( + checkForExistingModelDownloadTask( client, trainedModelConfig.getModelId(), request.isWaitForCompletion(), finalResponseListener, - handlePackageAndTagsListener, + () -> handlePackageAndTagsListener.onResponse(null), request.ackTimeout() ); } @@ -371,14 +371,26 @@ void callVerifyMlNodesAndModelArchitectures( } /** - * This method is package private for testing + * Check if the model is being downloaded. + * If the download is in progress then the response will be on + * the {@code isBeingDownloadedListener} otherwise {@code createModelAction} + * is called to trigger the next step in the model install. + * Should only be called for Elasticsearch hosted models. + * + * @param client Client + * @param modelId Model Id + * @param isWaitForCompletion Wait for the download to complete + * @param isBeingDownloadedListener The listener called if the download is in progress + * @param createModelAction If no download is in progress this is called to continue + * the model install process. + * @param timeout Model download timeout */ - static void checkForExistingTask( + static void checkForExistingModelDownloadTask( Client client, String modelId, boolean isWaitForCompletion, - ActionListener sendResponseListener, - ActionListener storeModelListener, + ActionListener isBeingDownloadedListener, + Runnable createModelAction, TimeValue timeout ) { TaskRetriever.getDownloadTaskInfo( @@ -389,12 +401,12 @@ static void checkForExistingTask( () -> "Timed out waiting for model download to complete", ActionListener.wrap(taskInfo -> { if (taskInfo != null) { - getModelInformation(client, modelId, sendResponseListener); + getModelInformation(client, modelId, isBeingDownloadedListener); } else { // no task exists so proceed with creating the model - storeModelListener.onResponse(null); + createModelAction.run(); } - }, sendResponseListener::onFailure) + }, isBeingDownloadedListener::onFailure) ); } @@ -554,5 +566,4 @@ static InferenceConfig parseInferenceConfigFromModelPackage(Map return inferenceConfig; } } - } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelActionTests.java index 73810e4e0046f..4a1a654a9a29f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportPutTrainedModelActionTests.java @@ -59,12 +59,12 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.xpack.ml.utils.TaskRetrieverTests.getTaskInfoListOfOne; import static org.elasticsearch.xpack.ml.utils.TaskRetrieverTests.mockClientWithTasksResponse; import static org.elasticsearch.xpack.ml.utils.TaskRetrieverTests.mockListTasksClient; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.same; import static org.mockito.Mockito.doAnswer; @@ -161,12 +161,12 @@ public void testCheckForExistingTaskCallsOnFailureForAnError() { var responseListener = new PlainActionFuture(); - TransportPutTrainedModelAction.checkForExistingTask( + TransportPutTrainedModelAction.checkForExistingModelDownloadTask( client, "inferenceEntityId", true, responseListener, - new PlainActionFuture(), + () -> {}, TIMEOUT ); @@ -178,18 +178,18 @@ public void testCheckForExistingTaskCallsOnFailureForAnError() { public void testCheckForExistingTaskCallsStoreModelListenerWhenNoTasksExist() { var client = mockClientWithTasksResponse(Collections.emptyList(), threadPool); - var storeListener = new PlainActionFuture(); + var createModelCalled = new AtomicBoolean(); - TransportPutTrainedModelAction.checkForExistingTask( + TransportPutTrainedModelAction.checkForExistingModelDownloadTask( client, "inferenceEntityId", true, new PlainActionFuture<>(), - storeListener, + () -> createModelCalled.set(Boolean.TRUE), TIMEOUT ); - assertThat(storeListener.actionGet(TIMEOUT), nullValue()); + assertTrue(createModelCalled.get()); } public void testCheckForExistingTaskThrowsNoModelFoundError() { @@ -197,12 +197,12 @@ public void testCheckForExistingTaskThrowsNoModelFoundError() { prepareGetTrainedModelResponse(client, Collections.emptyList()); var respListener = new PlainActionFuture(); - TransportPutTrainedModelAction.checkForExistingTask( + TransportPutTrainedModelAction.checkForExistingModelDownloadTask( client, "inferenceEntityId", true, respListener, - new PlainActionFuture<>(), + () -> {}, TIMEOUT ); @@ -224,12 +224,12 @@ public void testCheckForExistingTaskReturnsTask() { prepareGetTrainedModelResponse(client, List.of(trainedModel)); var respListener = new PlainActionFuture(); - TransportPutTrainedModelAction.checkForExistingTask( + TransportPutTrainedModelAction.checkForExistingModelDownloadTask( client, "inferenceEntityId", true, respListener, - new PlainActionFuture<>(), + () -> {}, TIMEOUT ); From 8e9e9bc6c886e35277c5d6e315f45405673afb86 Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Tue, 11 Jun 2024 11:53:18 +0200 Subject: [PATCH 28/31] Relaxed resulting docs checks (#109560) --- .../180_update_dense_vector_type.yml | 104 +++++++++--------- 1 file changed, 53 insertions(+), 51 deletions(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml index 0780b789e92a1..3502a5e643087 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/180_update_dense_vector_type.yml @@ -2,6 +2,9 @@ setup: - requires: cluster_features: "gte_v8.15.0" reason: 'updatable dense vector field types was added in 8.15' + - skip: + reason: "contains is a newly added assertion" + features: contains --- "Test create and update dense vector mapping with per-doc indexing and flush": - do: @@ -110,9 +113,9 @@ setup: - match: { hits.total.value: 10 } - length: {hits.hits: 3} - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "3" } } - do: indices.put_mapping: @@ -216,9 +219,9 @@ setup: - match: { hits.total.value: 20 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "11" } - - match: { hits.hits.2._id: "2" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "11" } } + - contains: { hits.hits: { _id: "2" } } - do: indices.put_mapping: @@ -323,10 +326,10 @@ setup: - match: { hits.total.value: 30 } - length: { hits.hits: 4 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "11" } - - match: { hits.hits.2._id: "2" } - - match: { hits.hits.3._id: "21" } + - contains: {hits.hits: {_id: "1"}} + - contains: {hits.hits: {_id: "11"}} + - contains: {hits.hits: {_id: "2"}} + - contains: {hits.hits: {_id: "21"}} - do: indices.put_mapping: @@ -431,12 +434,11 @@ setup: - match: { hits.total.value: 40 } - length: { hits.hits: 5 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "11" } - - match: { hits.hits.2._id: "31" } - - match: { hits.hits.3._id: "2" } - - match: { hits.hits.4._id: "21" } - + - contains: {hits.hits: {_id: "1"}} + - contains: {hits.hits: {_id: "11"}} + - contains: {hits.hits: {_id: "2"}} + - contains: {hits.hits: {_id: "21"}} + - contains: {hits.hits: {_id: "31"}} --- "Test create and update dense vector mapping with bulk indexing": @@ -501,9 +503,9 @@ setup: - match: { hits.total.value: 10 } - length: {hits.hits: 3} - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "3" } } - do: indices.put_mapping: @@ -561,9 +563,9 @@ setup: - match: { hits.total.value: 20 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "11" } } - do: indices.put_mapping: @@ -622,10 +624,10 @@ setup: - match: { hits.total.value: 30 } - length: { hits.hits: 4 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "21" } - - match: { hits.hits.3._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "11" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "21" } } - do: indices.put_mapping: @@ -684,11 +686,11 @@ setup: - match: { hits.total.value: 40 } - length: { hits.hits: 5 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "21" } - - match: { hits.hits.3._id: "31" } - - match: { hits.hits.4._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "11" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "21" } } + - contains: { hits.hits: { _id: "31" } } --- "Index, update and merge": @@ -753,9 +755,9 @@ setup: - match: { hits.total.value: 10 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "3" } } - do: indices.put_mapping: @@ -793,9 +795,9 @@ setup: - match: { hits.total.value: 10 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "3" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "3" } } - do: bulk: @@ -835,9 +837,9 @@ setup: - match: { hits.total.value: 20 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "11" } } - do: indices.put_mapping: @@ -871,9 +873,9 @@ setup: - match: { hits.total.value: 20 } - length: { hits.hits: 3 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "11" } } - do: bulk: @@ -913,10 +915,10 @@ setup: - match: { hits.total.value: 30 } - length: { hits.hits: 4 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "21" } - - match: { hits.hits.3._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "11" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "21" } } - do: indices.forcemerge: @@ -936,10 +938,10 @@ setup: - match: { hits.total.value: 30 } - length: { hits.hits: 4 } - - match: { hits.hits.0._id: "1" } - - match: { hits.hits.1._id: "2" } - - match: { hits.hits.2._id: "21" } - - match: { hits.hits.3._id: "11" } + - contains: { hits.hits: { _id: "1" } } + - contains: { hits.hits: { _id: "11" } } + - contains: { hits.hits: { _id: "2" } } + - contains: { hits.hits: { _id: "21" } } --- From c471b01f9e6bf8357c4260fabcd8d94995b1a426 Mon Sep 17 00:00:00 2001 From: Simon Cooper Date: Tue, 11 Jun 2024 11:40:10 +0100 Subject: [PATCH 29/31] Add permission to secure access to certain config files specified by settings (#108895) --- docs/changelog/108895.yaml | 5 + ...=> SecuredConfigFileAccessPermission.java} | 11 +- ...uredConfigFileSettingAccessPermission.java | 26 ++++ .../elasticsearch/bootstrap/PolicyUtil.java | 6 +- .../org/elasticsearch/bootstrap/Security.java | 111 +++++++++++++----- 5 files changed, 123 insertions(+), 36 deletions(-) create mode 100644 docs/changelog/108895.yaml rename server/src/main/java/org/elasticsearch/{SecuredFileAccessPermission.java => SecuredConfigFileAccessPermission.java} (53%) create mode 100644 server/src/main/java/org/elasticsearch/SecuredConfigFileSettingAccessPermission.java diff --git a/docs/changelog/108895.yaml b/docs/changelog/108895.yaml new file mode 100644 index 0000000000000..15293896b20c5 --- /dev/null +++ b/docs/changelog/108895.yaml @@ -0,0 +1,5 @@ +pr: 108895 +summary: Add permission to secure access to certain config files specified by settings +area: "Security" +type: bug +issues: [] diff --git a/server/src/main/java/org/elasticsearch/SecuredFileAccessPermission.java b/server/src/main/java/org/elasticsearch/SecuredConfigFileAccessPermission.java similarity index 53% rename from server/src/main/java/org/elasticsearch/SecuredFileAccessPermission.java rename to server/src/main/java/org/elasticsearch/SecuredConfigFileAccessPermission.java index 3d24a9bc5ddb3..d6372b5ef9885 100644 --- a/server/src/main/java/org/elasticsearch/SecuredFileAccessPermission.java +++ b/server/src/main/java/org/elasticsearch/SecuredConfigFileAccessPermission.java @@ -13,13 +13,14 @@ /** * A permission granted to ensure secured access to a file in the config directory. *

    - * By granting this permission, all code that does not have the same permission on the same file - * will be denied all read/write access to that file. - * Note that you also need to wrap any access to the secured files in an {@code AccessController.doPrivileged()} block + * By granting this permission with a file relative to the config directory, + * the file is secured from general access by Elasticsearch and other Elasticsearch plugins. + * All code that does not have a secured permission on the same file will be denied all read/write access to that file. + * Note that you also need to wrap any access to secured files in an {@code AccessController.doPrivileged()} block * as Elasticsearch itself is denied access to files secured by plugins. */ -public class SecuredFileAccessPermission extends BasicPermission { - public SecuredFileAccessPermission(String path) { +public class SecuredConfigFileAccessPermission extends BasicPermission { + public SecuredConfigFileAccessPermission(String path) { super(path, ""); } } diff --git a/server/src/main/java/org/elasticsearch/SecuredConfigFileSettingAccessPermission.java b/server/src/main/java/org/elasticsearch/SecuredConfigFileSettingAccessPermission.java new file mode 100644 index 0000000000000..fdea47d449a15 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/SecuredConfigFileSettingAccessPermission.java @@ -0,0 +1,26 @@ +/* + * 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; + +import java.security.BasicPermission; + +/** + * A permission granted to ensure secured access to a file specified by a setting in the config directory. + *

    + * By granting this permission with a setting key (wildcards are supported), + * the files pointed to by the settings are secured from general access by Elasticsearch and other Elasticsearch plugins. + * All code that does not have a secured permission on the same file will be denied all read/write access to that file. + * Note that you also need to wrap any access to secured files in an {@code AccessController.doPrivileged()} block + * as Elasticsearch itself is denied access to files secured by plugins. + */ +public class SecuredConfigFileSettingAccessPermission extends BasicPermission { + public SecuredConfigFileSettingAccessPermission(String setting) { + super(setting, ""); + } +} diff --git a/server/src/main/java/org/elasticsearch/bootstrap/PolicyUtil.java b/server/src/main/java/org/elasticsearch/bootstrap/PolicyUtil.java index 3279bc5b1bfdf..b9574f1a29ae8 100644 --- a/server/src/main/java/org/elasticsearch/bootstrap/PolicyUtil.java +++ b/server/src/main/java/org/elasticsearch/bootstrap/PolicyUtil.java @@ -8,7 +8,8 @@ package org.elasticsearch.bootstrap; -import org.elasticsearch.SecuredFileAccessPermission; +import org.elasticsearch.SecuredConfigFileAccessPermission; +import org.elasticsearch.SecuredConfigFileSettingAccessPermission; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.PathUtils; import org.elasticsearch.core.SuppressForbidden; @@ -169,7 +170,8 @@ public boolean test(Permission permission) { entry(PrivateCredentialPermission.class, ALLOW_ALL_NAMES), entry(SQLPermission.class, List.of("callAbort", "setNetworkTimeout")), entry(ClassPermission.class, ALLOW_ALL_NAMES), - entry(SecuredFileAccessPermission.class, ALLOW_ALL_NAMES) + entry(SecuredConfigFileAccessPermission.class, ALLOW_ALL_NAMES), + entry(SecuredConfigFileSettingAccessPermission.class, ALLOW_ALL_NAMES) ).collect(Collectors.toMap(e -> e.getKey().getCanonicalName(), Map.Entry::getValue)); PermissionCollection pluginPermissionCollection = new Permissions(); namedPermissions.forEach(pluginPermissionCollection::add); diff --git a/server/src/main/java/org/elasticsearch/bootstrap/Security.java b/server/src/main/java/org/elasticsearch/bootstrap/Security.java index e24e13dfff372..12edf344c72a2 100644 --- a/server/src/main/java/org/elasticsearch/bootstrap/Security.java +++ b/server/src/main/java/org/elasticsearch/bootstrap/Security.java @@ -9,13 +9,16 @@ package org.elasticsearch.bootstrap; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.SecuredFileAccessPermission; +import org.elasticsearch.SecuredConfigFileAccessPermission; +import org.elasticsearch.SecuredConfigFileSettingAccessPermission; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.PathUtils; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.env.Environment; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.jdk.JarHell; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; import org.elasticsearch.plugins.PluginsUtils; import org.elasticsearch.secure_sm.SecureSM; import org.elasticsearch.transport.TcpTransport; @@ -46,7 +49,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Consumer; -import java.util.stream.Stream; +import java.util.regex.Pattern; import static java.lang.invoke.MethodType.methodType; import static org.elasticsearch.bootstrap.ESPolicy.POLICY_RESOURCE; @@ -104,6 +107,8 @@ */ final class Security { + private static Logger logger; // not init'd until configure call below + static { prepopulateSecurityCaller(); } @@ -122,6 +127,8 @@ static void setSecurityManager(@SuppressWarnings("removal") SecurityManager sm) * @param filterBadDefaults true if we should filter out bad java defaults in the system policy. */ static void configure(Environment environment, boolean filterBadDefaults, Path pidFile) throws IOException { + logger = LogManager.getLogger(Security.class); + // enable security policy: union of template and environment-based paths, and possibly plugin permissions Map codebases = PolicyUtil.getCodebaseJarMap(JarHell.parseModulesAndClassPath()); Policy mainPolicy = PolicyUtil.readPolicy(ESPolicy.class.getResource(POLICY_RESOURCE), codebases); @@ -133,7 +140,7 @@ static void configure(Environment environment, boolean filterBadDefaults, Path p pluginPolicies, filterBadDefaults, createRecursiveDataPathPermission(environment), - readSecuredFiles(environment, mainPolicy, codebases.values(), pluginPolicies) + readSecuredConfigFiles(environment, mainPolicy, codebases.values(), pluginPolicies) ) ); @@ -196,55 +203,101 @@ private static List createRecursiveDataPathPermission(Environmen return toFilePermissions(policy); } - private static Map> readSecuredFiles( + private static Map> readSecuredConfigFiles( Environment environment, Policy template, Collection mainCodebases, Map pluginPolicies ) throws IOException { - Map> securedFiles = new HashMap<>(); + Map> securedConfigFiles = new HashMap<>(); + Map> securedSettingKeys = new HashMap<>(); for (URL url : mainCodebases) { - PolicyUtil.getPolicyPermissions(url, template, environment.tmpFile()) - .stream() - .flatMap(Security::extractSecuredFileName) - .map(environment.configFile()::resolve) - .forEach(f -> securedFiles.computeIfAbsent(f.toString(), k -> new HashSet<>()).add(url)); + for (Permission p : PolicyUtil.getPolicyPermissions(url, template, environment.tmpFile())) { + readSecuredConfigFilePermissions(environment, url, p, securedConfigFiles, securedSettingKeys); + } } for (var pp : pluginPolicies.entrySet()) { - PolicyUtil.getPolicyPermissions(pp.getKey(), pp.getValue(), environment.tmpFile()) - .stream() - .flatMap(Security::extractSecuredFileName) - .map(environment.configFile()::resolve) - .forEach(f -> securedFiles.computeIfAbsent(f.toString(), k -> new HashSet<>()).add(pp.getKey())); + for (Permission p : PolicyUtil.getPolicyPermissions(pp.getKey(), pp.getValue(), environment.tmpFile())) { + readSecuredConfigFilePermissions(environment, pp.getKey(), p, securedConfigFiles, securedSettingKeys); + } + } + + // compile a Pattern for each setting key we'll be looking for + // the key could include a * wildcard + List>> settingPatterns = securedSettingKeys.entrySet() + .stream() + .map(e -> Map.entry(Pattern.compile(e.getKey()), e.getValue())) + .toList(); + + for (String setting : environment.settings().keySet()) { + for (Map.Entry> ps : settingPatterns) { + if (ps.getKey().matcher(setting).matches()) { + // add the setting value to the secured files for these codebase URLs + Path file = environment.configFile().resolve(environment.settings().get(setting)); + if (file.startsWith(environment.configFile()) == false) { + throw new IllegalStateException(ps.getValue() + " tried to grant access to file outside config directory " + file); + } + if (logger.isDebugEnabled()) { + ps.getValue() + .forEach( + url -> logger.debug("Jar {} securing access to config file {} through setting {}", url, file, setting) + ); + } + securedConfigFiles.computeIfAbsent(file.toString(), k -> new HashSet<>()).addAll(ps.getValue()); + } + } } // always add some config files as exclusive files that no one can access // there's no reason for anyone to read these once the security manager is initialized // so if something has tried to grant itself access, crash out with an error - addSpeciallySecuredFile(securedFiles, environment.configFile().resolve("elasticsearch.yml").toString()); - addSpeciallySecuredFile(securedFiles, environment.configFile().resolve("jvm.options").toString()); - addSpeciallySecuredFile(securedFiles, environment.configFile().resolve("jvm.options.d/-").toString()); + addSpeciallySecuredConfigFile(securedConfigFiles, environment.configFile().resolve("elasticsearch.yml").toString()); + addSpeciallySecuredConfigFile(securedConfigFiles, environment.configFile().resolve("jvm.options").toString()); + addSpeciallySecuredConfigFile(securedConfigFiles, environment.configFile().resolve("jvm.options.d/-").toString()); - return Collections.unmodifiableMap(securedFiles); + return Collections.unmodifiableMap(securedConfigFiles); } - private static void addSpeciallySecuredFile(Map> securedFiles, String path) { - Set attemptedToGrant = securedFiles.put(path, Set.of()); - if (attemptedToGrant != null) { - throw new IllegalStateException(attemptedToGrant + " tried to grant access to special config file " + path); + private static void readSecuredConfigFilePermissions( + Environment environment, + URL url, + Permission p, + Map> securedFiles, + Map> securedSettingKeys + ) { + String securedFileName = extractSecuredName(p, SecuredConfigFileAccessPermission.class); + if (securedFileName != null) { + Path securedFile = environment.configFile().resolve(securedFileName); + if (securedFile.startsWith(environment.configFile()) == false) { + throw new IllegalStateException("[" + url + "] tried to grant access to file outside config directory " + securedFile); + } + logger.debug("Jar {} securing access to config file {}", url, securedFile); + securedFiles.computeIfAbsent(securedFile.toString(), k -> new HashSet<>()).add(url); + } + + String securedKey = extractSecuredName(p, SecuredConfigFileSettingAccessPermission.class); + if (securedKey != null) { + securedSettingKeys.computeIfAbsent(securedKey, k -> new HashSet<>()).add(url); } } - private static Stream extractSecuredFileName(Permission p) { - if (p instanceof SecuredFileAccessPermission) { - return Stream.of(p.getName()); + private static String extractSecuredName(Permission p, Class permissionType) { + if (permissionType.isInstance(p)) { + return p.getName(); + } else if (p instanceof UnresolvedPermission up && up.getUnresolvedType().equals(permissionType.getCanonicalName())) { + return up.getUnresolvedName(); + } else { + return null; } - if (p instanceof UnresolvedPermission up && up.getUnresolvedType().equals(SecuredFileAccessPermission.class.getCanonicalName())) { - return Stream.of(up.getUnresolvedName()); + } + + private static void addSpeciallySecuredConfigFile(Map> securedFiles, String path) { + Set attemptedToGrant = securedFiles.put(path, Set.of()); + if (attemptedToGrant != null) { + throw new IllegalStateException(attemptedToGrant + " tried to grant access to special config file " + path); } - return Stream.empty(); } /** Adds access to classpath jars/classes for jar hell scan, etc */ From 9647f6bd12f08b947203abcceefaeb15dc7d12cf Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 11 Jun 2024 12:52:20 +0200 Subject: [PATCH 30/31] Don't set number of shards to 0 for system indices that use auto-expand replicas (#108994) Many of our system indices that rely on auto_expand_replicas get created providing a manual number of replicas. Such number will be immediately overridden by the auto expand replicas functionality according to the number of data nodes available. While this causes no harm, it seems misleading and unnecessary, a potential misuse that we can avoid for indices that we create ourselves. Ideally we'd even prevent this from happening by rejecting such index creation requests, but that would be a breaking change that we'd prefer not to make at this time. --- .../java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java | 1 - .../java/org/elasticsearch/system/indices/SystemIndicesQA.java | 2 -- .../elasticsearch/synonyms/SynonymsManagementAPIService.java | 1 - .../elasticsearch/xpack/core/async/AsyncTaskIndexService.java | 1 - .../xpack/application/rules/QueryRulesIndexService.java | 1 - .../application/search/SearchApplicationIndexService.java | 1 - .../xpack/security/support/SecuritySystemIndices.java | 3 --- .../src/main/java/org/elasticsearch/xpack/watcher/Watcher.java | 1 - 8 files changed, 11 deletions(-) diff --git a/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java b/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java index e5756652a9842..9d0f9848d97b6 100644 --- a/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java +++ b/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java @@ -188,7 +188,6 @@ public Collection getSystemIndexDescriptors(Settings sett .setSettings( Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .build() ) diff --git a/qa/system-indices/src/main/java/org/elasticsearch/system/indices/SystemIndicesQA.java b/qa/system-indices/src/main/java/org/elasticsearch/system/indices/SystemIndicesQA.java index 9fc256e79873e..1d69ae5c1ee4a 100644 --- a/qa/system-indices/src/main/java/org/elasticsearch/system/indices/SystemIndicesQA.java +++ b/qa/system-indices/src/main/java/org/elasticsearch/system/indices/SystemIndicesQA.java @@ -72,7 +72,6 @@ public Collection getSystemIndexDescriptors(Settings sett .setSettings( Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .build() ) @@ -95,7 +94,6 @@ public Collection getSystemIndexDescriptors(Settings sett .setSettings( Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .build() ) diff --git a/server/src/main/java/org/elasticsearch/synonyms/SynonymsManagementAPIService.java b/server/src/main/java/org/elasticsearch/synonyms/SynonymsManagementAPIService.java index 9409aef96d8be..2cd35fd6889bd 100644 --- a/server/src/main/java/org/elasticsearch/synonyms/SynonymsManagementAPIService.java +++ b/server/src/main/java/org/elasticsearch/synonyms/SynonymsManagementAPIService.java @@ -483,7 +483,6 @@ private static String internalSynonymRuleId(String synonymsSetId, String synonym static Settings settings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-all") .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), SYNONYMS_INDEX_FORMAT) .build(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java index 2d92ded3b5454..e44af60a45e08 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java @@ -99,7 +99,6 @@ static Settings settings() { return Settings.builder() .put("index.codec", "best_compression") .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .build(); } diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java index adcd5da988b82..86192d8070cc1 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/rules/QueryRulesIndexService.java @@ -102,7 +102,6 @@ public static SystemIndexDescriptor getSystemIndexDescriptor() { private static Settings getIndexSettings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 100) .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), QueryRulesIndexMappingVersion.latest().id) diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java index 0ccef9acba088..9e8a8f750b764 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/SearchApplicationIndexService.java @@ -139,7 +139,6 @@ public static SystemIndexDescriptor getSystemIndexDescriptor() { private static Settings getIndexSettings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 100) .put("index.refresh_interval", "1s") diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecuritySystemIndices.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecuritySystemIndices.java index 75937bf3e2c18..4c5ce703f48ad 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecuritySystemIndices.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecuritySystemIndices.java @@ -152,7 +152,6 @@ private SystemIndexDescriptor getSecurityMainIndexDescriptor() { private static Settings getMainIndexSettings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 1000) .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), INTERNAL_MAIN_INDEX_FORMAT) @@ -667,7 +666,6 @@ private static SystemIndexDescriptor getSecurityTokenIndexDescriptor() { private static Settings getTokenIndexSettings() { return Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 1000) .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), INTERNAL_TOKENS_INDEX_FORMAT) @@ -868,7 +866,6 @@ private SystemIndexDescriptor getSecurityProfileIndexDescriptor(Settings setting private static Settings getProfileIndexSettings(Settings settings) { final Settings.Builder settingsBuilder = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, "0-1") .put(IndexMetadata.SETTING_PRIORITY, 1000) .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), INTERNAL_PROFILE_INDEX_FORMAT) diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index f8f910c38c080..2d71aef08ea13 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -874,7 +874,6 @@ public String getFeatureDescription() { private static Settings getWatchesIndexSettings() { return Settings.builder() .put("index.number_of_shards", 1) - .put("index.number_of_replicas", 0) .put("index.auto_expand_replicas", "0-1") .put(IndexMetadata.INDEX_FORMAT_SETTING.getKey(), 6) .put(IndexMetadata.SETTING_PRIORITY, 800) From 90ab2558b03f59c38b8223c2b881f97da7cd4394 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 11 Jun 2024 08:07:40 -0400 Subject: [PATCH 31/31] Adjusting bwc version after backport of #109423 (#109469) Co-authored-by: Elastic Machine --- .../170_knn_search_hex_encoded_byte_vectors.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/170_knn_search_hex_encoded_byte_vectors.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/170_knn_search_hex_encoded_byte_vectors.yml index be1e619d046ac..c4d8b1f0929b0 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/170_knn_search_hex_encoded_byte_vectors.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/170_knn_search_hex_encoded_byte_vectors.yml @@ -164,8 +164,8 @@ setup: --- "Dynamic dimensions for hex-encoded string": - requires: - cluster_features: "gte_v8.15.0" - reason: 'hex encoding for byte vectors fixed in 8.15' + cluster_features: "gte_v8.14.1" + reason: 'hex encoding for byte vectors fixed in 8.14.1' - do: indices.create: