diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java index c32aa1184ddaa..7580808be36ad 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java @@ -45,7 +45,6 @@ @State(Scope.Thread) @Fork(1) public class MultivalueDedupeBenchmark { - private static final BigArrays BIG_ARRAYS = BigArrays.NON_RECYCLING_INSTANCE; // TODO real big arrays? private static final BlockFactory blockFactory = BlockFactory.getInstance( new NoopCircuitBreaker("noop"), BigArrays.NON_RECYCLING_INSTANCE diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java index 66389c9e11ded..647f5d294e1f7 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java @@ -31,6 +31,7 @@ import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; @@ -96,8 +97,12 @@ public class ValuesSourceReaderBenchmark { for (String name : ValuesSourceReaderBenchmark.class.getField("name").getAnnotationsByType(Param.class)[0].value()) { benchmark.layout = layout; benchmark.name = name; - benchmark.setupPages(); - benchmark.benchmark(); + try { + benchmark.setupPages(); + benchmark.benchmark(); + } catch (Exception e) { + throw new AssertionError("error initializing [" + layout + "/" + name + "]", e); + } } } } finally { @@ -111,11 +116,11 @@ public class ValuesSourceReaderBenchmark { private static List fields(String name) { return switch (name) { case "3_stored_keywords" -> List.of( - new ValuesSourceReaderOperator.FieldInfo("keyword_1", List.of(blockLoader("stored_keyword_1"))), - new ValuesSourceReaderOperator.FieldInfo("keyword_2", List.of(blockLoader("stored_keyword_2"))), - new ValuesSourceReaderOperator.FieldInfo("keyword_3", List.of(blockLoader("stored_keyword_3"))) + new ValuesSourceReaderOperator.FieldInfo("keyword_1", ElementType.BYTES_REF, shardIdx -> blockLoader("stored_keyword_1")), + new ValuesSourceReaderOperator.FieldInfo("keyword_2", ElementType.BYTES_REF, shardIdx -> blockLoader("stored_keyword_2")), + new ValuesSourceReaderOperator.FieldInfo("keyword_3", ElementType.BYTES_REF, shardIdx -> blockLoader("stored_keyword_3")) ); - default -> List.of(new ValuesSourceReaderOperator.FieldInfo(name, List.of(blockLoader(name)))); + default -> List.of(new ValuesSourceReaderOperator.FieldInfo(name, elementType(name), shardIdx -> blockLoader(name))); }; } @@ -125,29 +130,38 @@ enum Where { STORED; } - private static BlockLoader blockLoader(String name) { - Where where = Where.DOC_VALUES; - if (name.startsWith("stored_")) { - name = name.substring("stored_".length()); - where = Where.STORED; - } else if (name.startsWith("source_")) { - name = name.substring("source_".length()); - where = Where.SOURCE; - } + private static ElementType elementType(String name) { + name = WhereAndBaseName.fromName(name).name; switch (name) { case "long": - return numericBlockLoader(name, where, NumberFieldMapper.NumberType.LONG); + return ElementType.LONG; case "int": - return numericBlockLoader(name, where, NumberFieldMapper.NumberType.INTEGER); + return ElementType.INT; case "double": - return numericBlockLoader(name, where, NumberFieldMapper.NumberType.DOUBLE); - case "keyword": - name = "keyword_1"; + return ElementType.DOUBLE; } if (name.startsWith("keyword")) { + return ElementType.BYTES_REF; + } + throw new UnsupportedOperationException("no element type for [" + name + "]"); + } + + private static BlockLoader blockLoader(String name) { + WhereAndBaseName w = WhereAndBaseName.fromName(name); + switch (w.name) { + case "long": + return numericBlockLoader(w, NumberFieldMapper.NumberType.LONG); + case "int": + return numericBlockLoader(w, NumberFieldMapper.NumberType.INTEGER); + case "double": + return numericBlockLoader(w, NumberFieldMapper.NumberType.DOUBLE); + case "keyword": + w = new WhereAndBaseName(w.where, "keyword_1"); + } + if (w.name.startsWith("keyword")) { boolean syntheticSource = false; FieldType ft = new FieldType(KeywordFieldMapper.Defaults.FIELD_TYPE); - switch (where) { + switch (w.where) { case DOC_VALUES: break; case SOURCE: @@ -161,7 +175,7 @@ private static BlockLoader blockLoader(String name) { } ft.freeze(); return new KeywordFieldMapper.KeywordFieldType( - name, + w.name, ft, Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, @@ -193,10 +207,21 @@ public String parentField(String field) { throw new IllegalArgumentException("can't read [" + name + "]"); } - private static BlockLoader numericBlockLoader(String name, Where where, NumberFieldMapper.NumberType numberType) { + private record WhereAndBaseName(Where where, String name) { + static WhereAndBaseName fromName(String name) { + if (name.startsWith("stored_")) { + return new WhereAndBaseName(Where.STORED, name.substring("stored_".length())); + } else if (name.startsWith("source_")) { + return new WhereAndBaseName(Where.SOURCE, name.substring("source_".length())); + } + return new WhereAndBaseName(Where.DOC_VALUES, name); + } + } + + private static BlockLoader numericBlockLoader(WhereAndBaseName w, NumberFieldMapper.NumberType numberType) { boolean stored = false; boolean docValues = true; - switch (where) { + switch (w.where) { case DOC_VALUES: break; case SOURCE: @@ -207,7 +232,7 @@ private static BlockLoader numericBlockLoader(String name, Where where, NumberFi throw new UnsupportedOperationException(); } return new NumberFieldMapper.NumberFieldType( - name, + w.name, numberType, true, stored, diff --git a/docs/changelog/101487.yaml b/docs/changelog/101487.yaml new file mode 100644 index 0000000000000..b4531f7fd6f75 --- /dev/null +++ b/docs/changelog/101487.yaml @@ -0,0 +1,5 @@ +pr: 101487 +summary: Wait for async searches to finish when shutting down +area: Infra/Node Lifecycle +type: enhancement +issues: [] diff --git a/docs/changelog/102207.yaml b/docs/changelog/102207.yaml new file mode 100644 index 0000000000000..8b247828845f4 --- /dev/null +++ b/docs/changelog/102207.yaml @@ -0,0 +1,6 @@ +pr: 102207 +summary: Fix disk computation when initializing unassigned shards in desired balance + computation +area: Allocation +type: bug +issues: [] diff --git a/docs/changelog/103453.yaml b/docs/changelog/103453.yaml new file mode 100644 index 0000000000000..4b7dab77c8b23 --- /dev/null +++ b/docs/changelog/103453.yaml @@ -0,0 +1,5 @@ +pr: 103453 +summary: Add expiration time to update api key api +area: Security +type: enhancement +issues: [] diff --git a/docs/changelog/103821.yaml b/docs/changelog/103821.yaml new file mode 100644 index 0000000000000..3279059acbe3e --- /dev/null +++ b/docs/changelog/103821.yaml @@ -0,0 +1,5 @@ +pr: 103821 +summary: "ESQL: Delay finding field load infrastructure" +area: ES|QL +type: enhancement +issues: [] diff --git a/docs/reference/esql/functions/types/add.asciidoc b/docs/reference/esql/functions/types/add.asciidoc index 72f9018503bb7..3665c112d802d 100644 --- a/docs/reference/esql/functions/types/add.asciidoc +++ b/docs/reference/esql/functions/types/add.asciidoc @@ -2,6 +2,7 @@ |=== lhs | rhs | result date_period | date_period | date_period +date_period | datetime | datetime datetime | date_period | datetime datetime | time_duration | datetime double | double | double diff --git a/docs/reference/rest-api/security/bulk-update-api-keys.asciidoc b/docs/reference/rest-api/security/bulk-update-api-keys.asciidoc index 2e7034caaae89..faf87c67d1ccc 100644 --- a/docs/reference/rest-api/security/bulk-update-api-keys.asciidoc +++ b/docs/reference/rest-api/security/bulk-update-api-keys.asciidoc @@ -30,7 +30,7 @@ This operation can greatly improve performance over making individual updates. It's not possible to update expired or <> API keys. -This API supports updates to API key access scope and metadata. +This API supports updates to API key access scope, metadata and expiration. The access scope of each API key is derived from the <> you specify in the request, and a snapshot of the owner user's permissions at the time of the request. The snapshot of the owner's permissions is updated automatically on every call. @@ -63,6 +63,9 @@ The structure of a role descriptor is the same as the request for the <>. -This API supports updates to an API key's access scope and metadata. +This API supports updates to an API key's access scope, metadata and expiration. The access scope of an API key is derived from the <> you specify in the request, and a snapshot of the owner user's permissions at the time of the request. The snapshot of the owner's permissions is updated automatically on every call. @@ -67,6 +67,9 @@ It supports nested data structure. Within the `metadata` object, top-level keys beginning with `_` are reserved for system usage. When specified, this fully replaces metadata previously associated with the API key. +`expiration`:: +(Optional, string) Expiration time for the API key. By default, API keys never expire. Can be omitted to leave unchanged. + [[security-api-update-api-key-response-body]] ==== {api-response-body-title} diff --git a/docs/reference/rest-api/security/update-cross-cluster-api-key.asciidoc b/docs/reference/rest-api/security/update-cross-cluster-api-key.asciidoc index f0dfb11f1c98b..c22a1347c8262 100644 --- a/docs/reference/rest-api/security/update-cross-cluster-api-key.asciidoc +++ b/docs/reference/rest-api/security/update-cross-cluster-api-key.asciidoc @@ -34,7 +34,7 @@ Use this API to update cross-cluster API keys created by the <>. -This API supports updates to an API key's access scope and metadata. +This API supports updates to an API key's access scope, metadata and expiration. The owner user's information, e.g. `username`, `realm`, is also updated automatically on every call. NOTE: This API cannot update <>, which should be updated by @@ -66,6 +66,9 @@ It supports nested data structure. Within the `metadata` object, top-level keys beginning with `_` are reserved for system usage. When specified, this fully replaces metadata previously associated with the API key. +`expiration`:: +(Optional, string) Expiration time for the API key. By default, API keys never expire. Can be omitted to leave unchanged. + [[security-api-update-cross-cluster-api-key-response-body]] ==== {api-response-body-title} diff --git a/docs/reference/tab-widgets/esql/esql-getting-started-enrich-policy.asciidoc b/docs/reference/tab-widgets/esql/esql-getting-started-enrich-policy.asciidoc index c51a46bdef3b3..3e9796f89e94d 100644 --- a/docs/reference/tab-widgets/esql/esql-getting-started-enrich-policy.asciidoc +++ b/docs/reference/tab-widgets/esql/esql-getting-started-enrich-policy.asciidoc @@ -58,9 +58,9 @@ DELETE /_enrich/policy/clientip_policy // tag::demo-env[] -On the demo environment at https://esql.demo.elastic.co/[esql.demo.elastic.co], +On the demo environment at https://ela.st/ql/[ela.st/ql], an enrich policy called `clientip_policy` has already been created an executed. The policy links an IP address to an environment ("Development", "QA", or -"Production") +"Production"). // end::demo-env[] 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 2a899a9f1ea33..d9b08b7281f77 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 @@ -43,6 +43,6 @@ PUT sample_data/_bulk The data set used in this guide has been preloaded into the Elastic {esql} public demo environment. Visit -https://esql.demo.elastic.co/[esql.demo.elastic.co] to start using it. +https://ela.st/ql[ela.st/ql] to start using it. // end::demo-env[] diff --git a/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java b/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java index d75519002f92e..b2df6b0fa01a3 100644 --- a/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java +++ b/qa/full-cluster-restart/src/javaRestTest/java/org/elasticsearch/upgrades/QueryBuilderBWCIT.java @@ -39,6 +39,7 @@ import org.elasticsearch.index.query.functionscore.RandomScoreFunctionBuilder; import org.elasticsearch.search.SearchModule; 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.elasticsearch.xcontent.XContentBuilder; @@ -76,6 +77,7 @@ public class QueryBuilderBWCIT extends ParameterizedFullClusterRestartTestCase { .version(getOldClusterTestVersion()) .nodes(2) .setting("xpack.security.enabled", "false") + .feature(FeatureFlag.FAILURE_STORE_ENABLED) .apply(() -> clusterConfig) .build(); diff --git a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/FieldCapsIT.java b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/FieldCapsIT.java index f3971d832be3e..860cd2c0e8617 100644 --- a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/FieldCapsIT.java +++ b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/FieldCapsIT.java @@ -11,6 +11,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import org.apache.http.HttpHost; +import org.apache.lucene.tests.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.Build; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; import org.elasticsearch.client.Request; @@ -38,6 +39,7 @@ * In 8.2 we also added the ability to filter fields by type and metadata, with some post-hoc filtering applied on * the co-ordinating node if older nodes were included in the system */ +@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103473") public class FieldCapsIT extends ParameterizedRollingUpgradeTestCase { public FieldCapsIT(@Name("upgradedNodes") int upgradedNodes) { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/ReloadSecureSettingsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/ReloadSecureSettingsIT.java index 94463cfa33271..954ef3d6d7887 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/ReloadSecureSettingsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/ReloadSecureSettingsIT.java @@ -65,9 +65,14 @@ private static void executeReloadSecureSettings( SecureString password, ActionListener listener ) { - final var request = new NodesReloadSecureSettingsRequest(nodeIds); - request.setSecureStorePassword(password); - client().execute(TransportNodesReloadSecureSettingsAction.TYPE, request, listener); + final var request = new NodesReloadSecureSettingsRequest(); + try { + request.nodesIds(nodeIds); + request.setSecureStorePassword(password); + client().execute(TransportNodesReloadSecureSettingsAction.TYPE, request, listener); + } finally { + request.decRef(); + } } private static SecureString emptyPassword() { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/gateway/ReplicaShardAllocatorSyncIdIT.java b/server/src/internalClusterTest/java/org/elasticsearch/gateway/ReplicaShardAllocatorSyncIdIT.java index 116a53f5dbfae..27e63e5614744 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/gateway/ReplicaShardAllocatorSyncIdIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/gateway/ReplicaShardAllocatorSyncIdIT.java @@ -14,7 +14,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.UpdateForV9; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; @@ -97,7 +96,7 @@ void syncFlush(String syncId) throws IOException { // make sure that background merges won't happen; otherwise, IndexWriter#hasUncommittedChanges can become true again forceMerge(false, 1, false, UUIDs.randomBase64UUID()); assertNotNull(indexWriter); - try (ReleasableLock ignored = readLock.acquire()) { + try (var ignored = acquireEnsureOpenRef()) { assertThat(getTranslogStats().getUncommittedOperations(), equalTo(0)); Map userData = new HashMap<>(getLastCommittedSegmentInfos().userData); SequenceNumbers.CommitInfo commitInfo = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(userData.entrySet()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 5dcd8b5b0e34f..bd400f9f0f6a1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -782,10 +782,13 @@ public Settings onNodeStopped(String nodeName) { * Tests shard recovery throttling on the target node. Node statistics should show throttling time on the target node, while no * throttling should be shown on the source node because the target will accept data more slowly than the source's throttling threshold. */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103204") public void testTargetThrottling() throws Exception { logger.info("--> starting node A with default settings"); - final String nodeA = internalCluster().startNode(); + final String nodeA = internalCluster().startNode( + Settings.builder() + // Use a high value so that when unthrottling recoveries we do not cause accidental throttling on the source node. + .put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "200mb") + ); logger.info("--> creating index on node A"); ByteSizeValue shardSize = createAndPopulateIndex(INDEX_NAME, 1, SHARD_COUNT_1, REPLICA_COUNT_0).getShards()[0].getStats() diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index eb4738b6723de..ce1f6d9ecaad5 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -176,6 +176,8 @@ static TransportVersion def(int id) { public static final TransportVersion ESQL_STATUS_INCLUDE_LUCENE_QUERIES = def(8_564_00_0); public static final TransportVersion ESQL_CLUSTER_ALIAS = def(8_565_00_0); public static final TransportVersion SNAPSHOTS_IN_PROGRESS_TRACKING_REMOVING_NODES_ADDED = def(8_566_00_0); + public static final TransportVersion SMALLER_RELOAD_SECURE_SETTINGS_REQUEST = def(8_567_00_0); + public static final TransportVersion UPDATE_API_KEY_EXPIRATION_TIME_ADDED = def(8_568_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsRequest.java index 0af12b5d47c58..c24833dca49ee 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsRequest.java @@ -8,26 +8,32 @@ package org.elasticsearch.action.admin.cluster.node.reload; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.nodes.BaseNodesRequest; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.CharArrays; import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.RefCounted; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.transport.TransportRequest; import java.io.IOException; import java.util.Arrays; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; /** * Request for a reload secure settings action */ -public class NodesReloadSecureSettingsRequest extends BaseNodesRequest implements Releasable { +public class NodesReloadSecureSettingsRequest extends BaseNodesRequest { /** * The password is used to re-read and decrypt the contents @@ -37,39 +43,12 @@ public class NodesReloadSecureSettingsRequest extends BaseNodesRequest Releasables.close(secureSettingsPassword))); + public NodesReloadSecureSettingsRequest() { super((String[]) null); } - public NodesReloadSecureSettingsRequest(StreamInput in) throws IOException { - super(in); - final BytesReference bytesRef = in.readOptionalBytesReference(); - if (bytesRef != null) { - byte[] bytes = BytesReference.toBytes(bytesRef); - try { - this.secureSettingsPassword = new SecureString(CharArrays.utf8BytesToChars(bytes)); - } finally { - Arrays.fill(bytes, (byte) 0); - } - } else { - this.secureSettingsPassword = null; - } - } - - /** - * Reload secure settings only on certain nodes, based on the nodes ids - * specified. If none are passed, secure settings will be reloaded on all the - * nodes. - */ - public NodesReloadSecureSettingsRequest(String... nodesIds) { - super(nodesIds); - } - - @Nullable - public SecureString getSecureSettingsPassword() { - return secureSettingsPassword; - } - public void setSecureStorePassword(SecureString secureStorePassword) { this.secureSettingsPassword = secureStorePassword; } @@ -80,64 +59,126 @@ boolean hasPassword() { @Override public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - if (this.secureSettingsPassword == null) { - out.writeOptionalBytesReference(null); - } else { - final byte[] passwordBytes = CharArrays.toUtf8Bytes(this.secureSettingsPassword.getChars()); - try { - out.writeOptionalBytesReference(new BytesArray(passwordBytes)); - } finally { - Arrays.fill(passwordBytes, (byte) 0); - } - } + TransportAction.localOnly(); } - // This field is intentionally not part of serialization - private final Set nodeRequests = ConcurrentHashMap.newKeySet(); + @Override + public void incRef() { + refs.incRef(); + } - NodeRequest newNodeRequest() { - final NodesReloadSecureSettingsRequest clone = new NodesReloadSecureSettingsRequest(nodesIds()); - if (hasPassword()) { - clone.setSecureStorePassword(getSecureSettingsPassword().clone()); - } - final NodeRequest nodeRequest = new NodeRequest(clone); - nodeRequests.add(nodeRequest); - return nodeRequest; + @Override + public boolean tryIncRef() { + return refs.tryIncRef(); } @Override - public void close() { - if (this.secureSettingsPassword != null) { - this.secureSettingsPassword.close(); - } - nodeRequests.forEach(NodeRequest::close); + public boolean decRef() { + return refs.decRef(); + } + + @Override + public boolean hasReferences() { + return refs.hasReferences(); + } + + NodeRequest newNodeRequest() { + refs.mustIncRef(); + return new NodeRequest(secureSettingsPassword, refs); } - public static class NodeRequest extends TransportRequest implements Releasable { + public static class NodeRequest extends TransportRequest { + + @Nullable + private final SecureString secureSettingsPassword; - // TODO don't wrap the whole top-level request, it contains heavy and irrelevant DiscoveryNode things; see #100878 - NodesReloadSecureSettingsRequest request; + private final RefCounted refs; NodeRequest(StreamInput in) throws IOException { super(in); - request = new NodesReloadSecureSettingsRequest(in); + + if (in.getTransportVersion().before(TransportVersions.SMALLER_RELOAD_SECURE_SETTINGS_REQUEST)) { + TaskId.readFromStream(in); + in.readStringArray(); + in.readOptionalArray(DiscoveryNode::new, DiscoveryNode[]::new); + in.readOptionalTimeValue(); + } + + final BytesReference bytesRef = in.readOptionalBytesReference(); + if (bytesRef != null) { + byte[] bytes = BytesReference.toBytes(bytesRef); + try { + this.secureSettingsPassword = new SecureString(CharArrays.utf8BytesToChars(bytes)); + this.refs = LeakTracker.wrap(AbstractRefCounted.of(() -> Releasables.close(this.secureSettingsPassword))); + } finally { + Arrays.fill(bytes, (byte) 0); + } + } else { + this.secureSettingsPassword = null; + this.refs = LeakTracker.wrap(AbstractRefCounted.of(() -> {})); + } } - NodeRequest(NodesReloadSecureSettingsRequest request) { - this.request = request; + NodeRequest(@Nullable SecureString secureSettingsPassword, RefCounted refs) { + assert secureSettingsPassword == null || secureSettingsPassword.getChars() != null; // ensures it's not closed + assert refs.hasReferences(); + this.secureSettingsPassword = secureSettingsPassword; + this.refs = refs; } @Override public void writeTo(StreamOutput out) throws IOException { + assert hasReferences(); super.writeTo(out); - request.writeTo(out); + + if (out.getTransportVersion().before(TransportVersions.SMALLER_RELOAD_SECURE_SETTINGS_REQUEST)) { + TaskId.EMPTY_TASK_ID.writeTo(out); + out.writeStringArray(Strings.EMPTY_ARRAY); + out.writeOptionalArray(StreamOutput::writeWriteable, null); + out.writeOptionalTimeValue(null); + } + + if (this.secureSettingsPassword == null) { + out.writeOptionalBytesReference(null); + } else { + final byte[] passwordBytes = CharArrays.toUtf8Bytes(this.secureSettingsPassword.getChars()); + try { + out.writeOptionalBytesReference(new BytesArray(passwordBytes)); + } finally { + Arrays.fill(passwordBytes, (byte) 0); + } + } + } + + boolean hasPassword() { + assert hasReferences(); + return this.secureSettingsPassword != null && this.secureSettingsPassword.length() > 0; + } + + @Nullable + public SecureString getSecureSettingsPassword() { + assert hasReferences(); + return secureSettingsPassword; + } + + @Override + public void incRef() { + refs.incRef(); + } + + @Override + public boolean tryIncRef() { + return refs.tryIncRef(); + } + + @Override + public boolean decRef() { + return refs.decRef(); } @Override - public void close() { - assert request.nodeRequests.isEmpty() : "potential circular reference"; - request.close(); + public boolean hasReferences() { + return refs.hasReferences(); } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java index 0165ccaeb8016..9598f378a188c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/reload/TransportNodesReloadSecureSettingsAction.java @@ -96,7 +96,6 @@ protected void doExecute( ActionListener listener ) { if (request.hasPassword() && isNodeLocal(request) == false && isNodeTransportTLSEnabled() == false) { - request.close(); listener.onFailure( new ElasticsearchException( "Secure settings cannot be updated cluster wide when TLS for the transport layer" @@ -104,16 +103,15 @@ protected void doExecute( ) ); } else { - super.doExecute(task, request, ActionListener.runBefore(listener, request::close)); + super.doExecute(task, request, listener); } } @Override protected NodesReloadSecureSettingsResponse.NodeResponse nodeOperation( - NodesReloadSecureSettingsRequest.NodeRequest nodeReloadRequest, + NodesReloadSecureSettingsRequest.NodeRequest request, Task task ) { - final NodesReloadSecureSettingsRequest request = nodeReloadRequest.request; // We default to using an empty string as the keystore password so that we mimic pre 7.3 API behavior try (KeyStoreWrapper keystore = KeyStoreWrapper.load(environment.configFile())) { // reread keystore from config file @@ -143,8 +141,6 @@ protected NodesReloadSecureSettingsResponse.NodeResponse nodeOperation( return new NodesReloadSecureSettingsResponse.NodeResponse(clusterService.localNode(), null); } catch (final Exception e) { return new NodesReloadSecureSettingsResponse.NodeResponse(clusterService.localNode(), e); - } finally { - request.close(); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterInfoSimulator.java b/server/src/main/java/org/elasticsearch/cluster/ClusterInfoSimulator.java index f9e56b5b2ff2d..9019ee465c936 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterInfoSimulator.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterInfoSimulator.java @@ -8,7 +8,9 @@ package org.elasticsearch.cluster; +import org.elasticsearch.cluster.ClusterInfo.NodeAndShard; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.util.CopyOnFirstWriteMap; import org.elasticsearch.index.shard.ShardId; @@ -16,20 +18,28 @@ import java.util.Map; import java.util.Objects; +import static org.elasticsearch.cluster.ClusterInfo.shardIdentifierFromRouting; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.shouldReserveSpaceForInitializingShard; +import static org.elasticsearch.cluster.routing.ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE; + public class ClusterInfoSimulator { + private final RoutingAllocation allocation; + private final Map leastAvailableSpaceUsage; private final Map mostAvailableSpaceUsage; private final CopyOnFirstWriteMap shardSizes; private final Map shardDataSetSizes; - private final Map dataPath; + private final Map dataPath; - public ClusterInfoSimulator(ClusterInfo clusterInfo) { - this.leastAvailableSpaceUsage = new HashMap<>(clusterInfo.getNodeLeastAvailableDiskUsages()); - this.mostAvailableSpaceUsage = new HashMap<>(clusterInfo.getNodeMostAvailableDiskUsages()); - this.shardSizes = new CopyOnFirstWriteMap<>(clusterInfo.shardSizes); - this.shardDataSetSizes = Map.copyOf(clusterInfo.shardDataSetSizes); - this.dataPath = Map.copyOf(clusterInfo.dataPath); + public ClusterInfoSimulator(RoutingAllocation allocation) { + this.allocation = allocation; + this.leastAvailableSpaceUsage = new HashMap<>(allocation.clusterInfo().getNodeLeastAvailableDiskUsages()); + this.mostAvailableSpaceUsage = new HashMap<>(allocation.clusterInfo().getNodeMostAvailableDiskUsages()); + this.shardSizes = new CopyOnFirstWriteMap<>(allocation.clusterInfo().shardSizes); + this.shardDataSetSizes = Map.copyOf(allocation.clusterInfo().shardDataSetSizes); + this.dataPath = Map.copyOf(allocation.clusterInfo().dataPath); } /** @@ -43,49 +53,50 @@ public ClusterInfoSimulator(ClusterInfo clusterInfo) { public void simulateShardStarted(ShardRouting shard) { assert shard.initializing(); - var size = getEstimatedShardSize(shard); - if (size != null && size > 0) { + var size = getExpectedShardSize( + shard, + UNAVAILABLE_EXPECTED_SHARD_SIZE, + getClusterInfo(), + allocation.snapshotShardSizeInfo(), + allocation.metadata(), + allocation.routingTable() + ); + if (size != UNAVAILABLE_EXPECTED_SHARD_SIZE) { if (shard.relocatingNodeId() != null) { // relocation modifyDiskUsage(shard.relocatingNodeId(), size); modifyDiskUsage(shard.currentNodeId(), -size); } else { // new shard - modifyDiskUsage(shard.currentNodeId(), -size); - shardSizes.put(ClusterInfo.shardIdentifierFromRouting(shard), size); + if (shouldReserveSpaceForInitializingShard(shard, allocation.metadata())) { + modifyDiskUsage(shard.currentNodeId(), -size); + } + shardSizes.put( + shardIdentifierFromRouting(shard), + allocation.metadata().getIndexSafe(shard.index()).ignoreDiskWatermarks() ? 0 : size + ); } } } - private Long getEstimatedShardSize(ShardRouting shard) { - if (shard.relocatingNodeId() != null) { - // relocation existing shard, get size of the source shard - return shardSizes.get(ClusterInfo.shardIdentifierFromRouting(shard)); - } else if (shard.primary() == false) { - // initializing new replica, get size of the source primary shard - return shardSizes.get(ClusterInfo.shardIdentifierFromRouting(shard.shardId(), true)); - } else { - // initializing new (empty?) primary - return shard.getExpectedShardSize(); + private void modifyDiskUsage(String nodeId, long freeDelta) { + if (freeDelta == 0) { + return; } - } - - private void modifyDiskUsage(String nodeId, long delta) { var diskUsage = mostAvailableSpaceUsage.get(nodeId); if (diskUsage == null) { return; } var path = diskUsage.getPath(); + updateDiskUsage(leastAvailableSpaceUsage, nodeId, path, freeDelta); + updateDiskUsage(mostAvailableSpaceUsage, nodeId, path, freeDelta); + } - var leastUsage = leastAvailableSpaceUsage.get(nodeId); - if (leastUsage != null && Objects.equals(leastUsage.getPath(), path)) { - // ensure new value is within bounds - leastAvailableSpaceUsage.put(nodeId, updateWithFreeBytes(leastUsage, delta)); - } - var mostUsage = mostAvailableSpaceUsage.get(nodeId); - if (mostUsage != null && Objects.equals(mostUsage.getPath(), path)) { + private void updateDiskUsage(Map availableSpaceUsage, String nodeId, String path, long freeDelta) { + var usage = availableSpaceUsage.get(nodeId); + if (usage != null && Objects.equals(usage.getPath(), path)) { // ensure new value is within bounds - mostAvailableSpaceUsage.put(nodeId, updateWithFreeBytes(mostUsage, delta)); + availableSpaceUsage.put(nodeId, updateWithFreeBytes(usage, freeDelta)); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java b/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java index 1f364e1ace6e4..46a45a058f123 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java @@ -48,6 +48,8 @@ public static boolean shouldReserveSpaceForInitializingShard(ShardRouting shard, case PEER -> true; // Snapshot restore (unless it is partial) require downloading all segments locally from the blobstore to start the shard. + // See org.elasticsearch.xpack.searchablesnapshots.action.TransportMountSearchableSnapshotAction.buildIndexSettings + // and DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS case SNAPSHOT -> metadata.getIndexSafe(shard.index()).isPartialSearchableSnapshot() == false; // shrink/split/clone operation is going to clone existing locally placed shards using file system hard links diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java index 60a6ec2e49899..effd5ec110c44 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java @@ -84,7 +84,7 @@ public DesiredBalance compute( final var knownNodeIds = routingNodes.getAllNodeIds(); final var changes = routingAllocation.changes(); final var ignoredShards = getIgnoredShardsWithDiscardedAllocationStatus(desiredBalanceInput.ignoredShards()); - final var clusterInfoSimulator = new ClusterInfoSimulator(routingAllocation.clusterInfo()); + final var clusterInfoSimulator = new ClusterInfoSimulator(routingAllocation); if (routingNodes.size() == 0) { return new DesiredBalance(desiredBalanceInput.index(), Map.of()); @@ -274,7 +274,7 @@ public DesiredBalance compute( routingAllocation.setSimulatedClusterInfo(clusterInfoSimulator.getClusterInfo()); logger.trace("running delegate allocator"); delegateAllocator.allocate(routingAllocation); - assert routingNodes.unassigned().size() == 0; // any unassigned shards should now be ignored + assert routingNodes.unassigned().isEmpty(); // any unassigned shards should now be ignored hasChanges = false; for (final var routingNode : routingNodes) { diff --git a/server/src/main/java/org/elasticsearch/common/settings/SecureString.java b/server/src/main/java/org/elasticsearch/common/settings/SecureString.java index 25fba6eebc6f9..84520c6a1dc48 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SecureString.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SecureString.java @@ -8,14 +8,15 @@ package org.elasticsearch.common.settings; -import java.io.Closeable; +import org.elasticsearch.core.Releasable; + import java.util.Arrays; import java.util.Objects; /** * A String implementations which allows clearing the underlying char array. */ -public final class SecureString implements CharSequence, Closeable { +public final class SecureString implements CharSequence, Releasable { private char[] chars; diff --git a/server/src/main/java/org/elasticsearch/common/util/CopyOnFirstWriteMap.java b/server/src/main/java/org/elasticsearch/common/util/CopyOnFirstWriteMap.java index 08d86c143fc1b..51491df02c37c 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CopyOnFirstWriteMap.java +++ b/server/src/main/java/org/elasticsearch/common/util/CopyOnFirstWriteMap.java @@ -12,6 +12,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Set; /** @@ -70,7 +71,11 @@ public V get(Object key) { @Override public V put(K key, V value) { - return getForUpdate().put(key, value); + if (Objects.equals(get(key), value)) { + return value; + } else { + return getForUpdate().put(key, value); + } } @Override 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 815ec11af923f..9b9cf8ad35c04 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -33,6 +33,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; @@ -43,8 +45,10 @@ import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.util.concurrent.UncategorizedExecutionException; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; @@ -91,7 +95,6 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; import java.util.function.Function; @@ -123,10 +126,13 @@ public abstract class Engine implements Closeable { private final CountDownLatch closedLatch = new CountDownLatch(1); protected final EventListener eventListener; protected final ReentrantLock failEngineLock = new ReentrantLock(); - protected final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); - protected final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); - protected final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); protected final SetOnce failedEngine = new SetOnce<>(); + + private final AtomicBoolean isClosing = new AtomicBoolean(); + private final SubscribableListener drainOnCloseListener = new SubscribableListener<>(); + private final RefCounted ensureOpenRefs = AbstractRefCounted.of(() -> drainOnCloseListener.onResponse(null)); + private final Releasable releaseEnsureOpenRef = ensureOpenRefs::decRef; // reuse this to avoid allocation for each op + /* * on {@code lastWriteNanos} we use System.nanoTime() to initialize this since: * - we use the value for figuring out if the shard / engine is active so if we startup and no write has happened yet we still @@ -1161,15 +1167,15 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { * indicating no flush and unknown generation. */ public final void flush(boolean force, boolean waitIfOngoing, ActionListener listener) throws EngineException { - try (var ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { flushHoldingLock(force, waitIfOngoing, listener); } } /** - * The actual implementation of {@link #flush(boolean, boolean, ActionListener)}, which should only be called when holding either {@link - * #readLock} (the normal case) or {@link #writeLock} (if this flush is happening because the shard is closing gracefully) + * The actual implementation of {@link #flush(boolean, boolean, ActionListener)}, to be called either when holding a ref that ensures + * the engine remains open, or holding {@code IndexShard#engineMutex} while closing the engine. + * */ protected abstract void flushHoldingLock(boolean force, boolean waitIfOngoing, ActionListener listener) throws EngineException; @@ -1865,32 +1871,82 @@ public void close() { } /** - * Method to close the engine while the write lock is held. - * Must decrement the supplied when closing work is done and resources are - * freed. + * Closes the engine without acquiring any refs or locks. The caller should either have changed {@link #isClosing} from {@code false} to + * {@code true} or else must hold the {@link #failEngineLock}. The implementation must decrement the supplied latch when done. */ protected abstract void closeNoLock(String reason, CountDownLatch closedLatch); + protected final boolean isDrainedForClose() { + return ensureOpenRefs.hasReferences() == false; + } + + protected final boolean isClosing() { + return isClosing.get(); + } + + protected final Releasable acquireEnsureOpenRef() { + if (isClosing() || ensureOpenRefs.tryIncRef() == false) { + ensureOpen(); // throws "engine is closed" exception if we're actually closed, otherwise ... + throw new AlreadyClosedException(shardId + " engine is closing", failedEngine.get()); + } + return Releasables.assertOnce(releaseEnsureOpenRef); + } + + /** + * When called for the first time, puts the engine into a closing state in which further calls to {@link #acquireEnsureOpenRef()} will + * fail with an {@link AlreadyClosedException} and waits for all outstanding ensure-open refs to be released, before returning {@code + * true}. If called again, returns {@code false} without waiting. + * + * @return a flag indicating whether this was the first call or not. + */ + private boolean drainForClose() { + if (isClosing.compareAndSet(false, true) == false) { + logger.trace("drainForClose(): already closing"); + return false; + } + + logger.debug("drainForClose(): draining ops"); + releaseEnsureOpenRef.close(); + final var future = new PlainActionFuture() { + @Override + protected boolean blockingAllowed() { + // TODO remove this blocking, or at least do it elsewhere, see https://github.com/elastic/elasticsearch/issues/89821 + return Thread.currentThread().getName().contains(ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME) + || super.blockingAllowed(); + } + }; + drainOnCloseListener.addListener(future); + try { + future.get(); + return true; + } catch (ExecutionException e) { + logger.error("failure while draining operations on close", e); + assert false : e; + throw new IllegalStateException(e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("interrupted while draining operations on close"); + throw new IllegalStateException(e); + } + } + /** - * Flush the engine (committing segments to disk and truncating the - * translog) and close it. + * Flush the engine (committing segments to disk and truncating the translog) and close it. */ public void flushAndClose() throws IOException { - if (isClosed.get() == false) { - logger.trace("flushAndClose now acquire writeLock"); - try (ReleasableLock lock = writeLock.acquire()) { - logger.trace("flushAndClose now acquired writeLock"); + logger.trace("flushAndClose() maybe draining ops"); + if (isClosed.get() == false && drainForClose()) { + logger.trace("flushAndClose drained ops"); + try { + logger.debug("flushing shard on close - this might take some time to sync files to disk"); try { - logger.debug("flushing shard on close - this might take some time to sync files to disk"); - try { - // TODO: We are not waiting for full durability here atm because we are on the cluster state update thread - flushHoldingLock(false, false, ActionListener.noop()); - } catch (AlreadyClosedException ex) { - logger.debug("engine already closed - skipping flushAndClose"); - } - } finally { - close(); // double close is not a problem + // TODO: We are not waiting for full durability here atm because we are on the cluster state update thread + flushHoldingLock(false, false, ActionListener.noop()); + } catch (AlreadyClosedException ex) { + logger.debug("engine already closed - skipping flushAndClose"); } + } finally { + closeNoLock("flushAndClose", closedLatch); } } awaitPendingClose(); @@ -1898,12 +1954,10 @@ public void flushAndClose() throws IOException { @Override public void close() throws IOException { - if (isClosed.get() == false) { // don't acquire the write lock if we are already closed - logger.debug("close now acquiring writeLock"); - try (ReleasableLock lock = writeLock.acquire()) { - logger.debug("close acquired writeLock"); - closeNoLock("api", closedLatch); - } + logger.debug("close() maybe draining ops"); + if (isClosed.get() == false && drainForClose()) { + logger.debug("close drained ops"); + closeNoLock("api", closedLatch); } awaitPendingClose(); } 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 ca9ed06d4d266..fd288df9b377f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -58,7 +58,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AsyncIOProcessor; import org.elasticsearch.common.util.concurrent.KeyedLock; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.Assertions; import org.elasticsearch.core.Booleans; @@ -516,8 +515,7 @@ final boolean assertSearcherIsWarmedUp(String source, SearcherScope scope) { @Override public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecoveryRunner) throws IOException { - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); try (Translog.Snapshot snapshot = getTranslog().newSnapshot(localCheckpoint + 1, Long.MAX_VALUE)) { return translogRecoveryRunner.run(this, snapshot); @@ -527,8 +525,7 @@ public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecove @Override public int fillSeqNoGaps(long primaryTerm) throws IOException { - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); final long maxSeqNo = localCheckpointTracker.getMaxSeqNo(); int numNoOpsAdded = 0; @@ -568,8 +565,7 @@ private void bootstrapAppendOnlyInfoFromWriter(IndexWriter writer) { @Override public void recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo, ActionListener listener) { ActionListener.run(listener, l -> { - try (ReleasableLock lock = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { if (pendingTranslogRecovery.get() == false) { throw new IllegalStateException("Engine has already been recovered"); } @@ -840,8 +836,7 @@ public GetResult get( Function searcherWrapper ) { assert assertGetUsesIdField(get); - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { if (get.realtime()) { var result = realtimeGetUnderLock(get, mappingLookup, documentParser, searcherWrapper, true); assert result != null : "real-time get result must not be null"; @@ -861,8 +856,7 @@ public GetResult getFromTranslog( Function searcherWrapper ) { assert assertGetUsesIdField(get); - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { return realtimeGetUnderLock(get, mappingLookup, documentParser, searcherWrapper, false); } } @@ -878,71 +872,75 @@ protected GetResult realtimeGetUnderLock( Function searcherWrapper, boolean getFromSearcher ) { - assert readLock.isHeldByCurrentThread(); + assert isDrainedForClose() == false; assert get.realtime(); final VersionValue versionValue; try (Releasable ignore = versionMap.acquireLock(get.uid().bytes())) { // we need to lock here to access the version map to do this truly in RT versionValue = getVersionFromMap(get.uid().bytes()); } - boolean getFromSearcherIfNotInTranslog = getFromSearcher; - if (versionValue != null) { - /* - * Once we've seen the ID in the live version map, in two cases it is still possible not to - * be able to follow up with serving the get from the translog: - * 1. It is possible that once attempt handling the get, we won't see the doc in the translog - * since it might have been moved out. - * TODO: ideally we should keep around translog entries long enough to cover this case - * 2. We might not be tracking translog locations in the live version map (see @link{trackTranslogLocation}) - * - * In these cases, we should always fall back to get the doc from the internal searcher. - */ + try { + boolean getFromSearcherIfNotInTranslog = getFromSearcher; + if (versionValue != null) { + /* + * Once we've seen the ID in the live version map, in two cases it is still possible not to + * be able to follow up with serving the get from the translog: + * 1. It is possible that once attempt handling the get, we won't see the doc in the translog + * since it might have been moved out. + * TODO: ideally we should keep around translog entries long enough to cover this case + * 2. We might not be tracking translog locations in the live version map (see @link{trackTranslogLocation}) + * + * In these cases, we should always fall back to get the doc from the internal searcher. + */ - getFromSearcherIfNotInTranslog = true; - if (versionValue.isDelete()) { - return GetResult.NOT_EXISTS; - } - if (get.versionType().isVersionConflictForReads(versionValue.version, get.version())) { - throw new VersionConflictEngineException( - shardId, - "[" + get.id() + "]", - get.versionType().explainConflictForReads(versionValue.version, get.version()) - ); - } - if (get.getIfSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO - && (get.getIfSeqNo() != versionValue.seqNo || get.getIfPrimaryTerm() != versionValue.term)) { - throw new VersionConflictEngineException( - shardId, - get.id(), - get.getIfSeqNo(), - get.getIfPrimaryTerm(), - versionValue.seqNo, - versionValue.term - ); - } - if (get.isReadFromTranslog()) { - if (versionValue.getLocation() != null) { - try { - final Translog.Operation operation = translog.readOperation(versionValue.getLocation()); - if (operation != null) { - return getFromTranslog(get, (Translog.Index) operation, mappingLookup, documentParser, searcherWrapper); + getFromSearcherIfNotInTranslog = true; + if (versionValue.isDelete()) { + return GetResult.NOT_EXISTS; + } + if (get.versionType().isVersionConflictForReads(versionValue.version, get.version())) { + throw new VersionConflictEngineException( + shardId, + "[" + get.id() + "]", + get.versionType().explainConflictForReads(versionValue.version, get.version()) + ); + } + if (get.getIfSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO + && (get.getIfSeqNo() != versionValue.seqNo || get.getIfPrimaryTerm() != versionValue.term)) { + throw new VersionConflictEngineException( + shardId, + get.id(), + get.getIfSeqNo(), + get.getIfPrimaryTerm(), + versionValue.seqNo, + versionValue.term + ); + } + if (get.isReadFromTranslog()) { + if (versionValue.getLocation() != null) { + try { + final Translog.Operation operation = translog.readOperation(versionValue.getLocation()); + if (operation != null) { + return getFromTranslog(get, (Translog.Index) operation, mappingLookup, documentParser, searcherWrapper); + } + } catch (IOException e) { + maybeFailEngine("realtime_get", e); // lets check if the translog has failed with a tragic event + throw new EngineException(shardId, "failed to read operation from translog", e); } - } catch (IOException e) { - maybeFailEngine("realtime_get", e); // lets check if the translog has failed with a tragic event - throw new EngineException(shardId, "failed to read operation from translog", e); + } else { + // We need to start tracking translog locations in the live version map. + trackTranslogLocation.set(true); } - } else { - // We need to start tracking translog locations in the live version map. - trackTranslogLocation.set(true); } + assert versionValue.seqNo >= 0 : versionValue; + refreshIfNeeded(REAL_TIME_GET_REFRESH_SOURCE, versionValue.seqNo); } - assert versionValue.seqNo >= 0 : versionValue; - refreshIfNeeded(REAL_TIME_GET_REFRESH_SOURCE, versionValue.seqNo); - } - if (getFromSearcherIfNotInTranslog) { - return getFromSearcher(get, acquireSearcher("realtime_get", SearcherScope.INTERNAL, searcherWrapper), false); + if (getFromSearcherIfNotInTranslog) { + return getFromSearcher(get, acquireSearcher("realtime_get", SearcherScope.INTERNAL, searcherWrapper), false); + } + return null; + } finally { + assert isDrainedForClose() == false; } - return null; } /** @@ -1140,8 +1138,7 @@ long doGenerateSeqNoForOperation(final Operation operation) { 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 (ReleasableLock releasableLock = readLock.acquire()) { - ensureOpen(); + try (var ignored1 = acquireEnsureOpenRef()) { assert assertIncomingSequenceNumber(index.origin(), index.seqNo()); int reservedDocs = 0; try ( @@ -1607,8 +1604,7 @@ public DeleteResult delete(Delete delete) throws IOException { 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 (ReleasableLock ignored = readLock.acquire(); Releasable ignored2 = versionMap.acquireLock(delete.uid().bytes())) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef(); Releasable ignored2 = versionMap.acquireLock(delete.uid().bytes())) { lastWriteNanos = delete.startTime(); final DeletionStrategy plan = deletionStrategyForOperation(delete); reservedDocs = plan.reservedDocs; @@ -1935,8 +1931,7 @@ public void maybePruneDeletes() { @Override public NoOpResult noOp(final NoOp noOp) throws IOException { final NoOpResult noOpResult; - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { noOpResult = innerNoOp(noOp); } catch (final Exception e) { try { @@ -1950,7 +1945,7 @@ public NoOpResult noOp(final NoOp noOp) throws IOException { } private NoOpResult innerNoOp(final NoOp noOp) throws IOException { - assert readLock.isHeldByCurrentThread(); + assert isDrainedForClose() == false; assert noOp.seqNo() > SequenceNumbers.NO_OPS_PERFORMED; final long seqNo = noOp.seqNo(); try (Releasable ignored = noOpKeyedLock.acquire(seqNo)) { @@ -2005,6 +2000,8 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { noOpResult.setTook(System.nanoTime() - noOp.startTime()); noOpResult.freeze(); return noOpResult; + } finally { + assert isDrainedForClose() == false; } } @@ -2180,6 +2177,7 @@ private boolean shouldPeriodicallyFlush(long flushThresholdSizeInBytes, long flu @Override protected void flushHoldingLock(boolean force, boolean waitIfOngoing, ActionListener listener) throws EngineException { + assert isClosed.get() == false; // might be closing, but not closed yet ensureOpen(); if (force && waitIfOngoing == false) { assert false : "wait_if_ongoing must be true for a force flush: force=" + force + " wait_if_ongoing=" + waitIfOngoing; @@ -2295,8 +2293,7 @@ private void refreshLastCommittedSegmentInfos() { @Override public void rollTranslogGeneration() throws EngineException { - try (ReleasableLock ignored = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { translog.rollGeneration(); translog.trimUnreferencedReaders(); } catch (AlreadyClosedException e) { @@ -2314,8 +2311,7 @@ public void rollTranslogGeneration() throws EngineException { @Override public void trimUnreferencedTranslogFiles() throws EngineException { - try (ReleasableLock lock = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { translog.trimUnreferencedReaders(); } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -2337,8 +2333,7 @@ public boolean shouldRollTranslogGeneration() { @Override public void trimOperationsFromTranslog(long belowTerm, long aboveSeqNo) throws EngineException { - try (ReleasableLock lock = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { translog.trimOperations(belowTerm, aboveSeqNo); } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -2524,7 +2519,8 @@ private boolean failOnTragicEvent(AlreadyClosedException ex) { } else if (translog.isOpen() == false && translog.getTragicException() != null) { failEngine("already closed by tragic event on the translog", translog.getTragicException()); engineFailed = true; - } else if (failedEngine.get() == null && isClosed.get() == false) { // we are closed but the engine is not failed yet? + } else if (failedEngine.get() == null && isClosing() == false && isClosed.get() == false) { + // we are closed but the engine is not failed yet? // this smells like a bug - we only expect ACE if we are in a fatal case ie. either translog or IW is closed by // a tragic event or has closed itself. if that is not the case we are in a buggy state and raise an assertion error throw new AssertionError("Unexpected AlreadyClosedException", ex); @@ -2576,7 +2572,7 @@ public long getIndexBufferRAMBytesUsed() { @Override public List segments() { - try (ReleasableLock lock = readLock.acquire()) { + try (var ignored = acquireEnsureOpenRef()) { Segment[] segmentsArr = getSegmentInfo(lastCommittedSegmentInfos); // fill in the merges flag @@ -2595,16 +2591,11 @@ public List segments() { } } - /** - * Closes the engine without acquiring the write lock. This should only be - * called while the write lock is hold or in a disaster condition ie. if the engine - * is failed. - */ @Override protected final void closeNoLock(String reason, CountDownLatch closedLatch) { if (isClosed.compareAndSet(false, true)) { - assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread() - : "Either the write lock must be held or the engine must be currently be failing itself"; + assert isDrainedForClose() || failEngineLock.isHeldByCurrentThread() + : "Either all operations must have been drained or the engine must be currently be failing itself"; try { this.versionMap.clear(); if (internalReaderManager != null) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index e8ae9d605b3f6..a666138492a20 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -16,7 +16,6 @@ import org.apache.lucene.index.SegmentReader; import org.apache.lucene.store.Directory; import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.Nullable; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -147,8 +146,7 @@ public DocsStats docStats() { public void trimUnreferencedTranslogFiles() { final Store store = this.engineConfig.getStore(); store.incRef(); - try (ReleasableLock lock = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { final List commits = DirectoryReader.listCommits(store.directory()); if (commits.size() == 1 && translogStats.getTranslogSizeInBytes() > translogStats.getUncommittedSizeInBytes()) { final Map commitUserData = getLastCommittedSegmentInfos().getUserData(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index 2efba314540ee..7d5410cf488d7 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.hash.MessageDigests; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Nullable; import org.elasticsearch.index.IndexVersion; @@ -527,8 +526,7 @@ public void recoverFromTranslog( ActionListener listener ) { ActionListener.run(listener, l -> { - try (ReleasableLock lock = readLock.acquire()) { - ensureOpen(); + try (var ignored = acquireEnsureOpenRef()) { try { translogRecoveryRunner.run(this, Translog.Snapshot.EMPTY); } catch (final Exception e) { diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 8f493977484a8..6da1dbc3e5c52 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -13,6 +13,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.Version; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.bootstrap.BootstrapCheck; import org.elasticsearch.bootstrap.BootstrapContext; import org.elasticsearch.client.internal.Client; @@ -40,6 +41,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.Assertions; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.PathUtils; @@ -75,6 +77,7 @@ import org.elasticsearch.snapshots.SnapshotShardsService; import org.elasticsearch.snapshots.SnapshotsService; import org.elasticsearch.tasks.TaskCancellationService; +import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.tasks.TaskResultsService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteClusterPortSettings; @@ -97,14 +100,16 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.FutureTask; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Function; import javax.net.ssl.SNIHostName; +import static org.elasticsearch.core.Strings.format; + /** * A node represent a node within a cluster ({@code cluster.name}). The {@link #client()} can be used * in order to use a {@link Client} to perform actions/operations against the cluster. @@ -148,6 +153,12 @@ public class Node implements Closeable { Property.NodeScope ); + public static final Setting MAXIMUM_SHUTDOWN_TIMEOUT_SETTING = Setting.positiveTimeSetting( + "node.maximum_shutdown_grace_period", + TimeValue.timeValueMillis(0), + Setting.Property.NodeScope + ); + private final Lifecycle lifecycle = new Lifecycle(); /** @@ -580,17 +591,92 @@ public synchronized void close() throws IOException { */ public void prepareForClose() { HttpServerTransport httpServerTransport = injector.getInstance(HttpServerTransport.class); - FutureTask stopper = new FutureTask<>(httpServerTransport::close, null); - new Thread(stopper, "http-server-transport-stop").start(); - + Map stoppers = new HashMap<>(); + TimeValue maxTimeout = MAXIMUM_SHUTDOWN_TIMEOUT_SETTING.get(this.settings()); + stoppers.put("http-server-transport-stop", httpServerTransport::close); + stoppers.put("async-search-stop", () -> this.awaitSearchTasksComplete(maxTimeout)); if (terminationHandler != null) { - terminationHandler.handleTermination(); + stoppers.put("termination-handler-stop", terminationHandler::handleTermination); + } + + Map> futures = new HashMap<>(stoppers.size()); + for (var stopperEntry : stoppers.entrySet()) { + var future = new CompletableFuture(); + new Thread(() -> { + try { + stopperEntry.getValue().run(); + } catch (Exception ex) { + logger.warn("unexpected exception in shutdown task [" + stopperEntry.getKey() + "]", ex); + } finally { + future.complete(null); + } + }, stopperEntry.getKey()).start(); + futures.put(stopperEntry.getKey(), future); } + @SuppressWarnings(value = "rawtypes") // Can't make an array of parameterized types, but it complains if you leave the type out + CompletableFuture allStoppers = CompletableFuture.allOf(futures.values().toArray(new CompletableFuture[stoppers.size()])); + try { - stopper.get(); - } catch (Exception e) { - logger.warn("unexpected exception while waiting for http server to close", e); + if (maxTimeout.millis() == 0) { + FutureUtils.get(allStoppers); + } else { + FutureUtils.get(allStoppers, maxTimeout.millis(), TimeUnit.MILLISECONDS); + } + + } catch (ElasticsearchTimeoutException t) { + var unfinishedTasks = futures.entrySet() + .stream() + .filter(entry -> entry.getValue().isDone() == false) + .map(Map.Entry::getKey) + .toList(); + logger.warn("timed out while waiting for graceful shutdown tasks: " + unfinishedTasks); + } + } + + private void awaitSearchTasksComplete(TimeValue asyncSearchTimeout) { + TaskManager taskManager = injector.getInstance(TransportService.class).getTaskManager(); + long millisWaited = 0; + while (true) { + long searchTasksRemaining = taskManager.getTasks() + .values() + .stream() + .filter(task -> TransportSearchAction.TYPE.name().equals(task.getAction())) + .count(); + if (searchTasksRemaining == 0) { + logger.debug("all search tasks complete"); + return; + } else { + // Let the system work on those searches for a while. We're on a dedicated thread to manage app shutdown, so we + // literally just want to wait and not take up resources on this thread for now. Poll period chosen to allow short + // response times, but checking the tasks list is relatively expensive, and we don't want to waste CPU time we could + // be spending on finishing those searches. + final TimeValue pollPeriod = TimeValue.timeValueMillis(500); + millisWaited += pollPeriod.millis(); + if (millisWaited >= asyncSearchTimeout.millis()) { + logger.warn( + format( + "timed out after waiting [%s] for [%d] search tasks to finish", + asyncSearchTimeout.toString(), + searchTasksRemaining + ) + ); + return; + } + logger.debug(format("waiting for [%s] search tasks to finish, next poll in [%s]", searchTasksRemaining, pollPeriod)); + try { + Thread.sleep(pollPeriod.millis()); + } catch (InterruptedException ex) { + logger.warn( + format( + "interrupted while waiting [%s] for [%d] search tasks to finish", + asyncSearchTimeout.toString(), + searchTasksRemaining + ) + ); + return; + } + } } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java index b5b0a65a98d6b..fce50eec6fc01 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java @@ -14,7 +14,9 @@ import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.core.Nullable; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestRequestFilter; import org.elasticsearch.rest.RestResponse; @@ -33,14 +35,16 @@ public final class RestReloadSecureSettingsAction extends BaseRestHandler implements RestRequestFilter { - static final ObjectParser PARSER = new ObjectParser<>( - "reload_secure_settings", - NodesReloadSecureSettingsRequest::new - ); + static final class ParsedRequestBody { + @Nullable + SecureString secureSettingsPassword; + } + + static final ObjectParser PARSER = new ObjectParser<>("reload_secure_settings", ParsedRequestBody::new); static { PARSER.declareString( - (request, value) -> request.setSecureStorePassword(new SecureString(value.toCharArray())), + (parsedRequestBody, value) -> parsedRequestBody.secureSettingsPassword = new SecureString(value.toCharArray()), new ParseField("secure_settings_password") ); } @@ -57,32 +61,42 @@ public List routes() { @Override public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { - final String[] nodesIds = Strings.splitStringByCommaToArray(request.param("nodeId")); - final NodesReloadSecureSettingsRequest reloadSecureSettingsRequest = new NodesReloadSecureSettingsRequest(nodesIds); + final NodesReloadSecureSettingsRequest reloadSecureSettingsRequest = new NodesReloadSecureSettingsRequest(); + reloadSecureSettingsRequest.nodesIds(Strings.splitStringByCommaToArray(request.param("nodeId"))); reloadSecureSettingsRequest.timeout(request.param("timeout")); request.withContentOrSourceParamParserOrNull(parser -> { if (parser != null) { - final NodesReloadSecureSettingsRequest nodesRequest = PARSER.parse(parser, null); - reloadSecureSettingsRequest.setSecureStorePassword(nodesRequest.getSecureSettingsPassword()); + final ParsedRequestBody parsedRequestBody = PARSER.parse(parser, null); + reloadSecureSettingsRequest.setSecureStorePassword(parsedRequestBody.secureSettingsPassword); } }); - return channel -> client.execute( - TransportNodesReloadSecureSettingsAction.TYPE, - reloadSecureSettingsRequest, - new RestBuilderListener<>(channel) { - @Override - public RestResponse buildResponse(NodesReloadSecureSettingsResponse response, XContentBuilder builder) throws Exception { - builder.startObject(); - RestActions.buildNodesHeader(builder, channel.request(), response); - builder.field("cluster_name", response.getClusterName().value()); - response.toXContent(builder, channel.request()); - builder.endObject(); - reloadSecureSettingsRequest.close(); - return new RestResponse(RestStatus.OK, builder); - } + return new RestChannelConsumer() { + @Override + public void accept(RestChannel channel) { + client.execute( + TransportNodesReloadSecureSettingsAction.TYPE, + reloadSecureSettingsRequest, + new RestBuilderListener<>(channel) { + @Override + public RestResponse buildResponse(NodesReloadSecureSettingsResponse response, XContentBuilder builder) + throws Exception { + builder.startObject(); + RestActions.buildNodesHeader(builder, channel.request(), response); + builder.field("cluster_name", response.getClusterName().value()); + response.toXContent(builder, channel.request()); + builder.endObject(); + return new RestResponse(RestStatus.OK, builder); + } + } + ); } - ); + + @Override + public void close() { + reloadSecureSettingsRequest.decRef(); + } + }; } @Override diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java index f81d99c55e84e..dd92589f0af89 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java @@ -158,7 +158,7 @@ public void testShouldReadSizeFromClonedShard() { var state = ClusterState.builder(ClusterName.DEFAULT) .metadata( metadata( - IndexMetadata.builder("source").settings(indexSettings(IndexVersion.current(), 2, 0)), + IndexMetadata.builder("source").settings(indexSettings(IndexVersion.current(), 1, 0)), IndexMetadata.builder("target") .settings( indexSettings(IndexVersion.current(), 1, 0) // diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterInfoSimulatorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterInfoSimulatorTests.java index 1d2a7f05ff1f2..7aecd611b931b 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterInfoSimulatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/ClusterInfoSimulatorTests.java @@ -9,54 +9,73 @@ package org.elasticsearch.cluster.routing.allocation.allocator; import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterInfo.NodeAndPath; +import org.elasticsearch.cluster.ClusterInfo.ReservedSpace; import org.elasticsearch.cluster.ClusterInfoSimulator; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.DiskUsage; +import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodeRole; -import org.elasticsearch.cluster.node.DiscoveryNodeUtils; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; -import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.InternalSnapshotsInfoService; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotShardSizeInfo; -import org.elasticsearch.test.ESTestCase; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; +import static org.elasticsearch.cluster.ClusterInfo.shardIdentifierFromRouting; +import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_RESIZE_SOURCE_NAME_KEY; +import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_RESIZE_SOURCE_UUID_KEY; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED; -import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider.SETTING_IGNORE_DISK_WATERMARKS; +import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.hamcrest.Matchers.equalTo; -public class ClusterInfoSimulatorTests extends ESTestCase { +public class ClusterInfoSimulatorTests extends ESAllocationTestCase { public void testInitializeNewPrimary() { - var newPrimary = newShardRouting("index-1", 0, "node-0", true, INITIALIZING); - - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode("node-0", new DiskUsageBuilder(1000, 1000)) - .withNode("node-1", new DiskUsageBuilder(1000, 1000)) - .withShard(newPrimary, 0) - .build() + var newPrimary = newShardRouting( + new ShardId("my-index", "_na_", 0), + "node-0", + true, + ShardRoutingState.INITIALIZING, + RecoverySource.EmptyStoreRecoverySource.INSTANCE ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000)) + .build(); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings(IndexVersion.current(), 1, 0)))) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(newPrimary); assertThat( @@ -64,32 +83,39 @@ public void testInitializeNewPrimary() { equalTo( new ClusterInfoTestBuilder() // .withNode("node-0", new DiskUsageBuilder(1000, 1000)) - .withNode("node-1", new DiskUsageBuilder(1000, 1000)) - .withShard(newPrimary, 0) .build() ) ); } - public void testInitializeNewPrimaryWithKnownExpectedSize() { - - var newPrimary = newShardRouting("index-1", 0, null, true, UNASSIGNED).initialize("node-0", null, 100); + public void testInitializePreviouslyExistingPrimary() { - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode("node-0", new DiskUsageBuilder(1000, 1000)) - .withNode("node-1", new DiskUsageBuilder(1000, 1000)) - .build() + var existingPrimary = newShardRouting( + new ShardId("my-index", "_na_", 0), + "node-0", + true, + ShardRoutingState.INITIALIZING, + RecoverySource.ExistingStoreRecoverySource.INSTANCE ); - simulator.simulateShardStarted(newPrimary); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 900)) + .withShard(existingPrimary, 100) + .build(); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings(IndexVersion.current(), 1, 0)))) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); + simulator.simulateShardStarted(existingPrimary); assertThat( simulator.getClusterInfo(), equalTo( new ClusterInfoTestBuilder() // .withNode("node-0", new DiskUsageBuilder(1000, 900)) - .withNode("node-1", new DiskUsageBuilder(1000, 1000)) - .withShard(newPrimary, 100) + .withShard(existingPrimary, 100) .build() ) ); @@ -97,17 +123,26 @@ public void testInitializeNewPrimaryWithKnownExpectedSize() { public void testInitializeNewReplica() { - var existingPrimary = newShardRouting("index-1", 0, "node-0", true, STARTED); - var newReplica = newShardRouting("index-1", 0, "node-1", false, INITIALIZING); - - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode("node-0", new DiskUsageBuilder(1000, 900)) - .withNode("node-1", new DiskUsageBuilder(1000, 1000)) - .withShard(existingPrimary, 100) - .withShard(newReplica, 0) - .build() + var existingPrimary = newShardRouting(new ShardId("my-index", "_na_", 0), "node-0", true, STARTED); + var newReplica = newShardRouting( + new ShardId("my-index", "_na_", 0), + "node-1", + false, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 900)) + .withNode("node-1", new DiskUsageBuilder(1000, 1000)) + .withShard(existingPrimary, 100) + .build(); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings(IndexVersion.current(), 1, 1)))) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(newReplica); assertThat( @@ -128,15 +163,26 @@ public void testRelocateShard() { var fromNodeId = "node-0"; var toNodeId = "node-1"; - var shard = newShardRouting("index-1", 0, toNodeId, fromNodeId, true, INITIALIZING); - - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode(fromNodeId, new DiskUsageBuilder(1000, 900)) - .withNode(toNodeId, new DiskUsageBuilder(1000, 1000)) - .withShard(shard, 100) - .build() + var shard = newShardRouting( + new ShardId("my-index", "_na_", 0), + toNodeId, + fromNodeId, + true, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode(fromNodeId, new DiskUsageBuilder(1000, 900)) + .withNode(toNodeId, new DiskUsageBuilder(1000, 1000)) + .withShard(shard, 100) + .build(); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings(IndexVersion.current(), 1, 0)))) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(shard); assertThat( @@ -151,20 +197,31 @@ public void testRelocateShard() { ); } - public void testRelocateShardWithMultipleDataPath1() { + public void testRelocateShardWithMultipleDataPath() { var fromNodeId = "node-0"; var toNodeId = "node-1"; - var shard = newShardRouting("index-1", 0, toNodeId, fromNodeId, true, INITIALIZING); - - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode(fromNodeId, new DiskUsageBuilder("/data-1", 1000, 500), new DiskUsageBuilder("/data-2", 1000, 750)) - .withNode(toNodeId, new DiskUsageBuilder("/data-1", 1000, 750), new DiskUsageBuilder("/data-2", 1000, 900)) - .withShard(shard, 100) - .build() + var shard = newShardRouting( + new ShardId("my-index", "_na_", 0), + toNodeId, + fromNodeId, + true, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode(fromNodeId, new DiskUsageBuilder("/data-1", 1000, 500), new DiskUsageBuilder("/data-2", 1000, 750)) + .withNode(toNodeId, new DiskUsageBuilder("/data-1", 1000, 750), new DiskUsageBuilder("/data-2", 1000, 900)) + .withShard(shard, 100) + .build(); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings(IndexVersion.current(), 1, 0)))) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(shard); assertThat( @@ -179,12 +236,201 @@ public void testRelocateShardWithMultipleDataPath1() { ); } + public void testInitializeShardFromSnapshot() { + + var shardSize = 100; + var indexSettings = indexSettings(IndexVersion.current(), 1, 0); + if (randomBoolean()) { + indexSettings.put(INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE); + } + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings))) + .build(); + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot-1", "na")); + var indexId = new IndexId("my-index", "_na_"); + var shard = newShardRouting( + new ShardId(state.metadata().index("my-index").getIndex(), 0), + "node-0", + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource(randomUUID(), snapshot, IndexVersion.current(), indexId) + ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000)) + .withNode("node-1", new DiskUsageBuilder(1000, 1000)) + .build(); + var snapshotShardSizeInfo = new SnapshotShardSizeInfoTestBuilder() // + .withShard(snapshot, indexId, shard.shardId(), shardSize) + .build(); + + var allocation = createRoutingAllocation(state, initialClusterInfo, snapshotShardSizeInfo); + var simulator = new ClusterInfoSimulator(allocation); + simulator.simulateShardStarted(shard); + + assertThat( + simulator.getClusterInfo(), + equalTo( + new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000 - shardSize)) + .withNode("node-1", new DiskUsageBuilder(1000, 1000)) + .withShard(shard, shardSize) + .build() + ) + ); + } + + public void testInitializeShardFromPartialSearchableSnapshot() { + + var shardSize = 100; + var indexSettings = indexSettings(IndexVersion.current(), 1, 0) // + .put(INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE) + .put(SNAPSHOT_PARTIAL_SETTING.getKey(), true) + .put(SETTING_IGNORE_DISK_WATERMARKS.getKey(), true); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings))) + .build(); + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot-1", "na")); + var indexId = new IndexId("my-index", "_na_"); + var shard = newShardRouting( + new ShardId(state.metadata().index("my-index").getIndex(), 0), + "node-0", + true, + ShardRoutingState.INITIALIZING, + new RecoverySource.SnapshotRecoverySource(randomUUID(), snapshot, IndexVersion.current(), indexId) + ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000)) + .withNode("node-1", new DiskUsageBuilder(1000, 1000)) + .build(); + var snapshotShardSizeInfo = new SnapshotShardSizeInfoTestBuilder() // + .withShard(snapshot, indexId, shard.shardId(), shardSize) + .build(); + + var allocation = createRoutingAllocation(state, initialClusterInfo, snapshotShardSizeInfo); + var simulator = new ClusterInfoSimulator(allocation); + simulator.simulateShardStarted(shard); + + assertThat( + simulator.getClusterInfo(), + equalTo( + new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000)) + .withNode("node-1", new DiskUsageBuilder(1000, 1000)) + .withShard(shard, 0) // partial searchable snapshot always reports 0 size + .build() + ) + ); + } + + public void testRelocatePartialSearchableSnapshotShard() { + + var shardSize = 100; + var indexSettings = indexSettings(IndexVersion.current(), 1, 0) // + .put(INDEX_STORE_TYPE_SETTING.getKey(), SEARCHABLE_SNAPSHOT_STORE_TYPE) + .put(SNAPSHOT_PARTIAL_SETTING.getKey(), true) + .put(SETTING_IGNORE_DISK_WATERMARKS.getKey(), true); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().put(IndexMetadata.builder("my-index").settings(indexSettings))) + .build(); + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot-1", "na")); + var indexId = new IndexId("my-index", "_na_"); + + var fromNodeId = "node-0"; + var toNodeId = "node-1"; + + var shard = newShardRouting( + new ShardId("my-index", "_na_", 0), + toNodeId, + fromNodeId, + true, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE + ); + + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode(fromNodeId, new DiskUsageBuilder(1000, 1000)) + .withNode(toNodeId, new DiskUsageBuilder(1000, 1000)) + .withShard(shard, 0) + .build(); + var snapshotShardSizeInfo = new SnapshotShardSizeInfoTestBuilder() // + .withShard(snapshot, indexId, shard.shardId(), shardSize) + .build(); + + var allocation = createRoutingAllocation(state, initialClusterInfo, snapshotShardSizeInfo); + var simulator = new ClusterInfoSimulator(allocation); + simulator.simulateShardStarted(shard); + + assertThat( + simulator.getClusterInfo(), + equalTo( + new ClusterInfoTestBuilder() // + .withNode(fromNodeId, new DiskUsageBuilder(1000, 1000)) + .withNode(toNodeId, new DiskUsageBuilder(1000, 1000)) + .withShard(shard, 0) // partial searchable snapshot always reports 0 size + .build() + ) + ); + } + + public void testInitializeShardFromClone() { + + var sourceShardSize = randomLongBetween(100, 1000); + var source = newShardRouting(new ShardId("source", "_na_", 0), randomIdentifier(), true, ShardRoutingState.STARTED); + var target = newShardRouting( + new ShardId("target", "_na_", 0), + randomIdentifier(), + true, + ShardRoutingState.INITIALIZING, + RecoverySource.LocalShardsRecoverySource.INSTANCE + ); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata( + Metadata.builder() + .put(IndexMetadata.builder("source").settings(indexSettings(IndexVersion.current(), 1, 0))) + .put( + IndexMetadata.builder("target") + .settings( + indexSettings(IndexVersion.current(), 1, 0) // + .put(INDEX_RESIZE_SOURCE_NAME_KEY, "source") // + .put(INDEX_RESIZE_SOURCE_UUID_KEY, "_na_") + ) + ) + ) + .routingTable(RoutingTable.builder().add(IndexRoutingTable.builder(source.index()).addShard(source))) + .build(); + + var initialClusterInfo = new ClusterInfoTestBuilder().withNode("node-0", new DiskUsageBuilder(1000, 1000 - sourceShardSize)) + .withShard(source, sourceShardSize) + .build(); + + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); + simulator.simulateShardStarted(target); + + assertThat( + simulator.getClusterInfo(), + equalTo( + new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder(1000, 1000 - sourceShardSize)) + .withShard(source, sourceShardSize) + .withShard(target, sourceShardSize) + .build() + ) + ); + } + public void testDiskUsageSimulationWithSingleDataPathAndDiskThresholdDecider() { - var discoveryNodesBuilder = DiscoveryNodes.builder() - .add(createDiscoveryNode("node-0", DiscoveryNodeRole.roles())) - .add(createDiscoveryNode("node-1", DiscoveryNodeRole.roles())) - .add(createDiscoveryNode("node-2", DiscoveryNodeRole.roles())); + var discoveryNodesBuilder = DiscoveryNodes.builder().add(newNode("node-0")).add(newNode("node-1")).add(newNode("node-2")); var metadataBuilder = Metadata.builder(); var routingTableBuilder = RoutingTable.builder(); @@ -192,28 +438,35 @@ public void testDiskUsageSimulationWithSingleDataPathAndDiskThresholdDecider() { var shard1 = newShardRouting("index-1", 0, "node-0", null, true, STARTED); addIndex(metadataBuilder, routingTableBuilder, shard1); - var shard2 = newShardRouting("index-2", 0, "node-0", "node-1", true, INITIALIZING); + var shard2 = newShardRouting( + new ShardId("index-2", "_na_", 0), + "node-0", + "node-1", + true, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE + ); addIndex(metadataBuilder, routingTableBuilder, shard2); var shard3 = newShardRouting("index-3", 0, "node-1", null, true, STARTED); addIndex(metadataBuilder, routingTableBuilder, shard3); - var clusterState = ClusterState.builder(ClusterName.DEFAULT) + var state = ClusterState.builder(ClusterName.DEFAULT) .nodes(discoveryNodesBuilder) .metadata(metadataBuilder) .routingTable(routingTableBuilder) .build(); - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode("node-0", new DiskUsageBuilder("/data-1", 1000, 500)) - .withNode("node-1", new DiskUsageBuilder("/data-1", 1000, 300)) - .withShard(shard1, 500) - .withShard(shard2, 400) - .withShard(shard3, 300) - .build() - ); + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder("/data-1", 1000, 500)) + .withNode("node-1", new DiskUsageBuilder("/data-1", 1000, 300)) + .withShard(shard1, 500) + .withShard(shard2, 400) + .withShard(shard3, 300) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(shard2); assertThat( @@ -229,42 +482,29 @@ public void testDiskUsageSimulationWithSingleDataPathAndDiskThresholdDecider() { ) ); - var decider = new DiskThresholdDecider( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - var allocation = new RoutingAllocation( - new AllocationDeciders(List.of(decider)), - clusterState, - simulator.getClusterInfo(), - SnapshotShardSizeInfo.EMPTY, - 0L - ); - var routingNodes = allocation.routingNodes(); + var decider = new DiskThresholdDecider(Settings.EMPTY, ClusterSettings.createBuiltInClusterSettings(Settings.EMPTY)); + allocation = createRoutingAllocation(state, simulator.getClusterInfo(), SnapshotShardSizeInfo.EMPTY, decider); assertThat( "Should keep index-1 on node-0", - decider.canRemain(clusterState.metadata().index("index-1"), shard1, routingNodes.node("node-0"), allocation).type(), + decider.canRemain(state.metadata().index("index-1"), shard1, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.YES) ); assertThat( "Should keep index-2 on node-0", - decider.canRemain(clusterState.metadata().index("index-2"), shard2, routingNodes.node("node-0"), allocation).type(), + decider.canRemain(state.metadata().index("index-2"), shard2, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.YES) ); assertThat( "Should not allocate index-3 on node-0 (not enough space)", - decider.canAllocate(shard3, routingNodes.node("node-0"), allocation).type(), + decider.canAllocate(shard3, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.NO) ); } public void testDiskUsageSimulationWithMultipleDataPathAndDiskThresholdDecider() { - var discoveryNodesBuilder = DiscoveryNodes.builder() - .add(createDiscoveryNode("node-0", DiscoveryNodeRole.roles())) - .add(createDiscoveryNode("node-1", DiscoveryNodeRole.roles())) - .add(createDiscoveryNode("node-2", DiscoveryNodeRole.roles())); + var discoveryNodesBuilder = DiscoveryNodes.builder().add(newNode("node-0")).add(newNode("node-1")).add(newNode("node-2")); var metadataBuilder = Metadata.builder(); var routingTableBuilder = RoutingTable.builder(); @@ -272,28 +512,35 @@ public void testDiskUsageSimulationWithMultipleDataPathAndDiskThresholdDecider() var shard1 = newShardRouting("index-1", 0, "node-0", null, true, STARTED); addIndex(metadataBuilder, routingTableBuilder, shard1); - var shard2 = newShardRouting("index-2", 0, "node-0", "node-1", true, INITIALIZING); + var shard2 = newShardRouting( + new ShardId("index-2", "_na_", 0), + "node-0", + "node-1", + true, + INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE + ); addIndex(metadataBuilder, routingTableBuilder, shard2); var shard3 = newShardRouting("index-3", 0, "node-1", null, true, STARTED); addIndex(metadataBuilder, routingTableBuilder, shard3); - var clusterState = ClusterState.builder(ClusterName.DEFAULT) + var state = ClusterState.builder(ClusterName.DEFAULT) .nodes(discoveryNodesBuilder) .metadata(metadataBuilder) .routingTable(routingTableBuilder) .build(); - var simulator = new ClusterInfoSimulator( - new ClusterInfoTestBuilder() // - .withNode("node-0", new DiskUsageBuilder("/data-1", 1000, 100), new DiskUsageBuilder("/data-2", 1000, 500)) - .withNode("node-1", new DiskUsageBuilder("/data-1", 1000, 100), new DiskUsageBuilder("/data-2", 1000, 300)) - .withShard(shard1, 500) - .withShard(shard2, 400) - .withShard(shard3, 300) - .build() - ); + var initialClusterInfo = new ClusterInfoTestBuilder() // + .withNode("node-0", new DiskUsageBuilder("/data-1", 1000, 100), new DiskUsageBuilder("/data-2", 1000, 500)) + .withNode("node-1", new DiskUsageBuilder("/data-1", 1000, 100), new DiskUsageBuilder("/data-2", 1000, 300)) + .withShard(shard1, 500) + .withShard(shard2, 400) + .withShard(shard3, 300) + .build(); + var allocation = createRoutingAllocation(state, initialClusterInfo, SnapshotShardSizeInfo.EMPTY); + var simulator = new ClusterInfoSimulator(allocation); simulator.simulateShardStarted(shard2); assertThat( @@ -309,53 +556,63 @@ public void testDiskUsageSimulationWithMultipleDataPathAndDiskThresholdDecider() ) ); - var decider = new DiskThresholdDecider( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - var allocation = new RoutingAllocation( - new AllocationDeciders(List.of(decider)), - clusterState, - simulator.getClusterInfo(), - SnapshotShardSizeInfo.EMPTY, - 0L - ); - var routingNodes = allocation.routingNodes(); + var decider = new DiskThresholdDecider(Settings.EMPTY, ClusterSettings.createBuiltInClusterSettings(Settings.EMPTY)); + allocation = createRoutingAllocation(state, simulator.getClusterInfo(), SnapshotShardSizeInfo.EMPTY, decider); assertThat( "Should keep index-1 on node-0", - decider.canRemain(clusterState.metadata().index("index-1"), shard1, routingNodes.node("node-0"), allocation).type(), + decider.canRemain(state.metadata().index("index-1"), shard1, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.YES) ); assertThat( "Should keep index-2 on node-0", - decider.canRemain(clusterState.metadata().index("index-2"), shard2, routingNodes.node("node-0"), allocation).type(), + decider.canRemain(state.metadata().index("index-2"), shard2, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.YES) ); assertThat( "Should not allocate index-3 on node-0 (not enough space)", - decider.canAllocate(shard3, routingNodes.node("node-0"), allocation).type(), + decider.canAllocate(shard3, allocation.routingNodes().node("node-0"), allocation).type(), equalTo(Decision.Type.NO) ); } - private static DiscoveryNode createDiscoveryNode(String id, Set roles) { - return DiscoveryNodeUtils.builder(id).name(id).externalId(UUIDs.randomBase64UUID(random())).roles(roles).build(); - } - private static void addIndex(Metadata.Builder metadataBuilder, RoutingTable.Builder routingTableBuilder, ShardRouting shardRouting) { var name = shardRouting.getIndexName(); metadataBuilder.put(IndexMetadata.builder(name).settings(indexSettings(IndexVersion.current(), 1, 0))); routingTableBuilder.add(IndexRoutingTable.builder(metadataBuilder.get(name).getIndex()).addShard(shardRouting)); } + private static RoutingAllocation createRoutingAllocation( + ClusterState state, + ClusterInfo clusterInfo, + SnapshotShardSizeInfo snapshotShardSizeInfo, + AllocationDecider... deciders + ) { + return new RoutingAllocation(new AllocationDeciders(List.of(deciders)), state, clusterInfo, snapshotShardSizeInfo, 0); + } + + private static class SnapshotShardSizeInfoTestBuilder { + + private final Map snapshotShardSizes = new HashMap<>(); + + public SnapshotShardSizeInfoTestBuilder withShard(Snapshot snapshot, IndexId indexId, ShardId shardId, long size) { + snapshotShardSizes.put(new InternalSnapshotsInfoService.SnapshotShard(snapshot, indexId, shardId), size); + return this; + } + + public SnapshotShardSizeInfo build() { + return new SnapshotShardSizeInfo(snapshotShardSizes); + } + } + private static class ClusterInfoTestBuilder { private final Map leastAvailableSpaceUsage = new HashMap<>(); private final Map mostAvailableSpaceUsage = new HashMap<>(); private final Map shardSizes = new HashMap<>(); + private final Map reservedSpace = new HashMap<>(); public ClusterInfoTestBuilder withNode(String name, DiskUsageBuilder diskUsageBuilderBuilder) { leastAvailableSpaceUsage.put(name, diskUsageBuilderBuilder.toDiskUsage(name)); @@ -370,12 +627,12 @@ public ClusterInfoTestBuilder withNode(String name, DiskUsageBuilder leastAvaila } public ClusterInfoTestBuilder withShard(ShardRouting shard, long size) { - shardSizes.put(ClusterInfo.shardIdentifierFromRouting(shard), size); + shardSizes.put(shardIdentifierFromRouting(shard), size); return this; } public ClusterInfo build() { - return new ClusterInfo(leastAvailableSpaceUsage, mostAvailableSpaceUsage, shardSizes, Map.of(), Map.of(), Map.of()); + return new ClusterInfo(leastAvailableSpaceUsage, mostAvailableSpaceUsage, shardSizes, Map.of(), Map.of(), reservedSpace); } } @@ -385,8 +642,8 @@ private DiskUsageBuilder(long total, long free) { this("/data", total, free); } - public DiskUsage toDiskUsage(String name) { - return new DiskUsage(name, name, name + path, total, free); + public DiskUsage toDiskUsage(String nodeId) { + return new DiskUsage(nodeId, nodeId, nodeId + path, total, free); } } } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputerTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputerTests.java index 5e3b6cd02f830..9fe168074f41e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputerTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputerTests.java @@ -10,8 +10,9 @@ import org.apache.logging.log4j.Level; import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterInfo.NodeAndPath; import org.elasticsearch.cluster.ClusterInfo.NodeAndShard; -import org.elasticsearch.cluster.ClusterModule; +import org.elasticsearch.cluster.ClusterInfo.ReservedSpace; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.DiskUsage; @@ -39,13 +40,13 @@ import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.Maps; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.InternalSnapshotsInfoService; import org.elasticsearch.snapshots.InternalSnapshotsInfoService.SnapshotShard; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; @@ -82,6 +83,7 @@ import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -804,29 +806,25 @@ public void testComputeConsideringShardSizes() { var node0RemainingBytes = (index0ReplicaShard.started() || index0ReplicaShard.relocating()) && Objects.equals(index0ReplicaShard.currentNodeId(), "node-0") ? 100 : 600; - var node0Usage = new DiskUsage("node-0", "node-0", "/data", 1000, node0RemainingBytes); - var node1Usage = new DiskUsage("node-1", "node-1", "/data", 1000, 100); - var node2Usage = new DiskUsage("node-2", "node-2", "/data", 1000, 1000); - var clusterInfo = createClusterInfo( - List.of(node0Usage, node1Usage, node2Usage), - Map.ofEntries( - // node-0 & node-1 - indexSize(clusterState, "index-0", 500, true), - indexSize(clusterState, "index-0", 500, false), - // node-0 - indexSize(clusterState, "index-1", 400, true), - // node-1 - indexSize(clusterState, "index-2", 50, true), - indexSize(clusterState, "index-3", 50, true), - indexSize(clusterState, "index-4", 50, true), - indexSize(clusterState, "index-5", 50, true), - indexSize(clusterState, "index-6", 50, true), - indexSize(clusterState, "index-7", 50, true), - indexSize(clusterState, "index-8", 50, true), - indexSize(clusterState, "index-9", 50, true) - ) - ); + var clusterInfo = new ClusterInfoTestBuilder().withNode("node-0", 1000, node0RemainingBytes) + .withNode("node-1", 1000, 100) + .withNode("node-2", 1000, 1000) + // node-0 & node-1 + .withShard(findShardId(clusterState, "index-0"), true, 500) + .withShard(findShardId(clusterState, "index-0"), false, 500) + // node-0 + .withShard(findShardId(clusterState, "index-1"), true, 400) + // node-1 + .withShard(findShardId(clusterState, "index-2"), true, 50) + .withShard(findShardId(clusterState, "index-3"), true, 50) + .withShard(findShardId(clusterState, "index-4"), true, 50) + .withShard(findShardId(clusterState, "index-5"), true, 50) + .withShard(findShardId(clusterState, "index-6"), true, 50) + .withShard(findShardId(clusterState, "index-7"), true, 50) + .withShard(findShardId(clusterState, "index-8"), true, 50) + .withShard(findShardId(clusterState, "index-9"), true, 50) + .build(); var settings = Settings.builder() // force as many iterations as possible to accumulate the diff @@ -839,11 +837,9 @@ public void testComputeConsideringShardSizes() { var initial = new DesiredBalance( 1, - Map.of( - findShardId(clusterState, "index-0"), - new ShardAssignment(Set.of("node-0", "node-1"), 2, 0, 0), - findShardId(clusterState, "index-1"), - new ShardAssignment(Set.of("node-0"), 1, 0, 0) + Map.ofEntries( + Map.entry(findShardId(clusterState, "index-0"), new ShardAssignment(Set.of("node-0", "node-1"), 2, 0, 0)), + Map.entry(findShardId(clusterState, "index-1"), new ShardAssignment(Set.of("node-0"), 1, 0, 0)) ) ); @@ -865,16 +861,182 @@ public void testComputeConsideringShardSizes() { assertThat(resultDiskUsage, allOf(aMapWithSize(2), hasEntry("node-0", 950L), hasEntry("node-1", 850L))); } - private static ClusterInfo createClusterInfo(List diskUsages, Map shardSizes) { - var diskUsage = diskUsages.stream().collect(toMap(DiskUsage::getNodeId, Function.identity())); - return new ClusterInfo(diskUsage, diskUsage, shardSizes, Map.of(), Map.of(), Map.of()); + public void testAccountForSizeOfMisplacedShardsDuringNewComputation() { + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot", randomUUID())); + + var clusterInfoBuilder = new ClusterInfoTestBuilder().withNode( + "node-1", + ByteSizeValue.ofGb(10).getBytes(), + ByteSizeValue.ofGb(2).getBytes() + ).withNode("node-2", ByteSizeValue.ofGb(10).getBytes(), ByteSizeValue.ofGb(2).getBytes()); + var snapshotShardSizes = Maps.newHashMapWithExpectedSize(5); + + var routingTableBuilder = RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY); + // index-1 is allocated according to the desired balance + var indexMetadata1 = IndexMetadata.builder("index-1").settings(indexSettings(IndexVersion.current(), 2, 0)).build(); + routingTableBuilder.add( + IndexRoutingTable.builder(indexMetadata1.getIndex()) + .addShard(newShardRouting(shardIdFrom(indexMetadata1, 0), "node-1", true, STARTED)) + .addShard(newShardRouting(shardIdFrom(indexMetadata1, 1), "node-2", true, STARTED)) + ); + clusterInfoBuilder.withShard(shardIdFrom(indexMetadata1, 0), true, ByteSizeValue.ofGb(8).getBytes()) + .withShard(shardIdFrom(indexMetadata1, 1), true, ByteSizeValue.ofGb(8).getBytes()); + + // index-2 is restored earlier but is not started on the desired node yet + var indexMetadata2 = IndexMetadata.builder("index-2").settings(indexSettings(IndexVersion.current(), 1, 0)).build(); + snapshotShardSizes.put( + new SnapshotShard(snapshot, indexIdFrom(indexMetadata2), shardIdFrom(indexMetadata2, 0)), + ByteSizeValue.ofGb(1).getBytes() + ); + var index2SnapshotRecoverySource = new RecoverySource.SnapshotRecoverySource( + "restore", + snapshot, + IndexVersion.current(), + indexIdFrom(indexMetadata2) + ); + switch (randomInt(3)) { + // index is still unassigned + case 0 -> routingTableBuilder.addAsNewRestore(indexMetadata2, index2SnapshotRecoverySource, Set.of()); + // index is initializing on desired node + case 1 -> { + ShardId index2ShardId = shardIdFrom(indexMetadata2, 0); + routingTableBuilder.add( + IndexRoutingTable.builder(indexMetadata2.getIndex()) + .addShard(newShardRouting(index2ShardId, "node-1", true, INITIALIZING, index2SnapshotRecoverySource)) + ); + if (randomBoolean()) { + // Shard is 75% downloaded + clusterInfoBuilder // + .withNodeUsedSpace("node-1", ByteSizeValue.ofMb(768).getBytes()) + .withReservedSpace("node-1", ByteSizeValue.ofMb(256).getBytes(), index2ShardId); + } + } + // index is initializing on undesired node + case 2 -> { + ShardId index2ShardId = shardIdFrom(indexMetadata2, 0); + routingTableBuilder.add( + IndexRoutingTable.builder(indexMetadata2.getIndex()) + .addShard(newShardRouting(index2ShardId, "node-2", true, INITIALIZING, index2SnapshotRecoverySource)) + ); + if (randomBoolean()) { + // Shard is 75% downloaded + clusterInfoBuilder // + .withNodeUsedSpace("node-2", ByteSizeValue.ofMb(768).getBytes()) + .withReservedSpace("node-2", ByteSizeValue.ofMb(256).getBytes(), index2ShardId); + } + } + // index is started on undesired node + case 3 -> { + routingTableBuilder.add( + IndexRoutingTable.builder(indexMetadata2.getIndex()) + .addShard(newShardRouting(shardIdFrom(indexMetadata2, 0), "node-2", true, STARTED)) + ); + clusterInfoBuilder.withNodeUsedSpace("node-2", ByteSizeValue.ofGb(1).getBytes()) + .withShard(shardIdFrom(indexMetadata2, 0), true, ByteSizeValue.ofGb(1).getBytes()); + } + default -> throw new AssertionError("unexpected randomization"); + } + + // index-3 is restored as new from snapshot + var indexMetadata3 = IndexMetadata.builder("index-3").settings(indexSettings(IndexVersion.current(), 2, 0)).build(); + routingTableBuilder.addAsNewRestore( + indexMetadata3, + new RecoverySource.SnapshotRecoverySource("restore", snapshot, IndexVersion.current(), indexIdFrom(indexMetadata3)), + Set.of() + ); + snapshotShardSizes.put( + new SnapshotShard(snapshot, indexIdFrom(indexMetadata3), shardIdFrom(indexMetadata3, 0)), + ByteSizeValue.ofMb(512).getBytes() + ); + snapshotShardSizes.put( + new SnapshotShard(snapshot, indexIdFrom(indexMetadata3), shardIdFrom(indexMetadata3, 1)), + ByteSizeValue.ofMb(512).getBytes() + ); + + var clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(newNode("node-1")).add(newNode("node-2"))) + .metadata(Metadata.builder().put(indexMetadata1, false).put(indexMetadata2, false).put(indexMetadata3, false).build()) + .routingTable(routingTableBuilder) + .customs( + Map.of( + RestoreInProgress.TYPE, + new RestoreInProgress.Builder().add( + new RestoreInProgress.Entry( + "restore", + snapshot, + RestoreInProgress.State.STARTED, + randomBoolean(), + List.of(indexMetadata2.getIndex().getName(), indexMetadata3.getIndex().getName()), + Map.ofEntries( + Map.entry(shardIdFrom(indexMetadata2, 0), new RestoreInProgress.ShardRestoreStatus(randomUUID())), + Map.entry(shardIdFrom(indexMetadata3, 0), new RestoreInProgress.ShardRestoreStatus(randomUUID())), + Map.entry(shardIdFrom(indexMetadata3, 1), new RestoreInProgress.ShardRestoreStatus(randomUUID())) + ) + ) + ).build() + ) + ) + .build(); + + var settings = Settings.EMPTY; + var allocation = new RoutingAllocation( + randomAllocationDeciders(settings, createBuiltInClusterSettings(settings)), + clusterState, + clusterInfoBuilder.build(), + new SnapshotShardSizeInfo(snapshotShardSizes), + 0L + ); + var initialDesiredBalance = new DesiredBalance( + 1, + Map.ofEntries( + Map.entry(shardIdFrom(indexMetadata1, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata1, 1), new ShardAssignment(Set.of("node-2"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata2, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)) + ) + ); + var nextDesiredBalance = createDesiredBalanceComputer(new BalancedShardsAllocator()).compute( + initialDesiredBalance, + new DesiredBalanceInput(2, allocation, List.of()), + queue(), + input -> true + ); + + // both node-1 and node-2 has enough space to allocate either only [index-2] shard or both [index-3] shards + assertThat( + nextDesiredBalance.assignments(), + anyOf( + equalTo( + Map.ofEntries( + Map.entry(shardIdFrom(indexMetadata1, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata1, 1), new ShardAssignment(Set.of("node-2"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata2, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata3, 0), new ShardAssignment(Set.of("node-2"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata3, 1), new ShardAssignment(Set.of("node-2"), 1, 0, 0)) + ) + ), + equalTo( + Map.ofEntries( + Map.entry(shardIdFrom(indexMetadata1, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata1, 1), new ShardAssignment(Set.of("node-2"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata2, 0), new ShardAssignment(Set.of("node-2"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata3, 0), new ShardAssignment(Set.of("node-1"), 1, 0, 0)), + Map.entry(shardIdFrom(indexMetadata3, 1), new ShardAssignment(Set.of("node-1"), 1, 0, 0)) + ) + ) + ) + ); } public void testAccountForSizeOfAllInitializingShardsDuringAllocation() { var snapshot = new Snapshot("repository", new SnapshotId("snapshot", randomUUID())); - var shardSizeInfo = Maps.newHashMapWithExpectedSize(5); + var clusterInfoBuilder = new ClusterInfoTestBuilder().withNode( + "node-1", + ByteSizeValue.ofGb(10).getBytes(), + ByteSizeValue.ofGb(2).getBytes() + ).withNode("node-2", ByteSizeValue.ofGb(10).getBytes(), ByteSizeValue.ofGb(2).getBytes()); var snapshotShardSizes = Maps.newHashMapWithExpectedSize(5); var routingTableBuilder = RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY); @@ -885,8 +1047,8 @@ public void testAccountForSizeOfAllInitializingShardsDuringAllocation() { .addShard(newShardRouting(shardIdFrom(indexMetadata1, 0), "node-1", true, STARTED)) .addShard(newShardRouting(shardIdFrom(indexMetadata1, 1), "node-2", true, STARTED)) ); - shardSizeInfo.put(shardIdentifierFromRouting(shardIdFrom(indexMetadata1, 0), true), ByteSizeValue.ofGb(8).getBytes()); - shardSizeInfo.put(shardIdentifierFromRouting(shardIdFrom(indexMetadata1, 1), true), ByteSizeValue.ofGb(8).getBytes()); + clusterInfoBuilder.withShard(shardIdFrom(indexMetadata1, 0), true, ByteSizeValue.ofGb(8).getBytes()) + .withShard(shardIdFrom(indexMetadata1, 1), true, ByteSizeValue.ofGb(8).getBytes()); // index-2 & index-3 are restored as new from snapshot var indexMetadata2 = IndexMetadata.builder("index-2") @@ -944,23 +1106,12 @@ public void testAccountForSizeOfAllInitializingShardsDuringAllocation() { ) .build(); - var clusterInfo = createClusterInfo( - List.of( - // node-1 has enough space to only allocate the only [index-2] shard - new DiskUsage("node-1", "data-1", "/data", ByteSizeValue.ofGb(10).getBytes(), ByteSizeValue.ofGb(2).getBytes()), - // node-2 has enough space to only allocate both shards of [index-3] - new DiskUsage("node-2", "data-2", "/data", ByteSizeValue.ofGb(10).getBytes(), ByteSizeValue.ofGb(2).getBytes()) - ), - shardSizeInfo - ); - var snapshotShardSizeInfo = new SnapshotShardSizeInfo(snapshotShardSizes); - var settings = Settings.EMPTY; var allocation = new RoutingAllocation( randomAllocationDeciders(settings, createBuiltInClusterSettings(settings)), clusterState, - clusterInfo, - snapshotShardSizeInfo, + clusterInfoBuilder.build(), + new SnapshotShardSizeInfo(snapshotShardSizes), 0L ); var initialDesiredBalance = new DesiredBalance( @@ -977,6 +1128,7 @@ public void testAccountForSizeOfAllInitializingShardsDuringAllocation() { input -> true ); + // both node-1 and node-2 has enough space to allocate either only [index-2] shard or both [index-3] shards assertThat( nextDesiredBalance.assignments(), anyOf( @@ -1002,6 +1154,46 @@ public void testAccountForSizeOfAllInitializingShardsDuringAllocation() { ); } + @Deprecated + private static ClusterInfo createClusterInfo(List diskUsages, Map shardSizes) { + var diskUsage = diskUsages.stream().collect(toMap(DiskUsage::getNodeId, Function.identity())); + return new ClusterInfo(diskUsage, diskUsage, shardSizes, Map.of(), Map.of(), Map.of()); + } + + private static class ClusterInfoTestBuilder { + + private final Map diskUsage = new HashMap<>(); + private final Map shardSizes = new HashMap<>(); + private final Map reservedSpace = new HashMap<>(); + + public ClusterInfoTestBuilder withNode(String nodeId, long totalBytes, long freeBytes) { + diskUsage.put(nodeId, new DiskUsage(nodeId, nodeId, "/path", totalBytes, freeBytes)); + return this; + } + + public ClusterInfoTestBuilder withNodeUsedSpace(String nodeId, long usedBytes) { + diskUsage.compute(nodeId, (key, usage) -> { + assertThat(usage, notNullValue()); + return new DiskUsage(usage.nodeId(), usage.nodeName(), usage.path(), usage.totalBytes(), usage.freeBytes() - usedBytes); + }); + return this; + } + + public ClusterInfoTestBuilder withShard(ShardId shardId, boolean primary, long size) { + shardSizes.put(shardIdentifierFromRouting(shardId, primary), size); + return this; + } + + public ClusterInfoTestBuilder withReservedSpace(String nodeId, long size, ShardId... shardIds) { + reservedSpace.put(new NodeAndPath(nodeId, "/path"), new ReservedSpace(size, Set.of(shardIds))); + return this; + } + + public ClusterInfo build() { + return new ClusterInfo(diskUsage, diskUsage, shardSizes, Map.of(), Map.of(), reservedSpace); + } + } + private static IndexId indexIdFrom(IndexMetadata indexMetadata) { return new IndexId(indexMetadata.getIndex().getName(), indexMetadata.getIndex().getUUID()); } @@ -1188,8 +1380,8 @@ public ShardAllocationDecision decideShardAllocation(ShardRouting shard, Routing }); } - private static DesiredBalanceComputer createDesiredBalanceComputer(ShardsAllocator shardsAllocator) { - return new DesiredBalanceComputer(createBuiltInClusterSettings(), mock(ThreadPool.class), shardsAllocator); + private static DesiredBalanceComputer createDesiredBalanceComputer(ShardsAllocator allocator) { + return new DesiredBalanceComputer(createBuiltInClusterSettings(), mock(ThreadPool.class), allocator); } private static void assertDesiredAssignments(DesiredBalance desiredBalance, Map expected) { @@ -1210,13 +1402,7 @@ private static RoutingAllocation routingAllocationWithDecidersOf( Settings settings ) { return new RoutingAllocation( - new AllocationDeciders( - ClusterModule.createAllocationDeciders( - settings, - new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), - List.of() - ) - ), + randomAllocationDeciders(settings, createBuiltInClusterSettings(settings)), clusterState, clusterInfo, SnapshotShardSizeInfo.EMPTY, diff --git a/server/src/test/java/org/elasticsearch/common/util/CopyOnFirstWriteMapTests.java b/server/src/test/java/org/elasticsearch/common/util/CopyOnFirstWriteMapTests.java index 41870862c437d..0c34bfa66524b 100644 --- a/server/src/test/java/org/elasticsearch/common/util/CopyOnFirstWriteMapTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/CopyOnFirstWriteMapTests.java @@ -21,7 +21,16 @@ public class CopyOnFirstWriteMapTests extends ESTestCase { public void testShouldNotCopyIfThereWereNoUpdates() { var source = Map.of("key", "value"); var copyOnFirstWrite = new CopyOnFirstWriteMap<>(source); - source.get("key"); + var copy = copyOnFirstWrite.toImmutableMap(); + + assertThat(copy, sameInstance(source)); + assertThat(copy, equalTo(source)); + } + + public void testShouldNotCopyWhenPuttingTheSameValue() { + var source = Map.of("key", "value"); + var copyOnFirstWrite = new CopyOnFirstWriteMap<>(source); + copyOnFirstWrite.put("key", "value"); var copy = copyOnFirstWrite.toImmutableMap(); assertThat(copy, sameInstance(source)); 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 cbb713f01ec03..c2994194bac4f 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -91,7 +91,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.TimeValue; @@ -1347,7 +1346,7 @@ public void testSyncedFlushVanishesOnReplay() throws IOException { } void syncFlush(IndexWriter writer, InternalEngine engine, String syncId) throws IOException { - try (ReleasableLock ignored = engine.readLock.acquire()) { + try (var ignored = engine.acquireEnsureOpenRef()) { Map userData = new HashMap<>(); writer.getLiveCommitData().forEach(e -> userData.put(e.getKey(), e.getValue())); userData.put(Engine.SYNC_COMMIT_ID, syncId); @@ -3676,6 +3675,7 @@ public void run() { /** * Tests that when the close method returns the engine is actually guaranteed to have cleaned up and that resources are closed */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103861") public void testConcurrentEngineClosed() throws BrokenBarrierException, InterruptedException { Thread[] closingThreads = new Thread[3]; CyclicBarrier barrier = new CyclicBarrier(1 + closingThreads.length + 1); diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsActionTests.java index bb26a616e0998..331148e1a0023 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/cluster/RestReloadSecureSettingsActionTests.java @@ -8,7 +8,6 @@ package org.elasticsearch.rest.action.admin.cluster; -import org.elasticsearch.action.admin.cluster.node.reload.NodesReloadSecureSettingsRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentParserConfiguration; @@ -22,16 +21,16 @@ public void testParserWithPassword() throws Exception { final String request = """ {"secure_settings_password": "secure_settings_password_string"}"""; try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, request)) { - NodesReloadSecureSettingsRequest reloadSecureSettingsRequest = RestReloadSecureSettingsAction.PARSER.parse(parser, null); - assertEquals("secure_settings_password_string", reloadSecureSettingsRequest.getSecureSettingsPassword().toString()); + RestReloadSecureSettingsAction.ParsedRequestBody parsedRequestBody = RestReloadSecureSettingsAction.PARSER.parse(parser, null); + assertEquals("secure_settings_password_string", parsedRequestBody.secureSettingsPassword.toString()); } } public void testParserWithoutPassword() throws Exception { final String request = "{}"; try (XContentParser parser = XContentType.JSON.xContent().createParser(XContentParserConfiguration.EMPTY, request)) { - NodesReloadSecureSettingsRequest reloadSecureSettingsRequest = RestReloadSecureSettingsAction.PARSER.parse(parser, null); - assertThat(reloadSecureSettingsRequest.getSecureSettingsPassword(), nullValue()); + RestReloadSecureSettingsAction.ParsedRequestBody parsedRequestBody = RestReloadSecureSettingsAction.PARSER.parse(parser, null); + assertThat(parsedRequestBody.secureSettingsPassword, nullValue()); } } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java index 1158e805ba3c1..1810b5cee76ec 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java @@ -84,6 +84,29 @@ public static ShardRouting newShardRouting( ); } + public static ShardRouting newShardRouting( + ShardId shardId, + String currentNodeId, + String relocatingNodeId, + boolean primary, + ShardRoutingState state, + RecoverySource recoverySource + ) { + return new ShardRouting( + shardId, + currentNodeId, + relocatingNodeId, + primary, + state, + recoverySource, + buildUnassignedInfo(state), + buildRelocationFailureInfo(state), + buildAllocationId(state), + -1, + ShardRouting.Role.DEFAULT + ); + } + public static ShardRouting newShardRouting( String index, int shardId, diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 349c4fe640aa3..eb255be092157 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -2376,7 +2376,7 @@ public static void setIgnoredErrorResponseCodes(Request request, RestStatus... r } public static void addXContentBody(Request request, ToXContent body) throws IOException { - final var xContentType = randomFrom(XContentType.values()); + final var xContentType = XContentType.JSON; final var bodyBytes = XContentHelper.toXContent(body, xContentType, EMPTY_PARAMS, randomBoolean()); request.setEntity( new InputStreamEntity( diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseBulkUpdateApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseBulkUpdateApiKeyRequest.java index 81c8479c47285..34b249d7a8233 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseBulkUpdateApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseBulkUpdateApiKeyRequest.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import java.io.IOException; @@ -27,9 +28,10 @@ public abstract class BaseBulkUpdateApiKeyRequest extends BaseUpdateApiKeyReques public BaseBulkUpdateApiKeyRequest( final List ids, @Nullable final List roleDescriptors, - @Nullable final Map metadata + @Nullable final Map metadata, + @Nullable final TimeValue expiration ) { - super(roleDescriptors, metadata); + super(roleDescriptors, metadata, expiration); this.ids = Objects.requireNonNull(ids, "API key IDs must not be null"); } @@ -56,4 +58,21 @@ public void writeTo(StreamOutput out) throws IOException { public List getIds() { return ids; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass() || super.equals(o)) return false; + + BaseBulkUpdateApiKeyRequest that = (BaseBulkUpdateApiKeyRequest) o; + return Objects.equals(getIds(), that.getIds()) + && Objects.equals(metadata, that.metadata) + && Objects.equals(expiration, that.expiration) + && Objects.equals(roleDescriptors, that.roleDescriptors); + } + + @Override + public int hashCode() { + return Objects.hash(getIds(), expiration, metadata, roleDescriptors); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseSingleUpdateApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseSingleUpdateApiKeyRequest.java index 7d89c0dd39b0c..725a9fb197b07 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseSingleUpdateApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseSingleUpdateApiKeyRequest.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import java.io.IOException; @@ -24,9 +25,10 @@ public abstract class BaseSingleUpdateApiKeyRequest extends BaseUpdateApiKeyRequ public BaseSingleUpdateApiKeyRequest( @Nullable final List roleDescriptors, @Nullable final Map metadata, + @Nullable final TimeValue expiration, String id ) { - super(roleDescriptors, metadata); + super(roleDescriptors, metadata, expiration); this.id = Objects.requireNonNull(id, "API key ID must not be null"); } @@ -44,4 +46,21 @@ public void writeTo(StreamOutput out) throws IOException { public String getId() { return id; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass() || super.equals(o)) return false; + + BaseSingleUpdateApiKeyRequest that = (BaseSingleUpdateApiKeyRequest) o; + return Objects.equals(getId(), that.getId()) + && Objects.equals(metadata, that.metadata) + && Objects.equals(expiration, that.expiration) + && Objects.equals(roleDescriptors, that.roleDescriptors); + } + + @Override + public int hashCode() { + return Objects.hash(getId(), expiration, metadata, roleDescriptors); + } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseUpdateApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseUpdateApiKeyRequest.java index b06f8868c53d1..a1acb6c9581f5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseUpdateApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BaseUpdateApiKeyRequest.java @@ -7,11 +7,13 @@ package org.elasticsearch.xpack.core.security.action.apikey; +import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.core.security.action.role.RoleDescriptorRequestValidator; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import org.elasticsearch.xpack.core.security.support.MetadataUtils; @@ -28,16 +30,28 @@ public abstract class BaseUpdateApiKeyRequest extends ActionRequest { protected final List roleDescriptors; @Nullable protected final Map metadata; + @Nullable + protected final TimeValue expiration; - public BaseUpdateApiKeyRequest(@Nullable final List roleDescriptors, @Nullable final Map metadata) { + public BaseUpdateApiKeyRequest( + @Nullable final List roleDescriptors, + @Nullable final Map metadata, + @Nullable final TimeValue expiration + ) { this.roleDescriptors = roleDescriptors; this.metadata = metadata; + this.expiration = expiration; } public BaseUpdateApiKeyRequest(StreamInput in) throws IOException { super(in); this.roleDescriptors = in.readOptionalCollectionAsList(RoleDescriptor::new); this.metadata = in.readMap(); + if (in.getTransportVersion().onOrAfter(TransportVersions.UPDATE_API_KEY_EXPIRATION_TIME_ADDED)) { + expiration = in.readOptionalTimeValue(); + } else { + expiration = null; + } } public Map getMetadata() { @@ -48,6 +62,10 @@ public List getRoleDescriptors() { return roleDescriptors; } + public TimeValue getExpiration() { + return expiration; + } + public abstract ApiKey.Type getType(); @Override @@ -72,5 +90,8 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeOptionalCollection(roleDescriptors); out.writeGenericMap(metadata); + if (out.getTransportVersion().onOrAfter(TransportVersions.UPDATE_API_KEY_EXPIRATION_TIME_ADDED)) { + out.writeOptionalTimeValue(expiration); + } } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequest.java index d4712abd2cfe2..f915781c6211a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequest.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import java.io.IOException; @@ -19,19 +20,25 @@ public final class BulkUpdateApiKeyRequest extends BaseBulkUpdateApiKeyRequest { public static BulkUpdateApiKeyRequest usingApiKeyIds(String... ids) { - return new BulkUpdateApiKeyRequest(Arrays.stream(ids).toList(), null, null); + return new BulkUpdateApiKeyRequest(Arrays.stream(ids).toList(), null, null, null); } public static BulkUpdateApiKeyRequest wrap(final UpdateApiKeyRequest request) { - return new BulkUpdateApiKeyRequest(List.of(request.getId()), request.getRoleDescriptors(), request.getMetadata()); + return new BulkUpdateApiKeyRequest( + List.of(request.getId()), + request.getRoleDescriptors(), + request.getMetadata(), + request.getExpiration() + ); } public BulkUpdateApiKeyRequest( final List ids, @Nullable final List roleDescriptors, - @Nullable final Map metadata + @Nullable final Map metadata, + @Nullable final TimeValue expiration ) { - super(ids, roleDescriptors, metadata); + super(ids, roleDescriptors, metadata, expiration); } public BulkUpdateApiKeyRequest(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequest.java index 456aa4d636335..c5c8bcc4fc87a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequest.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import java.io.IOException; @@ -17,15 +18,16 @@ public final class UpdateApiKeyRequest extends BaseSingleUpdateApiKeyRequest { public static UpdateApiKeyRequest usingApiKeyId(final String id) { - return new UpdateApiKeyRequest(id, null, null); + return new UpdateApiKeyRequest(id, null, null, null); } public UpdateApiKeyRequest( final String id, @Nullable final List roleDescriptors, - @Nullable final Map metadata + @Nullable final Map metadata, + @Nullable final TimeValue expiration ) { - super(roleDescriptors, metadata, id); + super(roleDescriptors, metadata, expiration, id); } public UpdateApiKeyRequest(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequest.java index ff6bb7da13660..184ce2c521ce0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequest.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequest.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import java.io.IOException; import java.util.List; @@ -22,9 +23,10 @@ public final class UpdateCrossClusterApiKeyRequest extends BaseSingleUpdateApiKe public UpdateCrossClusterApiKeyRequest( final String id, @Nullable CrossClusterApiKeyRoleDescriptorBuilder roleDescriptorBuilder, - @Nullable final Map metadata + @Nullable final Map metadata, + @Nullable TimeValue expiration ) { - super(roleDescriptorBuilder == null ? null : List.of(roleDescriptorBuilder.build()), metadata, id); + super(roleDescriptorBuilder == null ? null : List.of(roleDescriptorBuilder.build()), metadata, expiration, id); } public UpdateCrossClusterApiKeyRequest(StreamInput in) throws IOException { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/ApiKeyTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/ApiKeyTests.java index 9e357915186a5..02bce50ed3483 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/ApiKeyTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/ApiKeyTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.XContentTestUtils; import org.elasticsearch.xcontent.ToXContent; @@ -152,6 +153,10 @@ public static Map randomMetadata() { return randomMetadata == null ? new HashMap<>() : new HashMap<>(randomMetadata); } + public static TimeValue randomFutureExpirationTime() { + return TimeValue.parseTimeValue(randomTimeValue(10, 20, "d", "h", "s", "m"), "expiration"); + } + public static ApiKey randomApiKeyInstance() { final String name = randomAlphaOfLengthBetween(4, 10); final String id = randomAlphaOfLength(20); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestSerializationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestSerializationTests.java new file mode 100644 index 0000000000000..3dd989b3fe074 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestSerializationTests.java @@ -0,0 +1,71 @@ +/* + * 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.security.action.apikey; + +import org.elasticsearch.TransportVersions; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.nullValue; + +public class BulkUpdateApiKeyRequestSerializationTests extends AbstractWireSerializingTestCase { + public void testSerializationBackwardsCompatibility() throws IOException { + BulkUpdateApiKeyRequest testInstance = createTestInstance(); + BulkUpdateApiKeyRequest deserializedInstance = copyInstance(testInstance, TransportVersions.V_8_500_064); + try { + // Transport is on a version before expiration was introduced, so should always be null + assertThat(deserializedInstance.getExpiration(), nullValue()); + } finally { + dispose(deserializedInstance); + } + } + + @Override + protected BulkUpdateApiKeyRequest createTestInstance() { + final boolean roleDescriptorsPresent = randomBoolean(); + final List descriptorList; + if (roleDescriptorsPresent == false) { + descriptorList = null; + } else { + final int numDescriptors = randomIntBetween(0, 4); + descriptorList = new ArrayList<>(); + for (int i = 0; i < numDescriptors; i++) { + descriptorList.add(new RoleDescriptor("role_" + i, new String[] { "all" }, null, null)); + } + } + + final var ids = randomList(randomInt(5), () -> randomAlphaOfLength(10)); + final var metadata = ApiKeyTests.randomMetadata(); + final TimeValue expiration = ApiKeyTests.randomFutureExpirationTime(); + return new BulkUpdateApiKeyRequest(ids, descriptorList, metadata, expiration); + } + + @Override + protected Writeable.Reader instanceReader() { + return BulkUpdateApiKeyRequest::new; + } + + @Override + protected BulkUpdateApiKeyRequest mutateInstance(BulkUpdateApiKeyRequest instance) throws IOException { + Map metadata = ApiKeyTests.randomMetadata(); + long days = randomValueOtherThan(instance.getExpiration().days(), () -> ApiKeyTests.randomFutureExpirationTime().getDays()); + return new BulkUpdateApiKeyRequest( + instance.getIds(), + instance.getRoleDescriptors(), + metadata, + TimeValue.parseTimeValue((days + 1) + "d", null, "expiration") + ); + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestTests.java index 8a0f384daedaa..583b336b3f6eb 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/BulkUpdateApiKeyRequestTests.java @@ -8,13 +8,10 @@ package org.elasticsearch.xpack.core.security.action.apikey; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; -import java.io.IOException; -import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -23,42 +20,13 @@ import static org.hamcrest.Matchers.equalTo; public class BulkUpdateApiKeyRequestTests extends ESTestCase { - - public void testSerialization() throws IOException { - final boolean roleDescriptorsPresent = randomBoolean(); - final List descriptorList; - if (roleDescriptorsPresent == false) { - descriptorList = null; - } else { - final int numDescriptors = randomIntBetween(0, 4); - descriptorList = new ArrayList<>(); - for (int i = 0; i < numDescriptors; i++) { - descriptorList.add(new RoleDescriptor("role_" + i, new String[] { "all" }, null, null)); - } - } - - final List ids = randomList(1, 5, () -> randomAlphaOfLength(10)); - final Map metadata = ApiKeyTests.randomMetadata(); - final var request = new BulkUpdateApiKeyRequest(ids, descriptorList, metadata); - - try (BytesStreamOutput out = new BytesStreamOutput()) { - request.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - final var serialized = new BulkUpdateApiKeyRequest(in); - assertEquals(ids, serialized.getIds()); - assertEquals(descriptorList, serialized.getRoleDescriptors()); - assertEquals(metadata, request.getMetadata()); - } - } - } - public void testNullValuesValidForNonIds() { final var request = BulkUpdateApiKeyRequest.usingApiKeyIds("id"); assertNull(request.validate()); } public void testEmptyIdsNotValid() { - final var request = new BulkUpdateApiKeyRequest(List.of(), null, null); + final var request = new BulkUpdateApiKeyRequest(List.of(), null, null, null); final ActionRequestValidationException ve = request.validate(); assertNotNull(ve); assertThat(ve.validationErrors().size(), equalTo(1)); @@ -68,10 +36,12 @@ public void testEmptyIdsNotValid() { public void testMetadataKeyValidation() { final var reservedKey = "_" + randomAlphaOfLengthBetween(0, 10); final var metadataValue = randomAlphaOfLengthBetween(1, 10); + final TimeValue expiration = ApiKeyTests.randomFutureExpirationTime(); final var request = new BulkUpdateApiKeyRequest( randomList(1, 5, () -> randomAlphaOfLength(10)), null, - Map.of(reservedKey, metadataValue) + Map.of(reservedKey, metadataValue), + expiration ); final ActionRequestValidationException ve = request.validate(); assertNotNull(ve); @@ -103,6 +73,7 @@ public void testRoleDescriptorValidation() { new RoleDescriptor.Restriction(unknownWorkflows) ) ), + null, null ); final ActionRequestValidationException ve = request.validate(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestSerializationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestSerializationTests.java new file mode 100644 index 0000000000000..be1e69d4d30e8 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestSerializationTests.java @@ -0,0 +1,72 @@ +/* + * 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.security.action.apikey; + +import org.elasticsearch.TransportVersions; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.nullValue; + +public class UpdateApiKeyRequestSerializationTests extends AbstractWireSerializingTestCase { + public void testSerializationBackwardsCompatibility() throws IOException { + UpdateApiKeyRequest testInstance = createTestInstance(); + UpdateApiKeyRequest deserializedInstance = copyInstance(testInstance, TransportVersions.V_8_500_064); + try { + // Transport is on a version before expiration was introduced, so should always be null + assertThat(deserializedInstance.getExpiration(), nullValue()); + } finally { + dispose(deserializedInstance); + } + } + + @Override + protected UpdateApiKeyRequest createTestInstance() { + final boolean roleDescriptorsPresent = randomBoolean(); + final List descriptorList; + if (roleDescriptorsPresent == false) { + descriptorList = null; + } else { + final int numDescriptors = randomIntBetween(0, 4); + descriptorList = new ArrayList<>(); + for (int i = 0; i < numDescriptors; i++) { + descriptorList.add(new RoleDescriptor("role_" + i, new String[] { "all" }, null, null)); + } + } + + final var id = randomAlphaOfLength(10); + final var metadata = ApiKeyTests.randomMetadata(); + final TimeValue expiration = ApiKeyTests.randomFutureExpirationTime(); + return new UpdateApiKeyRequest(id, descriptorList, metadata, expiration); + } + + @Override + protected Writeable.Reader instanceReader() { + return UpdateApiKeyRequest::new; + } + + @Override + protected UpdateApiKeyRequest mutateInstance(UpdateApiKeyRequest instance) throws IOException { + Map metadata = ApiKeyTests.randomMetadata(); + long days = randomValueOtherThan(instance.getExpiration().days(), () -> ApiKeyTests.randomFutureExpirationTime().getDays()); + return new UpdateApiKeyRequest( + instance.getId(), + instance.getRoleDescriptors(), + metadata, + TimeValue.parseTimeValue(days + "d", null, "expiration") + ); + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestTests.java index cf4015f6fd4cc..7b85c71c7519f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateApiKeyRequestTests.java @@ -8,13 +8,10 @@ package org.elasticsearch.xpack.core.security.action.apikey; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; import org.elasticsearch.xpack.core.security.authz.restriction.WorkflowResolver; -import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -22,49 +19,19 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsStringIgnoringCase; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; public class UpdateApiKeyRequestTests extends ESTestCase { public void testNullValuesValidForNonIds() { - final var request = new UpdateApiKeyRequest("id", null, null); + final var request = new UpdateApiKeyRequest("id", null, null, null); assertNull(request.validate()); } - public void testSerialization() throws IOException { - final boolean roleDescriptorsPresent = randomBoolean(); - final List descriptorList; - if (roleDescriptorsPresent == false) { - descriptorList = null; - } else { - final int numDescriptors = randomIntBetween(0, 4); - descriptorList = new ArrayList<>(); - for (int i = 0; i < numDescriptors; i++) { - descriptorList.add(new RoleDescriptor("role_" + i, new String[] { "all" }, null, null)); - } - } - - final var id = randomAlphaOfLength(10); - final var metadata = ApiKeyTests.randomMetadata(); - final var request = new UpdateApiKeyRequest(id, descriptorList, metadata); - assertThat(request.getType(), is(ApiKey.Type.REST)); - - try (BytesStreamOutput out = new BytesStreamOutput()) { - request.writeTo(out); - try (StreamInput in = out.bytes().streamInput()) { - final var serialized = new UpdateApiKeyRequest(in); - assertEquals(id, serialized.getId()); - assertEquals(descriptorList, serialized.getRoleDescriptors()); - assertEquals(metadata, serialized.getMetadata()); - assertEquals(request.getType(), serialized.getType()); - } - } - } - public void testMetadataKeyValidation() { final var reservedKey = "_" + randomAlphaOfLengthBetween(0, 10); final var metadataValue = randomAlphaOfLengthBetween(1, 10); - UpdateApiKeyRequest request = new UpdateApiKeyRequest(randomAlphaOfLength(10), null, Map.of(reservedKey, metadataValue)); + + UpdateApiKeyRequest request = new UpdateApiKeyRequest(randomAlphaOfLength(10), null, Map.of(reservedKey, metadataValue), null); final ActionRequestValidationException ve = request.validate(); assertNotNull(ve); assertThat(ve.validationErrors().size(), equalTo(1)); @@ -98,6 +65,7 @@ public void testRoleDescriptorValidation() { new RoleDescriptor.Restriction(workflows.toArray(String[]::new)) ) ), + null, null ); final ActionRequestValidationException ve1 = request1.validate(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequestTests.java index 89a6f5b650b5a..f9faa2731dcc0 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/apikey/UpdateCrossClusterApiKeyRequestTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -25,15 +26,15 @@ public class UpdateCrossClusterApiKeyRequestTests extends ESTestCase { public void testSerialization() throws IOException { final var metadata = ApiKeyTests.randomMetadata(); - + final TimeValue expiration = ApiKeyTests.randomFutureExpirationTime(); final CrossClusterApiKeyRoleDescriptorBuilder roleDescriptorBuilder; - if (metadata == null || randomBoolean()) { + if (randomBoolean()) { roleDescriptorBuilder = CrossClusterApiKeyRoleDescriptorBuilder.parse(randomCrossClusterApiKeyAccessField()); } else { roleDescriptorBuilder = null; } - final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), roleDescriptorBuilder, metadata); + final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), roleDescriptorBuilder, metadata, expiration); assertThat(request.getType(), is(ApiKey.Type.CROSS_CLUSTER)); assertThat(request.validate(), nullValue()); @@ -44,13 +45,14 @@ public void testSerialization() throws IOException { assertEquals(request.getId(), serialized.getId()); assertEquals(request.getRoleDescriptors(), serialized.getRoleDescriptors()); assertEquals(metadata, serialized.getMetadata()); + assertEquals(expiration, serialized.getExpiration()); assertEquals(request.getType(), serialized.getType()); } } } public void testNotEmptyUpdateValidation() { - final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), null, null); + final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), null, null, null); final ActionRequestValidationException ve = request.validate(); assertThat(ve, notNullValue()); assertThat(ve.validationErrors(), contains("must update either [access] or [metadata] for cross-cluster API keys")); @@ -59,7 +61,7 @@ public void testNotEmptyUpdateValidation() { public void testMetadataKeyValidation() { final var reservedKey = "_" + randomAlphaOfLengthBetween(0, 10); final var metadataValue = randomAlphaOfLengthBetween(1, 10); - final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), null, Map.of(reservedKey, metadataValue)); + final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), null, Map.of(reservedKey, metadataValue), null); final ActionRequestValidationException ve = request.validate(); assertThat(ve, notNullValue()); assertThat(ve.validationErrors(), contains("API key metadata keys may not start with [_]")); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/ManageOwnApiKeyClusterPrivilegeTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/ManageOwnApiKeyClusterPrivilegeTests.java index 22e6a6f005919..66ed55fadb734 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/ManageOwnApiKeyClusterPrivilegeTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/ManageOwnApiKeyClusterPrivilegeTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.xpack.core.security.action.apikey.ApiKeyTests; import org.elasticsearch.xpack.core.security.action.apikey.BulkUpdateApiKeyRequest; import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyRequest; import org.elasticsearch.xpack.core.security.action.apikey.CreateCrossClusterApiKeyAction; @@ -88,7 +89,7 @@ public void testAuthenticationForBulkUpdateApiKeyAllowsAll() { .build(); final List apiKeyIds = randomList(1, 5, () -> randomAlphaOfLengthBetween(4, 7)); final Authentication authentication = AuthenticationTestHelper.builder().build(); - final TransportRequest bulkUpdateApiKeyRequest = new BulkUpdateApiKeyRequest(apiKeyIds, null, null); + final TransportRequest bulkUpdateApiKeyRequest = new BulkUpdateApiKeyRequest(apiKeyIds, null, null, null); assertTrue(clusterPermission.check("cluster:admin/xpack/security/api_key/update", bulkUpdateApiKeyRequest, authentication)); } @@ -315,7 +316,8 @@ public void testCheckUpdateCrossClusterApiKeyRequestDenied() { final UpdateCrossClusterApiKeyRequest request = new UpdateCrossClusterApiKeyRequest( randomAlphaOfLengthBetween(4, 7), null, - Map.of() + Map.of(), + ApiKeyTests.randomFutureExpirationTime() ); assertFalse(clusterPermission.check(UpdateCrossClusterApiKeyAction.NAME, request, AuthenticationTestHelper.builder().build())); } diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java index 0e794d6fa533f..032eb0cee223c 100644 --- a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java @@ -130,7 +130,6 @@ private TypeSpec type() { builder.addField(WARNINGS, "warnings", Modifier.PRIVATE, Modifier.FINAL); } - builder.addField(DRIVER_CONTEXT, "driverContext", Modifier.PRIVATE, Modifier.FINAL); builder.addMethod(ctor()); builder.addMethod(name()); @@ -159,12 +158,11 @@ private MethodSpec ctor() { builder.addParameter(SOURCE, "source"); } builder.addParameter(EXPRESSION_EVALUATOR, "field"); - builder.addStatement("super($L)", "field"); + builder.addStatement("super(driverContext, field)"); if (warnExceptions.isEmpty() == false) { builder.addStatement("this.warnings = new Warnings(source)"); } builder.addParameter(DRIVER_CONTEXT, "driverContext"); - builder.addStatement("this.driverContext = driverContext"); return builder.build(); } diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java index d7d0856963019..2cdb7f616e684 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java @@ -50,6 +50,9 @@ private BooleanArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -105,7 +108,7 @@ public BooleanBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); BooleanArrayBlock expanded = new BooleanArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayBlock.java index d75e988fe3a84..57863d2d89dd0 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayBlock.java @@ -51,6 +51,9 @@ private BooleanBigArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -106,7 +109,7 @@ public BooleanBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); BooleanBigArrayBlock expanded = new BooleanBigArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java index b236949e92018..492f99015ff03 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java @@ -53,6 +53,9 @@ private BytesRefArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -109,7 +112,7 @@ public BytesRefBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); BytesRefArrayBlock expanded = new BytesRefArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java index 6089679904e48..58e85b4c05015 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java @@ -50,6 +50,9 @@ private DoubleArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -105,7 +108,7 @@ public DoubleBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); DoubleArrayBlock expanded = new DoubleArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayBlock.java index adc09d5755b53..d7b5b644b721a 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayBlock.java @@ -51,6 +51,9 @@ private DoubleBigArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -106,7 +109,7 @@ public DoubleBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); DoubleBigArrayBlock expanded = new DoubleBigArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java index 1e5ae7a3de448..b33a70525b8f7 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java @@ -50,6 +50,9 @@ private IntArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -105,7 +108,7 @@ public IntBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); IntArrayBlock expanded = new IntArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayBlock.java index 068f550a56d5b..ebb3f470bdcee 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayBlock.java @@ -51,6 +51,9 @@ private IntBigArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -106,7 +109,7 @@ public IntBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); IntBigArrayBlock expanded = new IntBigArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java index 0d8464a95ed5b..5236bf32a5a10 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java @@ -50,6 +50,9 @@ private LongArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -105,7 +108,7 @@ public LongBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); LongArrayBlock expanded = new LongArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayBlock.java index eb1a353352ac9..1d3dde8357fa9 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayBlock.java @@ -51,6 +51,9 @@ private LongBigArrayBlock( ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -106,7 +109,7 @@ public LongBlock expand() { // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); LongBigArrayBlock expanded = new LongBigArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java index fe1ecbec92e5b..d6046f0bda085 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java @@ -52,7 +52,7 @@ public final MvOrdering mvOrdering() { } protected BitSet shiftNullsToExpandedPositions() { - BitSet expanded = new BitSet(getTotalValueCount()); + BitSet expanded = new BitSet(nullsMask.size()); int next = -1; while ((next = nullsMask.nextSetBit(next + 1)) != -1) { expanded.set(getFirstValueIndex(next)); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java index 8ce6ef9ab78ab..0c5207133f71d 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java @@ -31,6 +31,7 @@ protected AbstractBlock(int positionCount, BlockFactory blockFactory) { this.blockFactory = blockFactory; this.firstValueIndexes = null; this.nullsMask = null; + assert assertInvariants(); } /** @@ -43,6 +44,26 @@ protected AbstractBlock(int positionCount, @Nullable int[] firstValueIndexes, @N this.firstValueIndexes = firstValueIndexes; this.nullsMask = nullsMask == null || nullsMask.isEmpty() ? null : nullsMask; assert nullsMask != null || firstValueIndexes != null : "Create VectorBlock instead"; + assert assertInvariants(); + } + + private boolean assertInvariants() { + if (firstValueIndexes != null) { + assert firstValueIndexes.length == getPositionCount() + 1; + for (int i = 0; i < getPositionCount(); i++) { + assert (firstValueIndexes[i + 1] - firstValueIndexes[i]) >= 0; + } + } + if (nullsMask != null) { + assert nullsMask.nextSetBit(getPositionCount() + 1) == -1; + } + if (firstValueIndexes != null && nullsMask != null) { + for (int i = 0; i < getPositionCount(); i++) { + // Either we have multi-values or a null but never both. + assert ((nullsMask.get(i) == false) || (firstValueIndexes[i + 1] - firstValueIndexes[i]) == 1); + } + } + return true; } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockRamUsageEstimator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockRamUsageEstimator.java index d1f1bac940714..93fd02c3cd879 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockRamUsageEstimator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockRamUsageEstimator.java @@ -27,6 +27,8 @@ public static long sizeOfBitSet(@Nullable BitSet bitset) { } public static long sizeOfBitSet(long size) { - return BITSET_BASE_RAM_USAGE + (size / Byte.SIZE); + // BitSet is normally made up of words, represented by longs. So we need to divide and round up. + long wordCount = (size + Long.SIZE - 1) / Long.SIZE; + return BITSET_BASE_RAM_USAGE + wordCount * Long.BYTES; } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st index 245089ff2a83e..e710f51ff939f 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st @@ -61,6 +61,9 @@ final class $Type$ArrayBlock extends AbstractArrayBlock implements $Type$Block { ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -124,7 +127,7 @@ $endif$ // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); $Type$ArrayBlock expanded = new $Type$ArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayBlock.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayBlock.java.st index a5f5001802cb7..1fb9e4ec27b2e 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayBlock.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayBlock.java.st @@ -51,6 +51,9 @@ public final class $Type$BigArrayBlock extends AbstractArrayBlock implements $Ty ) { super(positionCount, firstValueIndexes, nulls, mvOrdering, blockFactory); this.vector = vector; + assert firstValueIndexes == null + ? vector.getPositionCount() == getPositionCount() + : firstValueIndexes[getPositionCount()] == vector.getPositionCount(); } @Override @@ -106,7 +109,7 @@ public final class $Type$BigArrayBlock extends AbstractArrayBlock implements $Ty // The following line is correct because positions with multi-values are never null. int expandedPositionCount = vector.getPositionCount(); - long bitSetRamUsedEstimate = BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount); + long bitSetRamUsedEstimate = Math.max(nullsMask.size(), BlockRamUsageEstimator.sizeOfBitSet(expandedPositionCount)); blockFactory().adjustBreaker(bitSetRamUsedEstimate, false); $Type$BigArrayBlock expanded = new $Type$BigArrayBlock( diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java index 967111a09f564..091506cf3cc4e 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java @@ -11,11 +11,8 @@ import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.query.SearchExecutionContext; -import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SearchLookup; -import java.util.ArrayList; -import java.util.List; import java.util.Set; /** @@ -26,56 +23,46 @@ private BlockReaderFactories() {} /** * Resolves *how* ESQL loads field values. - * @param searchContexts a search context per search index we're loading - * field from + * @param ctx a search context for the index we're loading field from * @param fieldName the name of the field to load * @param asUnsupportedSource should the field be loaded as "unsupported"? * These will always have {@code null} values */ - public static List loaders(List searchContexts, String fieldName, boolean asUnsupportedSource) { - List loaders = new ArrayList<>(searchContexts.size()); - - for (SearchContext searchContext : searchContexts) { - SearchExecutionContext ctx = searchContext.getSearchExecutionContext(); - if (asUnsupportedSource) { - loaders.add(BlockLoader.CONSTANT_NULLS); - continue; - } - MappedFieldType fieldType = ctx.getFieldType(fieldName); - if (fieldType == null) { - // the field does not exist in this context - loaders.add(BlockLoader.CONSTANT_NULLS); - continue; + public static BlockLoader loader(SearchExecutionContext ctx, String fieldName, boolean asUnsupportedSource) { + if (asUnsupportedSource) { + return BlockLoader.CONSTANT_NULLS; + } + MappedFieldType fieldType = ctx.getFieldType(fieldName); + if (fieldType == null) { + // the field does not exist in this context + return BlockLoader.CONSTANT_NULLS; + } + BlockLoader loader = fieldType.blockLoader(new MappedFieldType.BlockLoaderContext() { + @Override + public String indexName() { + return ctx.getFullyQualifiedIndex().getName(); } - BlockLoader loader = fieldType.blockLoader(new MappedFieldType.BlockLoaderContext() { - @Override - public String indexName() { - return ctx.getFullyQualifiedIndex().getName(); - } - @Override - public SearchLookup lookup() { - return ctx.lookup(); - } + @Override + public SearchLookup lookup() { + return ctx.lookup(); + } - @Override - public Set sourcePaths(String name) { - return ctx.sourcePath(name); - } + @Override + public Set sourcePaths(String name) { + return ctx.sourcePath(name); + } - @Override - public String parentField(String field) { - return ctx.parentPath(field); - } - }); - if (loader == null) { - HeaderWarning.addWarning("Field [{}] cannot be retrieved, it is unsupported or not indexed; returning null", fieldName); - loaders.add(BlockLoader.CONSTANT_NULLS); - continue; + @Override + public String parentField(String field) { + return ctx.parentPath(field); } - loaders.add(loader); + }); + if (loader == null) { + HeaderWarning.addWarning("Field [{}] cannot be retrieved, it is unsupported or not indexed; returning null", fieldName); + return BlockLoader.CONSTANT_NULLS; } - return loaders; + return loader; } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java index 826c25f3e7828..b9be899cec4f3 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -27,6 +27,7 @@ import org.elasticsearch.compute.operator.AbstractPageMappingOperator; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.core.Assertions; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; @@ -43,6 +44,7 @@ import java.util.Map; import java.util.Objects; import java.util.TreeMap; +import java.util.function.IntFunction; import java.util.function.Supplier; /** @@ -95,22 +97,25 @@ public String describe() { } } + /** + * Configuration for a field to load. + * + * {@code blockLoader} maps shard index to the {@link BlockLoader}s + * which load the actual blocks. + */ + public record FieldInfo(String name, ElementType type, IntFunction blockLoader) {} + public record ShardContext(IndexReader reader, Supplier newSourceLoader) {} - private final List fields; + private final FieldWork[] fields; private final List shardContexts; private final int docChannel; private final BlockFactory blockFactory; private final Map readersBuilt = new TreeMap<>(); - /** - * Configuration for a field to load. - * - * {@code blockLoaders} is a list, one entry per shard, of - * {@link BlockLoader}s which load the actual blocks. - */ - public record FieldInfo(String name, List blockLoaders) {} + int lastShard = -1; + int lastSegment = -1; /** * Creates a new extractor @@ -118,7 +123,7 @@ public record FieldInfo(String name, List blockLoaders) {} * @param docChannel the channel containing the shard, leaf/segment and doc id */ public ValuesSourceReaderOperator(BlockFactory blockFactory, List fields, List shardContexts, int docChannel) { - this.fields = fields.stream().map(f -> new FieldWork(f)).toList(); + this.fields = fields.stream().map(f -> new FieldWork(f)).toArray(FieldWork[]::new); this.shardContexts = shardContexts; this.docChannel = docChannel; this.blockFactory = blockFactory; @@ -128,13 +133,21 @@ public ValuesSourceReaderOperator(BlockFactory blockFactory, List fie protected Page process(Page page) { DocVector docVector = page.getBlock(docChannel).asVector(); - Block[] blocks = new Block[fields.size()]; + Block[] blocks = new Block[fields.length]; boolean success = false; try { if (docVector.singleSegmentNonDecreasing()) { loadFromSingleLeaf(blocks, docVector); } else { - loadFromManyLeaves(blocks, docVector); + try (LoadFromMany many = new LoadFromMany(blocks, docVector)) { + many.run(); + } + } + if (Assertions.ENABLED) { + for (int f = 0; f < fields.length; f++) { + assert blocks[f].elementType() == ElementType.NULL || blocks[f].elementType() == fields[f].info.type + : blocks[f].elementType() + " NOT IN (NULL, " + fields[f].info.type + ")"; + } } success = true; } catch (IOException e) { @@ -147,10 +160,51 @@ protected Page process(Page page) { return page.appendBlocks(blocks); } + private void positionFieldWork(int shard, int segment, int firstDoc) { + if (lastShard == shard) { + if (lastSegment == segment) { + for (FieldWork w : fields) { + w.sameSegment(firstDoc); + } + return; + } + lastSegment = segment; + for (FieldWork w : fields) { + w.sameShardNewSegment(); + } + return; + } + lastShard = shard; + lastSegment = segment; + for (FieldWork w : fields) { + w.newShard(shard); + } + } + + private boolean positionFieldWorkDocGuarteedAscending(int shard, int segment) { + if (lastShard == shard) { + if (lastSegment == segment) { + return false; + } + lastSegment = segment; + for (FieldWork w : fields) { + w.sameShardNewSegment(); + } + return true; + } + lastShard = shard; + lastSegment = segment; + for (FieldWork w : fields) { + w.newShard(shard); + } + return true; + } + private void loadFromSingleLeaf(Block[] blocks, DocVector docVector) throws IOException { int shard = docVector.shards().getInt(0); int segment = docVector.segments().getInt(0); int firstDoc = docVector.docs().getInt(0); + positionFieldWork(shard, segment, firstDoc); IntVector docs = docVector.docs(); BlockLoader.Docs loaderDocs = new BlockLoader.Docs() { @Override @@ -164,24 +218,24 @@ public int get(int i) { } }; StoredFieldsSpec storedFieldsSpec = StoredFieldsSpec.NO_REQUIREMENTS; - List rowStrideReaders = new ArrayList<>(fields.size()); + List rowStrideReaders = new ArrayList<>(fields.length); ComputeBlockLoaderFactory loaderBlockFactory = new ComputeBlockLoaderFactory(blockFactory, docs.getPositionCount()); + LeafReaderContext ctx = ctx(shard, segment); try { - for (int b = 0; b < fields.size(); b++) { - FieldWork field = fields.get(b); - BlockLoader.ColumnAtATimeReader columnAtATime = field.columnAtATime.reader(shard, segment, firstDoc); + for (int f = 0; f < fields.length; f++) { + FieldWork field = fields[f]; + BlockLoader.ColumnAtATimeReader columnAtATime = field.columnAtATime(ctx); if (columnAtATime != null) { - blocks[b] = (Block) columnAtATime.read(loaderBlockFactory, loaderDocs); + blocks[f] = (Block) columnAtATime.read(loaderBlockFactory, loaderDocs); } else { - BlockLoader.RowStrideReader rowStride = field.rowStride.reader(shard, segment, firstDoc); rowStrideReaders.add( new RowStrideReaderWork( - rowStride, - (Block.Builder) field.info.blockLoaders.get(shard).builder(loaderBlockFactory, docs.getPositionCount()), - b + field.rowStride(ctx), + (Block.Builder) field.loader.builder(loaderBlockFactory, docs.getPositionCount()), + f ) ); - storedFieldsSpec = storedFieldsSpec.merge(field.info.blockLoaders.get(shard).rowStrideStoredFieldSpec()); + storedFieldsSpec = storedFieldsSpec.merge(field.loader.rowStrideStoredFieldSpec()); } } @@ -193,7 +247,6 @@ public int get(int i) { "found row stride readers [" + rowStrideReaders + "] without stored fields [" + storedFieldsSpec + "]" ); } - LeafReaderContext ctx = ctx(shard, segment); StoredFieldLoader storedFieldLoader; if (useSequentialStoredFieldsReader(docVector.docs())) { storedFieldLoader = StoredFieldLoader.fromSpecSequential(storedFieldsSpec); @@ -203,7 +256,6 @@ public int get(int i) { trackStoredFields(storedFieldsSpec, false); } BlockLoaderStoredFieldsFromLeafLoader storedFields = new BlockLoaderStoredFieldsFromLeafLoader( - // TODO enable the optimization by passing non-null to docs if correct storedFieldLoader.getLoader(ctx, null), storedFieldsSpec.requiresSource() ? shardContexts.get(shard).newSourceLoader.get().leaf(ctx.reader(), null) : null ); @@ -226,50 +278,91 @@ public int get(int i) { } } - private void loadFromManyLeaves(Block[] blocks, DocVector docVector) throws IOException { - IntVector shards = docVector.shards(); - IntVector segments = docVector.segments(); - IntVector docs = docVector.docs(); - Block.Builder[] builders = new Block.Builder[blocks.length]; - int[] forwards = docVector.shardSegmentDocMapForwards(); - ComputeBlockLoaderFactory loaderBlockFactory = new ComputeBlockLoaderFactory(blockFactory, docs.getPositionCount()); - try { - for (int b = 0; b < fields.size(); b++) { - FieldWork field = fields.get(b); - builders[b] = builderFromFirstNonNull(loaderBlockFactory, field, docs.getPositionCount()); + private class LoadFromMany implements Releasable { + private final Block[] target; + private final IntVector shards; + private final IntVector segments; + private final IntVector docs; + private final int[] forwards; + private final int[] backwards; + private final Block.Builder[] builders; + private final BlockLoader.RowStrideReader[] rowStride; + + BlockLoaderStoredFieldsFromLeafLoader storedFields; + + LoadFromMany(Block[] target, DocVector docVector) { + this.target = target; + shards = docVector.shards(); + segments = docVector.segments(); + docs = docVector.docs(); + forwards = docVector.shardSegmentDocMapForwards(); + backwards = docVector.shardSegmentDocMapBackwards(); + builders = new Block.Builder[target.length]; + rowStride = new BlockLoader.RowStrideReader[target.length]; + } + + void run() throws IOException { + for (int f = 0; f < fields.length; f++) { + /* + * Important note: each block loader has a method to build an + * optimized block loader, but we have *many* fields and some + * of those block loaders may not be compatible with each other. + * So! We take the least common denominator which is the loader + * from the element expected element type. + */ + builders[f] = fields[f].info.type.newBlockBuilder(docs.getPositionCount(), blockFactory); } - int lastShard = -1; - int lastSegment = -1; - BlockLoaderStoredFieldsFromLeafLoader storedFields = null; - for (int i = 0; i < forwards.length; i++) { - int p = forwards[i]; - int shard = shards.getInt(p); - int segment = segments.getInt(p); - int doc = docs.getInt(p); - if (shard != lastShard || segment != lastSegment) { - lastShard = shard; - lastSegment = segment; - StoredFieldsSpec storedFieldsSpec = storedFieldsSpecForShard(shard); - LeafReaderContext ctx = ctx(shard, segment); - storedFields = new BlockLoaderStoredFieldsFromLeafLoader( - StoredFieldLoader.fromSpec(storedFieldsSpec).getLoader(ctx, null), - storedFieldsSpec.requiresSource() ? shardContexts.get(shard).newSourceLoader.get().leaf(ctx.reader(), null) : null - ); - if (false == storedFieldsSpec.equals(StoredFieldsSpec.NO_REQUIREMENTS)) { - trackStoredFields(storedFieldsSpec, false); - } + int p = forwards[0]; + int shard = shards.getInt(p); + int segment = segments.getInt(p); + int firstDoc = docs.getInt(p); + positionFieldWork(shard, segment, firstDoc); + LeafReaderContext ctx = ctx(shard, segment); + fieldsMoved(ctx, shard); + read(firstDoc); + for (int i = 1; i < forwards.length; i++) { + p = forwards[i]; + shard = shards.getInt(p); + segment = segments.getInt(p); + boolean changedSegment = positionFieldWorkDocGuarteedAscending(shard, segment); + if (changedSegment) { + ctx = ctx(shard, segment); + fieldsMoved(ctx, shard); } - storedFields.advanceTo(doc); - for (int r = 0; r < blocks.length; r++) { - fields.get(r).rowStride.reader(shard, segment, doc).read(doc, storedFields, builders[r]); + read(docs.getInt(p)); + } + for (int f = 0; f < builders.length; f++) { + try (Block orig = builders[f].build()) { + target[f] = orig.filter(backwards); } } - for (int r = 0; r < blocks.length; r++) { - try (Block orig = builders[r].build()) { - blocks[r] = orig.filter(docVector.shardSegmentDocMapBackwards()); + } + + private void fieldsMoved(LeafReaderContext ctx, int shard) throws IOException { + StoredFieldsSpec storedFieldsSpec = StoredFieldsSpec.NO_REQUIREMENTS; + for (int f = 0; f < fields.length; f++) { + FieldWork field = fields[f]; + rowStride[f] = field.rowStride(ctx); + storedFieldsSpec = storedFieldsSpec.merge(field.loader.rowStrideStoredFieldSpec()); + storedFields = new BlockLoaderStoredFieldsFromLeafLoader( + StoredFieldLoader.fromSpec(storedFieldsSpec).getLoader(ctx, null), + storedFieldsSpec.requiresSource() ? shardContexts.get(shard).newSourceLoader.get().leaf(ctx.reader(), null) : null + ); + if (false == storedFieldsSpec.equals(StoredFieldsSpec.NO_REQUIREMENTS)) { + trackStoredFields(storedFieldsSpec, false); } } - } finally { + } + + private void read(int doc) throws IOException { + storedFields.advanceTo(doc); + for (int f = 0; f < builders.length; f++) { + rowStride[f].read(doc, storedFields, builders[f]); + } + } + + @Override + public void close() { Releasables.closeExpectNoException(builders); } } @@ -298,83 +391,55 @@ private void trackStoredFields(StoredFieldsSpec spec, boolean sequential) { ); } - /** - * Returns a builder from the first non - {@link BlockLoader#CONSTANT_NULLS} loader - * in the list. If they are all the null loader then returns a null builder. - */ - private Block.Builder builderFromFirstNonNull(BlockLoader.BlockFactory loaderBlockFactory, FieldWork field, int positionCount) { - for (BlockLoader loader : field.info.blockLoaders) { - if (loader != BlockLoader.CONSTANT_NULLS) { - return (Block.Builder) loader.builder(loaderBlockFactory, positionCount); - } - } - // All null, just let the first one build the null block loader. - return (Block.Builder) field.info.blockLoaders.get(0).builder(loaderBlockFactory, positionCount); - } - - private StoredFieldsSpec storedFieldsSpecForShard(int shard) { - StoredFieldsSpec storedFieldsSpec = StoredFieldsSpec.NO_REQUIREMENTS; - for (int b = 0; b < fields.size(); b++) { - FieldWork field = fields.get(b); - storedFieldsSpec = storedFieldsSpec.merge(field.info.blockLoaders.get(shard).rowStrideStoredFieldSpec()); - } - return storedFieldsSpec; - } - private class FieldWork { final FieldInfo info; - final GuardedReader columnAtATime = new GuardedReader<>() { - @Override - BlockLoader.ColumnAtATimeReader build(BlockLoader loader, LeafReaderContext ctx) throws IOException { - return loader.columnAtATimeReader(ctx); - } - @Override - String type() { - return "column_at_a_time"; - } - }; + BlockLoader loader; + BlockLoader.ColumnAtATimeReader columnAtATime; + BlockLoader.RowStrideReader rowStride; - final GuardedReader rowStride = new GuardedReader<>() { - @Override - BlockLoader.RowStrideReader build(BlockLoader loader, LeafReaderContext ctx) throws IOException { - return loader.rowStrideReader(ctx); - } + FieldWork(FieldInfo info) { + this.info = info; + } - @Override - String type() { - return "row_stride"; + void sameSegment(int firstDoc) { + if (columnAtATime != null && columnAtATime.canReuse(firstDoc) == false) { + columnAtATime = null; } - }; + if (rowStride != null && rowStride.canReuse(firstDoc) == false) { + rowStride = null; + } + } - FieldWork(FieldInfo info) { - this.info = info; + void sameShardNewSegment() { + columnAtATime = null; + rowStride = null; } - private abstract class GuardedReader { - private int lastShard = -1; - private int lastSegment = -1; - V lastReader; + void newShard(int shard) { + loader = info.blockLoader.apply(shard); + columnAtATime = null; + rowStride = null; + } - V reader(int shard, int segment, int startingDocId) throws IOException { - if (lastShard == shard && lastSegment == segment) { - if (lastReader == null) { - return null; - } - if (lastReader.canReuse(startingDocId)) { - return lastReader; - } - } - lastShard = shard; - lastSegment = segment; - lastReader = build(info.blockLoaders.get(shard), ctx(shard, segment)); - readersBuilt.merge(info.name + ":" + type() + ":" + lastReader, 1, (prev, one) -> prev + one); - return lastReader; + BlockLoader.ColumnAtATimeReader columnAtATime(LeafReaderContext ctx) throws IOException { + if (columnAtATime == null) { + columnAtATime = loader.columnAtATimeReader(ctx); + trackReader("column_at_a_time", this.columnAtATime); } + return columnAtATime; + } - abstract V build(BlockLoader loader, LeafReaderContext ctx) throws IOException; + BlockLoader.RowStrideReader rowStride(LeafReaderContext ctx) throws IOException { + if (rowStride == null) { + rowStride = loader.rowStrideReader(ctx); + trackReader("row_stride", this.rowStride); + } + return rowStride; + } - abstract String type(); + private void trackReader(String type, BlockLoader.Reader reader) { + readersBuilt.merge(info.name + ":" + type + ":" + reader, 1, (prev, one) -> prev + one); } } @@ -393,7 +458,7 @@ private LeafReaderContext ctx(int shard, int segment) { public String toString() { StringBuilder sb = new StringBuilder(); sb.append("ValuesSourceReaderOperator[fields = ["); - if (fields.size() < 10) { + if (fields.length < 10) { boolean first = true; for (FieldWork f : fields) { if (first) { @@ -404,7 +469,7 @@ public String toString() { sb.append(f.info.name); } } else { - sb.append(fields.size()).append(" fields"); + sb.append(fields.length).append(" fields"); } return sb.append("]]").toString(); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java index a895525add46f..da60044b5e036 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.IntFunction; import java.util.function.Supplier; import static java.util.Objects.requireNonNull; @@ -52,7 +53,7 @@ */ public class OrdinalsGroupingOperator implements Operator { public record OrdinalsGroupingOperatorFactory( - List blockLoaders, + IntFunction blockLoaders, List shardContexts, ElementType groupingElementType, int docChannel, @@ -83,7 +84,7 @@ public String describe() { } } - private final List blockLoaders; + private final IntFunction blockLoaders; private final List shardContexts; private final int docChannel; private final String groupingField; @@ -102,7 +103,7 @@ public String describe() { private ValuesAggregator valuesAggregator; public OrdinalsGroupingOperator( - List blockLoaders, + IntFunction blockLoaders, List shardContexts, ElementType groupingElementType, int docChannel, @@ -136,7 +137,7 @@ public void addInput(Page page) { requireNonNull(page, "page is null"); DocVector docVector = page.getBlock(docChannel).asVector(); final int shardIndex = docVector.shards().getInt(0); - final var blockLoader = blockLoaders.get(shardIndex); + final var blockLoader = blockLoaders.apply(shardIndex); boolean pagePassed = false; try { if (docVector.singleSegmentNonDecreasing() && blockLoader.supportsOrdinals()) { @@ -464,7 +465,7 @@ private static class ValuesAggregator implements Releasable { private final HashAggregationOperator aggregator; ValuesAggregator( - List blockLoaders, + IntFunction blockLoaders, List shardContexts, ElementType groupingElementType, int docChannel, @@ -476,7 +477,7 @@ private static class ValuesAggregator implements Releasable { ) { this.extractor = new ValuesSourceReaderOperator( driverContext.blockFactory(), - List.of(new ValuesSourceReaderOperator.FieldInfo(groupingField, blockLoaders)), + List.of(new ValuesSourceReaderOperator.FieldInfo(groupingField, groupingElementType, blockLoaders)), shardContexts, docChannel ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java index 9008c9a257877..5ef4bdc9e0188 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java @@ -228,7 +228,7 @@ public String toString() { } }, new OrdinalsGroupingOperator( - List.of(new KeywordFieldMapper.KeywordFieldType("g").blockLoader(null)), + shardIdx -> new KeywordFieldMapper.KeywordFieldType("g").blockLoader(null), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), ElementType.BYTES_REF, 0, @@ -347,7 +347,7 @@ public static void assertDriverContext(DriverContext driverContext) { } static LuceneOperator.Factory luceneOperatorFactory(IndexReader reader, Query query, int limit) { - final SearchContext searchContext = mockSearchContext(reader); + final SearchContext searchContext = mockSearchContext(reader, 0); return new LuceneSourceOperator.Factory( List.of(searchContext), ctx -> query, diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java index 9acf188a4010d..9a8b7a4b9941a 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneCountOperatorTests.java @@ -83,7 +83,7 @@ private LuceneCountOperator.Factory simple(BigArrays bigArrays, DataPartitioning throw new RuntimeException(e); } - SearchContext ctx = LuceneSourceOperatorTests.mockSearchContext(reader); + SearchContext ctx = LuceneSourceOperatorTests.mockSearchContext(reader, 0); when(ctx.getSearchExecutionContext().getIndexReader()).thenReturn(reader); final Query query; if (enableShortcut && randomBoolean()) { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java index 74e9d7b122718..6c449ba9aab4f 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java @@ -18,6 +18,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.AnyOperatorTestCase; @@ -96,7 +97,7 @@ private LuceneSourceOperator.Factory simple(BigArrays bigArrays, DataPartitionin throw new RuntimeException(e); } - SearchContext ctx = mockSearchContext(reader); + SearchContext ctx = mockSearchContext(reader, 0); when(ctx.getSearchExecutionContext().getFieldType(anyString())).thenAnswer(inv -> { String name = inv.getArgument(0); return switch (name) { @@ -176,7 +177,7 @@ public void testShardDataPartitioningWithCranky() { private void testSimple(DriverContext ctx, int size, int limit) { LuceneSourceOperator.Factory factory = simple(ctx.bigArrays(), DataPartitioning.SHARD, size, limit); - Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD, ElementType.LONG); List results = new ArrayList<>(); @@ -204,7 +205,7 @@ private void testSimple(DriverContext ctx, int size, int limit) { * Creates a mock search context with the given index reader. * The returned mock search context can be used to test with {@link LuceneOperator}. */ - public static SearchContext mockSearchContext(IndexReader reader) { + public static SearchContext mockSearchContext(IndexReader reader, int shardId) { try { ContextIndexSearcher searcher = new ContextIndexSearcher( reader, @@ -218,7 +219,7 @@ public static SearchContext mockSearchContext(IndexReader reader) { SearchExecutionContext searchExecutionContext = mock(SearchExecutionContext.class); when(searchContext.getSearchExecutionContext()).thenReturn(searchExecutionContext); when(searchExecutionContext.getFullyQualifiedIndex()).thenReturn(new Index("test", "uid")); - when(searchExecutionContext.getShardId()).thenReturn(0); + when(searchExecutionContext.getShardId()).thenReturn(shardId); return searchContext; } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java index 445e3e0f80264..8bb3d235f3599 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java @@ -17,6 +17,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.AnyOperatorTestCase; @@ -87,7 +88,7 @@ private LuceneTopNSourceOperator.Factory simple(BigArrays bigArrays, DataPartiti throw new RuntimeException(e); } - SearchContext ctx = LuceneSourceOperatorTests.mockSearchContext(reader); + SearchContext ctx = LuceneSourceOperatorTests.mockSearchContext(reader, 0); when(ctx.getSearchExecutionContext().getFieldType(anyString())).thenAnswer(inv -> { String name = inv.getArgument(0); return switch (name) { @@ -173,7 +174,7 @@ private void testEmpty(DriverContext context) { private void testSimple(DriverContext ctx, int size, int limit) { LuceneTopNSourceOperator.Factory factory = simple(ctx.bigArrays(), DataPartitioning.SHARD, size, limit); - Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD, ElementType.LONG); List results = new ArrayList<>(); OperatorTestCase.runDriver( diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java index f917056a03026..d6d66c7a30e33 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; @@ -69,12 +70,14 @@ import org.elasticsearch.index.mapper.TextFieldMapper; import org.elasticsearch.index.mapper.TextSearchInfo; import org.elasticsearch.index.mapper.TsidExtractingIdFieldMapper; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; import org.hamcrest.Matcher; import org.junit.After; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -82,6 +85,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.elasticsearch.compute.lucene.LuceneSourceOperatorTests.mockSearchContext; @@ -129,19 +135,20 @@ protected Operator.OperatorFactory simple(BigArrays bigArrays) { throw new RuntimeException(e); } } - return factory(reader, docValuesNumberField("long", NumberFieldMapper.NumberType.LONG)); + return factory(reader, docValuesNumberField("long", NumberFieldMapper.NumberType.LONG), ElementType.LONG); } - static Operator.OperatorFactory factory(IndexReader reader, MappedFieldType ft) { - return factory(reader, ft.name(), ft.blockLoader(null)); + static Operator.OperatorFactory factory(IndexReader reader, MappedFieldType ft, ElementType elementType) { + return factory(reader, ft.name(), elementType, ft.blockLoader(null)); } - static Operator.OperatorFactory factory(IndexReader reader, String name, BlockLoader loader) { - return new ValuesSourceReaderOperator.Factory( - List.of(new ValuesSourceReaderOperator.FieldInfo(name, List.of(loader))), - List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), - 0 - ); + static Operator.OperatorFactory factory(IndexReader reader, String name, ElementType elementType, BlockLoader loader) { + return new ValuesSourceReaderOperator.Factory(List.of(new ValuesSourceReaderOperator.FieldInfo(name, elementType, shardIdx -> { + if (shardIdx != 0) { + fail("unexpected shardIdx [" + shardIdx + "]"); + } + return loader; + })), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), 0); } @Override @@ -160,7 +167,7 @@ private SourceOperator simpleInput(DriverContext context, int size, int commitEv throw new RuntimeException(e); } var luceneFactory = new LuceneSourceOperator.Factory( - List.of(mockSearchContext(reader)), + List.of(mockSearchContext(reader, 0)), ctx -> new MatchAllDocsQuery(), DataPartitioning.SHARD, randomIntBetween(1, 10), @@ -172,6 +179,10 @@ private SourceOperator simpleInput(DriverContext context, int size, int commitEv private void initIndex(int size, int commitEvery) throws IOException { keyToTags.clear(); + reader = initIndex(directory, size, commitEvery); + } + + private IndexReader initIndex(Directory directory, int size, int commitEvery) throws IOException { try ( IndexWriter writer = new IndexWriter( directory, @@ -240,7 +251,7 @@ private void initIndex(int size, int commitEvery) throws IOException { } } } - reader = DirectoryReader.open(directory); + return DirectoryReader.open(directory); } @Override @@ -308,12 +319,13 @@ public void testManySingleDocPages() { Checks checks = new Checks(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); FieldCase testCase = new FieldCase( new KeywordFieldMapper.KeywordFieldType("kwd"), + ElementType.BYTES_REF, checks::tags, StatusChecks::keywordsFromDocValues ); operators.add( new ValuesSourceReaderOperator.Factory( - List.of(testCase.info, fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER))), + List.of(testCase.info, fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER), ElementType.INT)), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), 0 ).get(driverContext) @@ -356,8 +368,17 @@ public void testLoadAllInOnePageShuffled() { loadSimpleAndAssert(driverContext, List.of(source), Block.MvOrdering.UNORDERED); } - private static ValuesSourceReaderOperator.FieldInfo fieldInfo(MappedFieldType ft) { - return new ValuesSourceReaderOperator.FieldInfo(ft.name(), List.of(ft.blockLoader(new MappedFieldType.BlockLoaderContext() { + private static ValuesSourceReaderOperator.FieldInfo fieldInfo(MappedFieldType ft, ElementType elementType) { + return new ValuesSourceReaderOperator.FieldInfo(ft.name(), elementType, shardIdx -> { + if (shardIdx != 0) { + fail("unexpected shardIdx [" + shardIdx + "]"); + } + return ft.blockLoader(blContext()); + }); + } + + private static MappedFieldType.BlockLoaderContext blContext() { + return new MappedFieldType.BlockLoaderContext() { @Override public String indexName() { return "test_index"; @@ -377,7 +398,7 @@ public Set sourcePaths(String name) { public String parentField(String field) { return null; } - }))); + }; } private void loadSimpleAndAssert(DriverContext driverContext, List input, Block.MvOrdering docValuesMvOrdering) { @@ -386,7 +407,7 @@ private void loadSimpleAndAssert(DriverContext driverContext, List input, List operators = new ArrayList<>(); operators.add( new ValuesSourceReaderOperator.Factory( - List.of(fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER))), + List.of(fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER), ElementType.INT)), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), 0 ).get(driverContext) @@ -439,13 +460,14 @@ interface CheckReadersWithName { } record FieldCase(ValuesSourceReaderOperator.FieldInfo info, CheckResults checkResults, CheckReadersWithName checkReaders) { - FieldCase(MappedFieldType ft, CheckResults checkResults, CheckReadersWithName checkReaders) { - this(fieldInfo(ft), checkResults, checkReaders); + FieldCase(MappedFieldType ft, ElementType elementType, CheckResults checkResults, CheckReadersWithName checkReaders) { + this(fieldInfo(ft, elementType), checkResults, checkReaders); } - FieldCase(MappedFieldType ft, CheckResults checkResults, CheckReaders checkReaders) { + FieldCase(MappedFieldType ft, ElementType elementType, CheckResults checkResults, CheckReaders checkReaders) { this( ft, + elementType, checkResults, (name, forcedRowByRow, pageCount, segmentCount, readersBuilt) -> checkReaders.check( forcedRowByRow, @@ -506,11 +528,17 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd Checks checks = new Checks(docValuesMvOrdering); List r = new ArrayList<>(); r.add( - new FieldCase(docValuesNumberField("long", NumberFieldMapper.NumberType.LONG), checks::longs, StatusChecks::longsFromDocValues) + new FieldCase( + docValuesNumberField("long", NumberFieldMapper.NumberType.LONG), + ElementType.LONG, + checks::longs, + StatusChecks::longsFromDocValues + ) ); r.add( new FieldCase( docValuesNumberField("mv_long", NumberFieldMapper.NumberType.LONG), + ElementType.LONG, checks::mvLongsFromDocValues, StatusChecks::mvLongsFromDocValues ) @@ -518,26 +546,39 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("missing_long", NumberFieldMapper.NumberType.LONG), + ElementType.LONG, checks::constantNulls, StatusChecks::constantNulls ) ); r.add( - new FieldCase(sourceNumberField("source_long", NumberFieldMapper.NumberType.LONG), checks::longs, StatusChecks::longsFromSource) + new FieldCase( + sourceNumberField("source_long", NumberFieldMapper.NumberType.LONG), + ElementType.LONG, + checks::longs, + StatusChecks::longsFromSource + ) ); r.add( new FieldCase( sourceNumberField("mv_source_long", NumberFieldMapper.NumberType.LONG), + ElementType.LONG, checks::mvLongsUnordered, StatusChecks::mvLongsFromSource ) ); r.add( - new FieldCase(docValuesNumberField("int", NumberFieldMapper.NumberType.INTEGER), checks::ints, StatusChecks::intsFromDocValues) + new FieldCase( + docValuesNumberField("int", NumberFieldMapper.NumberType.INTEGER), + ElementType.INT, + checks::ints, + StatusChecks::intsFromDocValues + ) ); r.add( new FieldCase( docValuesNumberField("mv_int", NumberFieldMapper.NumberType.INTEGER), + ElementType.INT, checks::mvIntsFromDocValues, StatusChecks::mvIntsFromDocValues ) @@ -545,16 +586,23 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("missing_int", NumberFieldMapper.NumberType.INTEGER), + ElementType.INT, checks::constantNulls, StatusChecks::constantNulls ) ); r.add( - new FieldCase(sourceNumberField("source_int", NumberFieldMapper.NumberType.INTEGER), checks::ints, StatusChecks::intsFromSource) + new FieldCase( + sourceNumberField("source_int", NumberFieldMapper.NumberType.INTEGER), + ElementType.INT, + checks::ints, + StatusChecks::intsFromSource + ) ); r.add( new FieldCase( sourceNumberField("mv_source_int", NumberFieldMapper.NumberType.INTEGER), + ElementType.INT, checks::mvIntsUnordered, StatusChecks::mvIntsFromSource ) @@ -562,6 +610,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("short", NumberFieldMapper.NumberType.SHORT), + ElementType.INT, checks::shorts, StatusChecks::shortsFromDocValues ) @@ -569,6 +618,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("mv_short", NumberFieldMapper.NumberType.SHORT), + ElementType.INT, checks::mvShorts, StatusChecks::mvShortsFromDocValues ) @@ -576,16 +626,23 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("missing_short", NumberFieldMapper.NumberType.SHORT), + ElementType.INT, checks::constantNulls, StatusChecks::constantNulls ) ); r.add( - new FieldCase(docValuesNumberField("byte", NumberFieldMapper.NumberType.BYTE), checks::bytes, StatusChecks::bytesFromDocValues) + new FieldCase( + docValuesNumberField("byte", NumberFieldMapper.NumberType.BYTE), + ElementType.INT, + checks::bytes, + StatusChecks::bytesFromDocValues + ) ); r.add( new FieldCase( docValuesNumberField("mv_byte", NumberFieldMapper.NumberType.BYTE), + ElementType.INT, checks::mvBytes, StatusChecks::mvBytesFromDocValues ) @@ -593,6 +650,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("missing_byte", NumberFieldMapper.NumberType.BYTE), + ElementType.INT, checks::constantNulls, StatusChecks::constantNulls ) @@ -600,6 +658,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("double", NumberFieldMapper.NumberType.DOUBLE), + ElementType.DOUBLE, checks::doubles, StatusChecks::doublesFromDocValues ) @@ -607,6 +666,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("mv_double", NumberFieldMapper.NumberType.DOUBLE), + ElementType.DOUBLE, checks::mvDoubles, StatusChecks::mvDoublesFromDocValues ) @@ -614,39 +674,106 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( docValuesNumberField("missing_double", NumberFieldMapper.NumberType.DOUBLE), + ElementType.DOUBLE, checks::constantNulls, StatusChecks::constantNulls ) ); - r.add(new FieldCase(new BooleanFieldMapper.BooleanFieldType("bool"), checks::bools, StatusChecks::boolFromDocValues)); - r.add(new FieldCase(new BooleanFieldMapper.BooleanFieldType("mv_bool"), checks::mvBools, StatusChecks::mvBoolFromDocValues)); - r.add(new FieldCase(new BooleanFieldMapper.BooleanFieldType("missing_bool"), checks::constantNulls, StatusChecks::constantNulls)); - r.add(new FieldCase(new KeywordFieldMapper.KeywordFieldType("kwd"), checks::tags, StatusChecks::keywordsFromDocValues)); + r.add( + new FieldCase( + new BooleanFieldMapper.BooleanFieldType("bool"), + ElementType.BOOLEAN, + checks::bools, + StatusChecks::boolFromDocValues + ) + ); + r.add( + new FieldCase( + new BooleanFieldMapper.BooleanFieldType("mv_bool"), + ElementType.BOOLEAN, + checks::mvBools, + StatusChecks::mvBoolFromDocValues + ) + ); + r.add( + new FieldCase( + new BooleanFieldMapper.BooleanFieldType("missing_bool"), + ElementType.BOOLEAN, + checks::constantNulls, + StatusChecks::constantNulls + ) + ); + r.add( + new FieldCase( + new KeywordFieldMapper.KeywordFieldType("kwd"), + ElementType.BYTES_REF, + checks::tags, + StatusChecks::keywordsFromDocValues + ) + ); r.add( new FieldCase( new KeywordFieldMapper.KeywordFieldType("mv_kwd"), + ElementType.BYTES_REF, checks::mvStringsFromDocValues, StatusChecks::mvKeywordsFromDocValues ) ); - r.add(new FieldCase(new KeywordFieldMapper.KeywordFieldType("missing_kwd"), checks::constantNulls, StatusChecks::constantNulls)); - r.add(new FieldCase(storedKeywordField("stored_kwd"), checks::strings, StatusChecks::keywordsFromStored)); - r.add(new FieldCase(storedKeywordField("mv_stored_kwd"), checks::mvStringsUnordered, StatusChecks::mvKeywordsFromStored)); - r.add(new FieldCase(sourceKeywordField("source_kwd"), checks::strings, StatusChecks::keywordsFromSource)); - r.add(new FieldCase(sourceKeywordField("mv_source_kwd"), checks::mvStringsUnordered, StatusChecks::mvKeywordsFromSource)); - r.add(new FieldCase(new TextFieldMapper.TextFieldType("source_text", false), checks::strings, StatusChecks::textFromSource)); + r.add( + new FieldCase( + new KeywordFieldMapper.KeywordFieldType("missing_kwd"), + ElementType.BYTES_REF, + checks::constantNulls, + StatusChecks::constantNulls + ) + ); + r.add(new FieldCase(storedKeywordField("stored_kwd"), ElementType.BYTES_REF, checks::strings, StatusChecks::keywordsFromStored)); + r.add( + new FieldCase( + storedKeywordField("mv_stored_kwd"), + ElementType.BYTES_REF, + checks::mvStringsUnordered, + StatusChecks::mvKeywordsFromStored + ) + ); + r.add(new FieldCase(sourceKeywordField("source_kwd"), ElementType.BYTES_REF, checks::strings, StatusChecks::keywordsFromSource)); + r.add( + new FieldCase( + sourceKeywordField("mv_source_kwd"), + ElementType.BYTES_REF, + checks::mvStringsUnordered, + StatusChecks::mvKeywordsFromSource + ) + ); + r.add( + new FieldCase( + new TextFieldMapper.TextFieldType("source_text", false), + ElementType.BYTES_REF, + checks::strings, + StatusChecks::textFromSource + ) + ); r.add( new FieldCase( new TextFieldMapper.TextFieldType("mv_source_text", false), + ElementType.BYTES_REF, checks::mvStringsUnordered, StatusChecks::mvTextFromSource ) ); - r.add(new FieldCase(storedTextField("stored_text"), checks::strings, StatusChecks::textFromStored)); - r.add(new FieldCase(storedTextField("mv_stored_text"), checks::mvStringsUnordered, StatusChecks::mvTextFromStored)); + r.add(new FieldCase(storedTextField("stored_text"), ElementType.BYTES_REF, checks::strings, StatusChecks::textFromStored)); + r.add( + new FieldCase( + storedTextField("mv_stored_text"), + ElementType.BYTES_REF, + checks::mvStringsUnordered, + StatusChecks::mvTextFromStored + ) + ); r.add( new FieldCase( textFieldWithDelegate("text_with_delegate", new KeywordFieldMapper.KeywordFieldType("kwd")), + ElementType.BYTES_REF, checks::tags, StatusChecks::textWithDelegate ) @@ -654,6 +781,7 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( textFieldWithDelegate("mv_text_with_delegate", new KeywordFieldMapper.KeywordFieldType("mv_kwd")), + ElementType.BYTES_REF, checks::mvStringsFromDocValues, StatusChecks::mvTextWithDelegate ) @@ -661,22 +789,27 @@ private List infoAndChecksForEachType(Block.MvOrdering docValuesMvOrd r.add( new FieldCase( textFieldWithDelegate("missing_text_with_delegate", new KeywordFieldMapper.KeywordFieldType("missing_kwd")), + ElementType.BYTES_REF, checks::constantNulls, StatusChecks::constantNullTextWithDelegate ) ); - r.add(new FieldCase(new ProvidedIdFieldMapper(() -> false).fieldType(), checks::ids, StatusChecks::id)); - r.add(new FieldCase(TsidExtractingIdFieldMapper.INSTANCE.fieldType(), checks::ids, StatusChecks::id)); + r.add(new FieldCase(new ProvidedIdFieldMapper(() -> false).fieldType(), ElementType.BYTES_REF, checks::ids, StatusChecks::id)); + r.add(new FieldCase(TsidExtractingIdFieldMapper.INSTANCE.fieldType(), ElementType.BYTES_REF, checks::ids, StatusChecks::id)); r.add( new FieldCase( - new ValuesSourceReaderOperator.FieldInfo("constant_bytes", List.of(BlockLoader.constantBytes(new BytesRef("foo")))), + new ValuesSourceReaderOperator.FieldInfo( + "constant_bytes", + ElementType.BYTES_REF, + shardIdx -> BlockLoader.constantBytes(new BytesRef("foo")) + ), checks::constantBytes, StatusChecks::constantBytes ) ); r.add( new FieldCase( - new ValuesSourceReaderOperator.FieldInfo("null", List.of(BlockLoader.CONSTANT_NULLS)), + new ValuesSourceReaderOperator.FieldInfo("null", ElementType.NULL, shardIdx -> BlockLoader.CONSTANT_NULLS), checks::constantNulls, StatusChecks::constantNulls ) @@ -1149,7 +1282,7 @@ public void testWithNulls() throws IOException { DriverContext driverContext = driverContext(); var luceneFactory = new LuceneSourceOperator.Factory( - List.of(mockSearchContext(reader)), + List.of(mockSearchContext(reader, 0)), ctx -> new MatchAllDocsQuery(), randomFrom(DataPartitioning.values()), randomIntBetween(1, 10), @@ -1161,10 +1294,10 @@ public void testWithNulls() throws IOException { driverContext, luceneFactory.get(driverContext), List.of( - factory(reader, intFt).get(driverContext), - factory(reader, longFt).get(driverContext), - factory(reader, doubleFt).get(driverContext), - factory(reader, kwFt).get(driverContext) + factory(reader, intFt, ElementType.INT).get(driverContext), + factory(reader, longFt, ElementType.LONG).get(driverContext), + factory(reader, doubleFt, ElementType.DOUBLE).get(driverContext), + factory(reader, kwFt, ElementType.BYTES_REF).get(driverContext) ), new PageConsumerOperator(page -> { try { @@ -1286,8 +1419,8 @@ public void testNullsShared() { List.of( new ValuesSourceReaderOperator.Factory( List.of( - new ValuesSourceReaderOperator.FieldInfo("null1", List.of(BlockLoader.CONSTANT_NULLS)), - new ValuesSourceReaderOperator.FieldInfo("null2", List.of(BlockLoader.CONSTANT_NULLS)) + new ValuesSourceReaderOperator.FieldInfo("null1", ElementType.NULL, shardIdx -> BlockLoader.CONSTANT_NULLS), + new ValuesSourceReaderOperator.FieldInfo("null2", ElementType.NULL, shardIdx -> BlockLoader.CONSTANT_NULLS) ), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), 0 @@ -1331,8 +1464,8 @@ private void testSequentialStoredFields(boolean sequential, int docCount) { assertTrue(source.get(0).getBlock(0).asVector().singleSegmentNonDecreasing()); Operator op = new ValuesSourceReaderOperator.Factory( List.of( - fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER)), - fieldInfo(storedTextField("stored_text")) + fieldInfo(docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER), ElementType.INT), + fieldInfo(storedTextField("stored_text"), ElementType.BYTES_REF) ), List.of(new ValuesSourceReaderOperator.ShardContext(reader, () -> SourceLoader.FROM_STORED_SOURCE)), 0 @@ -1368,4 +1501,63 @@ public void testDescriptionOfMany() { assertThat(op.toString(), equalTo("ValuesSourceReaderOperator[fields = [" + cases.size() + " fields]]")); } } + + public void testManyShards() throws IOException { + int shardCount = between(2, 10); + int size = between(100, 1000); + Directory[] dirs = new Directory[shardCount]; + IndexReader[] readers = new IndexReader[shardCount]; + Closeable[] closeMe = new Closeable[shardCount * 2]; + Set seenShards = new TreeSet<>(); + Map keyCounts = new TreeMap<>(); + try { + for (int d = 0; d < dirs.length; d++) { + closeMe[d * 2 + 1] = dirs[d] = newDirectory(); + closeMe[d * 2] = readers[d] = initIndex(dirs[d], size, between(10, size * 2)); + } + List contexts = new ArrayList<>(); + List readerShardContexts = new ArrayList<>(); + for (int s = 0; s < shardCount; s++) { + contexts.add(mockSearchContext(readers[s], s)); + readerShardContexts.add(new ValuesSourceReaderOperator.ShardContext(readers[s], () -> SourceLoader.FROM_STORED_SOURCE)); + } + var luceneFactory = new LuceneSourceOperator.Factory( + contexts, + ctx -> new MatchAllDocsQuery(), + DataPartitioning.SHARD, + randomIntBetween(1, 10), + 1000, + LuceneOperator.NO_LIMIT + ); + MappedFieldType ft = docValuesNumberField("key", NumberFieldMapper.NumberType.INTEGER); + var readerFactory = new ValuesSourceReaderOperator.Factory( + List.of(new ValuesSourceReaderOperator.FieldInfo("key", ElementType.INT, shardIdx -> { + seenShards.add(shardIdx); + return ft.blockLoader(blContext()); + })), + readerShardContexts, + 0 + ); + DriverContext driverContext = driverContext(); + List results = drive( + readerFactory.get(driverContext), + CannedSourceOperator.collectPages(luceneFactory.get(driverContext)).iterator(), + driverContext + ); + assertThat(seenShards, equalTo(IntStream.range(0, shardCount).boxed().collect(Collectors.toCollection(TreeSet::new)))); + for (Page p : results) { + IntBlock keyBlock = p.getBlock(1); + IntVector keys = keyBlock.asVector(); + for (int i = 0; i < keys.getPositionCount(); i++) { + keyCounts.merge(keys.getInt(i), 1, (prev, one) -> prev + one); + } + } + assertThat(keyCounts.keySet(), hasSize(size)); + for (int k = 0; k < size; k++) { + assertThat(keyCounts.get(k), equalTo(shardCount)); + } + } finally { + IOUtils.close(closeMe); + } + } } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersQueryIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersQueryIT.java index 02f85292f05b3..35e019e3a140b 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersQueryIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClustersQueryIT.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.action; +import org.elasticsearch.Build; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; @@ -121,6 +122,7 @@ public void testMetadataIndex() { } public void testProfile() { + assumeTrue("pragmas only enabled on snapshot builds", Build.current().isSnapshot()); final int localOnlyProfiles; // uses shard partitioning as segments can be merged during these queries var pragmas = new QueryPragmas(Settings.builder().put(QueryPragmas.DATA_PARTITIONING.getKey(), DataPartitioning.SHARD).build()); diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java index 6a9278efd2f6a..d87444746d2c6 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java @@ -18,11 +18,8 @@ * This class is generated. Do not edit it. */ public final class MvAvgDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvAvgDoubleEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java index 8f2abc5e759b4..63e6b4eb12106 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java @@ -19,11 +19,8 @@ * This class is generated. Do not edit it. */ public final class MvAvgIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvAvgIntEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java index b01424846c4a7..d699070747b49 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java @@ -19,11 +19,8 @@ * This class is generated. Do not edit it. */ public final class MvAvgLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvAvgLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java index 41e18cf1424a3..5c63508fa3560 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java @@ -19,12 +19,9 @@ * This class is generated. Do not edit it. */ public final class MvAvgUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvAvgUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java index 46155b23d7512..44b4432edbf6f 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java @@ -17,12 +17,9 @@ * This class is generated. Do not edit it. */ public final class MvMaxBooleanEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMaxBooleanEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java index 6f1469e365336..4e2180f2ec467 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java @@ -18,12 +18,9 @@ * This class is generated. Do not edit it. */ public final class MvMaxBytesRefEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMaxBytesRefEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java index 34e51c2d6f221..4a1be1673bb7c 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMaxDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMaxDoubleEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java index 5382f2dff2fd8..b0ed499efd84f 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMaxIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMaxIntEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java index 331d070315ea6..24397b64c9ccc 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMaxLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMaxLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java index 4870712f8f2fb..c3ea505a29e88 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java @@ -17,12 +17,9 @@ * This class is generated. Do not edit it. */ public final class MvMedianDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMedianDoubleEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java index 83376cb634a8f..d07dc41e1d04b 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMedianIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMedianIntEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java index bf324d4db4f72..f1cd87aefd3d0 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java @@ -17,12 +17,9 @@ * This class is generated. Do not edit it. */ public final class MvMedianLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMedianLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java index 3f95ba060f825..031280b767b41 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java @@ -17,12 +17,9 @@ * This class is generated. Do not edit it. */ public final class MvMedianUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMedianUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java index a8546837479a8..ea8b04cc5c4a5 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java @@ -17,12 +17,9 @@ * This class is generated. Do not edit it. */ public final class MvMinBooleanEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMinBooleanEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java index f00e7272ae378..31d41ff61e196 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java @@ -18,12 +18,9 @@ * This class is generated. Do not edit it. */ public final class MvMinBytesRefEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMinBytesRefEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java index 5cd7ee9039a33..5390350751ee7 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMinDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMinDoubleEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java index 93b4612f898ad..918b049780905 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMinIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMinIntEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java index 9c974caecc40d..37a6709d46d4d 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java @@ -17,11 +17,8 @@ * This class is generated. Do not edit it. */ public final class MvMinLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvMinLongEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java index cc54ebad77667..b49a92404ecd1 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java @@ -18,11 +18,8 @@ * This class is generated. Do not edit it. */ public final class MvSumDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { - private final DriverContext driverContext; - public MvSumDoubleEvaluator(EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java index bd24d4a917e84..20ae9a4047385 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java @@ -21,13 +21,10 @@ public final class MvSumIntEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { private final Warnings warnings; - private final DriverContext driverContext; - public MvSumIntEvaluator(Source source, EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); + super(driverContext, field); this.warnings = new Warnings(source); - this.driverContext = driverContext; } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java index 823d6fa17bee2..bff596a76d697 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java @@ -21,13 +21,10 @@ public final class MvSumLongEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { private final Warnings warnings; - private final DriverContext driverContext; - public MvSumLongEvaluator(Source source, EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); + super(driverContext, field); this.warnings = new Warnings(source); - this.driverContext = driverContext; } @Override diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java index 8203b46b57a51..28ae5e5a2da3b 100644 --- a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java @@ -21,13 +21,10 @@ public final class MvSumUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { private final Warnings warnings; - private final DriverContext driverContext; - public MvSumUnsignedLongEvaluator(Source source, EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { - super(field); + super(driverContext, field); this.warnings = new Warnings(source); - this.driverContext = driverContext; } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java index 6d57b239e94a0..a533c373ad2ca 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java @@ -42,6 +42,7 @@ import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.shard.ShardId; @@ -273,12 +274,23 @@ private void doLookup( NamedExpression extractField = extractFields.get(i); final ElementType elementType = PlannerUtils.toElementType(extractField.dataType()); mergingTypes[i] = elementType; - var loaders = BlockReaderFactories.loaders( - List.of(searchContext), + BlockLoader loader = BlockReaderFactories.loader( + searchContext.getSearchExecutionContext(), extractField instanceof Alias a ? ((NamedExpression) a.child()).name() : extractField.name(), EsqlDataTypes.isUnsupported(extractField.dataType()) ); - fields.add(new ValuesSourceReaderOperator.FieldInfo(extractField.name(), loaders)); + fields.add( + new ValuesSourceReaderOperator.FieldInfo( + extractField.name(), + PlannerUtils.toElementType(extractField.dataType()), + shardIdx -> { + if (shardIdx != 0) { + throw new IllegalStateException("only one shard"); + } + return loader; + } + ) + ); } intermediateOperators.add( new ValuesSourceReaderOperator( diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java index e3bb8212aebab..9edb67db668d0 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java @@ -9,6 +9,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.EvalOperator; import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator; import org.elasticsearch.core.Releasables; @@ -54,8 +55,8 @@ public final ExpressionEvaluator.Factory toEvaluator(java.util.function.Function * Base evaluator that can handle both nulls- and no-nulls-containing blocks. */ public abstract static class AbstractEvaluator extends AbstractNullableEvaluator { - protected AbstractEvaluator(EvalOperator.ExpressionEvaluator field) { - super(field); + protected AbstractEvaluator(DriverContext driverContext, EvalOperator.ExpressionEvaluator field) { + super(driverContext, field); } /** @@ -102,9 +103,11 @@ public final Block eval(Page page) { * Base evaluator that can handle evaluator-checked exceptions; i.e. for expressions that can be evaluated to null. */ public abstract static class AbstractNullableEvaluator implements EvalOperator.ExpressionEvaluator { + protected final DriverContext driverContext; protected final EvalOperator.ExpressionEvaluator field; - protected AbstractNullableEvaluator(EvalOperator.ExpressionEvaluator field) { + protected AbstractNullableEvaluator(DriverContext driverContext, EvalOperator.ExpressionEvaluator field) { + this.driverContext = driverContext; this.field = field; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java index b7d9a3a73929e..9e4482bd48682 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java @@ -92,11 +92,8 @@ public String toString() { } private static class Evaluator extends AbstractEvaluator { - private final DriverContext driverContext; - protected Evaluator(DriverContext driverContext, EvalOperator.ExpressionEvaluator field) { - super(field); - this.driverContext = driverContext; + super(driverContext, field); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index f1647ff15d9d0..493ff2eb2bef6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -43,6 +43,7 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Function; +import java.util.function.IntFunction; import static org.elasticsearch.common.lucene.search.Queries.newNonNestedFilter; import static org.elasticsearch.compute.lucene.LuceneSourceOperator.NO_LIMIT; @@ -74,9 +75,15 @@ public final PhysicalOperation fieldExtractPhysicalOperation(FieldExtractExec fi } layout.append(attr); DataType dataType = attr.dataType(); + ElementType elementType = PlannerUtils.toElementType(dataType); String fieldName = attr.name(); - List loaders = BlockReaderFactories.loaders(searchContexts, fieldName, EsqlDataTypes.isUnsupported(dataType)); - fields.add(new ValuesSourceReaderOperator.FieldInfo(fieldName, loaders)); + boolean isSupported = EsqlDataTypes.isUnsupported(dataType); + IntFunction loader = s -> BlockReaderFactories.loader( + searchContexts.get(s).getSearchExecutionContext(), + fieldName, + isSupported + ); + fields.add(new ValuesSourceReaderOperator.FieldInfo(fieldName, elementType, loader)); } return source.with(new ValuesSourceReaderOperator.Factory(fields, readers, docChannel), layout.build()); } @@ -165,8 +172,13 @@ public final Operator.OperatorFactory ordinalGroupingOperatorFactory( .toList(); // The grouping-by values are ready, let's group on them directly. // Costin: why are they ready and not already exposed in the layout? + boolean isUnsupported = EsqlDataTypes.isUnsupported(attrSource.dataType()); return new OrdinalsGroupingOperator.OrdinalsGroupingOperatorFactory( - BlockReaderFactories.loaders(searchContexts, attrSource.name(), EsqlDataTypes.isUnsupported(attrSource.dataType())), + shardIdx -> BlockReaderFactories.loader( + searchContexts.get(shardIdx).getSearchExecutionContext(), + attrSource.name(), + isUnsupported + ), shardContexts, groupElementType, docChannel, diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java index 23ec1c0262dd4..4e84830b4395a 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java @@ -338,9 +338,6 @@ public void testTsdbDataStreams() throws Exception { rolloverMaxOneDocCondition(client(), dataStream); String rollupIndex = waitAndGetRollupIndexName(client(), backingIndexName, fixedInterval); - if (rollupIndex == null) { - logger.warn("explain:" + explainIndex(client(), backingIndexName)); - } assertNotNull(String.format(Locale.ROOT, "Cannot retrieve rollup index [%s]", rollupIndex), rollupIndex); assertBusy(() -> assertTrue("Rollup index does not exist", indexExists(rollupIndex)), 30, TimeUnit.SECONDS); assertBusy(() -> assertFalse("Source index should have been deleted", indexExists(backingIndexName)), 30, TimeUnit.SECONDS); @@ -607,7 +604,7 @@ public void testDownsampleTwiceSameInterval() throws Exception { * @return the name of the rollup index for a given index, null if none exist */ public String waitAndGetRollupIndexName(RestClient client, String originalIndexName, DateHistogramInterval fixedInterval) - throws InterruptedException { + throws InterruptedException, IOException { final String[] rollupIndexName = new String[1]; waitUntil(() -> { try { @@ -617,7 +614,15 @@ public String waitAndGetRollupIndexName(RestClient client, String originalIndexN return false; } }, 120, TimeUnit.SECONDS); // High timeout in case we're unlucky and end_time has been increased. - logger.info("--> original index name is [{}], rollup index name is [{}]", originalIndexName, rollupIndexName[0]); + if (rollupIndexName[0] == null) { + logger.warn( + "--> original index name is [{}], rollup index name is NULL, possible explanation: {}", + originalIndexName, + explainIndex(client(), originalIndexName) + ); + } else { + logger.info("--> original index name is [{}], rollup index name is [{}]", originalIndexName, rollupIndexName[0]); + } return rollupIndexName[0]; } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportCloseJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportCloseJobAction.java index 7512aa2b42acf..02801864a3e78 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportCloseJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportCloseJobAction.java @@ -244,7 +244,7 @@ static class OpenAndClosingIds { } List openJobIds; - List closingJobIds; + final List closingJobIds; } /** @@ -616,8 +616,8 @@ private void normalCloseJob( } static class WaitForCloseRequest { - List> persistentTasks = new ArrayList<>(); - List jobsToFinalize = new ArrayList<>(); + final List> persistentTasks = new ArrayList<>(); + final List jobsToFinalize = new ArrayList<>(); public boolean hasJobsToWaitFor() { return persistentTasks.isEmpty() == false; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteTrainedModelAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteTrainedModelAction.java index 49f73056cd8bd..d19871d0e1b2f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteTrainedModelAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteTrainedModelAction.java @@ -231,7 +231,6 @@ private void deleteModel(DeleteTrainedModelAction.Request request, ClusterState id ) ); - return; } } else { deleteAliasesAndModel(request, modelAliases, listener); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetOverallBucketsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetOverallBucketsAction.java index 10fb54ee8ae4c..d9dfd0fb23eeb 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetOverallBucketsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetOverallBucketsAction.java @@ -14,7 +14,6 @@ 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.core.TimeValue; @@ -66,7 +65,6 @@ public class TransportGetOverallBucketsAction extends HandledTransportAction< private final ThreadPool threadPool; private final Client client; - private final ClusterService clusterService; private final JobManager jobManager; @Inject @@ -74,7 +72,6 @@ public TransportGetOverallBucketsAction( ThreadPool threadPool, TransportService transportService, ActionFilters actionFilters, - ClusterService clusterService, JobManager jobManager, Client client ) { @@ -86,7 +83,6 @@ public TransportGetOverallBucketsAction( EsExecutors.DIRECT_EXECUTOR_SERVICE ); this.threadPool = threadPool; - this.clusterService = clusterService; this.client = client; this.jobManager = jobManager; } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutDatafeedAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutDatafeedAction.java index d71e99040177f..8cdb8050bd257 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutDatafeedAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutDatafeedAction.java @@ -71,7 +71,7 @@ protected void masterOperation( ClusterState state, ActionListener listener ) { - datafeedManager.putDatafeed(request, state, licenseState, securityContext, threadPool, listener); + datafeedManager.putDatafeed(request, state, securityContext, threadPool, listener); } @Override diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutJobAction.java index 767ec08078b42..55f89a993ce61 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportPutJobAction.java @@ -99,7 +99,6 @@ protected void masterOperation( new PutDatafeedAction.Request(jobCreated.getResponse().getDatafeedConfig().get()), // Use newer state from cluster service as the job creation may have created shared indexes clusterService.state(), - licenseState, securityContext, threadPool, ActionListener.wrap(createdDatafeed -> { 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 5206799735c52..195cc44dcaa65 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 @@ -6,8 +6,6 @@ */ package org.elasticsearch.xpack.ml.action; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceNotFoundException; @@ -29,7 +27,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.logging.HeaderWarning; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; @@ -42,10 +39,8 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentFactory; @@ -55,7 +50,6 @@ import org.elasticsearch.xpack.core.XPackField; import org.elasticsearch.xpack.core.ml.MachineLearningField; import org.elasticsearch.xpack.core.ml.MlConfigVersion; -import org.elasticsearch.xpack.core.ml.MlTasks; import org.elasticsearch.xpack.core.ml.action.GetTrainedModelsAction; import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction; import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction.Request; @@ -87,22 +81,18 @@ import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; -import static org.elasticsearch.xpack.core.ml.MlTasks.downloadModelTaskDescription; public class TransportPutTrainedModelAction extends TransportMasterNodeAction { private static final ByteSizeValue MAX_NATIVE_DEFINITION_INDEX_SIZE = ByteSizeValue.ofGb(50); - private static final Logger logger = LogManager.getLogger(TransportPutTrainedModelAction.class); private final TrainedModelProvider trainedModelProvider; private final XPackLicenseState licenseState; private final NamedXContentRegistry xContentRegistry; private final OriginSettingClient client; - private final Settings settings; @Inject public TransportPutTrainedModelAction( - Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, @@ -128,7 +118,6 @@ public TransportPutTrainedModelAction( this.trainedModelProvider = trainedModelProvider; this.xContentRegistry = xContentRegistry; this.client = new OriginSettingClient(client, ML_ORIGIN); - this.settings = settings; } @Override @@ -248,13 +237,15 @@ protected void masterOperation( return; } - ActionListener finalResponseAction = ActionListener.wrap((configToReturn) -> { - finalResponseListener.onResponse(new PutTrainedModelAction.Response(configToReturn)); - }, finalResponseListener::onFailure); + ActionListener finalResponseAction = ActionListener.wrap( + (configToReturn) -> finalResponseListener.onResponse(new Response(configToReturn)), + finalResponseListener::onFailure + ); - ActionListener verifyClusterAndModelArchitectures = ActionListener.wrap((configToReturn) -> { - verifyMlNodesAndModelArchitectures(configToReturn, client, threadPool, finalResponseAction); - }, finalResponseListener::onFailure); + ActionListener verifyClusterAndModelArchitectures = ActionListener.wrap( + (configToReturn) -> verifyMlNodesAndModelArchitectures(configToReturn, client, threadPool, finalResponseAction), + finalResponseListener::onFailure + ); ActionListener finishedStoringListener = ActionListener.wrap(bool -> { TrainedModelConfig configToReturn = trainedModelConfig.clearDefinition().build(); @@ -361,7 +352,7 @@ void verifyMlNodesAndModelArchitectures( ThreadPool threadPool, ActionListener configToReturnListener ) { - ActionListener addWarningHeaderOnFailureListener = new ActionListener() { + ActionListener addWarningHeaderOnFailureListener = new ActionListener<>() { @Override public void onResponse(TrainedModelConfig config) { assert Objects.equals(config, configToReturn); @@ -413,38 +404,9 @@ static void checkForExistingTask( }, sendResponseListener::onFailure), timeout); } - private static void getExistingTaskInfo(Client client, String modelId, boolean waitForCompletion, ActionListener listener) { - client.admin() - .cluster() - .prepareListTasks() - .setActions(MlTasks.MODEL_IMPORT_TASK_ACTION) - .setDetailed(true) - .setWaitForCompletion(waitForCompletion) - .setDescriptions(downloadModelTaskDescription(modelId)) - .execute(ActionListener.wrap((response) -> { - var tasks = response.getTasks(); - - if (tasks.size() > 0) { - // there really shouldn't be more than a single task but if there is we'll just use the first one - listener.onResponse(tasks.get(0)); - } else { - listener.onResponse(null); - } - }, e -> { - listener.onFailure( - new ElasticsearchStatusException( - "Unable to retrieve task information for model id [{}]", - RestStatus.INTERNAL_SERVER_ERROR, - e, - modelId - ) - ); - })); - } - private static void getModelInformation(Client client, String modelId, ActionListener listener) { client.execute(GetTrainedModelsAction.INSTANCE, new GetTrainedModelsAction.Request(modelId), ActionListener.wrap(models -> { - if (models.getResources().results().size() == 0) { + if (models.getResources().results().isEmpty()) { listener.onFailure( new ElasticsearchStatusException( "No model information found for a concurrent create model execution for model id [{}]", @@ -563,11 +525,7 @@ static void setTrainedModelConfigFieldsFromPackagedModel( trainedModelConfig.setPlatformArchitecture(resolvedModelPackageConfig.getPlatformArchitecture()); trainedModelConfig.setMetadata(resolvedModelPackageConfig.getMetadata()); trainedModelConfig.setInferenceConfig( - parseInferenceConfigFromModelPackage( - resolvedModelPackageConfig.getInferenceConfigSource(), - xContentRegistry, - LoggingDeprecationHandler.INSTANCE - ) + parseInferenceConfigFromModelPackage(resolvedModelPackageConfig.getInferenceConfigSource(), xContentRegistry) ); trainedModelConfig.setTags(resolvedModelPackageConfig.getTags()); trainedModelConfig.setPrefixStrings(resolvedModelPackageConfig.getPrefixStrings()); @@ -578,16 +536,14 @@ static void setTrainedModelConfigFieldsFromPackagedModel( trainedModelConfig.setLocation(trainedModelConfig.getModelType().getDefaultLocation(trainedModelConfig.getModelId())); } - static InferenceConfig parseInferenceConfigFromModelPackage( - Map source, - NamedXContentRegistry namedXContentRegistry, - DeprecationHandler deprecationHandler - ) throws IOException { + static InferenceConfig parseInferenceConfigFromModelPackage(Map source, NamedXContentRegistry namedXContentRegistry) + throws IOException { try ( XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().map(source); XContentParser sourceParser = XContentType.JSON.xContent() .createParser( - XContentParserConfiguration.EMPTY.withRegistry(namedXContentRegistry).withDeprecationHandler(deprecationHandler), + XContentParserConfiguration.EMPTY.withRegistry(namedXContentRegistry) + .withDeprecationHandler(LoggingDeprecationHandler.INSTANCE), BytesReference.bytes(xContentBuilder).streamInput() ) ) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartTrainedModelDeploymentAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartTrainedModelDeploymentAction.java index ecfe4c8aac6c6..28475dc70569f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartTrainedModelDeploymentAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartTrainedModelDeploymentAction.java @@ -43,7 +43,6 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xpack.core.XPackField; import org.elasticsearch.xpack.core.ml.MachineLearningField; import org.elasticsearch.xpack.core.ml.action.CreateTrainedModelAssignmentAction; @@ -108,7 +107,6 @@ public TransportStartTrainedModelDeploymentAction( XPackLicenseState licenseState, IndexNameExpressionResolver indexNameExpressionResolver, TrainedModelAssignmentService trainedModelAssignmentService, - NamedXContentRegistry xContentRegistry, MlMemoryTracker memoryTracker, InferenceAuditor auditor ) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopTrainedModelDeploymentAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopTrainedModelDeploymentAction.java index 5b2c3fdeddf43..a3eb15a372d2a 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopTrainedModelDeploymentAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStopTrainedModelDeploymentAction.java @@ -17,8 +17,6 @@ import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.tasks.TransportTasksAction; -import org.elasticsearch.client.internal.Client; -import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -48,7 +46,6 @@ import java.util.Set; import static org.elasticsearch.core.Strings.format; -import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.ml.action.TransportDeleteTrainedModelAction.getModelAliases; import static org.elasticsearch.xpack.ml.action.TransportDeleteTrainedModelAction.getReferencedModelKeys; @@ -66,7 +63,6 @@ public class TransportStopTrainedModelDeploymentAction extends TransportTasksAct private static final Logger logger = LogManager.getLogger(TransportStopTrainedModelDeploymentAction.class); - private final Client client; private final IngestService ingestService; private final TrainedModelAssignmentClusterService trainedModelAssignmentClusterService; private final InferenceAuditor auditor; @@ -76,7 +72,6 @@ public TransportStopTrainedModelDeploymentAction( ClusterService clusterService, TransportService transportService, ActionFilters actionFilters, - Client client, IngestService ingestService, TrainedModelAssignmentClusterService trainedModelAssignmentClusterService, InferenceAuditor auditor @@ -91,7 +86,6 @@ public TransportStopTrainedModelDeploymentAction( StopTrainedModelDeploymentAction.Response::new, EsExecutors.DIRECT_EXECUTOR_SERVICE ); - this.client = new OriginSettingClient(client, ML_ORIGIN); this.ingestService = ingestService; this.trainedModelAssignmentClusterService = trainedModelAssignmentClusterService; this.auditor = Objects.requireNonNull(auditor); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java index c1e77d953ab54..dc53c02389184 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportUpdateFilterAction.java @@ -18,7 +18,6 @@ import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; -import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -60,8 +59,7 @@ public TransportUpdateFilterAction( TransportService transportService, ActionFilters actionFilters, Client client, - JobManager jobManager, - ClusterService clusterService + JobManager jobManager ) { super( UpdateFilterAction.NAME, diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregationBuilder.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregationBuilder.java index 6fce8aa20ed16..9262ac65b5cfd 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregationBuilder.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/CategorizeTextAggregationBuilder.java @@ -23,7 +23,6 @@ import org.elasticsearch.xcontent.ObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xpack.core.ml.job.config.CategorizationAnalyzerConfig; import org.elasticsearch.xpack.core.ml.job.messages.Messages; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; @@ -85,10 +84,6 @@ public class CategorizeTextAggregationBuilder extends AbstractAggregationBuilder PARSER.declareInt(CategorizeTextAggregationBuilder::size, REQUIRED_SIZE_FIELD_NAME); } - public static CategorizeTextAggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException { - return PARSER.parse(parser, new CategorizeTextAggregationBuilder(aggregationName), null); - } - private TermsAggregator.BucketCountThresholds bucketCountThresholds = new TermsAggregator.BucketCountThresholds( DEFAULT_BUCKET_COUNT_THRESHOLDS ); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/frequentitemsets/CountingItemSetTraverser.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/frequentitemsets/CountingItemSetTraverser.java index 35a0bd9e4c43f..a651b0c85eb40 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/frequentitemsets/CountingItemSetTraverser.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/frequentitemsets/CountingItemSetTraverser.java @@ -7,8 +7,6 @@ package org.elasticsearch.xpack.ml.aggs.frequentitemsets; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.xpack.ml.aggs.frequentitemsets.TransactionStore.TopItemIds; @@ -31,7 +29,6 @@ * if [a, b] is not in T, [a, b, c] can not be in T either */ final class CountingItemSetTraverser implements Releasable { - private static final Logger logger = LogManager.getLogger(CountingItemSetTraverser.class); // start size and size increment for the occurences stack private static final int OCCURENCES_SIZE_INCREMENT = 10; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/heuristic/PValueScore.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/heuristic/PValueScore.java index 324da870b1a40..8fe9f1ccd5415 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/heuristic/PValueScore.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/heuristic/PValueScore.java @@ -11,11 +11,9 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.aggregations.bucket.terms.heuristic.NXYSignificanceHeuristic; -import org.elasticsearch.search.aggregations.bucket.terms.heuristic.SignificanceHeuristic; import org.elasticsearch.xcontent.ConstructingObjectParser; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentParser; import java.io.IOException; import java.util.Objects; @@ -104,10 +102,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - public static SignificanceHeuristic parse(XContentParser parser) throws IOException { - return PARSER.apply(parser, null); - } - /** * This finds the p-value that the frequency of a category is unchanged on set subset assuming * we observe subsetFreq out of subset values in total relative to set supersetFreq where it accounts @@ -200,28 +194,4 @@ private static double eps(double value) { return Math.max(0.05 * value + 0.5, 1.0); } - public static class PValueScoreBuilder extends NXYBuilder { - private final long normalizeAbove; - - public PValueScoreBuilder(boolean backgroundIsSuperset, Long normalizeAbove) { - super(true, backgroundIsSuperset); - this.normalizeAbove = normalizeAbove == null ? 0L : normalizeAbove; - if (normalizeAbove != null && normalizeAbove <= 0) { - throw new IllegalArgumentException( - "[" + NORMALIZE_ABOVE.getPreferredName() + "] must be a positive value, provided [" + normalizeAbove + "]" - ); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(NAME); - builder.field(BACKGROUND_IS_SUPERSET.getPreferredName(), backgroundIsSuperset); - if (normalizeAbove > 0) { - builder.field(NORMALIZE_ABOVE.getPreferredName(), normalizeAbove); - } - builder.endObject(); - return builder; - } - } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/NodeFakeAvailabilityZoneMapper.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/NodeFakeAvailabilityZoneMapper.java index df2f66f6c5a42..1eb6fe3d03cd2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/NodeFakeAvailabilityZoneMapper.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/NodeFakeAvailabilityZoneMapper.java @@ -7,8 +7,6 @@ package org.elasticsearch.xpack.ml.autoscaling; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; @@ -27,8 +25,6 @@ */ public class NodeFakeAvailabilityZoneMapper extends AbstractNodeAvailabilityZoneMapper { - private static final Logger logger = LogManager.getLogger(NodeFakeAvailabilityZoneMapper.class); - public NodeFakeAvailabilityZoneMapper(Settings settings, ClusterSettings clusterSettings) { this(settings, clusterSettings, null); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java index 999d85b6dd549..ede57764a0813 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java @@ -20,7 +20,6 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.license.RemoteClusterLicenseChecker; -import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; @@ -100,7 +99,6 @@ public DatafeedManager( public void putDatafeed( PutDatafeedAction.Request request, ClusterState state, - XPackLicenseState licenseState, SecurityContext securityContext, ThreadPool threadPool, ActionListener listener diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/TrainedModelAssignmentClusterService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/TrainedModelAssignmentClusterService.java index 471615e8bbd6a..637ad9d7bbbb2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/TrainedModelAssignmentClusterService.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/TrainedModelAssignmentClusterService.java @@ -78,8 +78,6 @@ public class TrainedModelAssignmentClusterService implements ClusterStateListene private static final TransportVersion RENAME_ALLOCATION_TO_ASSIGNMENT_TRANSPORT_VERSION = TransportVersions.V_8_3_0; public static final TransportVersion DISTRIBUTED_MODEL_ALLOCATION_TRANSPORT_VERSION = TransportVersions.V_8_4_0; - private static final TransportVersion NEW_ALLOCATION_MEMORY_VERSION = TransportVersions.V_8_500_064; - private final ClusterService clusterService; private final ThreadPool threadPool; private final NodeLoadDetector nodeLoadDetector; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/planning/AssignmentPlan.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/planning/AssignmentPlan.java index 1dce7f0bb46ba..d9cb0f08a6cd0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/planning/AssignmentPlan.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/assignment/planning/AssignmentPlan.java @@ -437,10 +437,6 @@ public Builder assignModelToNode(Deployment deployment, Node node, int allocatio return this; } - private boolean isAlreadyAssigned(Deployment deployment, Node node) { - return deployment.currentAllocationsByNodeId().containsKey(node.id()) || assignments.get(deployment).get(node) > 0; - } - private int getAssignedAllocations(Deployment deployment, Node node) { int currentAllocations = getCurrentAllocations(deployment, node); int assignmentAllocations = assignments.get(deployment).get(node); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/pytorch/process/PyTorchProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/pytorch/process/PyTorchProcessManager.java deleted file mode 100644 index c812e490217ed..0000000000000 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/pytorch/process/PyTorchProcessManager.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.ml.inference.pytorch.process; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -public class PyTorchProcessManager { - - private static final Logger logger = LogManager.getLogger(PyTorchProcessManager.class); - - public PyTorchProcessManager() { - - } - - public void start(String taskId) { - - } -} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java index 24383e51b0ed2..a392996fbb448 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java @@ -144,7 +144,7 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws if (weightedTokensSupplier.get() == null) { return this; } - return weightedTokensToQuery(fieldName, weightedTokensSupplier.get(), queryRewriteContext); + return weightedTokensToQuery(fieldName, weightedTokensSupplier.get()); } CoordinatedInferenceAction.Request inferRequest = CoordinatedInferenceAction.Request.forTextInput( @@ -196,11 +196,7 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws return new TextExpansionQueryBuilder(this, textExpansionResultsSupplier); } - private QueryBuilder weightedTokensToQuery( - String fieldName, - TextExpansionResults textExpansionResults, - QueryRewriteContext queryRewriteContext - ) { + private QueryBuilder weightedTokensToQuery(String fieldName, TextExpansionResults textExpansionResults) { if (tokenPruningConfig != null) { WeightedTokensQueryBuilder weightedTokensQueryBuilder = new WeightedTokensQueryBuilder( fieldName, 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 84c49ba95b522..6ed7a3311c94a 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 @@ -16,8 +16,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.TimeValue; import org.elasticsearch.rest.RestStatus; @@ -115,8 +113,7 @@ public void testParseInferenceConfigFromModelPackage() throws IOException { assertNotNull(inferenceConfigMap); InferenceConfig parsedInferenceConfig = TransportPutTrainedModelAction.parseInferenceConfigFromModelPackage( Collections.singletonMap(inferenceConfig.getWriteableName(), inferenceConfigMap), - xContentRegistry(), - LoggingDeprecationHandler.INSTANCE + xContentRegistry() ); assertEquals(inferenceConfig, parsedInferenceConfig); @@ -278,7 +275,6 @@ private TransportPutTrainedModelAction createTransportPutTrainedModelAction() { doReturn(threadPool).when(mockClient).threadPool(); return new TransportPutTrainedModelAction( - Settings.EMPTY, mockTransportService, mockClusterService, threadPool, diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/aggs/changepoint/ChangePointAggregatorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/aggs/changepoint/ChangePointAggregatorTests.java index 2f4ecec87509a..f6922475e63af 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/aggs/changepoint/ChangePointAggregatorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/aggs/changepoint/ChangePointAggregatorTests.java @@ -72,6 +72,7 @@ public void testStationaryFalsePositiveRate() throws IOException { assertThat(fp, lessThan(5)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/103848") public void testSampledDistributionTestFalsePositiveRate() throws IOException { NormalDistribution normal = new NormalDistribution(RandomGeneratorFactory.createRandomGenerator(Randomness.get()), 0.0, 1.0); int fp = 0; diff --git a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/apikey/ApiKeyRestIT.java b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/apikey/ApiKeyRestIT.java index 3c3b06c84da2a..6c4aaeada74c7 100644 --- a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/apikey/ApiKeyRestIT.java +++ b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/apikey/ApiKeyRestIT.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.Strings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; import org.elasticsearch.test.XContentTestUtils; import org.elasticsearch.test.rest.ObjectPath; @@ -22,6 +23,7 @@ import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xcontent.json.JsonXContent; import org.elasticsearch.xpack.core.security.action.apikey.ApiKey; +import org.elasticsearch.xpack.core.security.action.apikey.ApiKeyTests; import org.elasticsearch.xpack.core.security.action.apikey.GetApiKeyResponse; import org.elasticsearch.xpack.core.security.action.apikey.GrantApiKeyAction; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; @@ -34,6 +36,7 @@ import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -436,6 +439,23 @@ public void testBulkUpdateApiKey() throws IOException { doTestAuthenticationWithApiKey(apiKeyExpectingNoop.name, apiKeyExpectingNoop.id, apiKeyExpectingNoop.encoded); } + public void testBulkUpdateExpirationTimeApiKey() throws IOException { + final EncodedApiKey apiKey1 = createApiKey("my-api-key-name", Map.of()); + final EncodedApiKey apiKey2 = createApiKey("my-other-api-key-name", Map.of()); + final var bulkUpdateApiKeyRequest = new Request("POST", "_security/api_key/_bulk_update"); + final TimeValue expiration = ApiKeyTests.randomFutureExpirationTime(); + bulkUpdateApiKeyRequest.setJsonEntity( + XContentTestUtils.convertToXContent(Map.of("ids", List.of(apiKey1.id, apiKey2.id), "expiration", expiration), XContentType.JSON) + .utf8ToString() + ); + final Response bulkUpdateApiKeyResponse = performRequestUsingRandomAuthMethod(bulkUpdateApiKeyRequest); + assertOK(bulkUpdateApiKeyResponse); + final Map response = responseAsMap(bulkUpdateApiKeyResponse); + assertEquals(List.of(apiKey1.id(), apiKey2.id()), response.get("updated")); + assertNull(response.get("errors")); + assertEquals(List.of(), response.get("noops")); + } + public void testGrantTargetCanUpdateApiKey() throws IOException { final var request = new Request("POST", "_security/api_key/grant"); request.setOptions( @@ -923,7 +943,7 @@ public void testUpdateCrossClusterApiKey() throws IOException { final ObjectPath createResponse = assertOKAndCreateObjectPath(client().performRequest(createRequest)); final String apiKeyId = createResponse.evaluate("id"); - // Update both access and metadata + // Update access, metadata and expiration final Request updateRequest1 = new Request("PUT", "/_security/cross_cluster/api_key/" + apiKeyId); updateRequest1.setJsonEntity(""" { @@ -940,7 +960,8 @@ public void testUpdateCrossClusterApiKey() throws IOException { } ] }, - "metadata": { "tag": "shared", "points": 0 } + "metadata": { "tag": "shared", "points": 0 }, + "expiration": "30d" }"""); setUserForRequest(updateRequest1, MANAGE_SECURITY_USER, END_USER_PASSWORD); final ObjectPath updateResponse1 = assertOKAndCreateObjectPath(client().performRequest(updateRequest1)); @@ -966,6 +987,7 @@ public void testUpdateCrossClusterApiKey() throws IOException { fetchResponse1.evaluate("api_keys.0.role_descriptors"), equalTo(Map.of("cross_cluster", XContentTestUtils.convertToMap(updatedRoleDescriptor1))) ); + assertThat(fetchResponse1.evaluate("api_keys.0.expiration"), notNullValue()); assertThat(fetchResponse1.evaluate("api_keys.0.access"), equalTo(XContentHelper.convertToMap(JsonXContent.jsonXContent, """ { "search": [ @@ -1465,6 +1487,40 @@ private void doTestAuthenticationWithApiKey(final String apiKeyName, final Strin assertThat(authenticate, hasEntry("api_key", Map.of("id", apiKeyId, "name", apiKeyName))); } + private static Map getRandomUpdateApiKeyRequestBody( + final Map oldMetadata, + boolean updateExpiration, + boolean updateMetadata + ) { + return getRandomUpdateApiKeyRequestBody(oldMetadata, updateExpiration, updateMetadata, List.of()); + } + + private static Map getRandomUpdateApiKeyRequestBody( + final Map oldMetadata, + boolean updateExpiration, + boolean updateMetadata, + List ids + ) { + Map updateRequestBody = new HashMap<>(); + + if (updateMetadata) { + updateRequestBody.put("metadata", Map.of("not", "returned (changed)", "foo", "bar")); + } else if (oldMetadata != null) { + updateRequestBody.put("metadata", oldMetadata); + } + + if (updateExpiration) { + updateRequestBody.put("expiration", ApiKeyTests.randomFutureExpirationTime()); + } + + if (ids.isEmpty() == false) { + updateRequestBody.put("ids", ids); + } + + return updateRequestBody; + } + + @SuppressWarnings({ "unchecked" }) private void doTestUpdateApiKey( final String apiKeyName, final String apiKeyId, @@ -1472,19 +1528,17 @@ private void doTestUpdateApiKey( final Map oldMetadata ) throws IOException { final var updateApiKeyRequest = new Request("PUT", "_security/api_key/" + apiKeyId); - final boolean updated = randomBoolean(); - final Map expectedApiKeyMetadata = updated ? Map.of("not", "returned (changed)", "foo", "bar") : oldMetadata; - final Map updateApiKeyRequestBody = expectedApiKeyMetadata == null - ? Map.of() - : Map.of("metadata", expectedApiKeyMetadata); - updateApiKeyRequest.setJsonEntity(XContentTestUtils.convertToXContent(updateApiKeyRequestBody, XContentType.JSON).utf8ToString()); + final boolean updateExpiration = randomBoolean(); + final boolean updateMetadata = randomBoolean(); + final Map updateRequestBody = getRandomUpdateApiKeyRequestBody(oldMetadata, updateExpiration, updateMetadata); + updateApiKeyRequest.setJsonEntity(XContentTestUtils.convertToXContent(updateRequestBody, XContentType.JSON).utf8ToString()); final Response updateApiKeyResponse = performRequestUsingRandomAuthMethod(updateApiKeyRequest); assertOK(updateApiKeyResponse); final Map updateApiKeyResponseMap = responseAsMap(updateApiKeyResponse); - assertEquals(updated, updateApiKeyResponseMap.get("updated")); - expectMetadata(apiKeyId, expectedApiKeyMetadata == null ? Map.of() : expectedApiKeyMetadata); + assertEquals(updateMetadata || updateExpiration, updateApiKeyResponseMap.get("updated")); + expectMetadata(apiKeyId, (Map) updateRequestBody.get("metadata")); // validate authentication still works after update doTestAuthenticationWithApiKey(apiKeyName, apiKeyId, apiKeyEncoded); } @@ -1497,28 +1551,29 @@ private void doTestUpdateApiKeyUsingBulkAction( final Map oldMetadata ) throws IOException { final var bulkUpdateApiKeyRequest = new Request("POST", "_security/api_key/_bulk_update"); - final boolean updated = randomBoolean(); - final Map expectedApiKeyMetadata = updated ? Map.of("not", "returned (changed)", "foo", "bar") : oldMetadata; - final Map bulkUpdateApiKeyRequestBody = expectedApiKeyMetadata == null - ? Map.of("ids", List.of(apiKeyId)) - : Map.of("ids", List.of(apiKeyId), "metadata", expectedApiKeyMetadata); - bulkUpdateApiKeyRequest.setJsonEntity( - XContentTestUtils.convertToXContent(bulkUpdateApiKeyRequestBody, XContentType.JSON).utf8ToString() + boolean updateMetadata = randomBoolean(); + boolean updateExpiration = randomBoolean(); + Map updateRequestBody = getRandomUpdateApiKeyRequestBody( + oldMetadata, + updateExpiration, + updateMetadata, + List.of(apiKeyId) ); + bulkUpdateApiKeyRequest.setJsonEntity(XContentTestUtils.convertToXContent(updateRequestBody, XContentType.JSON).utf8ToString()); final Response bulkUpdateApiKeyResponse = performRequestUsingRandomAuthMethod(bulkUpdateApiKeyRequest); assertOK(bulkUpdateApiKeyResponse); final Map bulkUpdateApiKeyResponseMap = responseAsMap(bulkUpdateApiKeyResponse); assertThat(bulkUpdateApiKeyResponseMap, not(hasKey("errors"))); - if (updated) { + if (updateMetadata || updateExpiration) { assertThat((List) bulkUpdateApiKeyResponseMap.get("noops"), empty()); assertThat((List) bulkUpdateApiKeyResponseMap.get("updated"), contains(apiKeyId)); } else { assertThat((List) bulkUpdateApiKeyResponseMap.get("updated"), empty()); assertThat((List) bulkUpdateApiKeyResponseMap.get("noops"), contains(apiKeyId)); } - expectMetadata(apiKeyId, expectedApiKeyMetadata == null ? Map.of() : expectedApiKeyMetadata); + expectMetadata(apiKeyId, (Map) updateRequestBody.get("metadata")); // validate authentication still works after update doTestAuthenticationWithApiKey(apiKeyName, apiKeyId, apiKeyEncoded); } @@ -1604,7 +1659,6 @@ private String headerFromRandomAuthMethod(final String username, final SecureStr } } - @SuppressWarnings({ "unchecked" }) private void expectMetadata(final String apiKeyId, final Map expectedMetadata) throws IOException { final var request = new Request("GET", "_security/api_key/"); request.addParameter("id", apiKeyId); @@ -1613,7 +1667,8 @@ private void expectMetadata(final String apiKeyId, final Map exp try (XContentParser parser = responseAsParser(response)) { final var apiKeyResponse = GetApiKeyResponse.fromXContent(parser); assertThat(apiKeyResponse.getApiKeyInfos().length, equalTo(1)); - assertThat(apiKeyResponse.getApiKeyInfos()[0].getMetadata(), equalTo(expectedMetadata)); + // ApiKey metadata is set to empty Map if null + assertThat(apiKeyResponse.getApiKeyInfos()[0].getMetadata(), equalTo(expectedMetadata == null ? Map.of() : expectedMetadata)); } } diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/ReloadRemoteClusterCredentialsIT.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/ReloadRemoteClusterCredentialsIT.java index 336257ff76069..c97b2b69c09ae 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/ReloadRemoteClusterCredentialsIT.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/ReloadRemoteClusterCredentialsIT.java @@ -280,31 +280,36 @@ private void reloadSecureSettings() { final CountDownLatch latch = new CountDownLatch(1); final SecureString emptyPassword = randomBoolean() ? new SecureString(new char[0]) : null; - final var request = new NodesReloadSecureSettingsRequest(Strings.EMPTY_ARRAY); - request.setSecureStorePassword(emptyPassword); - client().execute(TransportNodesReloadSecureSettingsAction.TYPE, request, new ActionListener<>() { - @Override - public void onResponse(NodesReloadSecureSettingsResponse nodesReloadResponse) { - try { - assertThat(nodesReloadResponse, notNullValue()); - final Map nodesMap = nodesReloadResponse.getNodesMap(); - assertThat(nodesMap.size(), equalTo(1)); - for (final NodesReloadSecureSettingsResponse.NodeResponse nodeResponse : nodesReloadResponse.getNodes()) { - assertThat(nodeResponse.reloadException(), nullValue()); + final var request = new NodesReloadSecureSettingsRequest(); + try { + request.nodesIds(Strings.EMPTY_ARRAY); + request.setSecureStorePassword(emptyPassword); + client().execute(TransportNodesReloadSecureSettingsAction.TYPE, request, new ActionListener<>() { + @Override + public void onResponse(NodesReloadSecureSettingsResponse nodesReloadResponse) { + try { + assertThat(nodesReloadResponse, notNullValue()); + final Map nodesMap = nodesReloadResponse.getNodesMap(); + assertThat(nodesMap.size(), equalTo(1)); + for (final NodesReloadSecureSettingsResponse.NodeResponse nodeResponse : nodesReloadResponse.getNodes()) { + assertThat(nodeResponse.reloadException(), nullValue()); + } + } catch (final AssertionError e) { + reloadSettingsError.set(e); + } finally { + latch.countDown(); } - } catch (final AssertionError e) { - reloadSettingsError.set(e); - } finally { - latch.countDown(); } - } - @Override - public void onFailure(Exception e) { - reloadSettingsError.set(new AssertionError("Nodes request failed", e)); - latch.countDown(); - } - }); + @Override + public void onFailure(Exception e) { + reloadSettingsError.set(new AssertionError("Nodes request failed", e)); + latch.countDown(); + } + }); + } finally { + request.decRef(); + } safeAwait(latch); if (reloadSettingsError.get() != null) { throw reloadSettingsError.get(); diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java index 72a6b6049932c..1329158f57d4d 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/ApiKeyIntegTests.java @@ -1959,7 +1959,7 @@ public void testUpdateApiKeysForSingleKey() throws Exception { null ) ); - final var request = new UpdateApiKeyRequest(apiKeyId, newRoleDescriptors, ApiKeyTests.randomMetadata()); + final var request = new UpdateApiKeyRequest(apiKeyId, newRoleDescriptors, ApiKeyTests.randomMetadata(), null); final UpdateApiKeyResponse response = updateSingleApiKeyMaybeUsingBulkAction(TEST_USER_NAME, request); @@ -2030,7 +2030,7 @@ public void testBulkUpdateApiKeysForMultipleKeys() throws ExecutionException, In BulkUpdateApiKeyResponse response = executeBulkUpdateApiKey( TEST_USER_NAME, - new BulkUpdateApiKeyRequest(apiKeyIds, newRoleDescriptors, newMetadata) + new BulkUpdateApiKeyRequest(apiKeyIds, newRoleDescriptors, newMetadata, ApiKeyTests.randomFutureExpirationTime()) ); assertNotNull(response); @@ -2070,7 +2070,7 @@ public void testBulkUpdateApiKeysForMultipleKeys() throws ExecutionException, In () -> randomValueOtherThanMany(apiKeyIds::contains, () -> randomAlphaOfLength(10)) ); newIds.addAll(notFoundIds); - final BulkUpdateApiKeyRequest request = new BulkUpdateApiKeyRequest(shuffledList(newIds), newRoleDescriptors, newMetadata); + final BulkUpdateApiKeyRequest request = new BulkUpdateApiKeyRequest(shuffledList(newIds), newRoleDescriptors, newMetadata, null); response = executeBulkUpdateApiKey(TEST_USER_NAME, request); @@ -2100,7 +2100,8 @@ public void testBulkUpdateApiKeysForMultipleKeys() throws ExecutionException, In final BulkUpdateApiKeyRequest requestWithSomeErrors = new BulkUpdateApiKeyRequest( shuffledList(apiKeyIds), randomValueOtherThan(null, this::randomRoleDescriptors), - randomValueOtherThan(null, ApiKeyTests::randomMetadata) + randomValueOtherThan(null, ApiKeyTests::randomMetadata), + ApiKeyTests.randomFutureExpirationTime() ); response = executeBulkUpdateApiKey(TEST_USER_NAME, requestWithSomeErrors); @@ -2124,7 +2125,7 @@ public void testBulkUpdateApiKeysWithDuplicates() throws ExecutionException, Int BulkUpdateApiKeyResponse response = executeBulkUpdateApiKey( TEST_USER_NAME, - new BulkUpdateApiKeyRequest(idsWithDuplicates, newRoleDescriptors, newMetadata) + new BulkUpdateApiKeyRequest(idsWithDuplicates, newRoleDescriptors, newMetadata, ApiKeyTests.randomFutureExpirationTime()) ); assertNotNull(response); @@ -2142,7 +2143,12 @@ public void testBulkUpdateApiKeysWithDuplicates() throws ExecutionException, Int response = executeBulkUpdateApiKey( TEST_USER_NAME, - new BulkUpdateApiKeyRequest(notFoundIdsWithDuplicates, newRoleDescriptors, newMetadata) + new BulkUpdateApiKeyRequest( + notFoundIdsWithDuplicates, + newRoleDescriptors, + newMetadata, + ApiKeyTests.randomFutureExpirationTime() + ) ); assertNotNull(response); @@ -2317,7 +2323,12 @@ public void testUpdateApiKeysNotFoundScenarios() throws Exception { final Tuple> createdApiKey = createApiKey(TEST_USER_NAME, null); final var apiKeyId = createdApiKey.v1().getId(); final var expectedRoleDescriptor = new RoleDescriptor(randomAlphaOfLength(10), new String[] { "all" }, null, null); - final var request = new UpdateApiKeyRequest(apiKeyId, List.of(expectedRoleDescriptor), ApiKeyTests.randomMetadata()); + final var request = new UpdateApiKeyRequest( + apiKeyId, + List.of(expectedRoleDescriptor), + ApiKeyTests.randomMetadata(), + ApiKeyTests.randomFutureExpirationTime() + ); // Validate can update own API key final UpdateApiKeyResponse response = updateSingleApiKeyMaybeUsingBulkAction(TEST_USER_NAME, request); @@ -2326,12 +2337,24 @@ public void testUpdateApiKeysNotFoundScenarios() throws Exception { // Test not found exception on non-existent API key final var otherApiKeyId = randomValueOtherThan(apiKeyId, () -> randomAlphaOfLength(20)); - doTestUpdateApiKeysNotFound(new UpdateApiKeyRequest(otherApiKeyId, request.getRoleDescriptors(), request.getMetadata())); + doTestUpdateApiKeysNotFound( + new UpdateApiKeyRequest( + otherApiKeyId, + request.getRoleDescriptors(), + request.getMetadata(), + ApiKeyTests.randomFutureExpirationTime() + ) + ); // Test not found exception on other user's API key final Tuple> otherUsersApiKey = createApiKey("user_with_manage_api_key_role", null); doTestUpdateApiKeysNotFound( - new UpdateApiKeyRequest(otherUsersApiKey.v1().getId(), request.getRoleDescriptors(), request.getMetadata()) + new UpdateApiKeyRequest( + otherUsersApiKey.v1().getId(), + request.getRoleDescriptors(), + request.getMetadata(), + ApiKeyTests.randomFutureExpirationTime() + ) ); // Test not found exception on API key of user with the same username but from a different realm @@ -2351,7 +2374,12 @@ public void testUpdateApiKeysNotFoundScenarios() throws Exception { "all" ).v1().get(0); doTestUpdateApiKeysNotFound( - new UpdateApiKeyRequest(apiKeyForNativeRealmUser.getId(), request.getRoleDescriptors(), request.getMetadata()) + new UpdateApiKeyRequest( + apiKeyForNativeRealmUser.getId(), + request.getRoleDescriptors(), + request.getMetadata(), + ApiKeyTests.randomFutureExpirationTime() + ) ); } @@ -2364,7 +2392,12 @@ public void testInvalidUpdateApiKeysScenarios() throws ExecutionException, Inter final var apiKeyId = createdApiKey.getId(); final var roleDescriptor = new RoleDescriptor(randomAlphaOfLength(10), new String[] { "manage_own_api_key" }, null, null); - final var request = new UpdateApiKeyRequest(apiKeyId, List.of(roleDescriptor), ApiKeyTests.randomMetadata()); + final var request = new UpdateApiKeyRequest( + apiKeyId, + List.of(roleDescriptor), + ApiKeyTests.randomMetadata(), + ApiKeyTests.randomFutureExpirationTime() + ); final PlainActionFuture updateListener = new PlainActionFuture<>(); client().filterWithHeader( Collections.singletonMap( @@ -2465,7 +2498,8 @@ public void testUpdateApiKeysNoopScenarios() throws Exception { List.of(new RoleDescriptor(randomAlphaOfLength(10), new String[] { "all" }, null, null)), // Ensure not `null` to set metadata since we use the initialRequest further down in the test to ensure that // metadata updates are non-noops - randomValueOtherThanMany(Objects::isNull, ApiKeyTests::randomMetadata) + randomValueOtherThanMany(Objects::isNull, ApiKeyTests::randomMetadata), + null // Expiration is relative current time, so must be null to cause noop ); UpdateApiKeyResponse response = updateSingleApiKeyMaybeUsingBulkAction(TEST_USER_NAME, initialRequest); assertNotNull(response); @@ -2501,14 +2535,17 @@ public void testUpdateApiKeysNoopScenarios() throws Exception { () -> RoleDescriptorTests.randomRoleDescriptor(false) ) ); - response = updateSingleApiKeyMaybeUsingBulkAction(TEST_USER_NAME, new UpdateApiKeyRequest(apiKeyId, newRoleDescriptors, null)); + response = updateSingleApiKeyMaybeUsingBulkAction( + TEST_USER_NAME, + new UpdateApiKeyRequest(apiKeyId, newRoleDescriptors, null, null) + ); assertNotNull(response); assertTrue(response.isUpdated()); // Update with re-ordered role descriptors is a noop response = updateSingleApiKeyMaybeUsingBulkAction( TEST_USER_NAME, - new UpdateApiKeyRequest(apiKeyId, List.of(newRoleDescriptors.get(1), newRoleDescriptors.get(0)), null) + new UpdateApiKeyRequest(apiKeyId, List.of(newRoleDescriptors.get(1), newRoleDescriptors.get(0)), null, null) ); assertNotNull(response); assertFalse(response.isUpdated()); @@ -2519,7 +2556,8 @@ public void testUpdateApiKeysNoopScenarios() throws Exception { new UpdateApiKeyRequest( apiKeyId, null, - randomValueOtherThanMany(md -> md == null || md.equals(initialRequest.getMetadata()), ApiKeyTests::randomMetadata) + randomValueOtherThanMany(md -> md == null || md.equals(initialRequest.getMetadata()), ApiKeyTests::randomMetadata), + null ) ); assertNotNull(response); @@ -2677,7 +2715,8 @@ public void testUpdateApiKeysClearsApiKeyDocCache() throws Exception { apiKey1.v1(), List.of(), // Set metadata to ensure update - Map.of(randomAlphaOfLength(5), randomAlphaOfLength(10)) + Map.of(randomAlphaOfLength(5), randomAlphaOfLength(10)), + ApiKeyTests.randomFutureExpirationTime() ) ); @@ -3251,7 +3290,12 @@ private UpdateApiKeyResponse updateSingleApiKeyMaybeUsingBulkAction(final String if (useBulkAction) { final BulkUpdateApiKeyResponse response = executeBulkUpdateApiKey( username, - new BulkUpdateApiKeyRequest(List.of(request.getId()), request.getRoleDescriptors(), request.getMetadata()) + new BulkUpdateApiKeyRequest( + List.of(request.getId()), + request.getRoleDescriptors(), + request.getMetadata(), + request.getExpiration() + ) ); return toUpdateApiKeyResponse(request.getId(), response); } else { diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/apikey/ApiKeySingleNodeTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/apikey/ApiKeySingleNodeTests.java index 91884086af959..33cd3de9e0685 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/apikey/ApiKeySingleNodeTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/apikey/ApiKeySingleNodeTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.security.action.Grant; import org.elasticsearch.xpack.core.security.action.apikey.ApiKey; +import org.elasticsearch.xpack.core.security.action.apikey.ApiKeyTests; import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyAction; import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyRequest; import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyRequestBuilder; @@ -706,7 +707,13 @@ public void testUpdateCrossClusterApiKey() throws IOException { updateMetadata = null; } - final var updateApiKeyRequest = new UpdateCrossClusterApiKeyRequest(apiKeyId, roleDescriptorBuilder, updateMetadata); + final boolean shouldUpdateExpiration = randomBoolean(); + TimeValue expiration = null; + if (shouldUpdateExpiration) { + ApiKeyTests.randomFutureExpirationTime(); + } + + final var updateApiKeyRequest = new UpdateCrossClusterApiKeyRequest(apiKeyId, roleDescriptorBuilder, updateMetadata, expiration); final UpdateApiKeyResponse updateApiKeyResponse = client().execute(UpdateCrossClusterApiKeyAction.INSTANCE, updateApiKeyRequest) .actionGet(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyAction.java index 011b95565e030..a47bbb0301ebc 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyAction.java @@ -50,7 +50,12 @@ void doExecuteUpdate( ) { apiKeyService.updateApiKeys( authentication, - new BaseBulkUpdateApiKeyRequest(List.of(request.getId()), request.getRoleDescriptors(), request.getMetadata()) { + new BaseBulkUpdateApiKeyRequest( + List.of(request.getId()), + request.getRoleDescriptors(), + request.getMetadata(), + request.getExpiration() + ) { @Override public ApiKey.Type getType() { return ApiKey.Type.CROSS_CLUSTER; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java index 6f82acfcebb44..9f2de14db2946 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java @@ -410,7 +410,7 @@ private void createApiKeyAndIndexIt( ActionListener listener ) { final Instant created = clock.instant(); - final Instant expiration = getApiKeyExpiration(created, request); + final Instant expiration = getApiKeyExpiration(created, request.getExpiration()); final SecureString apiKey = UUIDs.randomBase64UUIDSecureString(); assert ApiKey.Type.CROSS_CLUSTER != request.getType() || API_KEY_SECRET_LENGTH == apiKey.length(); final Version version = clusterService.state().nodes().getMinNodeVersion(); @@ -743,7 +743,8 @@ static XContentBuilder maybeBuildUpdatedDocument( final Version targetDocVersion, final Authentication authentication, final BaseUpdateApiKeyRequest request, - final Set userRoleDescriptors + final Set userRoleDescriptors, + final Clock clock ) throws IOException { assert currentApiKeyDoc.type == request.getType(); if (isNoop(apiKeyId, currentApiKeyDoc, targetDocVersion, authentication, request, userRoleDescriptors)) { @@ -755,9 +756,14 @@ static XContentBuilder maybeBuildUpdatedDocument( .field("doc_type", "api_key") .field("type", currentApiKeyDoc.type.value()) .field("creation_time", currentApiKeyDoc.creationTime) - .field("expiration_time", currentApiKeyDoc.expirationTime == -1 ? null : currentApiKeyDoc.expirationTime) .field("api_key_invalidated", false); + if (request.getExpiration() != null) { + builder.field("expiration_time", getApiKeyExpiration(clock.instant(), request.getExpiration()).toEpochMilli()); + } else { + builder.field("expiration_time", currentApiKeyDoc.expirationTime == -1 ? null : currentApiKeyDoc.expirationTime); + } + addApiKeyHash(builder, currentApiKeyDoc.hash.toCharArray()); final List keyRoles = request.getRoleDescriptors(); @@ -808,6 +814,11 @@ private static boolean isNoop( return false; } + if (request.getExpiration() != null) { + // Since expiration is relative current time, it's not likely that it matches the stored value to the ms, so assume update + return false; + } + final Map currentCreator = apiKeyDoc.creator; final var user = authentication.getEffectiveSubject().getUser(); final var sourceRealm = authentication.getEffectiveSubject().getRealm(); @@ -1280,9 +1291,9 @@ protected void verifyKeyAgainstHash(String apiKeyHash, ApiKeyCredentials credent })); } - private static Instant getApiKeyExpiration(Instant now, AbstractCreateApiKeyRequest request) { - if (request.getExpiration() != null) { - return now.plusSeconds(request.getExpiration().getSeconds()); + private static Instant getApiKeyExpiration(Instant now, @Nullable TimeValue expiration) { + if (expiration != null) { + return now.plusSeconds(expiration.getSeconds()); } else { return null; } @@ -1472,7 +1483,8 @@ private IndexRequest maybeBuildIndexRequest( targetDocVersion, authentication, request, - userRoleDescriptors + userRoleDescriptors, + clock ); final boolean isNoop = builder == null; return isNoop diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestBulkUpdateApiKeyAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestBulkUpdateApiKeyAction.java index c436370d67579..584ad08704ddd 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestBulkUpdateApiKeyAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestBulkUpdateApiKeyAction.java @@ -9,6 +9,7 @@ import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.Scope; @@ -35,7 +36,12 @@ public final class RestBulkUpdateApiKeyAction extends ApiKeyBaseRestHandler { @SuppressWarnings("unchecked") static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "bulk_update_api_key_request", - a -> new BulkUpdateApiKeyRequest((List) a[0], (List) a[1], (Map) a[2]) + a -> new BulkUpdateApiKeyRequest( + (List) a[0], + (List) a[1], + (Map) a[2], + TimeValue.parseTimeValue((String) a[3], null, "expiration") + ) ); static { @@ -45,6 +51,7 @@ public final class RestBulkUpdateApiKeyAction extends ApiKeyBaseRestHandler { return RoleDescriptor.parse(n, p, false); }, new ParseField("role_descriptors")); PARSER.declareObject(optionalConstructorArg(), (p, c) -> p.map(), new ParseField("metadata")); + PARSER.declareString(optionalConstructorArg(), new ParseField("expiration")); } public RestBulkUpdateApiKeyAction(final Settings settings, final XPackLicenseState licenseState) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateApiKeyAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateApiKeyAction.java index 16c323eaca76e..d64e7f4007387 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateApiKeyAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateApiKeyAction.java @@ -9,6 +9,7 @@ import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.Scope; @@ -33,7 +34,11 @@ public final class RestUpdateApiKeyAction extends ApiKeyBaseRestHandler { @SuppressWarnings("unchecked") static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "update_api_key_request_payload", - a -> new Payload((List) a[0], (Map) a[1]) + a -> new Payload( + (List) a[0], + (Map) a[1], + TimeValue.parseTimeValue((String) a[2], null, "expiration") + ) ); static { @@ -42,6 +47,7 @@ public final class RestUpdateApiKeyAction extends ApiKeyBaseRestHandler { return RoleDescriptor.parse(n, p, false); }, new ParseField("role_descriptors")); PARSER.declareObject(optionalConstructorArg(), (p, c) -> p.map(), new ParseField("metadata")); + PARSER.declareString(optionalConstructorArg(), new ParseField("expiration")); } public RestUpdateApiKeyAction(final Settings settings, final XPackLicenseState licenseState) { @@ -64,13 +70,13 @@ protected RestChannelConsumer innerPrepareRequest(final RestRequest request, fin // `RestClearApiKeyCacheAction` and our current REST implementation requires that path params have the same wildcard if their paths // share a prefix final var apiKeyId = request.param("ids"); - final var payload = request.hasContent() == false ? new Payload(null, null) : PARSER.parse(request.contentParser(), null); + final var payload = request.hasContent() == false ? new Payload(null, null, null) : PARSER.parse(request.contentParser(), null); return channel -> client.execute( UpdateApiKeyAction.INSTANCE, - new UpdateApiKeyRequest(apiKeyId, payload.roleDescriptors, payload.metadata), + new UpdateApiKeyRequest(apiKeyId, payload.roleDescriptors, payload.metadata, payload.expiration), new RestToXContentListener<>(channel) ); } - record Payload(List roleDescriptors, Map metadata) {} + record Payload(List roleDescriptors, Map metadata, TimeValue expiration) {} } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateCrossClusterApiKeyAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateCrossClusterApiKeyAction.java index a642723667639..e9244eaea0ec5 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateCrossClusterApiKeyAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/apikey/RestUpdateCrossClusterApiKeyAction.java @@ -9,6 +9,7 @@ import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestRequest; @@ -32,12 +33,17 @@ public final class RestUpdateCrossClusterApiKeyAction extends ApiKeyBaseRestHand @SuppressWarnings("unchecked") static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "update_cross_cluster_api_key_request_payload", - a -> new Payload((CrossClusterApiKeyRoleDescriptorBuilder) a[0], (Map) a[1]) + a -> new Payload( + (CrossClusterApiKeyRoleDescriptorBuilder) a[0], + (Map) a[1], + TimeValue.parseTimeValue((String) a[2], null, "expiration") + ) ); static { PARSER.declareObject(optionalConstructorArg(), CrossClusterApiKeyRoleDescriptorBuilder.PARSER, new ParseField("access")); PARSER.declareObject(optionalConstructorArg(), (p, c) -> p.map(), new ParseField("metadata")); + PARSER.declareString(optionalConstructorArg(), new ParseField("expiration")); } public RestUpdateCrossClusterApiKeyAction(final Settings settings, final XPackLicenseState licenseState) { @@ -61,7 +67,7 @@ protected RestChannelConsumer innerPrepareRequest(final RestRequest request, fin return channel -> client.execute( UpdateCrossClusterApiKeyAction.INSTANCE, - new UpdateCrossClusterApiKeyRequest(apiKeyId, payload.builder, payload.metadata), + new UpdateCrossClusterApiKeyRequest(apiKeyId, payload.builder, payload.metadata, payload.expiration), new RestToXContentListener<>(channel) ); } @@ -75,5 +81,5 @@ protected Exception innerCheckFeatureAvailable(RestRequest request) { } } - record Payload(CrossClusterApiKeyRoleDescriptorBuilder builder, Map metadata) {} + record Payload(CrossClusterApiKeyRoleDescriptorBuilder builder, Map metadata, TimeValue expiration) {} } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyActionTests.java index 7ce920506d7d1..70190b70f3f1a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/apikey/TransportUpdateCrossClusterApiKeyActionTests.java @@ -74,7 +74,12 @@ public void testExecute() throws IOException { } final String id = randomAlphaOfLength(10); - final var request = new UpdateCrossClusterApiKeyRequest(id, roleDescriptorBuilder, metadata); + final var request = new UpdateCrossClusterApiKeyRequest( + id, + roleDescriptorBuilder, + metadata, + ApiKeyTests.randomFutureExpirationTime() + ); final int updateStatus = randomIntBetween(0, 2); // 0 - success, 1 - noop, 2 - error doAnswer(invocation -> { @@ -129,7 +134,12 @@ public void testAuthenticationCheck() { mock(ApiKeyService.class), securityContext ); - final var request = new UpdateCrossClusterApiKeyRequest(randomAlphaOfLength(10), null, Map.of()); + final var request = new UpdateCrossClusterApiKeyRequest( + randomAlphaOfLength(10), + null, + Map.of(), + ApiKeyTests.randomFutureExpirationTime() + ); // null authentication error when(securityContext.getAuthentication()).thenReturn(null); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java index 3385b02147890..9c48354b951d8 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java @@ -46,6 +46,7 @@ import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.security.action.apikey.ApiKeyTests; import org.elasticsearch.xpack.core.security.action.apikey.BulkUpdateApiKeyAction; import org.elasticsearch.xpack.core.security.action.apikey.BulkUpdateApiKeyRequest; import org.elasticsearch.xpack.core.security.action.apikey.CreateApiKeyAction; @@ -629,7 +630,8 @@ public void testSecurityConfigChangeEventFormattingForRoles() throws IOException final var updateApiKeyRequest = new UpdateApiKeyRequest( keyId, randomBoolean() ? null : keyRoleDescriptors, - metadataWithSerialization.metadata() + metadataWithSerialization.metadata(), + ApiKeyTests.randomFutureExpirationTime() ); auditTrail.accessGranted(requestId, authentication, UpdateApiKeyAction.NAME, updateApiKeyRequest, authorizationInfo); final var expectedUpdateKeyAuditEventString = String.format( @@ -661,7 +663,8 @@ public void testSecurityConfigChangeEventFormattingForRoles() throws IOException final var bulkUpdateApiKeyRequest = new BulkUpdateApiKeyRequest( keyIds, randomBoolean() ? null : keyRoleDescriptors, - metadataWithSerialization.metadata() + metadataWithSerialization.metadata(), + ApiKeyTests.randomFutureExpirationTime() ); auditTrail.accessGranted(requestId, authentication, BulkUpdateApiKeyAction.NAME, bulkUpdateApiKeyRequest, authorizationInfo); final var expectedBulkUpdateKeyAuditEventString = String.format( @@ -875,7 +878,8 @@ public void testSecurityConfigChangeEventForCrossClusterApiKeys() throws IOExcep final var updateRequest = new UpdateCrossClusterApiKeyRequest( createRequest.getId(), updateAccess, - updateMetadataWithSerialization.metadata() + updateMetadataWithSerialization.metadata(), + ApiKeyTests.randomFutureExpirationTime() ); auditTrail.accessGranted(requestId, authentication, UpdateCrossClusterApiKeyAction.NAME, updateRequest, authorizationInfo); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java index cd297abe8f28d..b921fef9fd917 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java @@ -2119,6 +2119,8 @@ public void testMaybeBuildUpdatedDocument() throws IOException { } else { oldKeyRoles = randomList(3, RoleDescriptorTests::randomRoleDescriptor); } + final long now = randomMillisUpToYear9999(); + when(clock.instant()).thenReturn(Instant.ofEpochMilli(now)); final Map oldMetadata = ApiKeyTests.randomMetadata(); final Version oldVersion = VersionUtils.randomVersion(random()); final ApiKeyDoc oldApiKeyDoc = ApiKeyDoc.fromXContent( @@ -2147,6 +2149,8 @@ public void testMaybeBuildUpdatedDocument() throws IOException { final boolean changeMetadata = randomBoolean(); final boolean changeVersion = randomBoolean(); final boolean changeCreator = randomBoolean(); + final boolean changeExpiration = randomBoolean(); + final Set newUserRoles = changeUserRoles ? randomValueOtherThan(oldUserRoles, () -> randomSet(0, 3, RoleDescriptorTests::randomRoleDescriptor)) : oldUserRoles; @@ -2180,11 +2184,14 @@ public void testMaybeBuildUpdatedDocument() throws IOException { .build(false) ) : oldAuthentication; + final TimeValue newExpiration = changeExpiration ? randomFrom(ApiKeyTests.randomFutureExpirationTime()) : null; final String apiKeyId = randomAlphaOfLength(10); final BaseUpdateApiKeyRequest request = mock(BaseUpdateApiKeyRequest.class); when(request.getType()).thenReturn(type); when(request.getRoleDescriptors()).thenReturn(newKeyRoles); when(request.getMetadata()).thenReturn(newMetadata); + when(request.getExpiration()).thenReturn(newExpiration); + final var service = createApiKeyService(); final XContentBuilder builder = ApiKeyService.maybeBuildUpdatedDocument( @@ -2193,10 +2200,16 @@ public void testMaybeBuildUpdatedDocument() throws IOException { newVersion, newAuthentication, request, - newUserRoles + newUserRoles, + clock ); - final boolean noop = (changeCreator || changeMetadata || changeKeyRoles || changeUserRoles || changeVersion) == false; + final boolean noop = (changeCreator + || changeMetadata + || changeKeyRoles + || changeUserRoles + || changeVersion + || changeExpiration) == false; if (noop) { assertNull(builder); } else { @@ -2207,7 +2220,6 @@ public void testMaybeBuildUpdatedDocument() throws IOException { assertEquals(oldApiKeyDoc.type, updatedApiKeyDoc.type); assertEquals(oldApiKeyDoc.name, updatedApiKeyDoc.name); assertEquals(oldApiKeyDoc.hash, updatedApiKeyDoc.hash); - assertEquals(oldApiKeyDoc.expirationTime, updatedApiKeyDoc.expirationTime); assertEquals(oldApiKeyDoc.creationTime, updatedApiKeyDoc.creationTime); assertEquals(oldApiKeyDoc.invalidated, updatedApiKeyDoc.invalidated); assertEquals(newVersion.id, updatedApiKeyDoc.version); @@ -2237,6 +2249,11 @@ public void testMaybeBuildUpdatedDocument() throws IOException { } else { assertEquals(newMetadata, XContentHelper.convertToMap(updatedApiKeyDoc.metadataFlattened, true, XContentType.JSON).v2()); } + if (newExpiration != null) { + assertEquals(clock.instant().plusSeconds(newExpiration.getSeconds()).toEpochMilli(), updatedApiKeyDoc.expirationTime); + } else { + assertEquals(oldApiKeyDoc.expirationTime, updatedApiKeyDoc.expirationTime); + } assertEquals(newAuthentication.getEffectiveSubject().getUser().principal(), updatedApiKeyDoc.creator.get("principal")); assertEquals(newAuthentication.getEffectiveSubject().getUser().fullName(), updatedApiKeyDoc.creator.get("full_name")); assertEquals(newAuthentication.getEffectiveSubject().getUser().email(), updatedApiKeyDoc.creator.get("email")); @@ -2602,7 +2619,8 @@ public void testCreateOrUpdateApiKeyWithWorkflowsRestrictionForUnsupportedVersio final BulkUpdateApiKeyRequest updateRequest = new BulkUpdateApiKeyRequest( randomList(1, 3, () -> randomAlphaOfLengthBetween(3, 5)), roleDescriptorsWithWorkflowsRestriction, - Map.of() + Map.of(), + ApiKeyTests.randomFutureExpirationTime() ); final PlainActionFuture updateFuture = new PlainActionFuture<>(); service.updateApiKeys(authentication, updateRequest, Set.of(), updateFuture); @@ -2664,7 +2682,8 @@ public void testValidateOwnerUserRoleDescriptorsWithWorkflowsRestriction() { final BulkUpdateApiKeyRequest updateRequest = new BulkUpdateApiKeyRequest( randomList(1, 3, () -> randomAlphaOfLengthBetween(3, 5)), requestRoleDescriptors, - Map.of() + Map.of(), + ApiKeyTests.randomFutureExpirationTime() ); final PlainActionFuture updateFuture = new PlainActionFuture<>(); service.updateApiKeys(authentication, updateRequest, userRoleDescriptorsWithWorkflowsRestriction, updateFuture); diff --git a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookTokenIntegrationTests.java b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookTokenIntegrationTests.java index 9724aa6e0a8ce..7da2c5b718356 100644 --- a/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookTokenIntegrationTests.java +++ b/x-pack/plugin/watcher/src/internalClusterTest/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookTokenIntegrationTests.java @@ -100,8 +100,12 @@ public void testWebhook() throws Exception { } // Reload the keystore to load the new settings NodesReloadSecureSettingsRequest reloadReq = new NodesReloadSecureSettingsRequest(); - reloadReq.setSecureStorePassword(new SecureString("".toCharArray())); - client().execute(TransportNodesReloadSecureSettingsAction.TYPE, reloadReq).get(); + try { + reloadReq.setSecureStorePassword(new SecureString("".toCharArray())); + client().execute(TransportNodesReloadSecureSettingsAction.TYPE, reloadReq).get(); + } finally { + reloadReq.decRef(); + } webServer.enqueue(new MockResponse().setResponseCode(200).setBody("body")); HttpRequestTemplate.Builder builder = HttpRequestTemplate.builder("localhost", webServer.getPort())