From 29965bccf0df9d27f1f7baab1b40529274864940 Mon Sep 17 00:00:00 2001
From: Parker Timmins
Date: Tue, 4 Feb 2025 17:59:21 -0600
Subject: [PATCH 01/29] Add pipeline to clean docs during data stream reindex
(#121617)
Add the pipeline "reindex-data-stream-pipeline" to the reindex request within ReindexDataStreamIndexAction. This cleans up documents as needed before inserting into the destination index. Currently, the pipeline only sets a timestamp field with a value of 0, if the document is missing a timestamp field. This is needed because existing indices which are added to a data stream may not contain a timestamp, but reindex validates that a timestamp field exists when creating data stream destination indices.
This pipeline is managed by ES, but can be overriden by users if necessary. To do this, the version field of the pipeline should be set to a value higher than the MigrateRegistry version.
---
.../core/security/user/InternalUsers.java | 1 +
.../reindex-data-stream-pipeline.json | 16 ++
x-pack/plugin/migrate/build.gradle | 1 +
...indexDatastreamIndexTransportActionIT.java | 146 ++++++++++++++++--
.../xpack/migrate/MigratePlugin.java | 13 ++
.../migrate/MigrateTemplateRegistry.java | 55 +++++++
...ReindexDataStreamIndexTransportAction.java | 2 +
.../upgrades/DataStreamsUpgradeIT.java | 2 +-
8 files changed, 223 insertions(+), 13 deletions(-)
create mode 100644 x-pack/plugin/core/template-resources/src/main/resources/reindex-data-stream-pipeline.json
create mode 100644 x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigrateTemplateRegistry.java
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/user/InternalUsers.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/user/InternalUsers.java
index 1229d62dce04..a704b350dba4 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/user/InternalUsers.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/user/InternalUsers.java
@@ -225,6 +225,7 @@ public class InternalUsers {
.build() },
null,
null,
+
new String[] {},
MetadataUtils.DEFAULT_RESERVED_METADATA,
Map.of()
diff --git a/x-pack/plugin/core/template-resources/src/main/resources/reindex-data-stream-pipeline.json b/x-pack/plugin/core/template-resources/src/main/resources/reindex-data-stream-pipeline.json
new file mode 100644
index 000000000000..e8c335213170
--- /dev/null
+++ b/x-pack/plugin/core/template-resources/src/main/resources/reindex-data-stream-pipeline.json
@@ -0,0 +1,16 @@
+{
+ "description": "This pipeline sanitizes documents that are being reindexed into a data stream using the reindex data stream API. It is an internal pipeline and should not be modified.",
+ "processors": [
+ {
+ "set": {
+ "field": "@timestamp",
+ "value": 0,
+ "override": false
+ }
+ }
+ ],
+ "_meta": {
+ "managed": true
+ },
+ "version": ${xpack.migrate.reindex.pipeline.version}
+}
diff --git a/x-pack/plugin/migrate/build.gradle b/x-pack/plugin/migrate/build.gradle
index 283362a637e7..f179a311e0fe 100644
--- a/x-pack/plugin/migrate/build.gradle
+++ b/x-pack/plugin/migrate/build.gradle
@@ -19,6 +19,7 @@ dependencies {
testImplementation project(xpackModule('ccr'))
testImplementation project(':modules:data-streams')
testImplementation project(path: ':modules:reindex')
+ testImplementation project(path: ':modules:ingest-common')
}
addQaCheckDependencies(project)
diff --git a/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDatastreamIndexTransportActionIT.java b/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDatastreamIndexTransportActionIT.java
index e22c26bd6b8b..1ffeaed0bd1d 100644
--- a/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDatastreamIndexTransportActionIT.java
+++ b/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDatastreamIndexTransportActionIT.java
@@ -24,12 +24,17 @@ import org.elasticsearch.action.admin.indices.template.put.TransportPutComposabl
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
+import org.elasticsearch.action.ingest.DeletePipelineTransportAction;
+import org.elasticsearch.action.ingest.PutPipelineRequest;
+import org.elasticsearch.action.ingest.PutPipelineTransportAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.ComposableIndexTemplate;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.MappingMetadata;
import org.elasticsearch.cluster.metadata.MetadataIndexStateService;
import org.elasticsearch.cluster.metadata.Template;
+import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateFormatter;
@@ -38,12 +43,15 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.datastreams.DataStreamsPlugin;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.DateFieldMapper;
+import org.elasticsearch.ingest.common.IngestCommonPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.reindex.ReindexPlugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.migrate.MigratePlugin;
+import org.elasticsearch.xpack.migrate.MigrateTemplateRegistry;
+import org.junit.After;
import java.io.IOException;
import java.time.Instant;
@@ -56,19 +64,27 @@ import static java.lang.Boolean.parseBoolean;
import static org.elasticsearch.cluster.metadata.MetadataIndexTemplateService.DEFAULT_TIMESTAMP_FIELD;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse;
import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
import static org.hamcrest.Matchers.equalTo;
public class ReindexDatastreamIndexTransportActionIT extends ESIntegTestCase {
+ @After
+ private void cleanupCluster() throws Exception {
+ clusterAdmin().execute(
+ DeletePipelineTransportAction.TYPE,
+ new DeletePipelineRequest(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, MigrateTemplateRegistry.REINDEX_DATA_STREAM_PIPELINE_NAME)
+ );
+ super.cleanUpCluster();
+ }
private static final String MAPPING = """
{
"_doc":{
"dynamic":"strict",
"properties":{
- "foo1":{
- "type":"text"
- }
+ "foo1": {"type":"text"},
+ "@timestamp": {"type":"date"}
}
}
}
@@ -76,7 +92,116 @@ public class ReindexDatastreamIndexTransportActionIT extends ESIntegTestCase {
@Override
protected Collection> nodePlugins() {
- return List.of(MigratePlugin.class, ReindexPlugin.class, MockTransportService.TestPlugin.class, DataStreamsPlugin.class);
+ return List.of(
+ MigratePlugin.class,
+ ReindexPlugin.class,
+ MockTransportService.TestPlugin.class,
+ DataStreamsPlugin.class,
+ IngestCommonPlugin.class
+ );
+ }
+
+ private static String DATA_STREAM_MAPPING = """
+ {
+ "dynamic": true,
+ "_data_stream_timestamp": {
+ "enabled": true
+ },
+ "properties": {
+ "@timestamp": {"type":"date"}
+ }
+ }
+ """;
+
+ public void testTimestamp0AddedIfMissing() {
+ var sourceIndex = randomAlphaOfLength(20).toLowerCase(Locale.ROOT);
+ indicesAdmin().create(new CreateIndexRequest(sourceIndex)).actionGet();
+
+ // add doc without timestamp
+ addDoc(sourceIndex, "{\"foo\":\"baz\"}");
+
+ // add timestamp to source mapping
+ indicesAdmin().preparePutMapping(sourceIndex).setSource(DATA_STREAM_MAPPING, XContentType.JSON).get();
+
+ // call reindex
+ var destIndex = client().execute(ReindexDataStreamIndexAction.INSTANCE, new ReindexDataStreamIndexAction.Request(sourceIndex))
+ .actionGet()
+ .getDestIndex();
+
+ assertResponse(prepareSearch(destIndex), response -> {
+ Map sourceAsMap = response.getHits().getAt(0).getSourceAsMap();
+ assertEquals(Integer.valueOf(0), sourceAsMap.get(DEFAULT_TIMESTAMP_FIELD));
+ });
+ }
+
+ public void testTimestampNotAddedIfExists() {
+ var sourceIndex = randomAlphaOfLength(20).toLowerCase(Locale.ROOT);
+ indicesAdmin().create(new CreateIndexRequest(sourceIndex)).actionGet();
+
+ // add doc with timestamp
+ String time = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(System.currentTimeMillis());
+ var doc = String.format(Locale.ROOT, "{\"%s\":\"%s\"}", DEFAULT_TIMESTAMP_FIELD, time);
+ addDoc(sourceIndex, doc);
+
+ // add timestamp to source mapping
+ indicesAdmin().preparePutMapping(sourceIndex).setSource(DATA_STREAM_MAPPING, XContentType.JSON).get();
+
+ // call reindex
+ var destIndex = client().execute(ReindexDataStreamIndexAction.INSTANCE, new ReindexDataStreamIndexAction.Request(sourceIndex))
+ .actionGet()
+ .getDestIndex();
+
+ assertResponse(prepareSearch(destIndex), response -> {
+ Map sourceAsMap = response.getHits().getAt(0).getSourceAsMap();
+ assertEquals(time, sourceAsMap.get(DEFAULT_TIMESTAMP_FIELD));
+ });
+ }
+
+ public void testCustomReindexPipeline() {
+ String customPipeline = """
+ {
+ "processors": [
+ {
+ "set": {
+ "field": "cheese",
+ "value": "gorgonzola"
+ }
+ }
+ ],
+ "version": 1000
+ }
+ """;
+
+ PutPipelineRequest putRequest = new PutPipelineRequest(
+ TEST_REQUEST_TIMEOUT,
+ TEST_REQUEST_TIMEOUT,
+ MigrateTemplateRegistry.REINDEX_DATA_STREAM_PIPELINE_NAME,
+ new BytesArray(customPipeline),
+ XContentType.JSON
+ );
+
+ clusterAdmin().execute(PutPipelineTransportAction.TYPE, putRequest).actionGet();
+
+ var sourceIndex = randomAlphaOfLength(20).toLowerCase(Locale.ROOT);
+ indicesAdmin().create(new CreateIndexRequest(sourceIndex)).actionGet();
+
+ // add doc with timestamp
+ String time = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(System.currentTimeMillis());
+ var doc = String.format(Locale.ROOT, "{\"%s\":\"%s\"}", DEFAULT_TIMESTAMP_FIELD, time);
+ addDoc(sourceIndex, doc);
+
+ // add timestamp to source mapping
+ indicesAdmin().preparePutMapping(sourceIndex).setSource(DATA_STREAM_MAPPING, XContentType.JSON).get();
+
+ String destIndex = client().execute(ReindexDataStreamIndexAction.INSTANCE, new ReindexDataStreamIndexAction.Request(sourceIndex))
+ .actionGet()
+ .getDestIndex();
+
+ assertResponse(prepareSearch(destIndex), response -> {
+ Map sourceAsMap = response.getHits().getAt(0).getSourceAsMap();
+ assertEquals("gorgonzola", sourceAsMap.get("cheese"));
+ assertEquals(time, sourceAsMap.get(DEFAULT_TIMESTAMP_FIELD));
+ });
}
public void testDestIndexDeletedIfExists() throws Exception {
@@ -200,7 +325,7 @@ public class ReindexDatastreamIndexTransportActionIT extends ESIntegTestCase {
assertEquals(refreshInterval, settingsResponse.getSetting(destIndex, IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey()));
}
- public void testMappingsAddedToDestIndex() throws Exception {
+ public void testMappingsAddedToDestIndex() {
var sourceIndex = randomAlphaOfLength(20).toLowerCase(Locale.ROOT);
indicesAdmin().create(new CreateIndexRequest(sourceIndex).mapping(MAPPING)).actionGet();
@@ -479,12 +604,9 @@ public class ReindexDatastreamIndexTransportActionIT extends ESIntegTestCase {
return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant);
}
- private static String getIndexUUID(String index) {
- return indicesAdmin().getIndex(new GetIndexRequest(TEST_REQUEST_TIMEOUT).indices(index))
- .actionGet()
- .getSettings()
- .get(index)
- .get(IndexMetadata.SETTING_INDEX_UUID);
+ void addDoc(String index, String doc) {
+ BulkRequest bulkRequest = new BulkRequest();
+ bulkRequest.add(new IndexRequest(index).opType(DocWriteRequest.OpType.CREATE).source(doc, XContentType.JSON));
+ client().bulk(bulkRequest).actionGet();
}
-
}
diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java
index f5f8beba26d8..7811e84ac9f5 100644
--- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java
+++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java
@@ -55,6 +55,7 @@ import org.elasticsearch.xpack.migrate.task.ReindexDataStreamTask;
import org.elasticsearch.xpack.migrate.task.ReindexDataStreamTaskParams;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.List;
import java.util.function.Predicate;
import java.util.function.Supplier;
@@ -64,6 +65,18 @@ import static org.elasticsearch.xpack.migrate.action.ReindexDataStreamIndexTrans
import static org.elasticsearch.xpack.migrate.task.ReindexDataStreamPersistentTaskExecutor.MAX_CONCURRENT_INDICES_REINDEXED_PER_DATA_STREAM_SETTING;
public class MigratePlugin extends Plugin implements ActionPlugin, PersistentTaskPlugin {
+ @Override
+ public Collection> createComponents(PluginServices services) {
+ var registry = new MigrateTemplateRegistry(
+ services.environment().settings(),
+ services.clusterService(),
+ services.threadPool(),
+ services.client(),
+ services.xContentRegistry()
+ );
+ registry.initialize();
+ return List.of(registry);
+ }
@Override
public List getRestHandlers(
diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigrateTemplateRegistry.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigrateTemplateRegistry.java
new file mode 100644
index 000000000000..2a9dc97e1635
--- /dev/null
+++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigrateTemplateRegistry.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+package org.elasticsearch.xpack.migrate;
+
+import org.elasticsearch.client.internal.Client;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.threadpool.ThreadPool;
+import org.elasticsearch.xcontent.NamedXContentRegistry;
+import org.elasticsearch.xpack.core.ClientHelper;
+import org.elasticsearch.xpack.core.template.IndexTemplateRegistry;
+import org.elasticsearch.xpack.core.template.IngestPipelineConfig;
+import org.elasticsearch.xpack.core.template.JsonIngestPipelineConfig;
+
+import java.util.List;
+
+public class MigrateTemplateRegistry extends IndexTemplateRegistry {
+
+ // This number must be incremented when we make changes to built-in pipeline.
+ // If a specific user pipeline is needed instead, its version should be set to a value higher than the REGISTRY_VERSION.
+ static final int REGISTRY_VERSION = 1;
+ public static final String REINDEX_DATA_STREAM_PIPELINE_NAME = "reindex-data-stream-pipeline";
+ private static final String TEMPLATE_VERSION_VARIABLE = "xpack.migrate.reindex.pipeline.version";
+
+ public MigrateTemplateRegistry(
+ Settings nodeSettings,
+ ClusterService clusterService,
+ ThreadPool threadPool,
+ Client client,
+ NamedXContentRegistry xContentRegistry
+ ) {
+ super(nodeSettings, clusterService, threadPool, client, xContentRegistry);
+ }
+
+ @Override
+ protected List getIngestPipelines() {
+ return List.of(
+ new JsonIngestPipelineConfig(
+ REINDEX_DATA_STREAM_PIPELINE_NAME,
+ "/" + REINDEX_DATA_STREAM_PIPELINE_NAME + ".json",
+ REGISTRY_VERSION,
+ TEMPLATE_VERSION_VARIABLE
+ )
+ );
+ }
+
+ @Override
+ protected String getOrigin() {
+ return ClientHelper.STACK_ORIGIN;
+ }
+}
diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java
index 456c61451f7f..792ec4ec2b6f 100644
--- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java
+++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java
@@ -53,6 +53,7 @@ import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.deprecation.DeprecatedIndexPredicate;
+import org.elasticsearch.xpack.migrate.MigrateTemplateRegistry;
import java.util.Locale;
import java.util.Map;
@@ -271,6 +272,7 @@ public class ReindexDataStreamIndexTransportAction extends HandledTransportActio
logger.debug("Reindex to destination index [{}] from source index [{}]", destIndexName, sourceIndexName);
var reindexRequest = new ReindexRequest();
reindexRequest.setSourceIndices(sourceIndexName);
+ reindexRequest.setDestPipeline(MigrateTemplateRegistry.REINDEX_DATA_STREAM_PIPELINE_NAME);
reindexRequest.getSearchRequest().allowPartialSearchResults(false);
reindexRequest.getSearchRequest().source().fetchSource(true);
reindexRequest.setDestIndex(destIndexName);
diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataStreamsUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataStreamsUpgradeIT.java
index e55e35ae0932..6f72382078a2 100644
--- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataStreamsUpgradeIT.java
+++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/DataStreamsUpgradeIT.java
@@ -195,7 +195,7 @@ public class DataStreamsUpgradeIT extends AbstractUpgradeTestCase {
createDataStreamFromNonDataStreamIndices(dataStreamFromNonDataStreamIndices);
} else if (CLUSTER_TYPE == ClusterType.UPGRADED) {
upgradeDataStream(dataStreamName, numRollovers, numRollovers + 1, 0);
- upgradeDataStream(dataStreamFromNonDataStreamIndices, 0, 0, 1);
+ upgradeDataStream(dataStreamFromNonDataStreamIndices, 0, 1, 0);
}
}
From e885da1e941e103d9dd3404616ea4a7327988a9b Mon Sep 17 00:00:00 2001
From: Oleksandr Kolomiiets
Date: Tue, 4 Feb 2025 16:13:20 -0800
Subject: [PATCH 02/29] Introduce FallbackSyntheticSourceBlockLoader and apply
it to keyword fields (#119546)
---
docs/changelog/119546.yaml | 5 +
.../FallbackSyntheticSourceBlockLoader.java | 270 ++++++++++++++++++
.../index/mapper/KeywordFieldMapper.java | 60 +++-
.../index/mapper/XContentDataHelper.java | 92 ++++++
.../KeywordFieldBlockLoaderTests.java | 30 +-
.../index/mapper/BlockLoaderTestCase.java | 115 ++++++--
.../DefaultMappingParametersHandler.java | 3 +
.../matchers/source/FieldSpecificMatcher.java | 59 ++++
.../matchers/source/SourceMatcher.java | 4 +-
9 files changed, 604 insertions(+), 34 deletions(-)
create mode 100644 docs/changelog/119546.yaml
create mode 100644 server/src/main/java/org/elasticsearch/index/mapper/FallbackSyntheticSourceBlockLoader.java
diff --git a/docs/changelog/119546.yaml b/docs/changelog/119546.yaml
new file mode 100644
index 000000000000..017bbb845c0a
--- /dev/null
+++ b/docs/changelog/119546.yaml
@@ -0,0 +1,5 @@
+pr: 119546
+summary: Introduce `FallbackSyntheticSourceBlockLoader` and apply it to keyword fields
+area: Mapping
+type: enhancement
+issues: []
diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FallbackSyntheticSourceBlockLoader.java b/server/src/main/java/org/elasticsearch/index/mapper/FallbackSyntheticSourceBlockLoader.java
new file mode 100644
index 000000000000..28ea37ef73e3
--- /dev/null
+++ b/server/src/main/java/org/elasticsearch/index/mapper/FallbackSyntheticSourceBlockLoader.java
@@ -0,0 +1,270 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side
+ * Public License v 1"; you may not use this file except in compliance with, at
+ * your election, the "Elastic License 2.0", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.index.mapper;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.elasticsearch.search.fetch.StoredFieldsSpec;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Block loader for fields that use fallback synthetic source implementation.
+ *
+ * Usually fields have doc_values or stored fields and block loaders use them directly. In some cases neither is available
+ * and we would fall back to (potentially synthetic) _source. However, in case of synthetic source, there is actually no need to
+ * construct the entire _source. We know that there is no doc_values and stored fields, and therefore we will be using fallback synthetic
+ * source. That is equivalent to just reading _ignored_source stored field directly and doing an in-place synthetic source just
+ * for this field.
+ *
+ * See {@link IgnoredSourceFieldMapper}.
+ */
+public abstract class FallbackSyntheticSourceBlockLoader implements BlockLoader {
+ private final Reader> reader;
+ private final String fieldName;
+
+ protected FallbackSyntheticSourceBlockLoader(Reader> reader, String fieldName) {
+ this.reader = reader;
+ this.fieldName = fieldName;
+ }
+
+ @Override
+ public ColumnAtATimeReader columnAtATimeReader(LeafReaderContext context) throws IOException {
+ return null;
+ }
+
+ @Override
+ public RowStrideReader rowStrideReader(LeafReaderContext context) throws IOException {
+ return new IgnoredSourceRowStrideReader<>(fieldName, reader);
+ }
+
+ @Override
+ public StoredFieldsSpec rowStrideStoredFieldSpec() {
+ return new StoredFieldsSpec(false, false, Set.of(IgnoredSourceFieldMapper.NAME));
+ }
+
+ @Override
+ public boolean supportsOrdinals() {
+ return false;
+ }
+
+ @Override
+ public SortedSetDocValues ordinals(LeafReaderContext context) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ private record IgnoredSourceRowStrideReader(String fieldName, Reader reader) implements RowStrideReader {
+ @Override
+ public void read(int docId, StoredFields storedFields, Builder builder) throws IOException {
+ var ignoredSource = storedFields.storedFields().get(IgnoredSourceFieldMapper.NAME);
+ if (ignoredSource == null) {
+ return;
+ }
+
+ Map> valuesForFieldAndParents = new HashMap<>();
+
+ // Contains name of the field and all its parents
+ Set fieldNames = new HashSet<>() {
+ {
+ add("_doc");
+ }
+ };
+
+ var current = new StringBuilder();
+ for (String part : fieldName.split("\\.")) {
+ if (current.isEmpty() == false) {
+ current.append('.');
+ }
+ current.append(part);
+ fieldNames.add(current.toString());
+ }
+
+ for (Object value : ignoredSource) {
+ IgnoredSourceFieldMapper.NameValue nameValue = IgnoredSourceFieldMapper.decode(value);
+ if (fieldNames.contains(nameValue.name())) {
+ valuesForFieldAndParents.computeIfAbsent(nameValue.name(), k -> new ArrayList<>()).add(nameValue);
+ }
+ }
+
+ // TODO figure out how to handle XContentDataHelper#voidValue()
+
+ var blockValues = new ArrayList();
+
+ var leafFieldValue = valuesForFieldAndParents.get(fieldName);
+ if (leafFieldValue != null) {
+ readFromFieldValue(leafFieldValue, blockValues);
+ } else {
+ readFromParentValue(valuesForFieldAndParents, blockValues);
+ }
+
+ if (blockValues.isEmpty() == false) {
+ if (blockValues.size() > 1) {
+ builder.beginPositionEntry();
+ }
+
+ reader.writeToBlock(blockValues, builder);
+
+ if (blockValues.size() > 1) {
+ builder.endPositionEntry();
+ }
+ } else {
+ builder.appendNull();
+ }
+ }
+
+ private void readFromFieldValue(List nameValues, List blockValues) throws IOException {
+ if (nameValues.isEmpty()) {
+ return;
+ }
+
+ for (var nameValue : nameValues) {
+ // Leaf field is stored directly (not as a part of a parent object), let's try to decode it.
+ Optional
*
- * The generation code will also look for a method called {@code combineValueCount}
- * which is called once per received block with a count of values. NOTE: We may
- * not need this after we convert AVG into a composite operation.
- *
- *
* The generation code also looks for the optional methods {@code combineIntermediate}
* and {@code evaluateFinal} which are used to combine intermediate states and
* produce the final output. If the first is missing then the generated code will
diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java
index 46881bf337c8..c62dc9ed24d8 100644
--- a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java
+++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java
@@ -78,7 +78,6 @@ public class AggregatorImplementer {
private final List warnExceptions;
private final ExecutableElement init;
private final ExecutableElement combine;
- private final ExecutableElement combineValueCount;
private final ExecutableElement combineIntermediate;
private final ExecutableElement evaluateFinal;
private final ClassName implementation;
@@ -115,7 +114,6 @@ public class AggregatorImplementer {
TypeName firstParamType = TypeName.get(e.getParameters().get(0).asType());
return firstParamType.isPrimitive() || firstParamType.toString().equals(stateType.toString());
});
- this.combineValueCount = findMethod(declarationType, "combineValueCount");
this.combineIntermediate = findMethod(declarationType, "combineIntermediate");
this.evaluateFinal = findMethod(declarationType, "evaluateFinal");
this.createParameters = init.getParameters()
@@ -415,9 +413,6 @@ public class AggregatorImplementer {
combineRawInput(builder, "vector");
}
builder.endControlFlow();
- if (combineValueCount != null) {
- builder.addStatement("$T.combineValueCount(state, vector.getPositionCount())", declarationType);
- }
return builder.build();
}
@@ -459,9 +454,6 @@ public class AggregatorImplementer {
}
}
builder.endControlFlow();
- if (combineValueCount != null) {
- builder.addStatement("$T.combineValueCount(state, block.getTotalValueCount())", declarationType);
- }
return builder.build();
}
From e4fd6c06aec23ab392e366c35cfeff19ece326a3 Mon Sep 17 00:00:00 2001
From: Armin Braun
Date: Wed, 5 Feb 2025 11:56:29 +0100
Subject: [PATCH 05/29] Simplify counting in AbstractSearchAsyncAction
(#120593)
No need to do this so complicated, just count down one when we're actually done with a specific shard id.
---
.../search/AbstractSearchAsyncAction.java | 64 ++++++-------------
.../cluster/routing/GroupShardsIterator.java | 20 ------
.../action/search/ExpandSearchPhaseTests.java | 1 -
.../routing/GroupShardsIteratorTests.java | 27 --------
4 files changed, 19 insertions(+), 93 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
index aeea0a5d65c8..9f47e1f9773a 100644
--- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
@@ -90,15 +90,13 @@ abstract class AbstractSearchAsyncAction exten
private final Object shardFailuresMutex = new Object();
private final AtomicBoolean hasShardResponse = new AtomicBoolean(false);
private final AtomicInteger successfulOps = new AtomicInteger();
- private final AtomicInteger skippedOps = new AtomicInteger();
private final SearchTimeProvider timeProvider;
private final SearchResponse.Clusters clusters;
protected final GroupShardsIterator toSkipShardsIts;
protected final GroupShardsIterator shardsIts;
private final SearchShardIterator[] shardIterators;
- private final int expectedTotalOps;
- private final AtomicInteger totalOps = new AtomicInteger();
+ private final AtomicInteger outstandingShards;
private final int maxConcurrentRequestsPerNode;
private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>();
private final boolean throttleConcurrentRequests;
@@ -139,18 +137,12 @@ abstract class AbstractSearchAsyncAction exten
}
this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators);
this.shardsIts = new GroupShardsIterator<>(iterators);
-
+ outstandingShards = new AtomicInteger(shardsIts.size());
this.shardIterators = iterators.toArray(new SearchShardIterator[0]);
// we later compute the shard index based on the natural order of the shards
// that participate in the search request. This means that this number is
// consistent between two requests that target the same shards.
Arrays.sort(shardIterators);
-
- // we need to add 1 for non active partition, since we count it in the total. This means for each shard in the iterator we sum up
- // it's number of active shards but use 1 as the default if no replica of a shard is active at this point.
- // on a per shards level we use shardIt.remaining() to increment the totalOps pointer but add 1 for the current shard result
- // we process hence we add one for the non active partition here.
- this.expectedTotalOps = shardsIts.totalSizeWith1ForEmpty();
this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode;
// in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle
this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size();
@@ -251,9 +243,8 @@ abstract class AbstractSearchAsyncAction exten
void skipShard(SearchShardIterator iterator) {
successfulOps.incrementAndGet();
- skippedOps.incrementAndGet();
assert iterator.skip();
- successfulShardExecution(iterator);
+ successfulShardExecution();
}
private static boolean assertExecuteOnStartThread() {
@@ -380,7 +371,7 @@ abstract class AbstractSearchAsyncAction exten
"Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})",
discrepancy,
successfulOps.get(),
- skippedOps.get(),
+ toSkipShardsIts.size(),
getNumShards(),
currentPhase
);
@@ -449,17 +440,14 @@ abstract class AbstractSearchAsyncAction exten
}
onShardGroupFailure(shardIndex, shard, e);
}
- final int totalOps = this.totalOps.incrementAndGet();
- if (totalOps == expectedTotalOps) {
- onPhaseDone();
- } else if (totalOps > expectedTotalOps) {
- throw new AssertionError(
- "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]",
- new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures())
- );
+ if (lastShard == false) {
+ performPhaseOnShard(shardIndex, shardIt, nextShard);
} else {
- if (lastShard == false) {
- performPhaseOnShard(shardIndex, shardIt, nextShard);
+ // count down outstanding shards, we're done with this shard as there's no more copies to try
+ final int outstanding = outstandingShards.decrementAndGet();
+ assert outstanding >= 0 : "outstanding: " + outstanding;
+ if (outstanding == 0) {
+ onPhaseDone();
}
}
}
@@ -535,10 +523,10 @@ abstract class AbstractSearchAsyncAction exten
if (logger.isTraceEnabled()) {
logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null);
}
- results.consumeResult(result, () -> onShardResultConsumed(result, shardIt));
+ results.consumeResult(result, () -> onShardResultConsumed(result));
}
- private void onShardResultConsumed(Result result, SearchShardIterator shardIt) {
+ private void onShardResultConsumed(Result result) {
successfulOps.incrementAndGet();
// clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level
// so its ok concurrency wise to miss potentially the shard failures being created because of another failure
@@ -552,28 +540,14 @@ abstract class AbstractSearchAsyncAction exten
// cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc.
// increment all the "future" shards to update the total ops since we some may work and some may not...
// and when that happens, we break on total ops, so we must maintain them
- successfulShardExecution(shardIt);
+ successfulShardExecution();
}
- private void successfulShardExecution(SearchShardIterator shardsIt) {
- final int remainingOpsOnIterator;
- if (shardsIt.skip()) {
- // It's possible that we're skipping a shard that's unavailable
- // but its range was available in the IndexMetadata, in that
- // case the shardsIt.remaining() would be 0, expectedTotalOps
- // accounts for unavailable shards too.
- remainingOpsOnIterator = Math.max(shardsIt.remaining(), 1);
- } else {
- remainingOpsOnIterator = shardsIt.remaining() + 1;
- }
- final int xTotalOps = totalOps.addAndGet(remainingOpsOnIterator);
- if (xTotalOps == expectedTotalOps) {
+ private void successfulShardExecution() {
+ final int outstanding = outstandingShards.decrementAndGet();
+ assert outstanding >= 0 : "outstanding: " + outstanding;
+ if (outstanding == 0) {
onPhaseDone();
- } else if (xTotalOps > expectedTotalOps) {
- throw new AssertionError(
- "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]",
- new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures())
- );
}
}
@@ -640,7 +614,7 @@ abstract class AbstractSearchAsyncAction exten
scrollId,
getNumShards(),
numSuccess,
- skippedOps.get(),
+ toSkipShardsIts.size(),
buildTookInMillis(),
failures,
clusters,
diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java b/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
index 32f9530e4b18..590a1bbb1692 100644
--- a/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
+++ b/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
@@ -41,26 +41,6 @@ public final class GroupShardsIterator & Cou
this.iterators = iterators;
}
- /**
- * Returns the total number of shards within all groups
- * @return total number of shards
- */
- public int totalSize() {
- return iterators.stream().mapToInt(Countable::size).sum();
- }
-
- /**
- * Returns the total number of shards plus the number of empty groups
- * @return number of shards and empty groups
- */
- public int totalSizeWith1ForEmpty() {
- int size = 0;
- for (ShardIt shard : iterators) {
- size += Math.max(1, shard.size());
- }
- return size;
- }
-
/**
* Return the number of groups
* @return number of groups
diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java
index 65fdec96c92f..5f8473959951 100644
--- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java
+++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java
@@ -144,7 +144,6 @@ public class ExpandSearchPhaseTests extends ESTestCase {
hits.decRef();
}
} finally {
- mockSearchPhaseContext.execute(() -> {});
var resp = mockSearchPhaseContext.searchResponse.get();
if (resp != null) {
resp.decRef();
diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/GroupShardsIteratorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/GroupShardsIteratorTests.java
index 8e111c367628..d354658396a0 100644
--- a/server/src/test/java/org/elasticsearch/cluster/routing/GroupShardsIteratorTests.java
+++ b/server/src/test/java/org/elasticsearch/cluster/routing/GroupShardsIteratorTests.java
@@ -38,33 +38,6 @@ public class GroupShardsIteratorTests extends ESTestCase {
return shardRoutings;
}
- public void testSize() {
- List list = new ArrayList<>();
- Index index = new Index("foo", "na");
- {
- ShardId shardId = new ShardId(index, 0);
- list.add(new PlainShardIterator(shardId, randomShardRoutings(shardId, 2)));
- }
- list.add(new PlainShardIterator(new ShardId(index, 1), Collections.emptyList()));
- {
- ShardId shardId = new ShardId(index, 2);
- list.add(new PlainShardIterator(shardId, randomShardRoutings(shardId, 0)));
- }
- index = new Index("foo_1", "na");
- {
- ShardId shardId = new ShardId(index, 0);
- list.add(new PlainShardIterator(shardId, randomShardRoutings(shardId, 0)));
- }
- {
- ShardId shardId = new ShardId(index, 1);
- list.add(new PlainShardIterator(shardId, randomShardRoutings(shardId, 0)));
- }
- GroupShardsIterator iter = new GroupShardsIterator<>(list);
- assertEquals(7, iter.totalSizeWith1ForEmpty());
- assertEquals(5, iter.size());
- assertEquals(6, iter.totalSize());
- }
-
public void testIterate() {
List list = new ArrayList<>();
Index index = new Index("foo", "na");
From 09bc343aabfc72aa93a61e0cee314a630e6867bd Mon Sep 17 00:00:00 2001
From: David Turner
Date: Wed, 5 Feb 2025 12:06:03 +0000
Subject: [PATCH 06/29] Migrate to v9 transport handshake (#121646)
This commit moves to sending out a v9-format handshake (with apparent
transport version `8_800_00_0`) and drops support for handshakes from v7
nodes.
---
.../transport/TransportHandshaker.java | 51 +------
.../transport/InboundDecoderTests.java | 140 ------------------
.../TransportHandshakerRawMessageTests.java | 59 +-------
.../transport/TransportHandshakerTests.java | 63 +++-----
.../AbstractSimpleTransportTestCase.java | 24 +--
5 files changed, 40 insertions(+), 297 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
index 1a9043d093fe..9b829fcaf837 100644
--- a/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
@@ -44,49 +44,17 @@ final class TransportHandshaker {
* ignores the body of the request. After the handshake, the OutboundHandler uses the min(local,remote) protocol version for all later
* messages.
*
- * This version supports three handshake protocols, v6080099, v7170099 and v8800000, which respectively have the same message structure
- * as the transport protocols of v6.8.0, v7.17.0, and v8.18.0. This node only sends v7170099 requests, but it can send a valid response
- * to any v6080099 or v8800000 requests that it receives.
+ * This version supports two handshake protocols, v7170099 and v8800000, which respectively have the same message structure as the
+ * transport protocols of v7.17.0, and v8.18.0. This node only sends v8800000 requests, but it can send a valid response to any v7170099
+ * requests that it receives.
*
* Note that these are not really TransportVersion constants as used elsewhere in ES, they're independent things that just happen to be
- * stored in the same location in the message header and which roughly match the same ID numbering scheme. Older versions of ES did
- * rely on them matching the real transport protocol (which itself matched the release version numbers), but these days that's no longer
+ * stored in the same location in the message header and which roughly match the same ID numbering scheme. Older versions of ES did rely
+ * on them matching the real transport protocol (which itself matched the release version numbers), but these days that's no longer
* true.
*
* Here are some example messages, broken down to show their structure. See TransportHandshakerRawMessageTests for supporting tests.
*
- * ## v6080099 Request:
- *
- * 45 53 -- 'ES' marker
- * 00 00 00 34 -- total message length
- * 00 00 00 00 00 00 00 01 -- request ID
- * 08 -- status flags (0b1000 == handshake request)
- * 00 5c c6 63 -- handshake protocol version (0x5cc663 == 6080099)
- * 00 -- no request headers [1]
- * 00 -- no response headers [1]
- * 01 -- one feature [2]
- * 06 -- feature name length
- * 78 2d 70 61 63 6b -- feature name 'x-pack'
- * 16 -- action string size
- * 69 6e 74 65 72 6e 61 6c }
- * 3a 74 63 70 2f 68 61 6e }- ASCII representation of HANDSHAKE_ACTION_NAME
- * 64 73 68 61 6b 65 }
- * 00 -- no parent task ID [3]
- * 04 -- payload length
- * 8b d5 b5 03 -- max acceptable protocol version (vInt: 00000011 10110101 11010101 10001011 == 7170699)
- *
- * ## v6080099 Response:
- *
- * 45 53 -- 'ES' marker
- * 00 00 00 13 -- total message length
- * 00 00 00 00 00 00 00 01 -- request ID (copied from request)
- * 09 -- status flags (0b1001 == handshake response)
- * 00 5c c6 63 -- handshake protocol version (0x5cc663 == 6080099, copied from request)
- * 00 -- no request headers [1]
- * 00 -- no response headers [1]
- * c3 f9 eb 03 -- max acceptable protocol version (vInt: 00000011 11101011 11111001 11000011 == 8060099)
- *
- *
* ## v7170099 Requests:
*
* 45 53 -- 'ES' marker
@@ -158,14 +126,9 @@ final class TransportHandshaker {
* [3] Parent task ID should be empty; see org.elasticsearch.tasks.TaskId.writeTo for its structure.
*/
- static final TransportVersion V7_HANDSHAKE_VERSION = TransportVersion.fromId(6_08_00_99);
static final TransportVersion V8_HANDSHAKE_VERSION = TransportVersion.fromId(7_17_00_99);
static final TransportVersion V9_HANDSHAKE_VERSION = TransportVersion.fromId(8_800_00_0);
- static final Set ALLOWED_HANDSHAKE_VERSIONS = Set.of(
- V7_HANDSHAKE_VERSION,
- V8_HANDSHAKE_VERSION,
- V9_HANDSHAKE_VERSION
- );
+ static final Set ALLOWED_HANDSHAKE_VERSIONS = Set.of(V8_HANDSHAKE_VERSION, V9_HANDSHAKE_VERSION);
static final String HANDSHAKE_ACTION_NAME = "internal:tcp/handshake";
private final ConcurrentMap pendingHandshakes = new ConcurrentHashMap<>();
@@ -203,7 +166,7 @@ final class TransportHandshaker {
);
boolean success = false;
try {
- handshakeRequestSender.sendRequest(node, channel, requestId, V8_HANDSHAKE_VERSION);
+ handshakeRequestSender.sendRequest(node, channel, requestId, V9_HANDSHAKE_VERSION);
threadPool.schedule(
() -> handler.handleLocalException(new ConnectTransportException(node, "handshake_timeout[" + timeout + "]")),
diff --git a/server/src/test/java/org/elasticsearch/transport/InboundDecoderTests.java b/server/src/test/java/org/elasticsearch/transport/InboundDecoderTests.java
index 9b56cd3bde53..cfb3cc68e035 100644
--- a/server/src/test/java/org/elasticsearch/transport/InboundDecoderTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/InboundDecoderTests.java
@@ -18,7 +18,6 @@ import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.MockPageCacheRecycler;
import org.elasticsearch.common.util.concurrent.ThreadContext;
-import org.elasticsearch.core.UpdateForV9;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TransportVersionUtils;
import org.elasticsearch.transport.InboundDecoder.ChannelType;
@@ -126,105 +125,6 @@ public class InboundDecoderTests extends ESTestCase {
}
- @UpdateForV9(owner = UpdateForV9.Owner.CORE_INFRA) // can delete test in v9
- public void testDecodePreHeaderSizeVariableInt() throws IOException {
- Compression.Scheme compressionScheme = randomFrom(Compression.Scheme.DEFLATE, Compression.Scheme.DEFLATE, null);
- String action = "test-request";
- long requestId = randomNonNegativeLong();
- final TransportVersion preHeaderVariableInt = TransportHandshaker.V7_HANDSHAKE_VERSION;
- final String contentValue = randomAlphaOfLength(100);
- // 8.0 is only compatible with handshakes on a pre-variable int version
- final OutboundMessage message = new OutboundMessage.Request(
- threadContext,
- new TestRequest(contentValue),
- preHeaderVariableInt,
- action,
- requestId,
- true,
- compressionScheme
- );
-
- try (RecyclerBytesStreamOutput os = new RecyclerBytesStreamOutput(recycler)) {
- final BytesReference totalBytes = message.serialize(os);
- int partialHeaderSize = TcpHeader.headerSize(preHeaderVariableInt);
-
- InboundDecoder decoder = new InboundDecoder(recycler);
- final ArrayList fragments = new ArrayList<>();
- final ReleasableBytesReference releasable1 = wrapAsReleasable(totalBytes);
- int bytesConsumed = decoder.decode(releasable1, fragments::add);
- assertEquals(partialHeaderSize, bytesConsumed);
- assertTrue(releasable1.hasReferences());
-
- final Header header = (Header) fragments.get(0);
- assertEquals(requestId, header.getRequestId());
- assertEquals(preHeaderVariableInt, header.getVersion());
- if (compressionScheme == null) {
- assertFalse(header.isCompressed());
- } else {
- assertTrue(header.isCompressed());
- }
- assertTrue(header.isHandshake());
- assertTrue(header.isRequest());
- assertTrue(header.needsToReadVariableHeader());
- fragments.clear();
-
- final BytesReference bytes2 = totalBytes.slice(bytesConsumed, totalBytes.length() - bytesConsumed);
- final ReleasableBytesReference releasable2 = wrapAsReleasable(bytes2);
- int bytesConsumed2 = decoder.decode(releasable2, fragments::add);
- if (compressionScheme == null) {
- assertEquals(2, fragments.size());
- } else {
- assertEquals(3, fragments.size());
- final Object body = fragments.get(1);
- assertThat(body, instanceOf(ReleasableBytesReference.class));
- ((ReleasableBytesReference) body).close();
- }
- assertEquals(InboundDecoder.END_CONTENT, fragments.get(fragments.size() - 1));
- assertEquals(totalBytes.length() - bytesConsumed, bytesConsumed2);
- }
- }
-
- public void testDecodeHandshakeV7Compatibility() throws IOException {
- String action = "test-request";
- long requestId = randomNonNegativeLong();
- final String headerKey = randomAlphaOfLength(10);
- final String headerValue = randomAlphaOfLength(20);
- threadContext.putHeader(headerKey, headerValue);
- TransportVersion handshakeCompat = TransportHandshaker.V7_HANDSHAKE_VERSION;
- OutboundMessage message = new OutboundMessage.Request(
- threadContext,
- new TestRequest(randomAlphaOfLength(100)),
- handshakeCompat,
- action,
- requestId,
- true,
- null
- );
-
- try (RecyclerBytesStreamOutput os = new RecyclerBytesStreamOutput(recycler)) {
- final BytesReference bytes = message.serialize(os);
- int totalHeaderSize = TcpHeader.headerSize(handshakeCompat);
-
- InboundDecoder decoder = new InboundDecoder(recycler);
- final ArrayList fragments = new ArrayList<>();
- final ReleasableBytesReference releasable1 = wrapAsReleasable(bytes);
- int bytesConsumed = decoder.decode(releasable1, fragments::add);
- assertEquals(totalHeaderSize, bytesConsumed);
- assertTrue(releasable1.hasReferences());
-
- final Header header = (Header) fragments.get(0);
- assertEquals(requestId, header.getRequestId());
- assertEquals(handshakeCompat, header.getVersion());
- assertFalse(header.isCompressed());
- assertTrue(header.isHandshake());
- assertTrue(header.isRequest());
- // TODO: On 9.0 this will be true because all compatible versions with contain the variable header int
- assertTrue(header.needsToReadVariableHeader());
- fragments.clear();
- }
-
- }
-
public void testDecodeHandshakeV8Compatibility() throws IOException {
doHandshakeCompatibilityTest(TransportHandshaker.V8_HANDSHAKE_VERSION, null);
doHandshakeCompatibilityTest(TransportHandshaker.V8_HANDSHAKE_VERSION, Compression.Scheme.DEFLATE);
@@ -453,46 +353,6 @@ public class InboundDecoderTests extends ESTestCase {
}
- public void testCompressedDecodeHandshakeCompatibility() throws IOException {
- String action = "test-request";
- long requestId = randomNonNegativeLong();
- final String headerKey = randomAlphaOfLength(10);
- final String headerValue = randomAlphaOfLength(20);
- threadContext.putHeader(headerKey, headerValue);
- TransportVersion handshakeCompat = TransportHandshaker.V7_HANDSHAKE_VERSION;
- OutboundMessage message = new OutboundMessage.Request(
- threadContext,
- new TestRequest(randomAlphaOfLength(100)),
- handshakeCompat,
- action,
- requestId,
- true,
- Compression.Scheme.DEFLATE
- );
-
- try (RecyclerBytesStreamOutput os = new RecyclerBytesStreamOutput(recycler)) {
- final BytesReference bytes = message.serialize(os);
- int totalHeaderSize = TcpHeader.headerSize(handshakeCompat);
-
- InboundDecoder decoder = new InboundDecoder(recycler);
- final ArrayList fragments = new ArrayList<>();
- final ReleasableBytesReference releasable1 = wrapAsReleasable(bytes);
- int bytesConsumed = decoder.decode(releasable1, fragments::add);
- assertEquals(totalHeaderSize, bytesConsumed);
- assertTrue(releasable1.hasReferences());
-
- final Header header = (Header) fragments.get(0);
- assertEquals(requestId, header.getRequestId());
- assertEquals(handshakeCompat, header.getVersion());
- assertTrue(header.isCompressed());
- assertTrue(header.isHandshake());
- assertTrue(header.isRequest());
- // TODO: On 9.0 this will be true because all compatible versions with contain the variable header int
- assertTrue(header.needsToReadVariableHeader());
- fragments.clear();
- }
- }
-
public void testVersionIncompatibilityDecodeException() throws IOException {
String action = "test-request";
long requestId = randomNonNegativeLong();
diff --git a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerRawMessageTests.java b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerRawMessageTests.java
index de44ca70f200..2bac41199ab8 100644
--- a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerRawMessageTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerRawMessageTests.java
@@ -20,7 +20,6 @@ import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.OutputStreamStreamOutput;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.core.UpdateForV10;
-import org.elasticsearch.core.UpdateForV9;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.TransportVersionUtils;
@@ -38,56 +37,6 @@ import static org.hamcrest.Matchers.lessThan;
public class TransportHandshakerRawMessageTests extends ESSingleNodeTestCase {
- @UpdateForV9(owner = UpdateForV9.Owner.CORE_INFRA) // remove support for v7 handshakes in v9
- public void testV7Handshake() throws Exception {
- final BytesRef handshakeRequestBytes;
- final var requestId = randomNonNegativeLong();
- try (var outputStream = new BytesStreamOutput()) {
- outputStream.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- outputStream.writeLong(requestId);
- outputStream.writeByte(TransportStatus.setRequest(TransportStatus.setHandshake((byte) 0)));
- outputStream.writeInt(TransportHandshaker.V7_HANDSHAKE_VERSION.id());
- outputStream.writeByte((byte) 0); // no request headers;
- outputStream.writeByte((byte) 0); // no response headers;
- outputStream.writeStringArray(new String[] { "x-pack" }); // one feature
- outputStream.writeString("internal:tcp/handshake");
- outputStream.writeByte((byte) 0); // no parent task ID;
-
- final var requestNodeTransportVersionId = TransportVersionUtils.randomCompatibleVersion(random()).id();
- assertThat(requestNodeTransportVersionId, allOf(greaterThanOrEqualTo(1 << 22), lessThan(1 << 28))); // 4-byte vInt
- outputStream.writeByte((byte) 4); // payload length
- outputStream.writeVInt(requestNodeTransportVersionId);
-
- handshakeRequestBytes = outputStream.bytes().toBytesRef();
- }
-
- final BytesRef handshakeResponseBytes;
- try (var socket = openTransportConnection()) {
- var streamOutput = new OutputStreamStreamOutput(socket.getOutputStream());
- streamOutput.write("ES".getBytes(StandardCharsets.US_ASCII));
- streamOutput.writeInt(handshakeRequestBytes.length);
- streamOutput.writeBytes(handshakeRequestBytes.bytes, handshakeRequestBytes.offset, handshakeRequestBytes.length);
- streamOutput.flush();
-
- var streamInput = new InputStreamStreamInput(socket.getInputStream());
- assertEquals((byte) 'E', streamInput.readByte());
- assertEquals((byte) 'S', streamInput.readByte());
- var responseLength = streamInput.readInt();
- handshakeResponseBytes = streamInput.readBytesRef(responseLength);
- }
-
- try (var inputStream = new BytesArray(handshakeResponseBytes).streamInput()) {
- assertEquals(requestId, inputStream.readLong());
- assertEquals(TransportStatus.setResponse(TransportStatus.setHandshake((byte) 0)), inputStream.readByte());
- assertEquals(TransportHandshaker.V7_HANDSHAKE_VERSION.id(), inputStream.readInt());
- assertEquals((byte) 0, inputStream.readByte()); // no request headers
- assertEquals((byte) 0, inputStream.readByte()); // no response headers
- inputStream.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- assertEquals(TransportVersion.current().id(), inputStream.readVInt());
- assertEquals(-1, inputStream.read());
- }
- }
-
@UpdateForV10(owner = UpdateForV10.Owner.CORE_INFRA) // remove support for v8 handshakes in v10
public void testV8Handshake() throws Exception {
final BytesRef handshakeRequestBytes;
@@ -223,11 +172,10 @@ public class TransportHandshakerRawMessageTests extends ESSingleNodeTestCase {
try (var inputStream = new BytesArray(handshakeRequestBytes).streamInput()) {
assertThat(inputStream.readLong(), greaterThan(0L));
assertEquals(TransportStatus.setRequest(TransportStatus.setHandshake((byte) 0)), inputStream.readByte());
- assertEquals(TransportHandshaker.V8_HANDSHAKE_VERSION.id(), inputStream.readInt());
- assertEquals(0x1a, inputStream.readInt()); // length of variable-length header, always 0x1a
+ assertEquals(TransportHandshaker.V9_HANDSHAKE_VERSION.id(), inputStream.readInt());
+ assertEquals(0x19, inputStream.readInt()); // length of variable-length header, always 0x19
assertEquals((byte) 0, inputStream.readByte()); // no request headers
assertEquals((byte) 0, inputStream.readByte()); // no response headers
- assertEquals((byte) 0, inputStream.readByte()); // no features
assertEquals("internal:tcp/handshake", inputStream.readString());
assertEquals((byte) 0, inputStream.readByte()); // no parent task
inputStream.setTransportVersion(TransportHandshaker.V8_HANDSHAKE_VERSION);
@@ -236,8 +184,9 @@ public class TransportHandshakerRawMessageTests extends ESSingleNodeTestCase {
}
try (var inputStream = new BytesArray(payloadBytes).streamInput()) {
- inputStream.setTransportVersion(TransportHandshaker.V8_HANDSHAKE_VERSION);
+ inputStream.setTransportVersion(TransportHandshaker.V9_HANDSHAKE_VERSION);
assertEquals(TransportVersion.current().id(), inputStream.readVInt());
+ assertEquals(Build.current().version(), inputStream.readString());
assertEquals(-1, inputStream.read());
}
}
diff --git a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
index d260d6615765..4fd82480c4e5 100644
--- a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
@@ -16,7 +16,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.core.TimeValue;
-import org.elasticsearch.core.UpdateForV9;
+import org.elasticsearch.core.UpdateForV10;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.TransportVersionUtils;
@@ -39,8 +39,8 @@ public class TransportHandshakerTests extends ESTestCase {
private TestThreadPool threadPool;
private TransportHandshaker.HandshakeRequestSender requestSender;
- @UpdateForV9(owner = UpdateForV9.Owner.CORE_INFRA)
- private static final TransportVersion HANDSHAKE_REQUEST_VERSION = TransportHandshaker.V8_HANDSHAKE_VERSION;
+ @UpdateForV10(owner = UpdateForV10.Owner.CORE_INFRA) // new handshake version required in v10
+ private static final TransportVersion HANDSHAKE_REQUEST_VERSION = TransportHandshaker.V9_HANDSHAKE_VERSION;
@Override
public void setUp() throws Exception {
@@ -133,10 +133,8 @@ public class TransportHandshakerTests extends ESTestCase {
verify(requestSender).sendRequest(node, channel, reqId, HANDSHAKE_REQUEST_VERSION);
- TransportHandshaker.HandshakeRequest handshakeRequest = new TransportHandshaker.HandshakeRequest(
- TransportVersion.current(),
- randomIdentifier()
- );
+ final var buildVersion = randomIdentifier();
+ final var handshakeRequest = new TransportHandshaker.HandshakeRequest(TransportVersion.current(), buildVersion);
BytesStreamOutput currentHandshakeBytes = new BytesStreamOutput();
currentHandshakeBytes.setTransportVersion(HANDSHAKE_REQUEST_VERSION);
handshakeRequest.writeTo(currentHandshakeBytes);
@@ -145,17 +143,27 @@ public class TransportHandshakerTests extends ESTestCase {
BytesStreamOutput futureHandshake = new BytesStreamOutput();
TaskId.EMPTY_TASK_ID.writeTo(lengthCheckingHandshake);
TaskId.EMPTY_TASK_ID.writeTo(futureHandshake);
+ final var extraDataSize = between(0, 1024);
try (BytesStreamOutput internalMessage = new BytesStreamOutput()) {
Version.writeVersion(Version.CURRENT, internalMessage);
+ internalMessage.writeString(buildVersion);
lengthCheckingHandshake.writeBytesReference(internalMessage.bytes());
- internalMessage.write(new byte[1024]);
+ internalMessage.write(new byte[extraDataSize]);
futureHandshake.writeBytesReference(internalMessage.bytes());
}
StreamInput futureHandshakeStream = futureHandshake.bytes().streamInput();
// We check that the handshake we serialize for this test equals the actual request.
// Otherwise, we need to update the test.
assertEquals(currentHandshakeBytes.bytes().length(), lengthCheckingHandshake.bytes().length());
- assertEquals(1031, futureHandshakeStream.available());
+ final var expectedInternalMessageSize = 4 /* transport version id */
+ + (1 + buildVersion.length()) /* length prefixed release version string */
+ + extraDataSize;
+ assertEquals(
+ 1 /* EMPTY_TASK_ID */
+ + (expectedInternalMessageSize < 0x80 ? 1 : 2) /* internalMessage size vInt */
+ + expectedInternalMessageSize /* internalMessage */,
+ futureHandshakeStream.available()
+ );
final PlainActionFuture responseFuture = new PlainActionFuture<>();
final TestTransportChannel channel = new TestTransportChannel(responseFuture);
handshaker.handleHandshake(channel, reqId, futureHandshakeStream);
@@ -166,43 +174,6 @@ public class TransportHandshakerTests extends ESTestCase {
assertEquals(TransportVersion.current(), response.getTransportVersion());
}
- @UpdateForV9(owner = UpdateForV9.Owner.CORE_INFRA) // v7 handshakes are not supported in v9
- public void testReadV7HandshakeRequest() throws IOException {
- final var transportVersion = TransportVersionUtils.randomCompatibleVersion(random());
-
- final var requestPayloadStreamOutput = new BytesStreamOutput();
- requestPayloadStreamOutput.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- requestPayloadStreamOutput.writeVInt(transportVersion.id());
-
- final var requestBytesStreamOutput = new BytesStreamOutput();
- requestBytesStreamOutput.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- TaskId.EMPTY_TASK_ID.writeTo(requestBytesStreamOutput);
- requestBytesStreamOutput.writeBytesReference(requestPayloadStreamOutput.bytes());
-
- final var requestBytesStream = requestBytesStreamOutput.bytes().streamInput();
- requestBytesStream.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- final var handshakeRequest = new TransportHandshaker.HandshakeRequest(requestBytesStream);
-
- assertEquals(transportVersion, handshakeRequest.transportVersion);
- assertEquals(transportVersion.toReleaseVersion(), handshakeRequest.releaseVersion);
- }
-
- @UpdateForV9(owner = UpdateForV9.Owner.CORE_INFRA) // v7 handshakes are not supported in v9
- public void testReadV7HandshakeResponse() throws IOException {
- final var transportVersion = TransportVersionUtils.randomCompatibleVersion(random());
-
- final var responseBytesStreamOutput = new BytesStreamOutput();
- responseBytesStreamOutput.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- responseBytesStreamOutput.writeVInt(transportVersion.id());
-
- final var responseBytesStream = responseBytesStreamOutput.bytes().streamInput();
- responseBytesStream.setTransportVersion(TransportHandshaker.V7_HANDSHAKE_VERSION);
- final var handshakeResponse = new TransportHandshaker.HandshakeResponse(responseBytesStream);
-
- assertEquals(transportVersion, handshakeResponse.getTransportVersion());
- assertEquals(transportVersion.toReleaseVersion(), handshakeResponse.getReleaseVersion());
- }
-
public void testReadV8HandshakeRequest() throws IOException {
final var transportVersion = TransportVersionUtils.randomCompatibleVersion(random());
diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
index 4595fbf28607..0df978fe4937 100644
--- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java
@@ -2758,8 +2758,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportStats transportStats = serviceC.transport.getStats(); // we did a single round-trip to do the initial handshake
assertEquals(1, transportStats.getRxCount());
assertEquals(1, transportStats.getTxCount());
- assertEquals(29, transportStats.getRxSize().getBytes());
- assertEquals(55, transportStats.getTxSize().getBytes());
+ assertEquals(35, transportStats.getRxSize().getBytes());
+ assertEquals(60, transportStats.getTxSize().getBytes());
});
serviceC.sendRequest(
connection,
@@ -2773,16 +2773,16 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportStats transportStats = serviceC.transport.getStats(); // request has been send
assertEquals(1, transportStats.getRxCount());
assertEquals(2, transportStats.getTxCount());
- assertEquals(29, transportStats.getRxSize().getBytes());
- assertEquals(114, transportStats.getTxSize().getBytes());
+ assertEquals(35, transportStats.getRxSize().getBytes());
+ assertEquals(119, transportStats.getTxSize().getBytes());
});
sendResponseLatch.countDown();
responseLatch.await();
stats = serviceC.transport.getStats(); // response has been received
assertEquals(2, stats.getRxCount());
assertEquals(2, stats.getTxCount());
- assertEquals(54, stats.getRxSize().getBytes());
- assertEquals(114, stats.getTxSize().getBytes());
+ assertEquals(60, stats.getRxSize().getBytes());
+ assertEquals(119, stats.getTxSize().getBytes());
} finally {
serviceC.close();
}
@@ -2873,8 +2873,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportStats transportStats = serviceC.transport.getStats(); // request has been sent
assertEquals(1, transportStats.getRxCount());
assertEquals(1, transportStats.getTxCount());
- assertEquals(29, transportStats.getRxSize().getBytes());
- assertEquals(55, transportStats.getTxSize().getBytes());
+ assertEquals(35, transportStats.getRxSize().getBytes());
+ assertEquals(60, transportStats.getTxSize().getBytes());
});
serviceC.sendRequest(
connection,
@@ -2888,8 +2888,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
TransportStats transportStats = serviceC.transport.getStats(); // request has been sent
assertEquals(1, transportStats.getRxCount());
assertEquals(2, transportStats.getTxCount());
- assertEquals(29, transportStats.getRxSize().getBytes());
- assertEquals(114, transportStats.getTxSize().getBytes());
+ assertEquals(35, transportStats.getRxSize().getBytes());
+ assertEquals(119, transportStats.getTxSize().getBytes());
});
sendResponseLatch.countDown();
responseLatch.await();
@@ -2904,8 +2904,8 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase {
String failedMessage = "Unexpected read bytes size. The transport exception that was received=" + exception;
// 57 bytes are the non-exception message bytes that have been received. It should include the initial
// handshake message and the header, version, etc bytes in the exception message.
- assertEquals(failedMessage, 57 + streamOutput.bytes().length(), stats.getRxSize().getBytes());
- assertEquals(114, stats.getTxSize().getBytes());
+ assertEquals(failedMessage, 63 + streamOutput.bytes().length(), stats.getRxSize().getBytes());
+ assertEquals(119, stats.getTxSize().getBytes());
} finally {
serviceC.close();
}
From 6a526755de4b560e6c4d9a211fb783723b1f2807 Mon Sep 17 00:00:00 2001
From: Salvatore Campagna
<93581129+salvatore-campagna@users.noreply.github.com>
Date: Wed, 5 Feb 2025 13:55:51 +0100
Subject: [PATCH 07/29] Use synthetic recovery source by default if synthetic
source is enabled (#119110)
We experimented with using synthetic source for recovery and observed quite positive impact
on indexing throughput by means of our nightly Rally benchmarks. As a result, here we enable
it by default when synthetic source is used. To be more precise, if `index.mapping.source.mode`
setting is `synthetic` we enable recovery source by means of synthetic source.
Moreover, enabling synthetic source recovery is done behind a feature flag. That would allow us
to enable it in snapshot builds which in turn will allow us to see performance results in Rally nightly
benchmarks.
---
muted-tests.yml | 6 +
qa/smoke-test-multinode/build.gradle | 2 +
.../test/update/100_synthetic_source.yml | 14 +-
.../common/settings/Setting.java | 9 +
.../elasticsearch/index/IndexSettings.java | 23 +-
.../elasticsearch/index/IndexVersions.java | 2 +
.../mapper/IgnoredSourceFieldMapperTests.java | 10 +-
.../index/mapper/SourceFieldMapperTests.java | 63 +++--
.../index/mapper/MapperServiceTestCase.java | 7 +-
x-pack/plugin/logsdb/build.gradle | 10 +
.../test/60_synthetic_source_recovery.yml | 261 ++++++++++++++++++
.../build.gradle | 4 +-
12 files changed, 382 insertions(+), 29 deletions(-)
create mode 100644 x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/60_synthetic_source_recovery.yml
diff --git a/muted-tests.yml b/muted-tests.yml
index 2326972a83c8..e3b74e44cebc 100644
--- a/muted-tests.yml
+++ b/muted-tests.yml
@@ -134,6 +134,12 @@ tests:
- class: org.elasticsearch.datastreams.DataStreamsClientYamlTestSuiteIT
method: test {p0=data_stream/120_data_streams_stats/Multiple data stream}
issue: https://github.com/elastic/elasticsearch/issues/118217
+ # TODO: re-enable after backporting https://github.com/elastic/elasticsearch/pull/119110
+- class: org.elasticsearch.test.rest.ClientYamlTestSuiteIT
+ method: test {yaml=update/100_synthetic_source/keyword}
+ # TODO: re-enable after backporting https://github.com/elastic/elasticsearch/pull/119110
+- class: org.elasticsearch.test.rest.ClientYamlTestSuiteIT
+ method: test {yaml=update/100_synthetic_source/stored text}
- class: org.elasticsearch.xpack.searchablesnapshots.RetrySearchIntegTests
method: testSearcherId
issue: https://github.com/elastic/elasticsearch/issues/118374
diff --git a/qa/smoke-test-multinode/build.gradle b/qa/smoke-test-multinode/build.gradle
index cc71a99562eb..14e102025cac 100644
--- a/qa/smoke-test-multinode/build.gradle
+++ b/qa/smoke-test-multinode/build.gradle
@@ -28,5 +28,7 @@ tasks.named("yamlRestTest").configure {
'cat.templates/10_basic/No templates',
'cat.templates/10_basic/Sort templates',
'cat.templates/10_basic/Multiple template',
+ 'update/100_synthetic_source/keyword',
+ 'update/100_synthetic_source/stored text'
].join(',')
}
diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/update/100_synthetic_source.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/update/100_synthetic_source.yml
index f4894692b6ca..219bc52c4e28 100644
--- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/update/100_synthetic_source.yml
+++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/update/100_synthetic_source.yml
@@ -6,8 +6,8 @@ setup:
---
keyword:
- requires:
- cluster_features: ["gte_v8.4.0"]
- reason: introduced in 8.4.0
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
- do:
indices.create:
@@ -60,13 +60,14 @@ keyword:
index: test
run_expensive_tasks: true
- is_false: test.fields._source
- - is_true: test.fields._recovery_source
+ # When synthetic source is used there is no _recovery_source field
+ - match: { test.fields._recovery_source: null }
---
stored text:
- requires:
- cluster_features: ["gte_v8.5.0"]
- reason: introduced in 8.5.0
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
- do:
indices.create:
@@ -121,4 +122,5 @@ stored text:
index: test
run_expensive_tasks: true
- is_false: test.fields._source
- - is_true: test.fields._recovery_source
+ # When synthetic source is used there is no _recovery_source field
+ - match: { test.fields._recovery_source: null }
diff --git a/server/src/main/java/org/elasticsearch/common/settings/Setting.java b/server/src/main/java/org/elasticsearch/common/settings/Setting.java
index f3319a8f85bf..ab8b390490c1 100644
--- a/server/src/main/java/org/elasticsearch/common/settings/Setting.java
+++ b/server/src/main/java/org/elasticsearch/common/settings/Setting.java
@@ -1580,6 +1580,15 @@ public class Setting implements ToXContentObject {
return new Setting<>(key, Boolean.toString(defaultValue), booleanParser(key, properties), validator, properties);
}
+ public static Setting boolSetting(
+ String key,
+ Function defaultValueFn,
+ Validator validator,
+ Property... properties
+ ) {
+ return new Setting<>(key, defaultValueFn, booleanParser(key, properties), validator, properties);
+ }
+
public static Setting boolSetting(String key, Function defaultValueFn, Property... properties) {
return new Setting<>(key, defaultValueFn, booleanParser(key, properties), properties);
}
diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java
index 4895930eaefe..525f90accdf3 100644
--- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java
+++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java
@@ -24,6 +24,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateUtils;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
+import org.elasticsearch.common.util.FeatureFlag;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.mapper.IgnoredSourceFieldMapper;
import org.elasticsearch.index.mapper.Mapper;
@@ -39,6 +40,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
+import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
@@ -722,9 +724,25 @@ public final class IndexSettings {
Setting.Property.ServerlessPublic
);
+ public static final FeatureFlag RECOVERY_USE_SYNTHETIC_SOURCE = new FeatureFlag("index_recovery_use_synthetic_source");
public static final Setting RECOVERY_USE_SYNTHETIC_SOURCE_SETTING = Setting.boolSetting(
"index.recovery.use_synthetic_source",
- false,
+ settings -> {
+ boolean isSyntheticSourceRecoveryFeatureFlagEnabled = RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled();
+ boolean isNewIndexVersion = SETTING_INDEX_VERSION_CREATED.get(settings)
+ .onOrAfter(IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BY_DEFAULT);
+ boolean isIndexVersionInBackportRange = SETTING_INDEX_VERSION_CREATED.get(settings)
+ .between(IndexVersions.USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BY_DEFAULT_BACKPORT, IndexVersions.UPGRADE_TO_LUCENE_10_0_0);
+
+ boolean useSyntheticRecoverySource = isSyntheticSourceRecoveryFeatureFlagEnabled
+ && (isNewIndexVersion || isIndexVersionInBackportRange);
+
+ return String.valueOf(
+ useSyntheticRecoverySource
+ && Objects.equals(INDEX_MAPPER_SOURCE_MODE_SETTING.get(settings), SourceFieldMapper.Mode.SYNTHETIC)
+ );
+
+ },
new Setting.Validator<>() {
@Override
public void validate(Boolean value) {}
@@ -1083,7 +1101,8 @@ public final class IndexSettings {
skipIgnoredSourceRead = scopedSettings.get(IgnoredSourceFieldMapper.SKIP_IGNORED_SOURCE_READ_SETTING);
indexMappingSourceMode = scopedSettings.get(INDEX_MAPPER_SOURCE_MODE_SETTING);
recoverySourceEnabled = RecoverySettings.INDICES_RECOVERY_SOURCE_ENABLED_SETTING.get(nodeSettings);
- recoverySourceSyntheticEnabled = scopedSettings.get(RECOVERY_USE_SYNTHETIC_SOURCE_SETTING);
+ recoverySourceSyntheticEnabled = DiscoveryNode.isStateless(nodeSettings) == false
+ && scopedSettings.get(RECOVERY_USE_SYNTHETIC_SOURCE_SETTING);
if (recoverySourceSyntheticEnabled) {
if (DiscoveryNode.isStateless(settings)) {
throw new IllegalArgumentException("synthetic recovery source is only allowed in stateful");
diff --git a/server/src/main/java/org/elasticsearch/index/IndexVersions.java b/server/src/main/java/org/elasticsearch/index/IndexVersions.java
index 3b173ace0ac7..64f4c356bb12 100644
--- a/server/src/main/java/org/elasticsearch/index/IndexVersions.java
+++ b/server/src/main/java/org/elasticsearch/index/IndexVersions.java
@@ -134,6 +134,7 @@ public class IndexVersions {
public static final IndexVersion UPGRADE_TO_LUCENE_9_12_1 = def(8_523_0_00, parseUnchecked("9.12.1"));
public static final IndexVersion INFERENCE_METADATA_FIELDS_BACKPORT = def(8_524_0_00, parseUnchecked("9.12.1"));
public static final IndexVersion LOGSB_OPTIONAL_SORTING_ON_HOST_NAME_BACKPORT = def(8_525_0_00, parseUnchecked("9.12.1"));
+ public static final IndexVersion USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BY_DEFAULT_BACKPORT = def(8_526_0_00, parseUnchecked("9.12.1"));
public static final IndexVersion UPGRADE_TO_LUCENE_10_0_0 = def(9_000_0_00, Version.LUCENE_10_0_0);
public static final IndexVersion LOGSDB_DEFAULT_IGNORE_DYNAMIC_BEYOND_LIMIT = def(9_001_0_00, Version.LUCENE_10_0_0);
public static final IndexVersion TIME_BASED_K_ORDERED_DOC_ID = def(9_002_0_00, Version.LUCENE_10_0_0);
@@ -144,6 +145,7 @@ public class IndexVersions {
public static final IndexVersion SOURCE_MAPPER_MODE_ATTRIBUTE_NOOP = def(9_007_0_00, Version.LUCENE_10_0_0);
public static final IndexVersion HOSTNAME_DOC_VALUES_SPARSE_INDEX = def(9_008_0_00, Version.LUCENE_10_0_0);
public static final IndexVersion UPGRADE_TO_LUCENE_10_1_0 = def(9_009_0_00, Version.LUCENE_10_1_0);
+ public static final IndexVersion USE_SYNTHETIC_SOURCE_FOR_RECOVERY_BY_DEFAULT = def(9_010_00_0, Version.LUCENE_10_1_0);
/*
* STOP! READ THIS FIRST! No, really,
* ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _
diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
index d12bf5dc2e34..c84ff2099871 100644
--- a/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
+++ b/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
@@ -2427,8 +2427,14 @@ public class IgnoredSourceFieldMapperTests extends MapperServiceTestCase {
// and since the copy is exact, contents of ignored source are different.
assertReaderEquals(
"round trip " + syntheticSource,
- new FieldMaskingReader(Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, IgnoredSourceFieldMapper.NAME), reader),
- new FieldMaskingReader(Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, IgnoredSourceFieldMapper.NAME), roundTripReader)
+ new FieldMaskingReader(
+ Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, IgnoredSourceFieldMapper.NAME, SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME),
+ reader
+ ),
+ new FieldMaskingReader(
+ Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, IgnoredSourceFieldMapper.NAME, SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME),
+ roundTripReader
+ )
);
}
}
diff --git a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java
index 8ad37908b2e9..70010084cdb9 100644
--- a/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java
+++ b/server/src/test/java/org/elasticsearch/index/mapper/SourceFieldMapperTests.java
@@ -489,8 +489,13 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
MapperService mapperService = createMapperService(settings, topMapping(b -> {}));
DocumentMapper docMapper = mapperService.documentMapper();
ParsedDocument doc = docMapper.parse(source(b -> b.field("field1", "value1")));
- assertNotNull(doc.rootDoc().getField("_recovery_source"));
- assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"field1\":\"value1\"}")));
+ if (IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled() == false) {
+ // TODO: remove this if branch when removing the 'index_recovery_use_synthetic_source' feature flag
+ assertNotNull(doc.rootDoc().getField("_recovery_source"));
+ assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"field1\":\"value1\"}")));
+ } else {
+ assertNull(doc.rootDoc().getField("_recovery_source"));
+ }
}
{
Settings settings = Settings.builder()
@@ -521,8 +526,16 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
MapperService mapperService = createMapperService(settings, mapping(b -> {}));
DocumentMapper docMapper = mapperService.documentMapper();
ParsedDocument doc = docMapper.parse(source(b -> { b.field("@timestamp", "2012-02-13"); }));
- assertNotNull(doc.rootDoc().getField("_recovery_source"));
- assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\"}")));
+ if (IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled() == false) {
+ // TODO: remove this if branch when removing the 'index_recovery_use_synthetic_source' feature flag
+ assertNotNull(doc.rootDoc().getField("_recovery_source"));
+ assertThat(
+ doc.rootDoc().getField("_recovery_source").binaryValue(),
+ equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\"}"))
+ );
+ } else {
+ assertNull(doc.rootDoc().getField("_recovery_source"));
+ }
}
{
Settings settings = Settings.builder()
@@ -715,8 +728,16 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
MapperService mapperService = createMapperService(settings, mappings);
DocumentMapper docMapper = mapperService.documentMapper();
ParsedDocument doc = docMapper.parse(source(b -> { b.field("@timestamp", "2012-02-13"); }));
- assertNotNull(doc.rootDoc().getField("_recovery_source"));
- assertThat(doc.rootDoc().getField("_recovery_source").binaryValue(), equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\"}")));
+ if (IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled() == false) {
+ // TODO: remove this if branch when removing the 'index_recovery_use_synthetic_source' feature flag
+ assertNotNull(doc.rootDoc().getField("_recovery_source"));
+ assertThat(
+ doc.rootDoc().getField("_recovery_source").binaryValue(),
+ equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\"}"))
+ );
+ } else {
+ assertNull(doc.rootDoc().getField("_recovery_source"));
+ }
}
{
Settings settings = Settings.builder()
@@ -742,11 +763,16 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
}));
DocumentMapper docMapper = mapperService.documentMapper();
ParsedDocument doc = docMapper.parse(source("123", b -> b.field("@timestamp", "2012-02-13").field("field", "value1"), null));
- assertNotNull(doc.rootDoc().getField("_recovery_source"));
- assertThat(
- doc.rootDoc().getField("_recovery_source").binaryValue(),
- equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\",\"field\":\"value1\"}"))
- );
+ if (IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled() == false) {
+ // TODO: remove this if branch when removing the 'index_recovery_use_synthetic_source' feature flag
+ assertNotNull(doc.rootDoc().getField("_recovery_source"));
+ assertThat(
+ doc.rootDoc().getField("_recovery_source").binaryValue(),
+ equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\",\"field\":\"value1\"}"))
+ );
+ } else {
+ assertNull(doc.rootDoc().getField("_recovery_source"));
+ }
}
{
Settings settings = Settings.builder()
@@ -790,11 +816,16 @@ public class SourceFieldMapperTests extends MetadataMapperTestCase {
MapperService mapperService = createMapperService(settings, mappings);
DocumentMapper docMapper = mapperService.documentMapper();
ParsedDocument doc = docMapper.parse(source("123", b -> b.field("@timestamp", "2012-02-13").field("field", "value1"), null));
- assertNotNull(doc.rootDoc().getField("_recovery_source"));
- assertThat(
- doc.rootDoc().getField("_recovery_source").binaryValue(),
- equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\",\"field\":\"value1\"}"))
- );
+ if (IndexSettings.RECOVERY_USE_SYNTHETIC_SOURCE.isEnabled() == false) {
+ // TODO: remove this if branch when removing the 'index_recovery_use_synthetic_source' feature flag
+ assertNotNull(doc.rootDoc().getField("_recovery_source"));
+ assertThat(
+ doc.rootDoc().getField("_recovery_source").binaryValue(),
+ equalTo(new BytesRef("{\"@timestamp\":\"2012-02-13\",\"field\":\"value1\"}"))
+ );
+ } else {
+ assertNull(doc.rootDoc().getField("_recovery_source"));
+ }
}
{
Settings settings = Settings.builder()
diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java
index 459480d1d731..b62e40082683 100644
--- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java
@@ -884,8 +884,11 @@ public abstract class MapperServiceTestCase extends FieldTypeTestCase {
throws IOException {
assertReaderEquals(
"round trip " + syntheticSource,
- new FieldMaskingReader(SourceFieldMapper.RECOVERY_SOURCE_NAME, reader),
- new FieldMaskingReader(SourceFieldMapper.RECOVERY_SOURCE_NAME, roundTripReader)
+ new FieldMaskingReader(Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME), reader),
+ new FieldMaskingReader(
+ Set.of(SourceFieldMapper.RECOVERY_SOURCE_NAME, SourceFieldMapper.RECOVERY_SOURCE_SIZE_NAME),
+ roundTripReader
+ )
);
}
diff --git a/x-pack/plugin/logsdb/build.gradle b/x-pack/plugin/logsdb/build.gradle
index bef07258a8e3..917f8207d2d2 100644
--- a/x-pack/plugin/logsdb/build.gradle
+++ b/x-pack/plugin/logsdb/build.gradle
@@ -42,3 +42,13 @@ tasks.named("javaRestTest").configure {
tasks.named('yamlRestTest') {
usesDefaultDistribution()
}
+
+tasks.named("yamlRestTest") {
+ if (buildParams.isSnapshotBuild() == false) {
+ systemProperty 'tests.rest.blacklist', [
+ "60_synthetic_source_recovery/*"
+ ].join(',')
+ }
+}
+
+
diff --git a/x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/60_synthetic_source_recovery.yml b/x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/60_synthetic_source_recovery.yml
new file mode 100644
index 000000000000..cc2216997c6d
--- /dev/null
+++ b/x-pack/plugin/logsdb/src/yamlRestTest/resources/rest-api-spec/test/60_synthetic_source_recovery.yml
@@ -0,0 +1,261 @@
+---
+synthetic recovery for synthetic source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_synthetic_recovery
+ body:
+ settings:
+ index:
+ mapping.source.mode: synthetic
+
+ - do:
+ indices.get_settings:
+ index: test_synthetic_recovery
+ include_defaults: true
+
+ - match: { test_synthetic_recovery.settings.index.mapping.source.mode: synthetic }
+ - match: { test_synthetic_recovery.defaults.index.recovery.use_synthetic_source: "true" }
+
+---
+synthetic recovery for stored source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_stored_recovery
+ body:
+ settings:
+ index:
+ mapping.source.mode: stored
+
+ - do:
+ indices.get_settings:
+ index: test_stored_recovery
+ include_defaults: true
+
+ - match: { test_stored_recovery.settings.index.mapping.source.mode: stored }
+ - match: { test_stored_recovery.defaults.index.recovery.use_synthetic_source: "false" }
+
+---
+synthetic recovery for disabled source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_disabled_recovery
+ body:
+ settings:
+ index:
+ mapping.source.mode: disabled
+
+ - do:
+ indices.get_settings:
+ index: test_disabled_recovery
+ include_defaults: true
+
+ - match: { test_disabled_recovery.settings.index.mapping.source.mode: disabled }
+ - match: { test_disabled_recovery.defaults.index.recovery.use_synthetic_source: "false" }
+
+---
+synthetic recovery for standard index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_standard_index_recovery
+ body:
+ settings:
+ index:
+ mode: standard
+
+ - do:
+ indices.get_settings:
+ index: test_standard_index_recovery
+ include_defaults: true
+
+ - match: { test_standard_index_recovery.defaults.index.recovery.use_synthetic_source: "false" }
+
+---
+synthetic recovery for logsdb index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_logsdb_index_recovery
+ body:
+ settings:
+ index:
+ mode: logsdb
+
+ - do:
+ indices.get_settings:
+ index: test_logsdb_index_recovery
+ include_defaults: true
+
+ - match: { test_logsdb_index_recovery.defaults.index.recovery.use_synthetic_source: "true" }
+
+---
+synthetic recovery for time_series index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_time_series_index_recovery
+ body:
+ settings:
+ index:
+ mode: time_series
+ routing_path: [ keyword ]
+ time_series:
+ start_time: 2021-04-28T00:00:00Z
+ end_time: 2021-04-29T00:00:00Z
+ mappings:
+ properties:
+ keyword:
+ type: keyword
+ time_series_dimension: true
+
+ - do:
+ indices.get_settings:
+ index: test_time_series_index_recovery
+ include_defaults: true
+
+ - match: { test_time_series_index_recovery.defaults.index.recovery.use_synthetic_source: "true" }
+
+---
+override synthetic recovery for synthetic source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_synthetic_recovery_override
+ body:
+ settings:
+ index:
+ mapping.source.mode: synthetic
+ recovery.use_synthetic_source: false
+
+ - do:
+ indices.get_settings:
+ index: test_synthetic_recovery_override
+ include_defaults: true
+
+ - match: { test_synthetic_recovery_override.settings.index.mapping.source.mode: synthetic }
+ - match: { test_synthetic_recovery_override.settings.index.recovery.use_synthetic_source: "false" }
+
+---
+override synthetic recovery for stored source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ catch: bad_request
+ indices.create:
+ index: test_stored_recovery_override
+ body:
+ settings:
+ index:
+ mapping.source.mode: stored
+ recovery.use_synthetic_source: true
+
+---
+override synthetic recovery for disabled source mode index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ catch: bad_request
+ indices.create:
+ index: test_disabled_recovery_override
+ body:
+ settings:
+ index:
+ mapping.source.mode: disabled
+ recovery.use_synthetic_source: true
+
+---
+override synthetic recovery for standard index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ catch: bad_request
+ indices.create:
+ index: test_standard_index_recovery_override
+ body:
+ settings:
+ index:
+ mode: standard
+ recovery.use_synthetic_source: true
+
+---
+override synthetic recovery for logsdb index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_logsdb_index_recovery_override
+ body:
+ settings:
+ index:
+ mode: logsdb
+ recovery.use_synthetic_source: false
+
+ - do:
+ indices.get_settings:
+ index: test_logsdb_index_recovery_override
+ include_defaults: true
+
+ - match: { test_logsdb_index_recovery_override.settings.index.recovery.use_synthetic_source: "false" }
+
+---
+override synthetic recovery for time_series index:
+ - requires:
+ cluster_features: [ "mapper.synthetic_recovery_source" ]
+ reason: requires synthetic recovery source
+
+ - do:
+ indices.create:
+ index: test_time_series_index_recovery_override
+ body:
+ settings:
+ index:
+ mode: time_series
+ recovery.use_synthetic_source: false
+ routing_path: [ keyword ]
+ time_series:
+ start_time: 2021-04-28T00:00:00Z
+ end_time: 2021-04-29T00:00:00Z
+ mappings:
+ properties:
+ keyword:
+ type: keyword
+ time_series_dimension: true
+
+ - do:
+ indices.get_settings:
+ index: test_time_series_index_recovery_override
+ include_defaults: true
+
+ - match: { test_time_series_index_recovery_override.settings.index.recovery.use_synthetic_source: "false" }
diff --git a/x-pack/qa/core-rest-tests-with-security/build.gradle b/x-pack/qa/core-rest-tests-with-security/build.gradle
index 65f2282014dc..a3f50089d542 100644
--- a/x-pack/qa/core-rest-tests-with-security/build.gradle
+++ b/x-pack/qa/core-rest-tests-with-security/build.gradle
@@ -31,7 +31,9 @@ restResources {
tasks.named("yamlRestTest").configure {
ArrayList blacklist = [
'index/10_with_id/Index with ID',
- 'indices.get_alias/10_basic/Get alias against closed indices'
+ 'indices.get_alias/10_basic/Get alias against closed indices',
+ 'update/100_synthetic_source/keyword',
+ 'update/100_synthetic_source/stored text'
];
if (buildParams.isSnapshotBuild() == false) {
blacklist += [
From 94d7f22a2ae0fe5acc0c4e3d700ce8eac6c9a519 Mon Sep 17 00:00:00 2001
From: Artem Prigoda
Date: Wed, 5 Feb 2025 14:16:04 +0100
Subject: [PATCH 08/29] Simplify TransportStats assertions in v9 (#114700)
Transport handling times were added in #80581 (8.1), we don't need assertions for version prior to that in 9.0
---
.../org/elasticsearch/TransportVersions.java | 2 +
.../transport/TransportStats.java | 79 ++++++-------------
.../transport/TransportStatsTests.java | 48 +----------
3 files changed, 32 insertions(+), 97 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java
index d3b862844572..d97246ab7f83 100644
--- a/server/src/main/java/org/elasticsearch/TransportVersions.java
+++ b/server/src/main/java/org/elasticsearch/TransportVersions.java
@@ -175,6 +175,8 @@ public class TransportVersions {
public static final TransportVersion COHERE_BIT_EMBEDDING_TYPE_SUPPORT_ADDED_BACKPORT_8_X = def(8_840_0_01);
public static final TransportVersion ELASTICSEARCH_9_0 = def(9_000_0_00);
public static final TransportVersion COHERE_BIT_EMBEDDING_TYPE_SUPPORT_ADDED = def(9_001_0_00);
+ public static final TransportVersion TRANSPORT_STATS_HANDLING_TIME_REQUIRED = def(9_002_0_00);
+
/*
* STOP! READ THIS FIRST! No, really,
* ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _
diff --git a/server/src/main/java/org/elasticsearch/transport/TransportStats.java b/server/src/main/java/org/elasticsearch/transport/TransportStats.java
index 46b161b01e9f..1163cfbcb270 100644
--- a/server/src/main/java/org/elasticsearch/transport/TransportStats.java
+++ b/server/src/main/java/org/elasticsearch/transport/TransportStats.java
@@ -18,7 +18,6 @@ import org.elasticsearch.common.network.HandlingTimeTracker;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ChunkedToXContent;
import org.elasticsearch.core.TimeValue;
-import org.elasticsearch.core.UpdateForV9;
import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder;
@@ -70,18 +69,16 @@ public class TransportStats implements Writeable, ChunkedToXContent {
rxSize = in.readVLong();
txCount = in.readVLong();
txSize = in.readVLong();
- if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_1_0) && in.readBoolean()) {
- inboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT];
- for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) {
- inboundHandlingTimeBucketFrequencies[i] = in.readVLong();
- }
- outboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT];
- for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) {
- outboundHandlingTimeBucketFrequencies[i] = in.readVLong();
- }
- } else {
- inboundHandlingTimeBucketFrequencies = new long[0];
- outboundHandlingTimeBucketFrequencies = new long[0];
+ if (in.getTransportVersion().before(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED)) {
+ in.readBoolean();
+ }
+ inboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT];
+ for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) {
+ inboundHandlingTimeBucketFrequencies[i] = in.readVLong();
+ }
+ outboundHandlingTimeBucketFrequencies = new long[HandlingTimeTracker.BUCKET_COUNT];
+ for (int i = 0; i < inboundHandlingTimeBucketFrequencies.length; i++) {
+ outboundHandlingTimeBucketFrequencies[i] = in.readVLong();
}
if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_8_0)) {
transportActionStats = Collections.unmodifiableMap(in.readOrderedMap(StreamInput::readString, TransportActionStats::new));
@@ -99,15 +96,16 @@ public class TransportStats implements Writeable, ChunkedToXContent {
out.writeVLong(rxSize);
out.writeVLong(txCount);
out.writeVLong(txSize);
- if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_1_0)) {
- assert (inboundHandlingTimeBucketFrequencies.length > 0) == (outboundHandlingTimeBucketFrequencies.length > 0);
- out.writeBoolean(inboundHandlingTimeBucketFrequencies.length > 0);
- for (long handlingTimeBucketFrequency : inboundHandlingTimeBucketFrequencies) {
- out.writeVLong(handlingTimeBucketFrequency);
- }
- for (long handlingTimeBucketFrequency : outboundHandlingTimeBucketFrequencies) {
- out.writeVLong(handlingTimeBucketFrequency);
- }
+ assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT;
+ assert outboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT;
+ if (out.getTransportVersion().before(TransportVersions.TRANSPORT_STATS_HANDLING_TIME_REQUIRED)) {
+ out.writeBoolean(true);
+ }
+ for (long handlingTimeBucketFrequency : inboundHandlingTimeBucketFrequencies) {
+ out.writeVLong(handlingTimeBucketFrequency);
+ }
+ for (long handlingTimeBucketFrequency : outboundHandlingTimeBucketFrequencies) {
+ out.writeVLong(handlingTimeBucketFrequency);
}
if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_8_0)) {
out.writeMap(transportActionStats, StreamOutput::writeWriteable);
@@ -166,24 +164,13 @@ public class TransportStats implements Writeable, ChunkedToXContent {
return transportActionStats;
}
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION)
- // Review and simplify the if-else blocks containing this symbol once v9 is released
- private static final boolean IMPOSSIBLE_IN_V9 = true;
-
private boolean assertHistogramsConsistent() {
assert inboundHandlingTimeBucketFrequencies.length == outboundHandlingTimeBucketFrequencies.length;
- if (inboundHandlingTimeBucketFrequencies.length == 0) {
- // Stats came from before v8.1
- assert IMPOSSIBLE_IN_V9;
- } else {
- assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT;
- }
+ assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT;
return true;
}
@Override
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION)
- // review the "if" blocks checking for non-empty once we have
public Iterator extends ToXContent> toXContentChunked(ToXContent.Params outerParams) {
return Iterators.concat(Iterators.single((builder, params) -> {
builder.startObject(Fields.TRANSPORT);
@@ -193,19 +180,10 @@ public class TransportStats implements Writeable, ChunkedToXContent {
builder.humanReadableField(Fields.RX_SIZE_IN_BYTES, Fields.RX_SIZE, ByteSizeValue.ofBytes(rxSize));
builder.field(Fields.TX_COUNT, txCount);
builder.humanReadableField(Fields.TX_SIZE_IN_BYTES, Fields.TX_SIZE, ByteSizeValue.ofBytes(txSize));
- if (inboundHandlingTimeBucketFrequencies.length > 0) {
- histogramToXContent(builder, inboundHandlingTimeBucketFrequencies, Fields.INBOUND_HANDLING_TIME_HISTOGRAM);
- histogramToXContent(builder, outboundHandlingTimeBucketFrequencies, Fields.OUTBOUND_HANDLING_TIME_HISTOGRAM);
- } else {
- // Stats came from before v8.1
- assert IMPOSSIBLE_IN_V9;
- }
- if (transportActionStats.isEmpty() == false) {
- builder.startObject(Fields.ACTIONS);
- } else {
- // Stats came from before v8.8
- assert IMPOSSIBLE_IN_V9;
- }
+ assert inboundHandlingTimeBucketFrequencies.length > 0;
+ histogramToXContent(builder, inboundHandlingTimeBucketFrequencies, Fields.INBOUND_HANDLING_TIME_HISTOGRAM);
+ histogramToXContent(builder, outboundHandlingTimeBucketFrequencies, Fields.OUTBOUND_HANDLING_TIME_HISTOGRAM);
+ builder.startObject(Fields.ACTIONS);
return builder;
}),
@@ -215,12 +193,7 @@ public class TransportStats implements Writeable, ChunkedToXContent {
return builder;
}),
- Iterators.single((builder, params) -> {
- if (transportActionStats.isEmpty() == false) {
- builder.endObject();
- }
- return builder.endObject();
- })
+ Iterators.single((builder, params) -> { return builder.endObject().endObject(); })
);
}
diff --git a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java
index c3965547abb5..1c9cb4c9afc0 100644
--- a/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TransportStatsTests.java
@@ -20,50 +20,8 @@ import java.util.Map;
public class TransportStatsTests extends ESTestCase {
public void testToXContent() {
- assertEquals(
- Strings.toString(
- new TransportStats(1, 2, 3, ByteSizeUnit.MB.toBytes(4), 5, ByteSizeUnit.MB.toBytes(6), new long[0], new long[0], Map.of()),
- false,
- true
- ),
- """
- {"transport":{"server_open":1,"total_outbound_connections":2,\
- "rx_count":3,"rx_size":"4mb","rx_size_in_bytes":4194304,\
- "tx_count":5,"tx_size":"6mb","tx_size_in_bytes":6291456\
- }}"""
- );
-
final var histogram = new long[HandlingTimeTracker.BUCKET_COUNT];
- assertEquals(
- Strings.toString(
- new TransportStats(1, 2, 3, ByteSizeUnit.MB.toBytes(4), 5, ByteSizeUnit.MB.toBytes(6), histogram, histogram, Map.of()),
- false,
- true
- ),
- """
- {"transport":{"server_open":1,"total_outbound_connections":2,\
- "rx_count":3,"rx_size":"4mb","rx_size_in_bytes":4194304,\
- "tx_count":5,"tx_size":"6mb","tx_size_in_bytes":6291456,\
- "inbound_handling_time_histogram":[],\
- "outbound_handling_time_histogram":[]\
- }}"""
- );
-
histogram[4] = 10;
- assertEquals(
- Strings.toString(
- new TransportStats(1, 2, 3, ByteSizeUnit.MB.toBytes(4), 5, ByteSizeUnit.MB.toBytes(6), histogram, histogram, Map.of()),
- false,
- true
- ),
- """
- {"transport":{"server_open":1,"total_outbound_connections":2,\
- "rx_count":3,"rx_size":"4mb","rx_size_in_bytes":4194304,\
- "tx_count":5,"tx_size":"6mb","tx_size_in_bytes":6291456,\
- "inbound_handling_time_histogram":[{"ge":"8ms","ge_millis":8,"lt":"16ms","lt_millis":16,"count":10}],\
- "outbound_handling_time_histogram":[{"ge":"8ms","ge_millis":8,"lt":"16ms","lt_millis":16,"count":10}]\
- }}"""
- );
final var requestSizeHistogram = new long[29];
requestSizeHistogram[2] = 9;
@@ -84,8 +42,8 @@ public class TransportStatsTests extends ESTestCase {
ByteSizeUnit.MB.toBytes(4),
5,
ByteSizeUnit.MB.toBytes(6),
- new long[0],
- new long[0],
+ histogram,
+ histogram,
Map.of("internal:test/action", exampleActionStats)
),
false,
@@ -95,6 +53,8 @@ public class TransportStatsTests extends ESTestCase {
{"transport":{"server_open":1,"total_outbound_connections":2,\
"rx_count":3,"rx_size":"4mb","rx_size_in_bytes":4194304,\
"tx_count":5,"tx_size":"6mb","tx_size_in_bytes":6291456,\
+ "inbound_handling_time_histogram":[{"ge":"8ms","ge_millis":8,"lt":"16ms","lt_millis":16,"count":10}],\
+ "outbound_handling_time_histogram":[{"ge":"8ms","ge_millis":8,"lt":"16ms","lt_millis":16,"count":10}],\
"actions":{"internal:test/action":%s}}}""", Strings.toString(exampleActionStats, false, true))
);
}
From 5728a7210b2066db7c76950f8fa8b9e1cd428970 Mon Sep 17 00:00:00 2001
From: "elastic-renovate-prod[bot]"
<174716857+elastic-renovate-prod[bot]@users.noreply.github.com>
Date: Thu, 6 Feb 2025 01:10:15 +1100
Subject: [PATCH 09/29] Migrate config renovate.json (#121759)
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
The Renovate config in this repository needs migrating. Typically this
is because one or more configuration options you are using have been
renamed.
You don't need to merge this PR right away, because Renovate will
continue to migrate these fields internally each time it runs. But later
some of these fields may be fully deprecated and the migrations removed.
So it's a good idea to merge this migration PR soon.
🔕 **Ignore**: Close this PR and you won't be reminded about config
migration again, but one day your current config may no longer be valid.
❓ Got questions? Does something look wrong to you? Please don't hesitate
to [request help
here](https://redirect.github.com/renovatebot/renovate/discussions).
---
This PR has been generated by [Renovate
Bot](https://redirect.github.com/renovatebot/renovate).
---
renovate.json | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/renovate.json b/renovate.json
index 29406c488f94..53919e027dc7 100644
--- a/renovate.json
+++ b/renovate.json
@@ -30,8 +30,8 @@
"matchDatasources": [
"docker"
],
- "matchPackagePatterns": [
- "^docker.elastic.co/wolfi/chainguard-base$"
+ "matchPackageNames": [
+ "/^docker.elastic.co/wolfi/chainguard-base$/"
]
}
],
From 5c147899d28f2786ccb5cb3b246be1817d634e51 Mon Sep 17 00:00:00 2001
From: elasticsearchmachine
<58790826+elasticsearchmachine@users.noreply.github.com>
Date: Thu, 6 Feb 2025 01:16:07 +1100
Subject: [PATCH 10/29] Mute
org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT
org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT #121411
---
muted-tests.yml | 2 ++
1 file changed, 2 insertions(+)
diff --git a/muted-tests.yml b/muted-tests.yml
index e3b74e44cebc..2386c8c9a02d 100644
--- a/muted-tests.yml
+++ b/muted-tests.yml
@@ -394,6 +394,8 @@ tests:
- class: org.elasticsearch.xpack.esql.heap_attack.HeapAttackIT
method: testLookupExplosionBigStringManyMatches
issue: https://github.com/elastic/elasticsearch/issues/121465
+- class: org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT
+ issue: https://github.com/elastic/elasticsearch/issues/121411
# Examples:
#
From 62f0fe869a3a95da522fc2bf7e84ef612df40f0e Mon Sep 17 00:00:00 2001
From: Artem Prigoda
Date: Wed, 5 Feb 2025 15:35:38 +0100
Subject: [PATCH 11/29] Remove the `failures` field from snapshot responses
(#114496)
Failure handling for snapshots was made stricter in #107191 (8.15), so this field is always empty since then. Clients don't need to check it anymore for failure handling, we can remove it from API responses in 9.0
---
.../http/snapshots/RestGetSnapshotsIT.java | 14 +----
.../snapshots/SnapshotStatusApisIT.java | 1 -
.../org/elasticsearch/TransportVersions.java | 3 +-
.../snapshots/get/GetSnapshotsResponse.java | 56 ++++---------------
.../get/TransportGetSnapshotsAction.java | 1 -
.../rest/action/cat/RestSnapshotAction.java | 20 -------
.../get/GetSnapshotsResponseTests.java | 14 -----
.../core/ilm/WaitForSnapshotStepTests.java | 2 -
8 files changed, 17 insertions(+), 94 deletions(-)
diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java
index b1e28de1a526..683990d51d4a 100644
--- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java
+++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java
@@ -10,7 +10,6 @@
package org.elasticsearch.http.snapshots;
import org.apache.http.client.methods.HttpGet;
-import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
@@ -37,7 +36,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -516,10 +514,9 @@ public class RestGetSnapshotsIT extends AbstractSnapshotRestTestCase {
true,
(args) -> new GetSnapshotsResponse(
(List) args[0],
- (Map) args[1],
- (String) args[2],
- args[3] == null ? UNKNOWN_COUNT : (int) args[3],
- args[4] == null ? UNKNOWN_COUNT : (int) args[4]
+ (String) args[1],
+ args[2] == null ? UNKNOWN_COUNT : (int) args[2],
+ args[3] == null ? UNKNOWN_COUNT : (int) args[3]
)
);
@@ -529,11 +526,6 @@ public class RestGetSnapshotsIT extends AbstractSnapshotRestTestCase {
(p, c) -> SnapshotInfoUtils.snapshotInfoFromXContent(p),
new ParseField("snapshots")
);
- GET_SNAPSHOT_PARSER.declareObject(
- ConstructingObjectParser.optionalConstructorArg(),
- (p, c) -> p.map(HashMap::new, ElasticsearchException::fromXContent),
- new ParseField("failures")
- );
GET_SNAPSHOT_PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField("next"));
GET_SNAPSHOT_PARSER.declareIntOrNull(ConstructingObjectParser.optionalConstructorArg(), UNKNOWN_COUNT, new ParseField("total"));
GET_SNAPSHOT_PARSER.declareIntOrNull(ConstructingObjectParser.optionalConstructorArg(), UNKNOWN_COUNT, new ParseField("remaining"));
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java
index 6922b21be37f..2f4014bf4d35 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java
@@ -316,7 +316,6 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
.get();
assertTrue(getSnapshotsResponse.getSnapshots().isEmpty());
- assertTrue(getSnapshotsResponse.getFailures().isEmpty());
}
public void testGetSnapshotsMultipleRepos() throws Exception {
diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java
index d97246ab7f83..0fde3c9c2b1e 100644
--- a/server/src/main/java/org/elasticsearch/TransportVersions.java
+++ b/server/src/main/java/org/elasticsearch/TransportVersions.java
@@ -175,7 +175,8 @@ public class TransportVersions {
public static final TransportVersion COHERE_BIT_EMBEDDING_TYPE_SUPPORT_ADDED_BACKPORT_8_X = def(8_840_0_01);
public static final TransportVersion ELASTICSEARCH_9_0 = def(9_000_0_00);
public static final TransportVersion COHERE_BIT_EMBEDDING_TYPE_SUPPORT_ADDED = def(9_001_0_00);
- public static final TransportVersion TRANSPORT_STATS_HANDLING_TIME_REQUIRED = def(9_002_0_00);
+ public static final TransportVersion REMOVE_SNAPSHOT_FAILURES = def(9_002_0_00);
+ public static final TransportVersion TRANSPORT_STATS_HANDLING_TIME_REQUIRED = def(9_003_0_00);
/*
* STOP! READ THIS FIRST! No, really,
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
index dc261177567c..2436a9e29887 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java
@@ -9,7 +9,7 @@
package org.elasticsearch.action.admin.cluster.snapshots.get;
-import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Iterators;
@@ -17,12 +17,10 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ChunkedToXContentObject;
import org.elasticsearch.core.Nullable;
-import org.elasticsearch.core.UpdateForV9;
import org.elasticsearch.snapshots.SnapshotInfo;
import org.elasticsearch.xcontent.ToXContent;
import java.io.IOException;
-import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -35,9 +33,6 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
private final List snapshots;
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION) // always empty, can be dropped
- private final Map failures;
-
@Nullable
private final String next;
@@ -45,15 +40,8 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
private final int remaining;
- public GetSnapshotsResponse(
- List snapshots,
- Map failures,
- @Nullable String next,
- final int total,
- final int remaining
- ) {
+ public GetSnapshotsResponse(List snapshots, @Nullable String next, final int total, final int remaining) {
this.snapshots = List.copyOf(snapshots);
- this.failures = failures == null ? Map.of() : Map.copyOf(failures);
this.next = next;
this.total = total;
this.remaining = remaining;
@@ -61,7 +49,10 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
public GetSnapshotsResponse(StreamInput in) throws IOException {
this.snapshots = in.readCollectionAsImmutableList(SnapshotInfo::readFrom);
- this.failures = Collections.unmodifiableMap(in.readMap(StreamInput::readException));
+ if (in.getTransportVersion().before(TransportVersions.REMOVE_SNAPSHOT_FAILURES)) {
+ // Deprecated `failures` field
+ in.readMap(StreamInput::readException);
+ }
this.next = in.readOptionalString();
this.total = in.readVInt();
this.remaining = in.readVInt();
@@ -76,25 +67,11 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
return snapshots;
}
- /**
- * Returns a map of repository name to {@link ElasticsearchException} for each unsuccessful response.
- */
- public Map getFailures() {
- return failures;
- }
-
@Nullable
public String next() {
return next;
}
- /**
- * Returns true if there is at least one failed response.
- */
- public boolean isFailed() {
- return failures.isEmpty() == false;
- }
-
public int totalCount() {
return total;
}
@@ -106,7 +83,10 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeCollection(snapshots);
- out.writeMap(failures, StreamOutput::writeException);
+ if (out.getTransportVersion().before(TransportVersions.REMOVE_SNAPSHOT_FAILURES)) {
+ // Deprecated `failures` field
+ out.writeMap(Map.of(), StreamOutput::writeException);
+ }
out.writeOptionalString(next);
out.writeVInt(total);
out.writeVInt(remaining);
@@ -120,18 +100,6 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
return b;
}), Iterators.map(getSnapshots().iterator(), snapshotInfo -> snapshotInfo::toXContentExternal), Iterators.single((b, p) -> {
b.endArray();
- if (failures.isEmpty() == false) {
- b.startObject("failures");
- for (Map.Entry error : failures.entrySet()) {
- b.field(error.getKey(), (bb, pa) -> {
- bb.startObject();
- error.getValue().toXContent(bb, pa);
- bb.endObject();
- return bb;
- });
- }
- b.endObject();
- }
if (next != null) {
b.field("next", next);
}
@@ -151,12 +119,12 @@ public class GetSnapshotsResponse extends ActionResponse implements ChunkedToXCo
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
GetSnapshotsResponse that = (GetSnapshotsResponse) o;
- return Objects.equals(snapshots, that.snapshots) && Objects.equals(failures, that.failures) && Objects.equals(next, that.next);
+ return Objects.equals(snapshots, that.snapshots) && Objects.equals(next, that.next);
}
@Override
public int hashCode() {
- return Objects.hash(snapshots, failures, next);
+ return Objects.hash(snapshots, next);
}
@Override
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
index 896b336d54d7..ec4a578ef25c 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java
@@ -543,7 +543,6 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction 0 ? sortBy.encodeAfterQueryParam(snapshotInfos.get(snapshotInfos.size() - 1)) : null,
totalCount.get(),
remaining
diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java
index 94fa60762800..f8dc26e9c468 100644
--- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java
+++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java
@@ -9,11 +9,9 @@
package org.elasticsearch.rest.action.cat;
-import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
import org.elasticsearch.client.internal.node.NodeClient;
-import org.elasticsearch.common.Strings;
import org.elasticsearch.common.Table;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.core.TimeValue;
@@ -99,24 +97,6 @@ public class RestSnapshotAction extends AbstractCatAction {
private Table buildTable(RestRequest req, GetSnapshotsResponse getSnapshotsResponse) {
Table table = getTableWithHeader(req);
- if (getSnapshotsResponse.isFailed()) {
- ElasticsearchException causes = null;
-
- for (ElasticsearchException e : getSnapshotsResponse.getFailures().values()) {
- if (causes == null) {
- causes = e;
- } else {
- causes.addSuppressed(e);
- }
- }
- throw new ElasticsearchException(
- "Repositories ["
- + Strings.collectionToCommaDelimitedString(getSnapshotsResponse.getFailures().keySet())
- + "] failed to retrieve snapshots",
- causes
- );
- }
-
for (SnapshotInfo snapshotStatus : getSnapshotsResponse.getSnapshots()) {
table.startRow();
diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java
index 13479d188536..d3e5c1c7268f 100644
--- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java
+++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java
@@ -9,7 +9,6 @@
package org.elasticsearch.action.admin.cluster.snapshots.get;
-import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.TransportVersion;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@@ -31,14 +30,10 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
import java.util.Collections;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
-import java.util.Map;
import java.util.Set;
-import static org.hamcrest.CoreMatchers.containsString;
-
public class GetSnapshotsResponseTests extends ESTestCase {
// We can not subclass AbstractSerializingTestCase because it
// can only be used for instances with equals and hashCode
@@ -60,12 +55,6 @@ public class GetSnapshotsResponseTests extends ESTestCase {
private void assertEqualInstances(GetSnapshotsResponse expectedInstance, GetSnapshotsResponse newInstance) {
assertEquals(expectedInstance.getSnapshots(), newInstance.getSnapshots());
assertEquals(expectedInstance.next(), newInstance.next());
- assertEquals(expectedInstance.getFailures().keySet(), newInstance.getFailures().keySet());
- for (Map.Entry expectedEntry : expectedInstance.getFailures().entrySet()) {
- ElasticsearchException expectedException = expectedEntry.getValue();
- ElasticsearchException newException = newInstance.getFailures().get(expectedEntry.getKey());
- assertThat(newException.getMessage(), containsString(expectedException.getMessage()));
- }
}
private List createSnapshotInfos(String repoName) {
@@ -99,7 +88,6 @@ public class GetSnapshotsResponseTests extends ESTestCase {
private GetSnapshotsResponse createTestInstance() {
Set repositories = new HashSet<>();
- Map failures = new HashMap<>();
List responses = new ArrayList<>();
for (int i = 0; i < randomIntBetween(0, 5); i++) {
@@ -111,12 +99,10 @@ public class GetSnapshotsResponseTests extends ESTestCase {
for (int i = 0; i < randomIntBetween(0, 5); i++) {
String repository = randomValueOtherThanMany(repositories::contains, () -> randomAlphaOfLength(10));
repositories.add(repository);
- failures.put(repository, new ElasticsearchException(randomAlphaOfLength(10)));
}
return new GetSnapshotsResponse(
responses,
- failures,
randomBoolean()
? Base64.getUrlEncoder()
.encodeToString(
diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForSnapshotStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForSnapshotStepTests.java
index ed1cb477c30e..6e31759fd6c2 100644
--- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForSnapshotStepTests.java
+++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/WaitForSnapshotStepTests.java
@@ -164,7 +164,6 @@ public class WaitForSnapshotStepTests extends AbstractStepTestCase
Date: Wed, 5 Feb 2025 09:41:00 -0500
Subject: [PATCH 12/29] [ML] Change format for Unified Chat error responses
(#121396)
Unified Chat Completion error responses now forward code, type, and
param to in the response payload. `reason` has been renamed to
`message`.
Notes:
- `XContentFormattedException` is a `ChunkedToXContent` so that the REST listener can call `toXContentChunked` to format the output structure. By default, the structure forwards to our existing ES exception structure.
- `UnifiedChatCompletionException` will override the structure to match the new unified format.
- The Rest, Transport, and Stream handlers all check the exception to verify it is a UnifiedChatCompletionException.
- OpenAI response handler now reads all the fields in the error message and forwards them to the user.
- In the event that a `Throwable` is a `Error`, we rethrow it on another thread so the JVM can catch and handle it. We also stop surfacing the JVM details to the user in the error message (but it's still logged for debugging purposes).
---
docs/changelog/121396.yaml | 5 +
.../UnifiedChatCompletionException.java | 117 +++++++++++++++
.../results/XContentFormattedException.java | 87 ++++++++++++
...rverSentEventsRestActionListenerTests.java | 33 +++++
.../action/BaseTransportInferenceAction.java | 12 +-
...sportUnifiedCompletionInferenceAction.java | 42 ++++++
...eUnifiedChatCompletionResponseHandler.java | 20 +++
.../http/retry/BaseResponseHandler.java | 37 ++---
.../OpenAiChatCompletionResponseHandler.java | 11 ++
.../openai/OpenAiResponseHandler.java | 13 +-
...iUnifiedChatCompletionResponseHandler.java | 102 ++++++++++++-
.../RestUnifiedCompletionInferenceAction.java | 6 +-
.../ServerSentEventsRestActionListener.java | 15 +-
.../BaseTransportInferenceActionTestCase.java | 28 ++--
...TransportUnifiedCompletionActionTests.java | 10 +-
...iedChatCompletionResponseHandlerTests.java | 134 ++++++++++++++++++
.../services/InferenceEventsAssertion.java | 11 ++
.../elastic/ElasticInferenceServiceTests.java | 66 +++++++++
.../services/openai/OpenAiServiceTests.java | 58 ++++++++
19 files changed, 761 insertions(+), 46 deletions(-)
create mode 100644 docs/changelog/121396.yaml
create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/UnifiedChatCompletionException.java
create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/XContentFormattedException.java
create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandlerTests.java
diff --git a/docs/changelog/121396.yaml b/docs/changelog/121396.yaml
new file mode 100644
index 000000000000..1d77a8fbb007
--- /dev/null
+++ b/docs/changelog/121396.yaml
@@ -0,0 +1,5 @@
+pr: 121396
+summary: Change format for Unified Chat
+area: Machine Learning
+type: bug
+issues: []
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/UnifiedChatCompletionException.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/UnifiedChatCompletionException.java
new file mode 100644
index 000000000000..f2844e6534a9
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/UnifiedChatCompletionException.java
@@ -0,0 +1,117 @@
+/*
+ * 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.inference.results;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.elasticsearch.ExceptionsHelper;
+import org.elasticsearch.common.xcontent.ChunkedToXContentHelper;
+import org.elasticsearch.core.Nullable;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.xcontent.ToXContent;
+
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.Objects;
+
+import static java.util.Collections.emptyIterator;
+import static org.elasticsearch.ExceptionsHelper.maybeError;
+import static org.elasticsearch.common.collect.Iterators.concat;
+import static org.elasticsearch.common.xcontent.ChunkedToXContentHelper.endObject;
+import static org.elasticsearch.common.xcontent.ChunkedToXContentHelper.startObject;
+
+public class UnifiedChatCompletionException extends XContentFormattedException {
+
+ private static final Logger log = LogManager.getLogger(UnifiedChatCompletionException.class);
+ private final String message;
+ private final String type;
+ @Nullable
+ private final String code;
+ @Nullable
+ private final String param;
+
+ public UnifiedChatCompletionException(RestStatus status, String message, String type, @Nullable String code) {
+ this(status, message, type, code, null);
+ }
+
+ public UnifiedChatCompletionException(RestStatus status, String message, String type, @Nullable String code, @Nullable String param) {
+ super(message, status);
+ this.message = Objects.requireNonNull(message);
+ this.type = Objects.requireNonNull(type);
+ this.code = code;
+ this.param = param;
+ }
+
+ public UnifiedChatCompletionException(
+ Throwable cause,
+ RestStatus status,
+ String message,
+ String type,
+ @Nullable String code,
+ @Nullable String param
+ ) {
+ super(message, cause, status);
+ this.message = Objects.requireNonNull(message);
+ this.type = Objects.requireNonNull(type);
+ this.code = code;
+ this.param = param;
+ }
+
+ @Override
+ public Iterator extends ToXContent> toXContentChunked(Params params) {
+ return concat(
+ startObject(),
+ startObject("error"),
+ optionalField("code", code),
+ field("message", message),
+ optionalField("param", param),
+ field("type", type),
+ endObject(),
+ endObject()
+ );
+ }
+
+ private static Iterator field(String key, String value) {
+ return ChunkedToXContentHelper.chunk((b, p) -> b.field(key, value));
+ }
+
+ private static Iterator optionalField(String key, String value) {
+ return value != null ? ChunkedToXContentHelper.chunk((b, p) -> b.field(key, value)) : emptyIterator();
+ }
+
+ public static UnifiedChatCompletionException fromThrowable(Throwable t) {
+ if (ExceptionsHelper.unwrapCause(t) instanceof UnifiedChatCompletionException e) {
+ return e;
+ } else {
+ return maybeError(t).map(error -> {
+ // we should never be throwing Error, but just in case we are, rethrow it on another thread so the JVM can handle it and
+ // return a vague error to the user so that they at least see something went wrong but don't leak JVM details to users
+ ExceptionsHelper.maybeDieOnAnotherThread(error);
+ var e = new RuntimeException("Fatal error while streaming response. Please retry the request.");
+ log.error(e.getMessage(), t);
+ return new UnifiedChatCompletionException(
+ RestStatus.INTERNAL_SERVER_ERROR,
+ e.getMessage(),
+ getExceptionName(e),
+ RestStatus.INTERNAL_SERVER_ERROR.name().toLowerCase(Locale.ROOT)
+ );
+ }).orElseGet(() -> {
+ log.atDebug().withThrowable(t).log("UnifiedChatCompletionException stack trace for debugging purposes.");
+ var status = ExceptionsHelper.status(t);
+ return new UnifiedChatCompletionException(
+ t,
+ status,
+ t.getMessage(),
+ getExceptionName(t),
+ status.name().toLowerCase(Locale.ROOT),
+ null
+ );
+ });
+ }
+ }
+}
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/XContentFormattedException.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/XContentFormattedException.java
new file mode 100644
index 000000000000..799953d452f0
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/XContentFormattedException.java
@@ -0,0 +1,87 @@
+/*
+ * 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.inference.results;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.common.collect.Iterators;
+import org.elasticsearch.common.xcontent.ChunkedToXContent;
+import org.elasticsearch.common.xcontent.ChunkedToXContentHelper;
+import org.elasticsearch.core.RestApiVersion;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.xcontent.ToXContent;
+import org.elasticsearch.xcontent.XContentBuilder;
+
+import java.util.Iterator;
+import java.util.Objects;
+
+/**
+ * Similar to {@link org.elasticsearch.ElasticsearchWrapperException}, this will wrap an Exception to generate an xContent using
+ * {@link ElasticsearchException#generateFailureXContent(XContentBuilder, Params, Exception, boolean)}.
+ * Extends {@link ElasticsearchException} to provide REST handlers the {@link #status()} method in order to set the response header.
+ */
+public class XContentFormattedException extends ElasticsearchException implements ChunkedToXContent {
+
+ public static final String X_CONTENT_PARAM = "detailedErrorsEnabled";
+ private final RestStatus status;
+ private final Throwable cause;
+
+ public XContentFormattedException(String message, RestStatus status) {
+ super(message);
+ this.status = Objects.requireNonNull(status);
+ this.cause = null;
+ }
+
+ public XContentFormattedException(Throwable cause, RestStatus status) {
+ super(cause);
+ this.status = Objects.requireNonNull(status);
+ this.cause = cause;
+ }
+
+ public XContentFormattedException(String message, Throwable cause, RestStatus status) {
+ super(message, cause);
+ this.status = Objects.requireNonNull(status);
+ this.cause = cause;
+ }
+
+ @Override
+ public RestStatus status() {
+ return status;
+ }
+
+ @Override
+ public Iterator extends ToXContent> toXContentChunked(Params params) {
+ return Iterators.concat(
+ ChunkedToXContentHelper.startObject(),
+ Iterators.single(
+ (b, p) -> ElasticsearchException.generateFailureXContent(
+ b,
+ p,
+ cause instanceof Exception e ? e : this,
+ params.paramAsBoolean(X_CONTENT_PARAM, false)
+ )
+ ),
+ Iterators.single((b, p) -> b.field("status", status.getStatus())),
+ ChunkedToXContentHelper.endObject()
+ );
+ }
+
+ @Override
+ public Iterator extends ToXContent> toXContentChunked(RestApiVersion restApiVersion, Params params) {
+ return ChunkedToXContent.super.toXContentChunked(restApiVersion, params);
+ }
+
+ @Override
+ public Iterator extends ToXContent> toXContentChunkedV8(Params params) {
+ return ChunkedToXContent.super.toXContentChunkedV8(params);
+ }
+
+ @Override
+ public boolean isFragment() {
+ return super.isFragment();
+ }
+}
diff --git a/x-pack/plugin/inference/src/internalClusterTest/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListenerTests.java b/x-pack/plugin/inference/src/internalClusterTest/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListenerTests.java
index a22e179479de..903961794b33 100644
--- a/x-pack/plugin/inference/src/internalClusterTest/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListenerTests.java
+++ b/x-pack/plugin/inference/src/internalClusterTest/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListenerTests.java
@@ -44,10 +44,12 @@ import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestHandler;
import org.elasticsearch.rest.RestRequest;
+import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xpack.core.inference.action.InferenceAction;
+import org.elasticsearch.xpack.core.inference.results.XContentFormattedException;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEvent;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventField;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventParser;
@@ -80,6 +82,7 @@ public class ServerSentEventsRestActionListenerTests extends ESIntegTestCase {
private static final String REQUEST_COUNT = "request_count";
private static final String WITH_ERROR = "with_error";
private static final String ERROR_ROUTE = "/_inference_error";
+ private static final String FORMATTED_ERROR_ROUTE = "/_formatted_inference_error";
private static final String NO_STREAM_ROUTE = "/_inference_no_stream";
private static final Exception expectedException = new IllegalStateException("hello there");
private static final String expectedExceptionAsServerSentEvent = """
@@ -88,6 +91,11 @@ public class ServerSentEventsRestActionListenerTests extends ESIntegTestCase {
"type":"illegal_state_exception","reason":"hello there"},"status":500\
}""";
+ private static final Exception expectedFormattedException = new XContentFormattedException(
+ expectedException,
+ RestStatus.INTERNAL_SERVER_ERROR
+ );
+
@Override
protected boolean addMockHttpTransport() {
return false;
@@ -145,6 +153,16 @@ public class ServerSentEventsRestActionListenerTests extends ESIntegTestCase {
public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) {
new ServerSentEventsRestActionListener(channel, threadPool).onFailure(expectedException);
}
+ }, new RestHandler() {
+ @Override
+ public List routes() {
+ return List.of(new Route(RestRequest.Method.POST, FORMATTED_ERROR_ROUTE));
+ }
+
+ @Override
+ public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) {
+ new ServerSentEventsRestActionListener(channel, threadPool).onFailure(expectedFormattedException);
+ }
}, new RestHandler() {
@Override
public List routes() {
@@ -424,6 +442,21 @@ public class ServerSentEventsRestActionListenerTests extends ESIntegTestCase {
assertThat(collector.stringsVerified.getLast(), equalTo(expectedExceptionAsServerSentEvent));
}
+ public void testFormattedError() throws IOException {
+ var request = new Request(RestRequest.Method.POST.name(), FORMATTED_ERROR_ROUTE);
+
+ try {
+ getRestClient().performRequest(request);
+ fail("Expected an exception to be thrown from the error route");
+ } catch (ResponseException e) {
+ var response = e.getResponse();
+ assertThat(response.getStatusLine().getStatusCode(), is(HttpStatus.SC_INTERNAL_SERVER_ERROR));
+ assertThat(EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8), equalTo("""
+ \uFEFFevent: error
+ data:\s""" + expectedExceptionAsServerSentEvent + "\n\n"));
+ }
+ }
+
public void testNoStream() {
var collector = new RandomStringCollector();
var expectedTestCount = randomIntBetween(2, 30);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java
index bcfeef9f4af9..4afafc5adf0c 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java
@@ -50,9 +50,11 @@ import org.elasticsearch.xpack.inference.telemetry.InferenceTimer;
import java.io.IOException;
import java.util.Random;
import java.util.concurrent.Executor;
+import java.util.concurrent.Flow;
import java.util.function.Supplier;
import java.util.stream.Collectors;
+import static org.elasticsearch.ExceptionsHelper.unwrapCause;
import static org.elasticsearch.core.Strings.format;
import static org.elasticsearch.xpack.inference.InferencePlugin.INFERENCE_API_FEATURE;
import static org.elasticsearch.xpack.inference.telemetry.InferenceStats.modelAttributes;
@@ -280,7 +282,9 @@ public abstract class BaseTransportInferenceAction streamErrorHandler(Flow.Processor upstream) {
+ return upstream;
+ }
+
private void recordMetrics(Model model, InferenceTimer timer, @Nullable Throwable t) {
try {
- inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, t));
+ inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, unwrapCause(t)));
} catch (Exception e) {
log.atDebug().withThrowable(e).log("Failed to record metrics with a parsed model, dropping metrics");
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java
index 2e3090f2afd5..1144a11d86cc 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java
@@ -11,6 +11,7 @@ import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.client.internal.node.NodeClient;
+import org.elasticsearch.common.xcontent.ChunkedToXContent;
import org.elasticsearch.inference.InferenceService;
import org.elasticsearch.inference.InferenceServiceRegistry;
import org.elasticsearch.inference.InferenceServiceResults;
@@ -20,14 +21,19 @@ import org.elasticsearch.inference.UnparsedModel;
import org.elasticsearch.injection.guice.Inject;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xpack.core.inference.action.InferenceAction;
import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager;
import org.elasticsearch.xpack.inference.common.InferenceServiceRateLimitCalculator;
import org.elasticsearch.xpack.inference.registry.ModelRegistry;
import org.elasticsearch.xpack.inference.telemetry.InferenceStats;
+import java.util.concurrent.Flow;
+
public class TransportUnifiedCompletionInferenceAction extends BaseTransportInferenceAction {
@Inject
@@ -86,4 +92,40 @@ public class TransportUnifiedCompletionInferenceAction extends BaseTransportInfe
) {
service.unifiedCompletionInfer(model, request.getUnifiedCompletionRequest(), null, listener);
}
+
+ @Override
+ protected void doExecute(Task task, UnifiedCompletionAction.Request request, ActionListener listener) {
+ super.doExecute(task, request, listener.delegateResponse((l, e) -> l.onFailure(UnifiedChatCompletionException.fromThrowable(e))));
+ }
+
+ /**
+ * If we get any errors, either in {@link #doExecute} via the listener.onFailure or while streaming, make sure that they are formatted
+ * as {@link UnifiedChatCompletionException}.
+ */
+ @Override
+ protected Flow.Publisher streamErrorHandler(Flow.Processor upstream) {
+ return downstream -> {
+ upstream.subscribe(new Flow.Subscriber<>() {
+ @Override
+ public void onSubscribe(Flow.Subscription subscription) {
+ downstream.onSubscribe(subscription);
+ }
+
+ @Override
+ public void onNext(ChunkedToXContent item) {
+ downstream.onNext(item);
+ }
+
+ @Override
+ public void onError(Throwable throwable) {
+ downstream.onError(UnifiedChatCompletionException.fromThrowable(throwable));
+ }
+
+ @Override
+ public void onComplete() {
+ downstream.onComplete();
+ }
+ });
+ };
+ }
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/elastic/ElasticInferenceServiceUnifiedChatCompletionResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/elastic/ElasticInferenceServiceUnifiedChatCompletionResponseHandler.java
index e1438dde76c9..db09317b7b79 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/elastic/ElasticInferenceServiceUnifiedChatCompletionResponseHandler.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/elastic/ElasticInferenceServiceUnifiedChatCompletionResponseHandler.java
@@ -9,13 +9,16 @@ package org.elasticsearch.xpack.inference.external.elastic;
import org.elasticsearch.inference.InferenceServiceResults;
import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.inference.external.http.HttpResult;
+import org.elasticsearch.xpack.inference.external.http.retry.ErrorResponse;
import org.elasticsearch.xpack.inference.external.http.retry.ResponseParser;
import org.elasticsearch.xpack.inference.external.openai.OpenAiUnifiedStreamingProcessor;
import org.elasticsearch.xpack.inference.external.request.Request;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventParser;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventProcessor;
+import java.util.Locale;
import java.util.concurrent.Flow;
public class ElasticInferenceServiceUnifiedChatCompletionResponseHandler extends ElasticInferenceServiceResponseHandler {
@@ -32,4 +35,21 @@ public class ElasticInferenceServiceUnifiedChatCompletionResponseHandler extends
serverSentEventProcessor.subscribe(openAiProcessor);
return new StreamingUnifiedChatCompletionResults(openAiProcessor);
}
+
+ @Override
+ protected Exception buildError(String message, Request request, HttpResult result, ErrorResponse errorResponse) {
+ assert request.isStreaming() : "Only streaming requests support this format";
+ var responseStatusCode = result.response().getStatusLine().getStatusCode();
+ if (request.isStreaming()) {
+ var restStatus = toRestStatus(responseStatusCode);
+ return new UnifiedChatCompletionException(
+ restStatus,
+ errorMessage(message, request, result, errorResponse, responseStatusCode),
+ "error",
+ restStatus.name().toLowerCase(Locale.ROOT)
+ );
+ } else {
+ return super.buildError(message, request, result, errorResponse);
+ }
+ }
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/retry/BaseResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/retry/BaseResponseHandler.java
index ed852e5177ac..cb5ed53fc558 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/retry/BaseResponseHandler.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/retry/BaseResponseHandler.java
@@ -107,33 +107,26 @@ public abstract class BaseResponseHandler implements ResponseHandler {
protected Exception buildError(String message, Request request, HttpResult result, ErrorResponse errorResponse) {
var responseStatusCode = result.response().getStatusLine().getStatusCode();
-
- if (errorResponse == null
- || errorResponse.errorStructureFound() == false
- || Strings.isNullOrEmpty(errorResponse.getErrorMessage())) {
- return new ElasticsearchStatusException(
- format(
- "%s for request from inference entity id [%s] status [%s]",
- message,
- request.getInferenceEntityId(),
- responseStatusCode
- ),
- toRestStatus(responseStatusCode)
- );
- }
-
return new ElasticsearchStatusException(
- format(
- "%s for request from inference entity id [%s] status [%s]. Error message: [%s]",
- message,
- request.getInferenceEntityId(),
- responseStatusCode,
- errorResponse.getErrorMessage()
- ),
+ errorMessage(message, request, result, errorResponse, responseStatusCode),
toRestStatus(responseStatusCode)
);
}
+ protected String errorMessage(String message, Request request, HttpResult result, ErrorResponse errorResponse, int statusCode) {
+ return (errorResponse == null
+ || errorResponse.errorStructureFound() == false
+ || Strings.isNullOrEmpty(errorResponse.getErrorMessage()))
+ ? format("%s for request from inference entity id [%s] status [%s]", message, request.getInferenceEntityId(), statusCode)
+ : format(
+ "%s for request from inference entity id [%s] status [%s]. Error message: [%s]",
+ message,
+ request.getInferenceEntityId(),
+ statusCode,
+ errorResponse.getErrorMessage()
+ );
+ }
+
public static RestStatus toRestStatus(int statusCode) {
RestStatus code = null;
if (statusCode < 500) {
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiChatCompletionResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiChatCompletionResponseHandler.java
index 7607e5e4ed3a..99f2a7c31e7d 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiChatCompletionResponseHandler.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiChatCompletionResponseHandler.java
@@ -8,15 +8,26 @@
package org.elasticsearch.xpack.inference.external.openai;
import org.elasticsearch.xpack.inference.external.http.HttpResult;
+import org.elasticsearch.xpack.inference.external.http.retry.ErrorResponse;
import org.elasticsearch.xpack.inference.external.http.retry.ResponseParser;
import org.elasticsearch.xpack.inference.external.http.retry.RetryException;
import org.elasticsearch.xpack.inference.external.request.Request;
+import java.util.function.Function;
+
public class OpenAiChatCompletionResponseHandler extends OpenAiResponseHandler {
public OpenAiChatCompletionResponseHandler(String requestType, ResponseParser parseFunction) {
super(requestType, parseFunction, true);
}
+ protected OpenAiChatCompletionResponseHandler(
+ String requestType,
+ ResponseParser parseFunction,
+ Function errorParseFunction
+ ) {
+ super(requestType, parseFunction, errorParseFunction, true);
+ }
+
@Override
protected RetryException buildExceptionHandling429(Request request, HttpResult result) {
// We don't retry, if the chat completion input is too large
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiResponseHandler.java
index e0bc341fc679..8698955868a7 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiResponseHandler.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiResponseHandler.java
@@ -13,6 +13,7 @@ import org.elasticsearch.xpack.core.inference.results.StreamingChatCompletionRes
import org.elasticsearch.xpack.inference.external.http.HttpResult;
import org.elasticsearch.xpack.inference.external.http.retry.BaseResponseHandler;
import org.elasticsearch.xpack.inference.external.http.retry.ContentTooLargeException;
+import org.elasticsearch.xpack.inference.external.http.retry.ErrorResponse;
import org.elasticsearch.xpack.inference.external.http.retry.ResponseParser;
import org.elasticsearch.xpack.inference.external.http.retry.RetryException;
import org.elasticsearch.xpack.inference.external.request.Request;
@@ -21,6 +22,7 @@ import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentE
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventProcessor;
import java.util.concurrent.Flow;
+import java.util.function.Function;
import static org.elasticsearch.xpack.inference.external.http.retry.ResponseHandlerUtils.getFirstHeaderOrUnknown;
@@ -42,7 +44,16 @@ public class OpenAiResponseHandler extends BaseResponseHandler {
static final String OPENAI_SERVER_BUSY = "Received a server busy error status code";
public OpenAiResponseHandler(String requestType, ResponseParser parseFunction, boolean canHandleStreamingResponses) {
- super(requestType, parseFunction, ErrorMessageResponseEntity::fromResponse, canHandleStreamingResponses);
+ this(requestType, parseFunction, ErrorMessageResponseEntity::fromResponse, canHandleStreamingResponses);
+ }
+
+ protected OpenAiResponseHandler(
+ String requestType,
+ ResponseParser parseFunction,
+ Function errorParseFunction,
+ boolean canHandleStreamingResponses
+ ) {
+ super(requestType, parseFunction, errorParseFunction, canHandleStreamingResponses);
}
/**
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java
index fce2556efc5e..2901b449f8a6 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java
@@ -7,19 +7,31 @@
package org.elasticsearch.xpack.inference.external.openai;
+import org.elasticsearch.core.Nullable;
import org.elasticsearch.inference.InferenceServiceResults;
+import org.elasticsearch.xcontent.ConstructingObjectParser;
+import org.elasticsearch.xcontent.ParseField;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xcontent.XContentParser;
+import org.elasticsearch.xcontent.XContentParserConfiguration;
+import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.inference.external.http.HttpResult;
+import org.elasticsearch.xpack.inference.external.http.retry.ErrorResponse;
import org.elasticsearch.xpack.inference.external.http.retry.ResponseParser;
import org.elasticsearch.xpack.inference.external.request.Request;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventParser;
import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventProcessor;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.Optional;
import java.util.concurrent.Flow;
public class OpenAiUnifiedChatCompletionResponseHandler extends OpenAiChatCompletionResponseHandler {
public OpenAiUnifiedChatCompletionResponseHandler(String requestType, ResponseParser parseFunction) {
- super(requestType, parseFunction);
+ super(requestType, parseFunction, OpenAiErrorResponse::fromResponse);
}
@Override
@@ -31,4 +43,92 @@ public class OpenAiUnifiedChatCompletionResponseHandler extends OpenAiChatComple
serverSentEventProcessor.subscribe(openAiProcessor);
return new StreamingUnifiedChatCompletionResults(openAiProcessor);
}
+
+ @Override
+ protected Exception buildError(String message, Request request, HttpResult result, ErrorResponse errorResponse) {
+ assert request.isStreaming() : "Only streaming requests support this format";
+ var responseStatusCode = result.response().getStatusLine().getStatusCode();
+ if (request.isStreaming()) {
+ var errorMessage = errorMessage(message, request, result, errorResponse, responseStatusCode);
+ var restStatus = toRestStatus(responseStatusCode);
+ return errorResponse instanceof OpenAiErrorResponse oer
+ ? new UnifiedChatCompletionException(restStatus, errorMessage, oer.type(), oer.code(), oer.param())
+ : new UnifiedChatCompletionException(
+ restStatus,
+ errorMessage,
+ errorResponse != null ? errorResponse.getClass().getSimpleName() : "unknown",
+ restStatus.name().toLowerCase(Locale.ROOT)
+ );
+ } else {
+ return super.buildError(message, request, result, errorResponse);
+ }
+ }
+
+ private static class OpenAiErrorResponse extends ErrorResponse {
+ private static final ConstructingObjectParser, Void> ERROR_PARSER = new ConstructingObjectParser<>(
+ "open_ai_error",
+ true,
+ args -> Optional.ofNullable((OpenAiErrorResponse) args[0])
+ );
+ private static final ConstructingObjectParser ERROR_BODY_PARSER = new ConstructingObjectParser<>(
+ "open_ai_error",
+ true,
+ args -> new OpenAiErrorResponse((String) args[0], (String) args[1], (String) args[2], (String) args[3])
+ );
+
+ static {
+ ERROR_BODY_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("message"));
+ ERROR_BODY_PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField("code"));
+ ERROR_BODY_PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField("param"));
+ ERROR_BODY_PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("type"));
+
+ ERROR_PARSER.declareObjectOrNull(
+ ConstructingObjectParser.optionalConstructorArg(),
+ ERROR_BODY_PARSER,
+ null,
+ new ParseField("error")
+ );
+ }
+
+ private static ErrorResponse fromResponse(HttpResult response) {
+ try (
+ XContentParser parser = XContentFactory.xContent(XContentType.JSON)
+ .createParser(XContentParserConfiguration.EMPTY, response.body())
+ ) {
+ return ERROR_PARSER.apply(parser, null).orElse(ErrorResponse.UNDEFINED_ERROR);
+ } catch (Exception e) {
+ // swallow the error
+ }
+
+ return ErrorResponse.UNDEFINED_ERROR;
+ }
+
+ @Nullable
+ private final String code;
+ @Nullable
+ private final String param;
+ private final String type;
+
+ OpenAiErrorResponse(String errorMessage, @Nullable String code, @Nullable String param, String type) {
+ super(errorMessage);
+ this.code = code;
+ this.param = param;
+ this.type = Objects.requireNonNull(type);
+ }
+
+ @Nullable
+ public String code() {
+ return code;
+ }
+
+ @Nullable
+ public String param() {
+ return param;
+ }
+
+ public String type() {
+ return type;
+ }
+ }
+
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java
index 51f1bc48c830..0efd31a6832c 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java
@@ -15,6 +15,7 @@ import org.elasticsearch.rest.Scope;
import org.elasticsearch.rest.ServerlessScope;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import java.io.IOException;
import java.util.List;
@@ -57,7 +58,10 @@ public class RestUnifiedCompletionInferenceAction extends BaseRestHandler {
return channel -> client.execute(
UnifiedCompletionAction.INSTANCE,
request,
- new ServerSentEventsRestActionListener(channel, threadPool)
+ new ServerSentEventsRestActionListener(channel, threadPool).delegateResponse((l, e) -> {
+ // format any validation exceptions from the rest -> transport path as UnifiedChatCompletionException
+ l.onFailure(UnifiedChatCompletionException.fromThrowable(e));
+ })
);
}
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListener.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListener.java
index 6991e1325f3b..cadf3e5f1806 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListener.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/ServerSentEventsRestActionListener.java
@@ -35,15 +35,19 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xpack.core.inference.action.InferenceAction;
+import org.elasticsearch.xpack.core.inference.results.XContentFormattedException;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Iterator;
+import java.util.Map;
import java.util.Objects;
import java.util.concurrent.Flow;
import java.util.concurrent.atomic.AtomicBoolean;
+import static org.elasticsearch.xpack.core.inference.results.XContentFormattedException.X_CONTENT_PARAM;
+
/**
* A version of {@link org.elasticsearch.rest.action.RestChunkedToXContentListener} that reads from a {@link Flow.Publisher} and encodes
* the response in Server-Sent Events.
@@ -72,7 +76,7 @@ public class ServerSentEventsRestActionListener implements ActionListener threadPool) {
this.channel = channel;
- this.params = params;
+ this.params = new ToXContent.DelegatingMapParams(Map.of(X_CONTENT_PARAM, String.valueOf(channel.detailedErrorsEnabled())), params);
this.threadPool = Objects.requireNonNull(threadPool);
}
@@ -150,6 +154,12 @@ public class ServerSentEventsRestActionListener implements ActionListener Iterators.concat(
ChunkedToXContentHelper.startObject(),
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java
index 562c99c0887b..56966ca40c47 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java
@@ -7,7 +7,7 @@
package org.elasticsearch.xpack.inference.action;
-import org.elasticsearch.ElasticsearchStatusException;
+import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.client.internal.node.NodeClient;
@@ -47,9 +47,9 @@ import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.assertArg;
-import static org.mockito.ArgumentMatchers.same;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
@@ -127,8 +127,7 @@ public abstract class BaseTransportInferenceActionTestCase {
assertThat(attributes.get("service"), nullValue());
@@ -148,7 +147,13 @@ public abstract class BaseTransportInferenceActionTestCase listener = mock();
+ ActionListener listener = spy(new ActionListener<>() {
+ @Override
+ public void onResponse(InferenceAction.Response o) {}
+
+ @Override
+ public void onFailure(Exception e) {}
+ });
action.doExecute(mock(), request, listener);
return listener;
}
@@ -161,9 +166,9 @@ public abstract class BaseTransportInferenceActionTestCase {
- assertThat(e, isA(ElasticsearchStatusException.class));
+ assertThat(e, isA(ElasticsearchException.class));
assertThat(e.getMessage(), is("Unknown service [" + serviceId + "] for model [" + inferenceId + "]. "));
- assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST));
+ assertThat(((ElasticsearchException) e).status(), is(RestStatus.BAD_REQUEST));
}));
verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> {
assertThat(attributes.get("service"), is(serviceId));
@@ -192,7 +197,7 @@ public abstract class BaseTransportInferenceActionTestCase {
- assertThat(e, isA(ElasticsearchStatusException.class));
+ assertThat(e, isA(ElasticsearchException.class));
assertThat(
e.getMessage(),
is(
@@ -203,7 +208,7 @@ public abstract class BaseTransportInferenceActionTestCase {
assertThat(attributes.get("service"), is(serviceId));
@@ -221,7 +226,6 @@ public abstract class BaseTransportInferenceActionTestCase {
assertThat(attributes.get("service"), is(serviceId));
assertThat(attributes.get("task_type"), is(taskType.toString()));
@@ -239,8 +243,8 @@ public abstract class BaseTransportInferenceActionTestCase {
- assertThat(e, isA(ElasticsearchStatusException.class));
- var ese = (ElasticsearchStatusException) e;
+ assertThat(e, isA(ElasticsearchException.class));
+ var ese = (ElasticsearchException) e;
assertThat(ese.getMessage(), is("Streaming is not allowed for service [" + serviceId + "]."));
assertThat(ese.status(), is(expectedStatus));
}));
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java
index e6b5c6d33613..7dac6a1015aa 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java
@@ -7,7 +7,6 @@
package org.elasticsearch.xpack.inference.action;
-import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.inference.InferenceServiceRegistry;
@@ -17,6 +16,7 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager;
import org.elasticsearch.xpack.inference.common.InferenceServiceRateLimitCalculator;
import org.elasticsearch.xpack.inference.registry.ModelRegistry;
@@ -81,12 +81,12 @@ public class TransportUnifiedCompletionActionTests extends BaseTransportInferenc
var listener = doExecute(requestTaskType);
verify(listener).onFailure(assertArg(e -> {
- assertThat(e, isA(ElasticsearchStatusException.class));
+ assertThat(e, isA(UnifiedChatCompletionException.class));
assertThat(
e.getMessage(),
is("Incompatible task_type for unified API, the requested type [" + requestTaskType + "] must be one of [chat_completion]")
);
- assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST));
+ assertThat(((UnifiedChatCompletionException) e).status(), is(RestStatus.BAD_REQUEST));
}));
verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> {
assertThat(attributes.get("service"), is(serviceId));
@@ -106,12 +106,12 @@ public class TransportUnifiedCompletionActionTests extends BaseTransportInferenc
var listener = doExecute(requestTaskType);
verify(listener).onFailure(assertArg(e -> {
- assertThat(e, isA(ElasticsearchStatusException.class));
+ assertThat(e, isA(UnifiedChatCompletionException.class));
assertThat(
e.getMessage(),
is("Incompatible task_type for unified API, the requested type [" + requestTaskType + "] must be one of [chat_completion]")
);
- assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST));
+ assertThat(((UnifiedChatCompletionException) e).status(), is(RestStatus.BAD_REQUEST));
}));
verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> {
assertThat(attributes.get("service"), is(serviceId));
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandlerTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandlerTests.java
new file mode 100644
index 000000000000..4853aa8d2c56
--- /dev/null
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandlerTests.java
@@ -0,0 +1,134 @@
+/*
+ * 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.inference.external.openai;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.StatusLine;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.xcontent.XContentFactory;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
+import org.elasticsearch.xpack.inference.external.http.HttpResult;
+import org.elasticsearch.xpack.inference.external.http.retry.RetryException;
+import org.elasticsearch.xpack.inference.external.request.Request;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+import static org.elasticsearch.ExceptionsHelper.unwrapCause;
+import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class OpenAiUnifiedChatCompletionResponseHandlerTests extends ESTestCase {
+ private final OpenAiUnifiedChatCompletionResponseHandler responseHandler = new OpenAiUnifiedChatCompletionResponseHandler(
+ "chat completions",
+ (a, b) -> mock()
+ );
+
+ public void testFailValidationWithAllFields() throws IOException {
+ var responseJson = """
+ {
+ "error": {
+ "type": "not_found_error",
+ "message": "a message",
+ "code": "ahh",
+ "param": "model"
+ }
+ }
+ """;
+
+ var errorJson = invalidResponseJson(responseJson);
+
+ assertThat(errorJson, is("""
+ {"error":{"code":"ahh","message":"Received a server error status code for request from inference entity id [abc] status [500]. \
+ Error message: [a message]","param":"model","type":"not_found_error"}}"""));
+ }
+
+ public void testFailValidationWithoutOptionalFields() throws IOException {
+ var responseJson = """
+ {
+ "error": {
+ "type": "not_found_error",
+ "message": "a message"
+ }
+ }
+ """;
+
+ var errorJson = invalidResponseJson(responseJson);
+
+ assertThat(errorJson, is("""
+ {"error":{"message":"Received a server error status code for request from inference entity id [abc] status [500]. \
+ Error message: [a message]","type":"not_found_error"}}"""));
+ }
+
+ public void testFailValidationWithInvalidJson() throws IOException {
+ var responseJson = """
+ what? this isn't a json
+ """;
+
+ var errorJson = invalidResponseJson(responseJson);
+
+ assertThat(errorJson, is("""
+ {"error":{"code":"bad_request","message":"Received a server error status code for request from inference entity id [abc] status\
+ [500]","type":"ErrorResponse"}}"""));
+ }
+
+ private String invalidResponseJson(String responseJson) throws IOException {
+ var exception = invalidResponse(responseJson);
+ assertThat(exception, isA(RetryException.class));
+ assertThat(unwrapCause(exception), isA(UnifiedChatCompletionException.class));
+ return toJson((UnifiedChatCompletionException) unwrapCause(exception));
+ }
+
+ private Exception invalidResponse(String responseJson) {
+ return expectThrows(
+ RetryException.class,
+ () -> responseHandler.validateResponse(
+ mock(),
+ mock(),
+ mockRequest(),
+ new HttpResult(mock500Response(), responseJson.getBytes(StandardCharsets.UTF_8))
+ )
+ );
+ }
+
+ private static Request mockRequest() {
+ var request = mock(Request.class);
+ when(request.getInferenceEntityId()).thenReturn("abc");
+ when(request.isStreaming()).thenReturn(true);
+ return request;
+ }
+
+ private static HttpResponse mock500Response() {
+ int statusCode = 500;
+ var statusLine = mock(StatusLine.class);
+ when(statusLine.getStatusCode()).thenReturn(statusCode);
+
+ var response = mock(HttpResponse.class);
+ when(response.getStatusLine()).thenReturn(statusLine);
+
+ return response;
+ }
+
+ private String toJson(UnifiedChatCompletionException e) throws IOException {
+ try (var builder = XContentFactory.jsonBuilder()) {
+ e.toXContentChunked(EMPTY_PARAMS).forEachRemaining(xContent -> {
+ try {
+ xContent.toXContent(builder, EMPTY_PARAMS);
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ });
+ return XContentHelper.convertToJson(BytesReference.bytes(builder), false, builder.contentType());
+ }
+ }
+}
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/InferenceEventsAssertion.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/InferenceEventsAssertion.java
index 7cfd231be39f..637ae726572a 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/InferenceEventsAssertion.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/InferenceEventsAssertion.java
@@ -11,6 +11,7 @@ import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.ChunkedToXContent;
import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.core.CheckedConsumer;
import org.elasticsearch.inference.InferenceServiceResults;
import org.elasticsearch.xcontent.XContentFactory;
import org.hamcrest.MatcherAssert;
@@ -85,6 +86,16 @@ public record InferenceEventsAssertion(Iterator events, Throwable error,
return this;
}
+ public InferenceEventsAssertion hasErrorMatching(CheckedConsumer matcher) {
+ hasError();
+ try {
+ matcher.accept(error);
+ } catch (Exception e) {
+ fail(e);
+ }
+ return this;
+ }
+
public InferenceEventsAssertion hasEvents(String... events) {
Arrays.stream(events).forEach(this::hasEvent);
return this;
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceServiceTests.java
index fdf8520b939f..5d66486731f5 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceServiceTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceServiceTests.java
@@ -27,14 +27,17 @@ import org.elasticsearch.inference.InputType;
import org.elasticsearch.inference.MinimalServiceSettings;
import org.elasticsearch.inference.Model;
import org.elasticsearch.inference.TaskType;
+import org.elasticsearch.inference.UnifiedCompletionRequest;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.http.MockResponse;
import org.elasticsearch.test.http.MockWebServer;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent;
+import org.elasticsearch.xcontent.XContentFactory;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.inference.action.InferenceAction;
import org.elasticsearch.xpack.core.inference.results.ChunkedInferenceEmbeddingSparse;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.core.ml.search.WeightedToken;
import org.elasticsearch.xpack.inference.external.http.HttpClientManager;
import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender;
@@ -44,11 +47,15 @@ import org.elasticsearch.xpack.inference.external.response.elastic.ElasticInfere
import org.elasticsearch.xpack.inference.logging.ThrottlerManager;
import org.elasticsearch.xpack.inference.registry.ModelRegistry;
import org.elasticsearch.xpack.inference.results.SparseEmbeddingResultsTests;
+import org.elasticsearch.xpack.inference.services.InferenceEventsAssertion;
import org.elasticsearch.xpack.inference.services.ServiceFields;
import org.elasticsearch.xpack.inference.services.elastic.authorization.ElasticInferenceServiceAuthorization;
import org.elasticsearch.xpack.inference.services.elastic.authorization.ElasticInferenceServiceAuthorizationHandler;
import org.elasticsearch.xpack.inference.services.elastic.authorization.ElasticInferenceServiceAuthorizationTests;
+import org.elasticsearch.xpack.inference.services.elastic.completion.ElasticInferenceServiceCompletionModel;
+import org.elasticsearch.xpack.inference.services.elastic.completion.ElasticInferenceServiceCompletionServiceSettings;
import org.elasticsearch.xpack.inference.services.elasticsearch.ElserModels;
+import org.elasticsearch.xpack.inference.services.settings.RateLimitSettings;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.junit.After;
@@ -61,8 +68,10 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
+import static org.elasticsearch.ExceptionsHelper.unwrapCause;
import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
+import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS;
import static org.elasticsearch.xpack.inference.Utils.getInvalidModel;
import static org.elasticsearch.xpack.inference.Utils.getModelListenerForException;
import static org.elasticsearch.xpack.inference.Utils.getPersistedConfigMap;
@@ -76,6 +85,7 @@ import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.isA;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@@ -951,6 +961,62 @@ public class ElasticInferenceServiceTests extends ESTestCase {
}
}
+ public void testUnifiedCompletionError() throws Exception {
+ var eisGatewayUrl = getUrl(webServer);
+ var senderFactory = HttpRequestSenderTests.createSenderFactory(threadPool, clientManager);
+ try (var service = createService(senderFactory, eisGatewayUrl)) {
+ var responseJson = """
+ {
+ "error": "The model `rainbow-sprinkles` does not exist or you do not have access to it."
+ }""";
+ webServer.enqueue(new MockResponse().setResponseCode(404).setBody(responseJson));
+ var model = new ElasticInferenceServiceCompletionModel(
+ "id",
+ TaskType.COMPLETION,
+ "elastic",
+ new ElasticInferenceServiceCompletionServiceSettings("model_id", new RateLimitSettings(100)),
+ EmptyTaskSettings.INSTANCE,
+ EmptySecretSettings.INSTANCE,
+ new ElasticInferenceServiceComponents(eisGatewayUrl)
+ );
+ PlainActionFuture listener = new PlainActionFuture<>();
+ service.unifiedCompletionInfer(
+ model,
+ UnifiedCompletionRequest.of(
+ List.of(new UnifiedCompletionRequest.Message(new UnifiedCompletionRequest.ContentString("hello"), "user", null, null))
+ ),
+ InferenceAction.Request.DEFAULT_TIMEOUT,
+ listener
+ );
+
+ var result = listener.actionGet(TIMEOUT);
+
+ InferenceEventsAssertion.assertThat(result).hasFinishedStream().hasNoEvents().hasErrorMatching(e -> {
+ e = unwrapCause(e);
+ assertThat(e, isA(UnifiedChatCompletionException.class));
+ try (var builder = XContentFactory.jsonBuilder()) {
+ ((UnifiedChatCompletionException) e).toXContentChunked(EMPTY_PARAMS).forEachRemaining(xContent -> {
+ try {
+ xContent.toXContent(builder, EMPTY_PARAMS);
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ });
+ var json = XContentHelper.convertToJson(BytesReference.bytes(builder), false, builder.contentType());
+
+ assertThat(json, is("""
+ {\
+ "error":{\
+ "code":"not_found",\
+ "message":"Received an unsuccessful status code for request from inference entity id [id] status \
+ [404]. Error message: [The model `rainbow-sprinkles` does not exist or you do not have access to it.]",\
+ "type":"error"\
+ }}"""));
+ }
+ });
+ }
+ }
+
private ElasticInferenceService createServiceWithMockSender() {
return createServiceWithMockSender(ElasticInferenceServiceAuthorizationTests.createEnabledAuth());
}
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java
index ee93677538b3..34539042c1f0 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java
@@ -33,9 +33,11 @@ import org.elasticsearch.test.http.MockResponse;
import org.elasticsearch.test.http.MockWebServer;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xcontent.ToXContent;
+import org.elasticsearch.xcontent.XContentFactory;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xpack.core.inference.action.InferenceAction;
import org.elasticsearch.xpack.core.inference.results.ChunkedInferenceEmbeddingFloat;
+import org.elasticsearch.xpack.core.inference.results.UnifiedChatCompletionException;
import org.elasticsearch.xpack.inference.chunking.ChunkingSettingsTests;
import org.elasticsearch.xpack.inference.external.http.HttpClientManager;
import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender;
@@ -62,8 +64,10 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
+import static org.elasticsearch.ExceptionsHelper.unwrapCause;
import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
+import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS;
import static org.elasticsearch.xpack.inference.Utils.getInvalidModel;
import static org.elasticsearch.xpack.inference.Utils.getPersistedConfigMap;
import static org.elasticsearch.xpack.inference.Utils.getRequestConfigMap;
@@ -85,6 +89,7 @@ import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.isA;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
@@ -1062,6 +1067,59 @@ public class OpenAiServiceTests extends ESTestCase {
}
}
+ public void testUnifiedCompletionError() throws Exception {
+ String responseJson = """
+ {
+ "error": {
+ "message": "The model `gpt-4awero` does not exist or you do not have access to it.",
+ "type": "invalid_request_error",
+ "param": null,
+ "code": "model_not_found"
+ }
+ }""";
+ webServer.enqueue(new MockResponse().setResponseCode(404).setBody(responseJson));
+
+ var senderFactory = HttpRequestSenderTests.createSenderFactory(threadPool, clientManager);
+ try (var service = new OpenAiService(senderFactory, createWithEmptySettings(threadPool))) {
+ var model = OpenAiChatCompletionModelTests.createChatCompletionModel(getUrl(webServer), "org", "secret", "model", "user");
+ PlainActionFuture listener = new PlainActionFuture<>();
+ service.unifiedCompletionInfer(
+ model,
+ UnifiedCompletionRequest.of(
+ List.of(new UnifiedCompletionRequest.Message(new UnifiedCompletionRequest.ContentString("hello"), "user", null, null))
+ ),
+ InferenceAction.Request.DEFAULT_TIMEOUT,
+ listener
+ );
+
+ var result = listener.actionGet(TIMEOUT);
+
+ InferenceEventsAssertion.assertThat(result).hasFinishedStream().hasNoEvents().hasErrorMatching(e -> {
+ e = unwrapCause(e);
+ assertThat(e, isA(UnifiedChatCompletionException.class));
+ try (var builder = XContentFactory.jsonBuilder()) {
+ ((UnifiedChatCompletionException) e).toXContentChunked(EMPTY_PARAMS).forEachRemaining(xContent -> {
+ try {
+ xContent.toXContent(builder, EMPTY_PARAMS);
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ });
+ var json = XContentHelper.convertToJson(BytesReference.bytes(builder), false, builder.contentType());
+
+ assertThat(json, is("""
+ {\
+ "error":{\
+ "code":"model_not_found",\
+ "message":"Received an unsuccessful status code for request from inference entity id [id] status \
+ [404]. Error message: [The model `gpt-4awero` does not exist or you do not have access to it.]",\
+ "type":"invalid_request_error"\
+ }}"""));
+ }
+ });
+ }
+ }
+
public void testInfer_StreamRequest() throws Exception {
String responseJson = """
data: {\
From b440e06c85436cdef391f8796b8a7a643d2e8d45 Mon Sep 17 00:00:00 2001
From: elasticsearchmachine
<58790826+elasticsearchmachine@users.noreply.github.com>
Date: Thu, 6 Feb 2025 01:41:19 +1100
Subject: [PATCH 13/29] Mute
org.elasticsearch.xpack.migrate.action.ReindexDatastreamIndexTransportActionIT
org.elasticsearch.xpack.migrate.action.ReindexDatastreamIndexTransportActionIT
#121737
---
muted-tests.yml | 2 ++
1 file changed, 2 insertions(+)
diff --git a/muted-tests.yml b/muted-tests.yml
index 2386c8c9a02d..b59fe87185d4 100644
--- a/muted-tests.yml
+++ b/muted-tests.yml
@@ -396,6 +396,8 @@ tests:
issue: https://github.com/elastic/elasticsearch/issues/121465
- class: org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT
issue: https://github.com/elastic/elasticsearch/issues/121411
+- class: org.elasticsearch.xpack.migrate.action.ReindexDatastreamIndexTransportActionIT
+ issue: https://github.com/elastic/elasticsearch/issues/121737
# Examples:
#
From 885a5510e1f2cbb0254619f5d59e0a5259f5c510 Mon Sep 17 00:00:00 2001
From: Artem Prigoda
Date: Wed, 5 Feb 2025 15:41:47 +0100
Subject: [PATCH 14/29] Don't return or accept `node_version` in the Desired
Nodes API (#119049)
Re-submission of #114580
> node_version was deprecated in #104209 (8.13) and shouldn't be set or returned in 9.0
Resolve ES-9443
---
.../upgrades/DesiredNodesUpgradeIT.java | 6 +-
rest-api-spec/build.gradle | 8 ++
.../test/cluster.desired_nodes/10_basic.yml | 95 -------------------
.../org/elasticsearch/TransportVersions.java | 2 +
.../cluster/metadata/DesiredNode.java | 87 ++---------------
.../metadata/DesiredNodeWithStatus.java | 5 +-
.../cluster/RestUpdateDesiredNodesAction.java | 10 --
7 files changed, 21 insertions(+), 192 deletions(-)
diff --git a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DesiredNodesUpgradeIT.java b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DesiredNodesUpgradeIT.java
index 30367bf55d8c..c0f12f95269e 100644
--- a/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DesiredNodesUpgradeIT.java
+++ b/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/DesiredNodesUpgradeIT.java
@@ -82,8 +82,7 @@ public class DesiredNodesUpgradeIT extends AbstractRollingUpgradeTestCase {
Settings.builder().put(NODE_NAME_SETTING.getKey(), nodeName).build(),
randomDoubleProcessorCount(),
ByteSizeValue.ofGb(randomIntBetween(10, 24)),
- ByteSizeValue.ofGb(randomIntBetween(128, 256)),
- null
+ ByteSizeValue.ofGb(randomIntBetween(128, 256))
)
)
.toList();
@@ -94,8 +93,7 @@ public class DesiredNodesUpgradeIT extends AbstractRollingUpgradeTestCase {
Settings.builder().put(NODE_NAME_SETTING.getKey(), nodeName).build(),
new DesiredNode.ProcessorsRange(minProcessors, minProcessors + randomIntBetween(10, 20)),
ByteSizeValue.ofGb(randomIntBetween(10, 24)),
- ByteSizeValue.ofGb(randomIntBetween(128, 256)),
- null
+ ByteSizeValue.ofGb(randomIntBetween(128, 256))
);
}).toList();
}
diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle
index 67f0b5a0714a..3bdaf029c364 100644
--- a/rest-api-spec/build.gradle
+++ b/rest-api-spec/build.gradle
@@ -74,4 +74,12 @@ tasks.named("yamlRestCompatTestTransform").configure ({ task ->
task.skipTest("index/91_metrics_no_subobjects/Metrics object indexing with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0")
task.skipTest("index/91_metrics_no_subobjects/Root without subobjects with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0")
task.skipTest("indices.create/20_synthetic_source/synthetic_source with copy_to inside nested object", "temporary until backported")
+ task.skipTest(
+ "cluster.desired_nodes/10_basic/Test delete desired nodes with node_version generates a warning",
+ "node_version warning is removed in 9.0"
+ )
+ task.skipTest(
+ "cluster.desired_nodes/10_basic/Test update desired nodes with node_version generates a warning",
+ "node_version warning is removed in 9.0"
+ )
})
diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/10_basic.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/10_basic.yml
index 1d1aa524ffb2..a45146a4e147 100644
--- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/10_basic.yml
+++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/cluster.desired_nodes/10_basic.yml
@@ -59,61 +59,6 @@ teardown:
- contains: { nodes: { settings: { node: { name: "instance-000187" } }, processors: 8.5, memory: "64gb", storage: "128gb" } }
- contains: { nodes: { settings: { node: { name: "instance-000188" } }, processors: 16.0, memory: "128gb", storage: "1tb" } }
---
-"Test update desired nodes with node_version generates a warning":
- - skip:
- reason: "contains is a newly added assertion"
- features: ["contains", "allowed_warnings"]
- - do:
- cluster.state: {}
-
- # Get master node id
- - set: { master_node: master }
-
- - do:
- nodes.info: {}
- - set: { nodes.$master.version: es_version }
-
- - do:
- _internal.update_desired_nodes:
- history_id: "test"
- version: 1
- body:
- nodes:
- - { settings: { "node.name": "instance-000187" }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
- allowed_warnings:
- - "[version removal] Specifying node_version in desired nodes requests is deprecated."
- - match: { replaced_existing_history_id: false }
-
- - do:
- _internal.get_desired_nodes: {}
- - match:
- $body:
- history_id: "test"
- version: 1
- nodes:
- - { settings: { node: { name: "instance-000187" } }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
-
- - do:
- _internal.update_desired_nodes:
- history_id: "test"
- version: 2
- body:
- nodes:
- - { settings: { "node.name": "instance-000187" }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version }
- - { settings: { "node.name": "instance-000188" }, processors: 16.0, memory: "128gb", storage: "1tb", node_version: $es_version }
- allowed_warnings:
- - "[version removal] Specifying node_version in desired nodes requests is deprecated."
- - match: { replaced_existing_history_id: false }
-
- - do:
- _internal.get_desired_nodes: {}
-
- - match: { history_id: "test" }
- - match: { version: 2 }
- - length: { nodes: 2 }
- - contains: { nodes: { settings: { node: { name: "instance-000187" } }, processors: 8.5, memory: "64gb", storage: "128gb", node_version: $es_version } }
- - contains: { nodes: { settings: { node: { name: "instance-000188" } }, processors: 16.0, memory: "128gb", storage: "1tb", node_version: $es_version } }
----
"Test update move to a new history id":
- skip:
reason: "contains is a newly added assertion"
@@ -199,46 +144,6 @@ teardown:
_internal.get_desired_nodes: {}
- match: { status: 404 }
---
-"Test delete desired nodes with node_version generates a warning":
- - skip:
- features: allowed_warnings
- - do:
- cluster.state: {}
-
- - set: { master_node: master }
-
- - do:
- nodes.info: {}
- - set: { nodes.$master.version: es_version }
-
- - do:
- _internal.update_desired_nodes:
- history_id: "test"
- version: 1
- body:
- nodes:
- - { settings: { "node.external_id": "instance-000187" }, processors: 8.0, memory: "64gb", storage: "128gb", node_version: $es_version }
- allowed_warnings:
- - "[version removal] Specifying node_version in desired nodes requests is deprecated."
- - match: { replaced_existing_history_id: false }
-
- - do:
- _internal.get_desired_nodes: {}
- - match:
- $body:
- history_id: "test"
- version: 1
- nodes:
- - { settings: { node: { external_id: "instance-000187" } }, processors: 8.0, memory: "64gb", storage: "128gb", node_version: $es_version }
-
- - do:
- _internal.delete_desired_nodes: {}
-
- - do:
- catch: missing
- _internal.get_desired_nodes: {}
- - match: { status: 404 }
----
"Test update desired nodes is idempotent":
- skip:
reason: "contains is a newly added assertion"
diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java
index 0fde3c9c2b1e..5065d84f8497 100644
--- a/server/src/main/java/org/elasticsearch/TransportVersions.java
+++ b/server/src/main/java/org/elasticsearch/TransportVersions.java
@@ -178,6 +178,8 @@ public class TransportVersions {
public static final TransportVersion REMOVE_SNAPSHOT_FAILURES = def(9_002_0_00);
public static final TransportVersion TRANSPORT_STATS_HANDLING_TIME_REQUIRED = def(9_003_0_00);
+ public static final TransportVersion REMOVE_DESIRED_NODE_VERSION = def(9_004_0_00);
+
/*
* STOP! READ THIS FIRST! No, really,
* ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _
diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNode.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNode.java
index 8366083b1907..b0d1758d895e 100644
--- a/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNode.java
+++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNode.java
@@ -11,10 +11,8 @@ package org.elasticsearch.cluster.metadata;
import org.elasticsearch.TransportVersion;
import org.elasticsearch.TransportVersions;
-import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
-import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
@@ -22,7 +20,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.Processors;
import org.elasticsearch.core.Nullable;
-import org.elasticsearch.core.UpdateForV9;
import org.elasticsearch.xcontent.ConstructingObjectParser;
import org.elasticsearch.xcontent.ObjectParser;
import org.elasticsearch.xcontent.ParseField;
@@ -36,7 +33,6 @@ import java.util.Locale;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
-import java.util.regex.Pattern;
import static java.lang.String.format;
import static org.elasticsearch.node.Node.NODE_EXTERNAL_ID_SETTING;
@@ -52,8 +48,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
private static final ParseField PROCESSORS_RANGE_FIELD = new ParseField("processors_range");
private static final ParseField MEMORY_FIELD = new ParseField("memory");
private static final ParseField STORAGE_FIELD = new ParseField("storage");
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION) // Remove deprecated field
- private static final ParseField VERSION_FIELD = new ParseField("node_version");
public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(
"desired_node",
@@ -63,8 +57,7 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
(Processors) args[1],
(ProcessorsRange) args[2],
(ByteSizeValue) args[3],
- (ByteSizeValue) args[4],
- (String) args[5]
+ (ByteSizeValue) args[4]
)
);
@@ -98,12 +91,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
STORAGE_FIELD,
ObjectParser.ValueType.STRING
);
- parser.declareField(
- ConstructingObjectParser.optionalConstructorArg(),
- (p, c) -> p.text(),
- VERSION_FIELD,
- ObjectParser.ValueType.STRING
- );
}
private final Settings settings;
@@ -112,21 +99,9 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
private final ByteSizeValue memory;
private final ByteSizeValue storage;
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION) // Remove deprecated version field
- private final String version;
private final String externalId;
private final Set roles;
- @Deprecated
- public DesiredNode(Settings settings, ProcessorsRange processorsRange, ByteSizeValue memory, ByteSizeValue storage, String version) {
- this(settings, null, processorsRange, memory, storage, version);
- }
-
- @Deprecated
- public DesiredNode(Settings settings, double processors, ByteSizeValue memory, ByteSizeValue storage, String version) {
- this(settings, Processors.of(processors), null, memory, storage, version);
- }
-
public DesiredNode(Settings settings, ProcessorsRange processorsRange, ByteSizeValue memory, ByteSizeValue storage) {
this(settings, null, processorsRange, memory, storage);
}
@@ -136,17 +111,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
}
DesiredNode(Settings settings, Processors processors, ProcessorsRange processorsRange, ByteSizeValue memory, ByteSizeValue storage) {
- this(settings, processors, processorsRange, memory, storage, null);
- }
-
- DesiredNode(
- Settings settings,
- Processors processors,
- ProcessorsRange processorsRange,
- ByteSizeValue memory,
- ByteSizeValue storage,
- @Deprecated String version
- ) {
assert settings != null;
assert memory != null;
assert storage != null;
@@ -180,7 +144,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
this.processorsRange = processorsRange;
this.memory = memory;
this.storage = storage;
- this.version = version;
this.externalId = NODE_EXTERNAL_ID_SETTING.get(settings);
this.roles = Collections.unmodifiableSortedSet(new TreeSet<>(DiscoveryNode.getRolesFromSettings(settings)));
}
@@ -198,25 +161,10 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
}
final var memory = ByteSizeValue.readFrom(in);
final var storage = ByteSizeValue.readFrom(in);
- final String version;
- if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_13_0)) {
- version = in.readOptionalString();
- } else {
- version = Version.readVersion(in).toString();
+ if (in.getTransportVersion().before(TransportVersions.REMOVE_DESIRED_NODE_VERSION)) {
+ in.readOptionalString();
}
- return new DesiredNode(settings, processors, processorsRange, memory, storage, version);
- }
-
- private static final Pattern SEMANTIC_VERSION_PATTERN = Pattern.compile("^(\\d+\\.\\d+\\.\\d+)\\D?.*");
-
- private static Version parseLegacyVersion(String version) {
- if (version != null) {
- var semanticVersionMatcher = SEMANTIC_VERSION_PATTERN.matcher(version);
- if (semanticVersionMatcher.matches()) {
- return Version.fromString(semanticVersionMatcher.group(1));
- }
- }
- return null;
+ return new DesiredNode(settings, processors, processorsRange, memory, storage);
}
@Override
@@ -232,16 +180,8 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
}
memory.writeTo(out);
storage.writeTo(out);
- if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_13_0)) {
- out.writeOptionalString(version);
- } else {
- Version parsedVersion = parseLegacyVersion(version);
- if (version == null) {
- // Some node is from before we made the version field not required. If so, fill in with the current node version.
- Version.writeVersion(Version.CURRENT, out);
- } else {
- Version.writeVersion(parsedVersion, out);
- }
+ if (out.getTransportVersion().before(TransportVersions.REMOVE_DESIRED_NODE_VERSION)) {
+ out.writeOptionalString(null);
}
}
@@ -269,14 +209,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
}
builder.field(MEMORY_FIELD.getPreferredName(), memory);
builder.field(STORAGE_FIELD.getPreferredName(), storage);
- addDeprecatedVersionField(builder);
- }
-
- @UpdateForV9(owner = UpdateForV9.Owner.DISTRIBUTED_COORDINATION) // Remove deprecated field from response
- private void addDeprecatedVersionField(XContentBuilder builder) throws IOException {
- if (version != null) {
- builder.field(VERSION_FIELD.getPreferredName(), version);
- }
}
public boolean hasMasterRole() {
@@ -356,7 +288,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
return Objects.equals(settings, that.settings)
&& Objects.equals(memory, that.memory)
&& Objects.equals(storage, that.storage)
- && Objects.equals(version, that.version)
&& Objects.equals(externalId, that.externalId)
&& Objects.equals(roles, that.roles);
}
@@ -369,7 +300,7 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
@Override
public int hashCode() {
- return Objects.hash(settings, processors, processorsRange, memory, storage, version, externalId, roles);
+ return Objects.hash(settings, processors, processorsRange, memory, storage, externalId, roles);
}
@Override
@@ -398,10 +329,6 @@ public final class DesiredNode implements Writeable, ToXContentObject, Comparabl
+ '}';
}
- public boolean hasVersion() {
- return Strings.isNullOrBlank(version) == false;
- }
-
public record ProcessorsRange(Processors min, @Nullable Processors max) implements Writeable, ToXContentObject {
private static final ParseField MIN_FIELD = new ParseField("min");
diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNodeWithStatus.java b/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNodeWithStatus.java
index 7b89406be9aa..606309adf205 100644
--- a/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNodeWithStatus.java
+++ b/server/src/main/java/org/elasticsearch/cluster/metadata/DesiredNodeWithStatus.java
@@ -44,13 +44,12 @@ public record DesiredNodeWithStatus(DesiredNode desiredNode, Status status)
(Processors) args[1],
(DesiredNode.ProcessorsRange) args[2],
(ByteSizeValue) args[3],
- (ByteSizeValue) args[4],
- (String) args[5]
+ (ByteSizeValue) args[4]
),
// An unknown status is expected during upgrades to versions >= STATUS_TRACKING_SUPPORT_VERSION
// the desired node status would be populated when a node in the newer version is elected as
// master, the desired nodes status update happens in NodeJoinExecutor.
- args[6] == null ? Status.PENDING : (Status) args[6]
+ args[5] == null ? Status.PENDING : (Status) args[5]
)
);
diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestUpdateDesiredNodesAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestUpdateDesiredNodesAction.java
index da7a7d3379ee..9ab7f275252a 100644
--- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestUpdateDesiredNodesAction.java
+++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestUpdateDesiredNodesAction.java
@@ -12,8 +12,6 @@ package org.elasticsearch.rest.action.admin.cluster;
import org.elasticsearch.action.admin.cluster.desirednodes.UpdateDesiredNodesAction;
import org.elasticsearch.action.admin.cluster.desirednodes.UpdateDesiredNodesRequest;
import org.elasticsearch.client.internal.node.NodeClient;
-import org.elasticsearch.cluster.metadata.DesiredNode;
-import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
@@ -27,10 +25,6 @@ import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
public class RestUpdateDesiredNodesAction extends BaseRestHandler {
- private final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(RestUpdateDesiredNodesAction.class);
- private static final String VERSION_DEPRECATION_MESSAGE =
- "[version removal] Specifying node_version in desired nodes requests is deprecated.";
-
@Override
public String getName() {
return "update_desired_nodes";
@@ -59,10 +53,6 @@ public class RestUpdateDesiredNodesAction extends BaseRestHandler {
);
}
- if (updateDesiredNodesRequest.getNodes().stream().anyMatch(DesiredNode::hasVersion)) {
- deprecationLogger.compatibleCritical("desired_nodes_version", VERSION_DEPRECATION_MESSAGE);
- }
-
return restChannel -> client.execute(
UpdateDesiredNodesAction.INSTANCE,
updateDesiredNodesRequest,
From c8053d4aca263776eb929de212ff07bf4a1d7321 Mon Sep 17 00:00:00 2001
From: Pat Whelan
Date: Wed, 5 Feb 2025 09:49:40 -0500
Subject: [PATCH 15/29] [ML] Skip Usage stats update when ML is disabled
(#121559)
Do not call ML's GetDeploymentStatsAction API when ML is disabled in the cluster, instead return the inference configurations as-is.
Fix #121532
---
docs/changelog/121559.yaml | 6 ++
.../ElasticsearchInternalService.java | 13 +++-
.../ElasticsearchInternalServiceTests.java | 69 +++++++++++++++++++
3 files changed, 87 insertions(+), 1 deletion(-)
create mode 100644 docs/changelog/121559.yaml
diff --git a/docs/changelog/121559.yaml b/docs/changelog/121559.yaml
new file mode 100644
index 000000000000..e3870609a454
--- /dev/null
+++ b/docs/changelog/121559.yaml
@@ -0,0 +1,6 @@
+pr: 121559
+summary: Skip Usage stats update when ML is disabled
+area: Machine Learning
+type: bug
+issues:
+ - 121532
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java
index 9a150a5e1377..c7f19adb269a 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java
@@ -15,6 +15,7 @@ import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.logging.DeprecationCategory;
import org.elasticsearch.common.logging.DeprecationLogger;
+import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.LazyInitializable;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.TimeValue;
@@ -33,6 +34,7 @@ import org.elasticsearch.inference.TaskType;
import org.elasticsearch.inference.UnifiedCompletionRequest;
import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType;
import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.xpack.core.XPackSettings;
import org.elasticsearch.xpack.core.inference.results.InferenceTextEmbeddingFloatResults;
import org.elasticsearch.xpack.core.inference.results.RankedDocsResults;
import org.elasticsearch.xpack.core.inference.results.SparseEmbeddingResults;
@@ -109,8 +111,11 @@ public class ElasticsearchInternalService extends BaseElasticsearchInternalServi
private static final Logger logger = LogManager.getLogger(ElasticsearchInternalService.class);
private static final DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.getLogger(ElasticsearchInternalService.class);
+ private final Settings settings;
+
public ElasticsearchInternalService(InferenceServiceExtension.InferenceServiceFactoryContext context) {
super(context);
+ this.settings = context.settings();
}
// for testing
@@ -119,6 +124,7 @@ public class ElasticsearchInternalService extends BaseElasticsearchInternalServi
Consumer> platformArch
) {
super(context, platformArch);
+ this.settings = context.settings();
}
@Override
@@ -837,12 +843,17 @@ public class ElasticsearchInternalService extends BaseElasticsearchInternalServi
@Override
public void updateModelsWithDynamicFields(List models, ActionListener> listener) {
-
if (models.isEmpty()) {
listener.onResponse(models);
return;
}
+ // if ML is disabled, do not update Deployment Stats (there won't be changes)
+ if (XPackSettings.MACHINE_LEARNING_ENABLED.get(settings) == false) {
+ listener.onResponse(models);
+ return;
+ }
+
var modelsByDeploymentIds = new HashMap();
for (var model : models) {
assert model instanceof ElasticsearchInternalModel;
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java
index 1615d46b349e..580871bb2c9a 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalServiceTests.java
@@ -13,6 +13,7 @@ import org.apache.logging.log4j.Level;
import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener;
+import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.service.ClusterService;
@@ -46,12 +47,14 @@ import org.elasticsearch.xpack.core.inference.results.ChunkedInferenceEmbeddingF
import org.elasticsearch.xpack.core.inference.results.ChunkedInferenceEmbeddingSparse;
import org.elasticsearch.xpack.core.inference.results.ChunkedInferenceError;
import org.elasticsearch.xpack.core.ml.MachineLearningField;
+import org.elasticsearch.xpack.core.ml.action.GetDeploymentStatsAction;
import org.elasticsearch.xpack.core.ml.action.GetTrainedModelsAction;
import org.elasticsearch.xpack.core.ml.action.InferModelAction;
import org.elasticsearch.xpack.core.ml.action.InferTrainedModelDeploymentAction;
import org.elasticsearch.xpack.core.ml.action.PutTrainedModelAction;
import org.elasticsearch.xpack.core.ml.inference.TrainedModelConfig;
import org.elasticsearch.xpack.core.ml.inference.TrainedModelPrefixStrings;
+import org.elasticsearch.xpack.core.ml.inference.assignment.AssignmentStats;
import org.elasticsearch.xpack.core.ml.inference.results.ErrorInferenceResults;
import org.elasticsearch.xpack.core.ml.inference.results.MlTextEmbeddingResults;
import org.elasticsearch.xpack.core.ml.inference.results.MlTextEmbeddingResultsTests;
@@ -67,11 +70,13 @@ import org.elasticsearch.xpack.inference.chunking.ChunkingSettingsTests;
import org.elasticsearch.xpack.inference.chunking.EmbeddingRequestChunker;
import org.elasticsearch.xpack.inference.chunking.WordBoundaryChunkingSettings;
import org.elasticsearch.xpack.inference.services.ServiceFields;
+import org.hamcrest.Matchers;
import org.junit.After;
import org.junit.Before;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
@@ -81,12 +86,14 @@ import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.common.xcontent.XContentHelper.toXContent;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertToXContentEquivalent;
+import static org.elasticsearch.xpack.core.ml.action.GetTrainedModelsStatsAction.Response.RESULTS_FIELD;
import static org.elasticsearch.xpack.inference.chunking.ChunkingSettingsTests.createRandomChunkingSettingsMap;
import static org.elasticsearch.xpack.inference.services.elasticsearch.ElasticsearchInternalService.MULTILINGUAL_E5_SMALL_MODEL_ID;
import static org.elasticsearch.xpack.inference.services.elasticsearch.ElasticsearchInternalService.MULTILINGUAL_E5_SMALL_MODEL_ID_LINUX_X86;
@@ -101,6 +108,7 @@ import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.same;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class ElasticsearchInternalServiceTests extends ESTestCase {
@@ -1632,6 +1640,67 @@ public class ElasticsearchInternalServiceTests extends ESTestCase {
}
}
+ public void testUpdateWithoutMlEnabled() throws IOException, InterruptedException {
+ var cs = mock(ClusterService.class);
+ var cSettings = new ClusterSettings(Settings.EMPTY, Set.of(MachineLearningField.MAX_LAZY_ML_NODES));
+ when(cs.getClusterSettings()).thenReturn(cSettings);
+ var context = new InferenceServiceExtension.InferenceServiceFactoryContext(
+ mock(),
+ threadPool,
+ cs,
+ Settings.builder().put("xpack.ml.enabled", false).build()
+ );
+ try (var service = new ElasticsearchInternalService(context)) {
+ var models = List.of(mock(Model.class));
+ var latch = new CountDownLatch(1);
+ service.updateModelsWithDynamicFields(models, ActionTestUtils.assertNoFailureListener(r -> {
+ latch.countDown();
+ assertThat(r, Matchers.sameInstance(models));
+ }));
+ assertTrue(latch.await(30, TimeUnit.SECONDS));
+ }
+ }
+
+ public void testUpdateWithMlEnabled() throws IOException, InterruptedException {
+ var deploymentId = "deploymentId";
+ var model = mock(ElasticsearchInternalModel.class);
+ when(model.mlNodeDeploymentId()).thenReturn(deploymentId);
+
+ AssignmentStats stats = mock();
+ when(stats.getDeploymentId()).thenReturn(deploymentId);
+ when(stats.getNumberOfAllocations()).thenReturn(3);
+
+ var client = mock(Client.class);
+ doAnswer(ans -> {
+ QueryPage queryPage = new QueryPage<>(List.of(stats), 1, RESULTS_FIELD);
+
+ GetDeploymentStatsAction.Response response = mock();
+ when(response.getStats()).thenReturn(queryPage);
+
+ ActionListener listener = ans.getArgument(2);
+ listener.onResponse(response);
+ return null;
+ }).when(client).execute(eq(GetDeploymentStatsAction.INSTANCE), any(), any());
+ when(client.threadPool()).thenReturn(threadPool);
+
+ var cs = mock(ClusterService.class);
+ var cSettings = new ClusterSettings(Settings.EMPTY, Set.of(MachineLearningField.MAX_LAZY_ML_NODES));
+ when(cs.getClusterSettings()).thenReturn(cSettings);
+ var context = new InferenceServiceExtension.InferenceServiceFactoryContext(
+ client,
+ threadPool,
+ cs,
+ Settings.builder().put("xpack.ml.enabled", true).build()
+ );
+ try (var service = new ElasticsearchInternalService(context)) {
+ List models = List.of(model);
+ var latch = new CountDownLatch(1);
+ service.updateModelsWithDynamicFields(models, ActionTestUtils.assertNoFailureListener(r -> latch.countDown()));
+ assertTrue(latch.await(30, TimeUnit.SECONDS));
+ verify(model).updateNumAllocations(3);
+ }
+ }
+
private ElasticsearchInternalService createService(Client client) {
var cs = mock(ClusterService.class);
var cSettings = new ClusterSettings(Settings.EMPTY, Set.of(MachineLearningField.MAX_LAZY_ML_NODES));
From 5ab175ebc19944787ac7508504f6fca7b73514d1 Mon Sep 17 00:00:00 2001
From: Ryan Ernst
Date: Wed, 5 Feb 2025 07:37:02 -0800
Subject: [PATCH 16/29] Move spi and system actions into separate classes
(#121555)
This commit moves methods used to test spi and system actions out of
RestEntitlementsCheckAction into separate classes, to ease future
merges.
---
.../entitlement/qa/test/EntitlementTest.java | 1 +
.../qa/test/RestEntitlementsCheckAction.java | 128 +-----------------
.../entitlement/qa/test/SpiActions.java | 76 +++++++++++
.../entitlement/qa/test/SystemActions.java | 67 +++++++++
4 files changed, 151 insertions(+), 121 deletions(-)
create mode 100644 libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SpiActions.java
create mode 100644 libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SystemActions.java
diff --git a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/EntitlementTest.java b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/EntitlementTest.java
index 953d02bccf1e..a4b9767c4c64 100644
--- a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/EntitlementTest.java
+++ b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/EntitlementTest.java
@@ -20,6 +20,7 @@ public @interface EntitlementTest {
enum ExpectedAccess {
PLUGINS,
ES_MODULES_ONLY,
+ SERVER_ONLY,
ALWAYS_DENIED
}
diff --git a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/RestEntitlementsCheckAction.java b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/RestEntitlementsCheckAction.java
index dfca49d12267..f2f6bd721e88 100644
--- a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/RestEntitlementsCheckAction.java
+++ b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/RestEntitlementsCheckAction.java
@@ -13,18 +13,6 @@ import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.core.CheckedRunnable;
import org.elasticsearch.core.SuppressForbidden;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyBreakIteratorProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyCalendarDataProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyCalendarNameProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyCollatorProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyCurrencyNameProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyDateFormatProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyDateFormatSymbolsProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyDecimalFormatSymbolsProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyLocaleNameProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyLocaleServiceProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyNumberFormatProvider;
-import org.elasticsearch.entitlement.qa.test.DummyImplementations.DummyTimeZoneNameProvider;
import org.elasticsearch.logging.LogManager;
import org.elasticsearch.logging.Logger;
import org.elasticsearch.rest.BaseRestHandler;
@@ -59,6 +47,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
+import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -75,7 +64,6 @@ import static org.elasticsearch.rest.RestRequest.Method.GET;
@SuppressWarnings("unused")
public class RestEntitlementsCheckAction extends BaseRestHandler {
private static final Logger logger = LogManager.getLogger(RestEntitlementsCheckAction.class);
- public static final Thread NO_OP_SHUTDOWN_HOOK = new Thread(() -> {}, "Shutdown hook for testing");
record CheckAction(CheckedRunnable action, boolean isAlwaysDeniedToPlugins, Integer fromJavaVersion) {
/**
@@ -94,11 +82,8 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
}
}
- private static final Map checkActions = Stream.concat(
+ private static final Map checkActions = Stream.of(
Stream.>of(
- entry("runtime_exit", deniedToPlugins(RestEntitlementsCheckAction::runtimeExit)),
- entry("runtime_halt", deniedToPlugins(RestEntitlementsCheckAction::runtimeHalt)),
- entry("system_exit", deniedToPlugins(RestEntitlementsCheckAction::systemExit)),
entry("create_classloader", forPlugins(RestEntitlementsCheckAction::createClassLoader)),
entry("processBuilder_start", deniedToPlugins(RestEntitlementsCheckAction::processBuilder_start)),
entry("processBuilder_startPipeline", deniedToPlugins(RestEntitlementsCheckAction::processBuilder_startPipeline)),
@@ -106,27 +91,10 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
entry("set_default_ssl_socket_factory", alwaysDenied(RestEntitlementsCheckAction::setDefaultSSLSocketFactory)),
entry("set_default_hostname_verifier", alwaysDenied(RestEntitlementsCheckAction::setDefaultHostnameVerifier)),
entry("set_default_ssl_context", alwaysDenied(RestEntitlementsCheckAction::setDefaultSSLContext)),
- entry("system_setIn", alwaysDenied(RestEntitlementsCheckAction::system$$setIn)),
- entry("system_setOut", alwaysDenied(RestEntitlementsCheckAction::system$$setOut)),
- entry("system_setErr", alwaysDenied(RestEntitlementsCheckAction::system$$setErr)),
- entry("runtime_addShutdownHook", alwaysDenied(RestEntitlementsCheckAction::runtime$addShutdownHook)),
- entry("runtime_removeShutdownHook", alwaysDenied(RestEntitlementsCheckAction::runtime$$removeShutdownHook)),
entry(
"thread_setDefaultUncaughtExceptionHandler",
alwaysDenied(RestEntitlementsCheckAction::thread$$setDefaultUncaughtExceptionHandler)
),
- entry("localeServiceProvider", alwaysDenied(RestEntitlementsCheckAction::localeServiceProvider$)),
- entry("breakIteratorProvider", alwaysDenied(RestEntitlementsCheckAction::breakIteratorProvider$)),
- entry("collatorProvider", alwaysDenied(RestEntitlementsCheckAction::collatorProvider$)),
- entry("dateFormatProvider", alwaysDenied(RestEntitlementsCheckAction::dateFormatProvider$)),
- entry("dateFormatSymbolsProvider", alwaysDenied(RestEntitlementsCheckAction::dateFormatSymbolsProvider$)),
- entry("decimalFormatSymbolsProvider", alwaysDenied(RestEntitlementsCheckAction::decimalFormatSymbolsProvider$)),
- entry("numberFormatProvider", alwaysDenied(RestEntitlementsCheckAction::numberFormatProvider$)),
- entry("calendarDataProvider", alwaysDenied(RestEntitlementsCheckAction::calendarDataProvider$)),
- entry("calendarNameProvider", alwaysDenied(RestEntitlementsCheckAction::calendarNameProvider$)),
- entry("currencyNameProvider", alwaysDenied(RestEntitlementsCheckAction::currencyNameProvider$)),
- entry("localeNameProvider", alwaysDenied(RestEntitlementsCheckAction::localeNameProvider$)),
- entry("timeZoneNameProvider", alwaysDenied(RestEntitlementsCheckAction::timeZoneNameProvider$)),
entry("logManager", alwaysDenied(RestEntitlementsCheckAction::logManager$)),
entry("locale_setDefault", alwaysDenied(WritePropertiesCheckActions::setDefaultLocale)),
@@ -230,8 +198,11 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
entry("symbol_lookup_name", new CheckAction(VersionSpecificNativeChecks::symbolLookupWithName, false, 22)),
entry("symbol_lookup_path", new CheckAction(VersionSpecificNativeChecks::symbolLookupWithPath, false, 22))
),
- getTestEntries(FileCheckActions.class)
+ getTestEntries(FileCheckActions.class),
+ getTestEntries(SpiActions.class),
+ getTestEntries(SystemActions.class)
)
+ .flatMap(Function.identity())
.filter(entry -> entry.getValue().fromJavaVersion() == null || Runtime.version().feature() >= entry.getValue().fromJavaVersion())
.collect(Collectors.toUnmodifiableMap(Entry::getKey, Entry::getValue));
@@ -267,7 +238,7 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
}
}
};
- boolean deniedToPlugins = testAnnotation.expectedAccess() == PLUGINS;
+ boolean deniedToPlugins = testAnnotation.expectedAccess() != PLUGINS;
Integer fromJavaVersion = testAnnotation.fromJavaVersion() == -1 ? null : testAnnotation.fromJavaVersion();
entries.add(entry(method.getName(), new CheckAction(runnable, deniedToPlugins, fromJavaVersion)));
}
@@ -323,21 +294,6 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
HttpsURLConnection.setDefaultSSLSocketFactory(new DummyImplementations.DummySSLSocketFactory());
}
- @SuppressForbidden(reason = "Specifically testing Runtime.exit")
- private static void runtimeExit() {
- Runtime.getRuntime().exit(123);
- }
-
- @SuppressForbidden(reason = "Specifically testing Runtime.halt")
- private static void runtimeHalt() {
- Runtime.getRuntime().halt(123);
- }
-
- @SuppressForbidden(reason = "Specifically testing System.exit")
- private static void systemExit() {
- System.exit(123);
- }
-
private static void createClassLoader() throws IOException {
try (var classLoader = new URLClassLoader("test", new URL[0], RestEntitlementsCheckAction.class.getClassLoader())) {
logger.info("Created URLClassLoader [{}]", classLoader.getName());
@@ -356,80 +312,10 @@ public class RestEntitlementsCheckAction extends BaseRestHandler {
new DummyImplementations.DummyHttpsURLConnection().setSSLSocketFactory(new DummyImplementations.DummySSLSocketFactory());
}
- private static void system$$setIn() {
- System.setIn(System.in);
- }
-
- @SuppressForbidden(reason = "This should be a no-op so we don't interfere with system streams")
- private static void system$$setOut() {
- System.setOut(System.out);
- }
-
- @SuppressForbidden(reason = "This should be a no-op so we don't interfere with system streams")
- private static void system$$setErr() {
- System.setErr(System.err);
- }
-
- private static void runtime$addShutdownHook() {
- Runtime.getRuntime().addShutdownHook(NO_OP_SHUTDOWN_HOOK);
- }
-
- private static void runtime$$removeShutdownHook() {
- Runtime.getRuntime().removeShutdownHook(NO_OP_SHUTDOWN_HOOK);
- }
-
private static void thread$$setDefaultUncaughtExceptionHandler() {
Thread.setDefaultUncaughtExceptionHandler(Thread.getDefaultUncaughtExceptionHandler());
}
- private static void localeServiceProvider$() {
- new DummyLocaleServiceProvider();
- }
-
- private static void breakIteratorProvider$() {
- new DummyBreakIteratorProvider();
- }
-
- private static void collatorProvider$() {
- new DummyCollatorProvider();
- }
-
- private static void dateFormatProvider$() {
- new DummyDateFormatProvider();
- }
-
- private static void dateFormatSymbolsProvider$() {
- new DummyDateFormatSymbolsProvider();
- }
-
- private static void decimalFormatSymbolsProvider$() {
- new DummyDecimalFormatSymbolsProvider();
- }
-
- private static void numberFormatProvider$() {
- new DummyNumberFormatProvider();
- }
-
- private static void calendarDataProvider$() {
- new DummyCalendarDataProvider();
- }
-
- private static void calendarNameProvider$() {
- new DummyCalendarNameProvider();
- }
-
- private static void currencyNameProvider$() {
- new DummyCurrencyNameProvider();
- }
-
- private static void localeNameProvider$() {
- new DummyLocaleNameProvider();
- }
-
- private static void timeZoneNameProvider$() {
- new DummyTimeZoneNameProvider();
- }
-
private static void logManager$() {
new java.util.logging.LogManager() {
};
diff --git a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SpiActions.java b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SpiActions.java
new file mode 100644
index 000000000000..d9ebd1705cb4
--- /dev/null
+++ b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SpiActions.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the "Elastic License
+ * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
+ * Public License v 1"; you may not use this file except in compliance with, at
+ * your election, the "Elastic License 2.0", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.entitlement.qa.test;
+
+import static org.elasticsearch.entitlement.qa.test.EntitlementTest.ExpectedAccess.ALWAYS_DENIED;
+
+class SpiActions {
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createBreakIteratorProvider() {
+ new DummyImplementations.DummyBreakIteratorProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createCollatorProvider() {
+ new DummyImplementations.DummyCollatorProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createDateFormatProvider() {
+ new DummyImplementations.DummyDateFormatProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createDateFormatSymbolsProvider() {
+ new DummyImplementations.DummyDateFormatSymbolsProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createDecimalFormatSymbolsProvider() {
+ new DummyImplementations.DummyDecimalFormatSymbolsProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createNumberFormatProvider() {
+ new DummyImplementations.DummyNumberFormatProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createCalendarDataProvider() {
+ new DummyImplementations.DummyCalendarDataProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createCalendarNameProvider() {
+ new DummyImplementations.DummyCalendarNameProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createCurrencyNameProvider() {
+ new DummyImplementations.DummyCurrencyNameProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createLocaleNameProvider() {
+ new DummyImplementations.DummyLocaleNameProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createTimeZoneNameProvider() {
+ new DummyImplementations.DummyTimeZoneNameProvider();
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void createLocaleServiceProvider() {
+ new DummyImplementations.DummyLocaleServiceProvider();
+ }
+
+ private SpiActions() {}
+}
diff --git a/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SystemActions.java b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SystemActions.java
new file mode 100644
index 000000000000..4df1b1dd26d6
--- /dev/null
+++ b/libs/entitlement/qa/entitlement-test-plugin/src/main/java/org/elasticsearch/entitlement/qa/test/SystemActions.java
@@ -0,0 +1,67 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side
+ * Public License v 1"; you may not use this file except in compliance with, at
+ * your election, the "Elastic License 2.0", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.entitlement.qa.test;
+
+import org.elasticsearch.core.SuppressForbidden;
+
+import static org.elasticsearch.entitlement.qa.test.EntitlementTest.ExpectedAccess.ALWAYS_DENIED;
+import static org.elasticsearch.entitlement.qa.test.EntitlementTest.ExpectedAccess.SERVER_ONLY;
+
+class SystemActions {
+
+ @SuppressForbidden(reason = "Specifically testing Runtime.exit")
+ @EntitlementTest(expectedAccess = SERVER_ONLY)
+ static void runtimeExit() {
+ Runtime.getRuntime().exit(123);
+ }
+
+ @SuppressForbidden(reason = "Specifically testing Runtime.halt")
+ @EntitlementTest(expectedAccess = SERVER_ONLY)
+ static void runtimeHalt() {
+ Runtime.getRuntime().halt(123);
+ }
+
+ @SuppressForbidden(reason = "Specifically testing System.exit")
+ @EntitlementTest(expectedAccess = SERVER_ONLY)
+ static void systemExit() {
+ System.exit(123);
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void systemSetIn() {
+ System.setIn(System.in);
+ }
+
+ @SuppressForbidden(reason = "This should be a no-op so we don't interfere with system streams")
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void systemSetOut() {
+ System.setOut(System.out);
+ }
+
+ @SuppressForbidden(reason = "This should be a no-op so we don't interfere with system streams")
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void systemSetErr() {
+ System.setErr(System.err);
+ }
+
+ private static final Thread NO_OP_SHUTDOWN_HOOK = new Thread(() -> {}, "Shutdown hook for testing");
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void runtimeAddShutdownHook() {
+ Runtime.getRuntime().addShutdownHook(NO_OP_SHUTDOWN_HOOK);
+ }
+
+ @EntitlementTest(expectedAccess = ALWAYS_DENIED)
+ static void runtimeRemoveShutdownHook() {
+ Runtime.getRuntime().removeShutdownHook(NO_OP_SHUTDOWN_HOOK);
+ }
+
+ private SystemActions() {}
+}
From 2ad64e18f4e8c2cd187be36eb5fac09eaa7ee52b Mon Sep 17 00:00:00 2001
From: Oleksandr Kolomiiets
Date: Wed, 5 Feb 2025 08:21:59 -0800
Subject: [PATCH 17/29] Fix synthetic source issue with deeply nested ignored
source fields (#121715)
* Fix synthetic source issue with deeply nested ignored source fields
* Update docs/changelog/121715.yaml
---
docs/changelog/121715.yaml | 5 ++++
.../index/mapper/ObjectMapper.java | 2 +-
.../mapper/IgnoredSourceFieldMapperTests.java | 28 +++++++++++++++++++
3 files changed, 34 insertions(+), 1 deletion(-)
create mode 100644 docs/changelog/121715.yaml
diff --git a/docs/changelog/121715.yaml b/docs/changelog/121715.yaml
new file mode 100644
index 000000000000..837bae57b4c9
--- /dev/null
+++ b/docs/changelog/121715.yaml
@@ -0,0 +1,5 @@
+pr: 121715
+summary: Fix synthetic source issue with deeply nested ignored source fields
+area: Mapping
+type: bug
+issues: []
diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java
index abca8e057f3b..4c9214015fba 100644
--- a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java
+++ b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java
@@ -1127,7 +1127,7 @@ public class ObjectMapper extends Mapper {
for (SourceLoader.SyntheticFieldLoader loader : fields) {
ignoredValuesPresent |= loader.setIgnoredValues(objectsWithIgnoredFields);
}
- return this.ignoredValues != null;
+ return ignoredValuesPresent;
}
@Override
diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
index c84ff2099871..6bcc94924d55 100644
--- a/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
+++ b/server/src/test/java/org/elasticsearch/index/mapper/IgnoredSourceFieldMapperTests.java
@@ -2420,6 +2420,34 @@ public class IgnoredSourceFieldMapperTests extends MapperServiceTestCase {
{"outer":{"inner":[{"a.b":"a.b","a.c":"a.c"}]}}""", syntheticSource);
}
+ public void testSingleDeepIgnoredField() throws IOException {
+ DocumentMapper documentMapper = createSytheticSourceMapperService(mapping(b -> {
+ b.startObject("top");
+ b.startObject("properties");
+ {
+ b.startObject("level1").startObject("properties");
+ {
+ b.startObject("level2").startObject("properties");
+ {
+ b.startObject("n")
+ .field("type", "integer")
+ .field("doc_values", "false")
+ .field("synthetic_source_keep", "all")
+ .endObject();
+ }
+ b.endObject().endObject();
+ }
+ b.endObject().endObject();
+ }
+ b.endObject().endObject();
+ })).documentMapper();
+
+ var syntheticSource = syntheticSource(documentMapper, b -> {
+ b.startObject("top").startObject("level1").startObject("level2").field("n", 25).endObject().endObject().endObject();
+ });
+ assertEquals("{\"top\":{\"level1\":{\"level2\":{\"n\":25}}}}", syntheticSource);
+ }
+
protected void validateRoundTripReader(String syntheticSource, DirectoryReader reader, DirectoryReader roundTripReader)
throws IOException {
// We exclude ignored source field since in some cases it contains an exact copy of a part of document source.
From 534e17127518e953aa057ef2a0114689e1be4c13 Mon Sep 17 00:00:00 2001
From: Jack Conradson
Date: Wed, 5 Feb 2025 08:57:12 -0800
Subject: [PATCH 18/29] Update policy parser to allow static methods for
entitlement creation (#121706)
This updates the PolicyParser to allow static methods to have an ExternalEntitlement annotation. This
removes a limitation where constructors cannot properly support type-erasure with different types of
data structures for internal entitlement generation and external entitlement generation (for example
List from the parser and List from an internal builder). We continue to enforce
that only one constructor/method may be annotated with ExternalEntitlement per Entitlement class.
---
.../runtime/policy/ExternalEntitlement.java | 2 +-
.../runtime/policy/PolicyParser.java | 34 ++++++-
.../policy/entitlements/FileEntitlement.java | 4 +-
.../runtime/policy/FileAccessTreeTests.java | 2 +-
.../runtime/policy/PolicyParserTests.java | 89 ++++++++++++++++++-
5 files changed, 122 insertions(+), 9 deletions(-)
diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/ExternalEntitlement.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/ExternalEntitlement.java
index b58e0d2fb87e..fef7b5d11aff 100644
--- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/ExternalEntitlement.java
+++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/ExternalEntitlement.java
@@ -22,7 +22,7 @@ import java.lang.annotation.Target;
* using this annotation is considered parseable as part of a policy file
* for entitlements.
*/
-@Target(ElementType.CONSTRUCTOR)
+@Target({ ElementType.CONSTRUCTOR, ElementType.METHOD })
@Retention(RetentionPolicy.RUNTIME)
public @interface ExternalEntitlement {
diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/PolicyParser.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/PolicyParser.java
index 992728b68186..2d3468165a59 100644
--- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/PolicyParser.java
+++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/PolicyParser.java
@@ -27,6 +27,8 @@ import java.io.InputStream;
import java.io.UncheckedIOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@@ -147,6 +149,7 @@ public class PolicyParser {
}
Constructor> entitlementConstructor = null;
+ Method entitlementMethod = null;
ExternalEntitlement entitlementMetadata = null;
for (var ctor : entitlementClass.getConstructors()) {
var metadata = ctor.getAnnotation(ExternalEntitlement.class);
@@ -161,8 +164,27 @@ public class PolicyParser {
entitlementConstructor = ctor;
entitlementMetadata = metadata;
}
-
}
+ for (var method : entitlementClass.getMethods()) {
+ var metadata = method.getAnnotation(ExternalEntitlement.class);
+ if (metadata != null) {
+ if (Modifier.isStatic(method.getModifiers()) == false) {
+ throw new IllegalStateException(
+ "entitlement class [" + entitlementClass.getName() + "] has non-static method annotated with ExternalEntitlement"
+ );
+ }
+ if (entitlementMetadata != null) {
+ throw new IllegalStateException(
+ "entitlement class ["
+ + entitlementClass.getName()
+ + "] has more than one constructor and/or method annotated with ExternalEntitlement"
+ );
+ }
+ entitlementMethod = method;
+ entitlementMetadata = metadata;
+ }
+ }
+
if (entitlementMetadata == null) {
throw newPolicyParserException(scopeName, "unknown entitlement type [" + entitlementType + "]");
}
@@ -171,7 +193,9 @@ public class PolicyParser {
throw newPolicyParserException("entitlement type [" + entitlementType + "] is allowed only on modules");
}
- Class>[] parameterTypes = entitlementConstructor.getParameterTypes();
+ Class>[] parameterTypes = entitlementConstructor != null
+ ? entitlementConstructor.getParameterTypes()
+ : entitlementMethod.getParameterTypes();
String[] parametersNames = entitlementMetadata.parameterNames();
if (parameterTypes.length != 0 || parametersNames.length != 0) {
@@ -204,7 +228,11 @@ public class PolicyParser {
}
try {
- return (Entitlement) entitlementConstructor.newInstance(parameterValues);
+ if (entitlementConstructor != null) {
+ return (Entitlement) entitlementConstructor.newInstance(parameterValues);
+ } else {
+ return (Entitlement) entitlementMethod.invoke(null, parameterValues);
+ }
} catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
if (e.getCause() instanceof PolicyValidationException piae) {
throw newPolicyParserException(startLocation, scopeName, entitlementType, piae);
diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
index f3a0ee1758a0..5a2492b1b231 100644
--- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
+++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
@@ -43,7 +43,7 @@ public record FileEntitlement(String path, Mode mode) implements Entitlement {
}
@ExternalEntitlement(parameterNames = { "path", "mode" }, esModulesOnly = false)
- public FileEntitlement(String path, String mode) {
- this(path, parseMode(mode));
+ public static FileEntitlement create(String path, String mode) {
+ return new FileEntitlement(path, parseMode(mode));
}
}
diff --git a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
index c133cf0f1242..28fec6da7389 100644
--- a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
+++ b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
@@ -85,6 +85,6 @@ public class FileAccessTreeTests extends ESTestCase {
FileEntitlement entitlement(String path, String mode) {
Path p = path(path);
- return new FileEntitlement(p.toString(), mode);
+ return FileEntitlement.create(p.toString(), mode);
}
}
diff --git a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/PolicyParserTests.java b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/PolicyParserTests.java
index 53cd5ee8aae0..85bffda369f3 100644
--- a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/PolicyParserTests.java
+++ b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/PolicyParserTests.java
@@ -40,6 +40,35 @@ public class PolicyParserTests extends ESTestCase {
public ManyConstructorsEntitlement(int i) {}
}
+ public static class ManyMethodsEntitlement implements Entitlement {
+ @ExternalEntitlement
+ public static ManyMethodsEntitlement create(String s) {
+ return new ManyMethodsEntitlement();
+ }
+
+ @ExternalEntitlement
+ public static ManyMethodsEntitlement create(int i) {
+ return new ManyMethodsEntitlement();
+ }
+ }
+
+ public static class ConstructorAndMethodEntitlement implements Entitlement {
+ @ExternalEntitlement
+ public static ConstructorAndMethodEntitlement create(String s) {
+ return new ConstructorAndMethodEntitlement(s);
+ }
+
+ @ExternalEntitlement
+ public ConstructorAndMethodEntitlement(String s) {}
+ }
+
+ public static class NonStaticMethodEntitlement implements Entitlement {
+ @ExternalEntitlement
+ public NonStaticMethodEntitlement create() {
+ return new NonStaticMethodEntitlement();
+ }
+ }
+
public void testGetEntitlementTypeName() {
assertEquals("create_class_loader", PolicyParser.getEntitlementTypeName(CreateClassLoaderEntitlement.class));
@@ -55,7 +84,7 @@ public class PolicyParserTests extends ESTestCase {
.parsePolicy();
Policy expected = new Policy(
"test-policy.yaml",
- List.of(new Scope("entitlement-module-name", List.of(new FileEntitlement("test/path/to/file", "read_write"))))
+ List.of(new Scope("entitlement-module-name", List.of(FileEntitlement.create("test/path/to/file", "read_write"))))
);
assertEquals(expected, parsedPolicy);
}
@@ -65,7 +94,7 @@ public class PolicyParserTests extends ESTestCase {
.parsePolicy();
Policy expected = new Policy(
"test-policy.yaml",
- List.of(new Scope("entitlement-module-name", List.of(new FileEntitlement("test/path/to/file", "read_write"))))
+ List.of(new Scope("entitlement-module-name", List.of(FileEntitlement.create("test/path/to/file", "read_write"))))
);
assertEquals(expected, parsedPolicy);
}
@@ -174,4 +203,60 @@ public class PolicyParserTests extends ESTestCase {
)
);
}
+
+ public void testMultipleMethodsAnnotated() throws IOException {
+ var parser = new PolicyParser(new ByteArrayInputStream("""
+ entitlement-module-name:
+ - many_methods
+ """.getBytes(StandardCharsets.UTF_8)), "test-policy.yaml", true, Map.of("many_methods", ManyMethodsEntitlement.class));
+
+ var e = expectThrows(IllegalStateException.class, parser::parsePolicy);
+ assertThat(
+ e.getMessage(),
+ equalTo(
+ "entitlement class "
+ + "[org.elasticsearch.entitlement.runtime.policy.PolicyParserTests$ManyMethodsEntitlement]"
+ + " has more than one constructor and/or method annotated with ExternalEntitlement"
+ )
+ );
+ }
+
+ public void testConstructorAndMethodAnnotated() throws IOException {
+ var parser = new PolicyParser(
+ new ByteArrayInputStream("""
+ entitlement-module-name:
+ - constructor_and_method
+ """.getBytes(StandardCharsets.UTF_8)),
+ "test-policy.yaml",
+ true,
+ Map.of("constructor_and_method", ConstructorAndMethodEntitlement.class)
+ );
+
+ var e = expectThrows(IllegalStateException.class, parser::parsePolicy);
+ assertThat(
+ e.getMessage(),
+ equalTo(
+ "entitlement class "
+ + "[org.elasticsearch.entitlement.runtime.policy.PolicyParserTests$ConstructorAndMethodEntitlement]"
+ + " has more than one constructor and/or method annotated with ExternalEntitlement"
+ )
+ );
+ }
+
+ public void testNonStaticMethodAnnotated() throws IOException {
+ var parser = new PolicyParser(new ByteArrayInputStream("""
+ entitlement-module-name:
+ - non_static
+ """.getBytes(StandardCharsets.UTF_8)), "test-policy.yaml", true, Map.of("non_static", NonStaticMethodEntitlement.class));
+
+ var e = expectThrows(IllegalStateException.class, parser::parsePolicy);
+ assertThat(
+ e.getMessage(),
+ equalTo(
+ "entitlement class "
+ + "[org.elasticsearch.entitlement.runtime.policy.PolicyParserTests$NonStaticMethodEntitlement]"
+ + " has non-static method annotated with ExternalEntitlement"
+ )
+ );
+ }
}
From 5302589639ca231222383ca49eeaf21691c1183d Mon Sep 17 00:00:00 2001
From: David Turner
Date: Wed, 5 Feb 2025 17:46:36 +0000
Subject: [PATCH 19/29] Validate transport handshake from known version
(#121747)
With parallel releases on multiple branches it's possible that an older
branch sees a transport version update that is not known to a
numerically newer but chronologically older version. In that case the
two nodes cannot intercommunicate, so with this commit we reject such
connection attempts at the version negotiation stage.
---
.../org/elasticsearch/TransportVersion.java | 8 ++
.../transport/TransportHandshaker.java | 72 +++++++++++----
.../transport/InboundHandlerTests.java | 2 +
.../transport/TransportHandshakerTests.java | 90 +++++++++++++++++++
4 files changed, 153 insertions(+), 19 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/TransportVersion.java b/server/src/main/java/org/elasticsearch/TransportVersion.java
index 032b10f0a30d..47cbe605b98e 100644
--- a/server/src/main/java/org/elasticsearch/TransportVersion.java
+++ b/server/src/main/java/org/elasticsearch/TransportVersion.java
@@ -118,6 +118,14 @@ public record TransportVersion(int id) implements VersionId {
return VersionsHolder.ALL_VERSIONS;
}
+ /**
+ * @return whether this is a known {@link TransportVersion}, i.e. one declared in {@link TransportVersions}. Other versions may exist
+ * in the wild (they're sent over the wire by numeric ID) but we don't know how to communicate using such versions.
+ */
+ public boolean isKnown() {
+ return VersionsHolder.ALL_VERSIONS_MAP.containsKey(id);
+ }
+
public static TransportVersion fromString(String str) {
return TransportVersion.fromId(Integer.parseInt(str));
}
diff --git a/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java b/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
index 9b829fcaf837..eb4e0394bb5a 100644
--- a/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
+++ b/server/src/main/java/org/elasticsearch/transport/TransportHandshaker.java
@@ -11,7 +11,6 @@ package org.elasticsearch.transport;
import org.elasticsearch.Build;
import org.elasticsearch.TransportVersion;
-import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.bytes.BytesReference;
@@ -19,8 +18,11 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.metrics.CounterMetric;
+import org.elasticsearch.core.Strings;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.core.UpdateForV9;
+import org.elasticsearch.logging.LogManager;
+import org.elasticsearch.logging.Logger;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.EOFException;
@@ -126,6 +128,8 @@ final class TransportHandshaker {
* [3] Parent task ID should be empty; see org.elasticsearch.tasks.TaskId.writeTo for its structure.
*/
+ private static final Logger logger = LogManager.getLogger(TransportHandshaker.class);
+
static final TransportVersion V8_HANDSHAKE_VERSION = TransportVersion.fromId(7_17_00_99);
static final TransportVersion V9_HANDSHAKE_VERSION = TransportVersion.fromId(8_800_00_0);
static final Set ALLOWED_HANDSHAKE_VERSIONS = Set.of(V8_HANDSHAKE_VERSION, V9_HANDSHAKE_VERSION);
@@ -159,7 +163,7 @@ final class TransportHandshaker {
ActionListener listener
) {
numHandshakes.inc();
- final HandshakeResponseHandler handler = new HandshakeResponseHandler(requestId, listener);
+ final HandshakeResponseHandler handler = new HandshakeResponseHandler(requestId, channel, listener);
pendingHandshakes.put(requestId, handler);
channel.addCloseListener(
ActionListener.running(() -> handler.handleLocalException(new TransportException("handshake failed because connection reset")))
@@ -185,9 +189,9 @@ final class TransportHandshaker {
}
void handleHandshake(TransportChannel channel, long requestId, StreamInput stream) throws IOException {
+ final HandshakeRequest handshakeRequest;
try {
- // Must read the handshake request to exhaust the stream
- new HandshakeRequest(stream);
+ handshakeRequest = new HandshakeRequest(stream);
} catch (Exception e) {
assert ignoreDeserializationErrors : e;
throw e;
@@ -206,9 +210,44 @@ final class TransportHandshaker {
assert ignoreDeserializationErrors : exception;
throw exception;
}
+ ensureCompatibleVersion(version, handshakeRequest.transportVersion, handshakeRequest.releaseVersion, channel);
channel.sendResponse(new HandshakeResponse(this.version, Build.current().version()));
}
+ static void ensureCompatibleVersion(
+ TransportVersion localTransportVersion,
+ TransportVersion remoteTransportVersion,
+ String releaseVersion,
+ Object channel
+ ) {
+ if (TransportVersion.isCompatible(remoteTransportVersion)) {
+ if (remoteTransportVersion.onOrAfter(localTransportVersion)) {
+ // Remote is newer than us, so we will be using our transport protocol and it's up to the other end to decide whether it
+ // knows how to do that.
+ return;
+ }
+ if (remoteTransportVersion.isKnown()) {
+ // Remote is older than us, so we will be using its transport protocol, which we can only do if and only if its protocol
+ // version is known to us.
+ return;
+ }
+ }
+
+ final var message = Strings.format(
+ """
+ Rejecting unreadable transport handshake from remote node with version [%s/%s] received on [%s] since this node has \
+ version [%s/%s] which has an incompatible wire format.""",
+ releaseVersion,
+ remoteTransportVersion,
+ channel,
+ Build.current().version(),
+ localTransportVersion
+ );
+ logger.warn(message);
+ throw new IllegalStateException(message);
+
+ }
+
TransportResponseHandler removeHandlerForHandshake(long requestId) {
return pendingHandshakes.remove(requestId);
}
@@ -224,11 +263,13 @@ final class TransportHandshaker {
private class HandshakeResponseHandler implements TransportResponseHandler {
private final long requestId;
+ private final TcpChannel channel;
private final ActionListener listener;
private final AtomicBoolean isDone = new AtomicBoolean(false);
- private HandshakeResponseHandler(long requestId, ActionListener listener) {
+ private HandshakeResponseHandler(long requestId, TcpChannel channel, ActionListener listener) {
this.requestId = requestId;
+ this.channel = channel;
this.listener = listener;
}
@@ -245,20 +286,13 @@ final class TransportHandshaker {
@Override
public void handleResponse(HandshakeResponse response) {
if (isDone.compareAndSet(false, true)) {
- TransportVersion responseVersion = response.transportVersion;
- if (TransportVersion.isCompatible(responseVersion) == false) {
- listener.onFailure(
- new IllegalStateException(
- "Received message from unsupported version: ["
- + responseVersion
- + "] minimal compatible version is: ["
- + TransportVersions.MINIMUM_COMPATIBLE
- + "]"
- )
- );
- } else {
- listener.onResponse(TransportVersion.min(TransportHandshaker.this.version, response.getTransportVersion()));
- }
+ ActionListener.completeWith(listener, () -> {
+ ensureCompatibleVersion(version, response.getTransportVersion(), response.getReleaseVersion(), channel);
+ final var resultVersion = TransportVersion.min(TransportHandshaker.this.version, response.getTransportVersion());
+ assert TransportVersion.current().before(version) // simulating a newer-version transport service for test purposes
+ || resultVersion.isKnown() : "negotiated unknown version " + resultVersion;
+ return resultVersion;
+ });
}
}
diff --git a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java
index cb266c58d70d..7f5c4fbb1c4d 100644
--- a/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/InboundHandlerTests.java
@@ -290,7 +290,9 @@ public class InboundHandlerTests extends ESTestCase {
);
BytesStreamOutput byteData = new BytesStreamOutput();
TaskId.EMPTY_TASK_ID.writeTo(byteData);
+ // simulate bytes of a transport handshake: vInt transport version then release version string
TransportVersion.writeVersion(remoteVersion, byteData);
+ byteData.writeString(randomIdentifier());
final InboundMessage requestMessage = new InboundMessage(
requestHeader,
ReleasableBytesReference.wrap(byteData.bytes()),
diff --git a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
index 4fd82480c4e5..f36a56d9237c 100644
--- a/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
+++ b/server/src/test/java/org/elasticsearch/transport/TransportHandshakerTests.java
@@ -8,8 +8,12 @@
*/
package org.elasticsearch.transport;
+import org.apache.logging.log4j.Level;
+import org.elasticsearch.Build;
import org.elasticsearch.TransportVersion;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.Version;
+import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
@@ -19,13 +23,17 @@ import org.elasticsearch.core.TimeValue;
import org.elasticsearch.core.UpdateForV10;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.test.MockLog;
import org.elasticsearch.test.TransportVersionUtils;
+import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.TestThreadPool;
import java.io.IOException;
import java.util.Collections;
+import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
+import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
@@ -93,6 +101,40 @@ public class TransportHandshakerTests extends ESTestCase {
assertEquals(TransportVersion.current(), versionFuture.actionGet());
}
+ @TestLogging(reason = "testing WARN logging", value = "org.elasticsearch.transport.TransportHandshaker:WARN")
+ public void testIncompatibleHandshakeRequest() throws IOException {
+ TransportHandshaker.HandshakeRequest handshakeRequest = new TransportHandshaker.HandshakeRequest(
+ getRandomIncompatibleTransportVersion(),
+ randomIdentifier()
+ );
+ BytesStreamOutput bytesStreamOutput = new BytesStreamOutput();
+ bytesStreamOutput.setTransportVersion(HANDSHAKE_REQUEST_VERSION);
+ handshakeRequest.writeTo(bytesStreamOutput);
+ StreamInput input = bytesStreamOutput.bytes().streamInput();
+ input.setTransportVersion(HANDSHAKE_REQUEST_VERSION);
+ final TestTransportChannel channel = new TestTransportChannel(ActionListener.running(() -> fail("should not complete")));
+
+ MockLog.assertThatLogger(
+ () -> assertThat(
+ expectThrows(IllegalStateException.class, () -> handshaker.handleHandshake(channel, randomNonNegativeLong(), input))
+ .getMessage(),
+ allOf(
+ containsString("Rejecting unreadable transport handshake"),
+ containsString("[" + handshakeRequest.releaseVersion + "/" + handshakeRequest.transportVersion + "]"),
+ containsString("[" + Build.current().version() + "/" + TransportVersion.current() + "]"),
+ containsString("which has an incompatible wire format")
+ )
+ ),
+ TransportHandshaker.class,
+ new MockLog.SeenEventExpectation(
+ "warning",
+ TransportHandshaker.class.getCanonicalName(),
+ Level.WARN,
+ "Rejecting unreadable transport handshake * incompatible wire format."
+ )
+ );
+ }
+
public void testHandshakeResponseFromOlderNode() throws Exception {
final PlainActionFuture versionFuture = new PlainActionFuture<>();
final long reqId = randomNonNegativeLong();
@@ -108,6 +150,54 @@ public class TransportHandshakerTests extends ESTestCase {
assertEquals(remoteVersion, versionFuture.result());
}
+ @TestLogging(reason = "testing WARN logging", value = "org.elasticsearch.transport.TransportHandshaker:WARN")
+ public void testHandshakeResponseFromOlderNodeWithPatchedProtocol() {
+ final PlainActionFuture versionFuture = new PlainActionFuture<>();
+ final long reqId = randomNonNegativeLong();
+ handshaker.sendHandshake(reqId, node, channel, SAFE_AWAIT_TIMEOUT, versionFuture);
+ TransportResponseHandler handler = handshaker.removeHandlerForHandshake(reqId);
+
+ assertFalse(versionFuture.isDone());
+
+ final var handshakeResponse = new TransportHandshaker.HandshakeResponse(
+ getRandomIncompatibleTransportVersion(),
+ randomIdentifier()
+ );
+
+ MockLog.assertThatLogger(
+ () -> handler.handleResponse(handshakeResponse),
+ TransportHandshaker.class,
+ new MockLog.SeenEventExpectation(
+ "warning",
+ TransportHandshaker.class.getCanonicalName(),
+ Level.WARN,
+ "Rejecting unreadable transport handshake * incompatible wire format."
+ )
+ );
+
+ assertTrue(versionFuture.isDone());
+ assertThat(
+ expectThrows(ExecutionException.class, IllegalStateException.class, versionFuture::result).getMessage(),
+ allOf(
+ containsString("Rejecting unreadable transport handshake"),
+ containsString("[" + handshakeResponse.getReleaseVersion() + "/" + handshakeResponse.getTransportVersion() + "]"),
+ containsString("[" + Build.current().version() + "/" + TransportVersion.current() + "]"),
+ containsString("which has an incompatible wire format")
+ )
+ );
+ }
+
+ private static TransportVersion getRandomIncompatibleTransportVersion() {
+ return randomBoolean()
+ // either older than MINIMUM_COMPATIBLE
+ ? new TransportVersion(between(1, TransportVersions.MINIMUM_COMPATIBLE.id() - 1))
+ // or between MINIMUM_COMPATIBLE and current but not known
+ : randomValueOtherThanMany(
+ TransportVersion::isKnown,
+ () -> new TransportVersion(between(TransportVersions.MINIMUM_COMPATIBLE.id(), TransportVersion.current().id()))
+ );
+ }
+
public void testHandshakeResponseFromNewerNode() throws Exception {
final PlainActionFuture versionFuture = new PlainActionFuture<>();
final long reqId = randomNonNegativeLong();
From c1deef4467094955eb3da0b769b520aa8460d433 Mon Sep 17 00:00:00 2001
From: Patrick Doyle <810052+prdoyle@users.noreply.github.com>
Date: Wed, 5 Feb 2025 14:04:43 -0500
Subject: [PATCH 20/29] Entitlement policies correct handling of prefixes that
are not directories (#121598)
* Fix FileAccessTree for prefixes that aren't parents
* Support backslashes
* Whoops, nio
* Move normalization responsibility to FileEntitlement
* Normalize to native separators
* Avoid forbidden API
---
.../runtime/policy/FileAccessTree.java | 25 +++++++++++++------
.../policy/entitlements/FileEntitlement.java | 16 ++++--------
.../runtime/policy/FileAccessTreeTests.java | 21 ++++++++++++++++
3 files changed, 43 insertions(+), 19 deletions(-)
diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTree.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTree.java
index d574609d1321..3333eefa4f71 100644
--- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTree.java
+++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTree.java
@@ -17,8 +17,11 @@ import java.util.Arrays;
import java.util.List;
import java.util.Objects;
+import static org.elasticsearch.core.PathUtils.getDefaultFileSystem;
+
public final class FileAccessTree {
public static final FileAccessTree EMPTY = new FileAccessTree(List.of());
+ private static final String FILE_SEPARATOR = getDefaultFileSystem().getSeparator();
private final String[] readPaths;
private final String[] writePaths;
@@ -27,11 +30,11 @@ public final class FileAccessTree {
List readPaths = new ArrayList<>();
List writePaths = new ArrayList<>();
for (FileEntitlement fileEntitlement : fileEntitlements) {
- var mode = fileEntitlement.mode();
- if (mode == FileEntitlement.Mode.READ_WRITE) {
- writePaths.add(fileEntitlement.path());
+ String path = normalizePath(Path.of(fileEntitlement.path()));
+ if (fileEntitlement.mode() == FileEntitlement.Mode.READ_WRITE) {
+ writePaths.add(path);
}
- readPaths.add(fileEntitlement.path());
+ readPaths.add(path);
}
readPaths.sort(String::compareTo);
@@ -46,14 +49,20 @@ public final class FileAccessTree {
}
boolean canRead(Path path) {
- return checkPath(normalize(path), readPaths);
+ return checkPath(normalizePath(path), readPaths);
}
boolean canWrite(Path path) {
- return checkPath(normalize(path), writePaths);
+ return checkPath(normalizePath(path), writePaths);
}
- private static String normalize(Path path) {
+ /**
+ * @return the "canonical" form of the given {@code path}, to be used for entitlement checks.
+ */
+ static String normalizePath(Path path) {
+ // Note that toAbsolutePath produces paths separated by the default file separator,
+ // so on Windows, if the given path uses forward slashes, this consistently
+ // converts it to backslashes.
return path.toAbsolutePath().normalize().toString();
}
@@ -64,7 +73,7 @@ public final class FileAccessTree {
int ndx = Arrays.binarySearch(paths, path);
if (ndx < -1) {
String maybeParent = paths[-ndx - 2];
- return path.startsWith(maybeParent);
+ return path.startsWith(maybeParent) && path.startsWith(FILE_SEPARATOR, maybeParent.length());
}
return ndx >= 0;
}
diff --git a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
index 5a2492b1b231..01d882e4d9e2 100644
--- a/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
+++ b/libs/entitlement/src/main/java/org/elasticsearch/entitlement/runtime/policy/entitlements/FileEntitlement.java
@@ -12,10 +12,12 @@ package org.elasticsearch.entitlement.runtime.policy.entitlements;
import org.elasticsearch.entitlement.runtime.policy.ExternalEntitlement;
import org.elasticsearch.entitlement.runtime.policy.PolicyValidationException;
-import java.nio.file.Paths;
-
/**
- * Describes a file entitlement with a path and mode.
+ * Describes entitlement to access files at a particular location.
+ *
+ * @param path the location of the files. For directories, implicitly includes access to
+ * all contained files and (recursively) subdirectories.
+ * @param mode the type of operation
*/
public record FileEntitlement(String path, Mode mode) implements Entitlement {
@@ -24,14 +26,6 @@ public record FileEntitlement(String path, Mode mode) implements Entitlement {
READ_WRITE
}
- public FileEntitlement {
- path = normalizePath(path);
- }
-
- private static String normalizePath(String path) {
- return Paths.get(path).toAbsolutePath().normalize().toString();
- }
-
private static Mode parseMode(String mode) {
if (mode.equals("read")) {
return Mode.READ;
diff --git a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
index 28fec6da7389..48c03cfd2f9b 100644
--- a/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
+++ b/libs/entitlement/src/test/java/org/elasticsearch/entitlement/runtime/policy/FileAccessTreeTests.java
@@ -16,6 +16,7 @@ import org.junit.BeforeClass;
import java.nio.file.Path;
import java.util.List;
+import static org.elasticsearch.core.PathUtils.getDefaultFileSystem;
import static org.hamcrest.Matchers.is;
public class FileAccessTreeTests extends ESTestCase {
@@ -41,7 +42,9 @@ public class FileAccessTreeTests extends ESTestCase {
var tree = FileAccessTree.of(List.of(entitlement("foo", "read")));
assertThat(tree.canRead(path("foo")), is(true));
assertThat(tree.canRead(path("foo/subdir")), is(true));
+ assertThat(tree.canRead(path("food")), is(false));
assertThat(tree.canWrite(path("foo")), is(false));
+ assertThat(tree.canWrite(path("food")), is(false));
assertThat(tree.canRead(path("before")), is(false));
assertThat(tree.canRead(path("later")), is(false));
@@ -51,7 +54,9 @@ public class FileAccessTreeTests extends ESTestCase {
var tree = FileAccessTree.of(List.of(entitlement("foo", "read_write")));
assertThat(tree.canWrite(path("foo")), is(true));
assertThat(tree.canWrite(path("foo/subdir")), is(true));
+ assertThat(tree.canWrite(path("food")), is(false));
assertThat(tree.canRead(path("foo")), is(true));
+ assertThat(tree.canRead(path("food")), is(false));
assertThat(tree.canWrite(path("before")), is(false));
assertThat(tree.canWrite(path("later")), is(false));
@@ -83,6 +88,22 @@ public class FileAccessTreeTests extends ESTestCase {
assertThat(tree.canRead(path("")), is(false));
}
+ public void testForwardSlashes() {
+ String sep = getDefaultFileSystem().getSeparator();
+ var tree = FileAccessTree.of(List.of(entitlement("a/b", "read"), entitlement("m" + sep + "n", "read")));
+
+ // Native separators work
+ assertThat(tree.canRead(path("a" + sep + "b")), is(true));
+ assertThat(tree.canRead(path("m" + sep + "n")), is(true));
+
+ // Forward slashes also work
+ assertThat(tree.canRead(path("a/b")), is(true));
+ assertThat(tree.canRead(path("m/n")), is(true));
+
+ // In case the native separator is a backslash, don't treat that as an escape
+ assertThat(tree.canRead(path("m\n")), is(false));
+ }
+
FileEntitlement entitlement(String path, String mode) {
Path p = path(path);
return FileEntitlement.create(p.toString(), mode);
From 229d89d343fd233ad38134209564e601eac98e02 Mon Sep 17 00:00:00 2001
From: Armin Braun
Date: Wed, 5 Feb 2025 21:08:00 +0100
Subject: [PATCH 21/29] Remove GroupShardsIterator and replace it with plain
List (#116891)
There is no point in having `GroupShardsIterator`, it's mostly an
unnecessary layer of indirection as it has no state and a single field
only. It's only value could be seen in it hiding the ability to mutate
the list it wraps, but that hardly justifies the overhead on the search
path and extra code complexity. Moreover, the list it references is not
copied/immutable in any way, so the value of hiding is limited also.
---
.../RemoveCorruptedShardDataCommandIT.java | 7 +-
.../index/store/CorruptedFileIT.java | 9 +-
.../index/suggest/stats/SuggestStatsIT.java | 6 +-
.../search/stats/SearchStatsIT.java | 5 +-
.../TransportClusterSearchShardsAction.java | 4 +-
.../TransportAnalyzeIndexDiskUsageAction.java | 10 +-
.../query/TransportValidateQueryAction.java | 3 +-
.../action/fieldcaps/RequestDispatcher.java | 5 +-
.../search/AbstractSearchAsyncAction.java | 19 ++-
.../search/CanMatchPreFilterSearchPhase.java | 34 ++--
.../SearchDfsQueryThenFetchAsyncAction.java | 3 +-
.../action/search/SearchPhase.java | 6 +-
.../action/search/SearchProgressListener.java | 6 +-
.../SearchQueryThenFetchAsyncAction.java | 4 +-
.../TransportOpenPointInTimeAction.java | 6 +-
.../action/search/TransportSearchAction.java | 15 +-
.../search/TransportSearchShardsAction.java | 19 ++-
.../broadcast/TransportBroadcastAction.java | 6 +-
.../TransportTermVectorsAction.java | 10 +-
.../cluster/routing/GroupShardsIterator.java | 60 --------
.../cluster/routing/OperationRouting.java | 10 +-
.../cluster/routing/RoutingTable.java | 15 +-
...sportAnalyzeIndexDiskUsageActionTests.java | 5 +-
.../AbstractSearchAsyncActionTests.java | 5 +-
.../CanMatchPreFilterSearchPhaseTests.java | 39 +++--
.../action/search/MockSearchPhaseContext.java | 3 +-
.../action/search/SearchAsyncActionTests.java | 19 +--
.../SearchQueryThenFetchAsyncActionTests.java | 4 +-
.../search/SearchShardIteratorTests.java | 33 ++--
.../search/TransportSearchActionTests.java | 6 +-
...rdFailedClusterStateTaskExecutorTests.java | 3 +-
.../routing/GroupShardsIteratorTests.java | 145 ------------------
.../routing/OperationRoutingTests.java | 29 +---
.../routing/PlainShardIteratorTests.java | 5 +-
.../structure/RoutingIteratorTests.java | 8 +-
.../action/TransportForgetFollowerAction.java | 4 +-
.../action/TransportTermsEnumAction.java | 4 +-
.../TransportDownsampleIndexerAction.java | 11 +-
.../action/EnrichShardMultiSearchAction.java | 7 +-
.../esql/enrich/AbstractLookupService.java | 3 +-
40 files changed, 159 insertions(+), 436 deletions(-)
delete mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
delete mode 100644 server/src/test/java/org/elasticsearch/cluster/routing/GroupShardsIteratorTests.java
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandIT.java
index 4e9e4b4d641d..a1fd526e886c 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandIT.java
@@ -32,7 +32,6 @@ import org.elasticsearch.cli.ProcessInfo;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
@@ -531,8 +530,7 @@ public class RemoveCorruptedShardDataCommandIT extends ESIntegTestCase {
nodeNameToNodeId.put(cursor.getValue().getName(), cursor.getKey());
}
- final GroupShardsIterator shardIterators = state.getRoutingTable()
- .activePrimaryShardsGrouped(new String[] { indexName }, false);
+ final List shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(new String[] { indexName }, false);
final List iterators = iterableAsArrayList(shardIterators);
final ShardRouting shardRouting = iterators.iterator().next().nextOrNull();
assertThat(shardRouting, notNullValue());
@@ -571,8 +569,7 @@ public class RemoveCorruptedShardDataCommandIT extends ESIntegTestCase {
private Path getPathToShardData(String indexName, String dirSuffix) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator shardIterators = state.getRoutingTable()
- .activePrimaryShardsGrouped(new String[] { indexName }, false);
+ List shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(new String[] { indexName }, false);
List iterators = iterableAsArrayList(shardIterators);
ShardIterator shardIterator = RandomPicks.randomFrom(random(), iterators);
ShardRouting shardRouting = shardIterator.nextOrNull();
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/store/CorruptedFileIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/store/CorruptedFileIT.java
index 1594514d2f41..141f24e283b0 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/index/store/CorruptedFileIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/index/store/CorruptedFileIT.java
@@ -34,7 +34,6 @@ import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
@@ -311,8 +310,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
}
assertThat(response.getStatus(), is(ClusterHealthStatus.RED));
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator shardIterators = state.getRoutingTable()
- .activePrimaryShardsGrouped(new String[] { "test" }, false);
+ List shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(new String[] { "test" }, false);
for (ShardIterator iterator : shardIterators) {
ShardRouting routing;
while ((routing = iterator.nextOrNull()) != null) {
@@ -667,7 +665,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
private int numShards(String... index) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator> shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(index, false);
+ List> shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(index, false);
return shardIterators.size();
}
@@ -695,8 +693,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
private ShardRouting corruptRandomPrimaryFile(final boolean includePerCommitFiles) throws IOException {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
Index test = state.metadata().index("test").getIndex();
- GroupShardsIterator shardIterators = state.getRoutingTable()
- .activePrimaryShardsGrouped(new String[] { "test" }, false);
+ List shardIterators = state.getRoutingTable().activePrimaryShardsGrouped(new String[] { "test" }, false);
List iterators = iterableAsArrayList(shardIterators);
ShardIterator shardIterator = RandomPicks.randomFrom(random(), iterators);
ShardRouting shardRouting = shardIterator.nextOrNull();
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/suggest/stats/SuggestStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/suggest/stats/SuggestStatsIT.java
index 5cb468da7099..9256065f0d0c 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/index/suggest/stats/SuggestStatsIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/index/suggest/stats/SuggestStatsIT.java
@@ -14,7 +14,6 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.index.search.stats.SearchStats;
@@ -24,6 +23,7 @@ import org.elasticsearch.search.suggest.term.TermSuggestionBuilder;
import org.elasticsearch.test.ESIntegTestCase;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@@ -146,7 +146,7 @@ public class SuggestStatsIT extends ESIntegTestCase {
private Set nodeIdsWithIndex(String... indices) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
+ List allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
Set nodes = new HashSet<>();
for (ShardIterator shardIterator : allAssignedShardsGrouped) {
for (ShardRouting routing : shardIterator) {
@@ -161,7 +161,7 @@ public class SuggestStatsIT extends ESIntegTestCase {
protected int numAssignedShards(String... indices) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator> allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
+ List> allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
return allAssignedShardsGrouped.size();
}
}
diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/stats/SearchStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/stats/SearchStatsIT.java
index 2530dd35946f..ee7f76e6be3f 100644
--- a/server/src/internalClusterTest/java/org/elasticsearch/search/stats/SearchStatsIT.java
+++ b/server/src/internalClusterTest/java/org/elasticsearch/search/stats/SearchStatsIT.java
@@ -14,7 +14,6 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetadata;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.settings.Settings;
@@ -165,7 +164,7 @@ public class SearchStatsIT extends ESIntegTestCase {
private Set nodeIdsWithIndex(String... indices) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
+ List allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
Set nodes = new HashSet<>();
for (ShardIterator shardIterator : allAssignedShardsGrouped) {
for (ShardRouting routing : shardIterator) {
@@ -248,7 +247,7 @@ public class SearchStatsIT extends ESIntegTestCase {
protected int numAssignedShards(String... indices) {
ClusterState state = clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState();
- GroupShardsIterator> allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
+ List> allAssignedShardsGrouped = state.routingTable().allAssignedShardsGrouped(indices, true);
return allAssignedShardsGrouped.size();
}
diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java
index dd21b11ad18c..b13d0b031c5d 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java
@@ -19,7 +19,6 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@@ -34,6 +33,7 @@ import org.elasticsearch.transport.TransportService;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -101,7 +101,7 @@ public class TransportClusterSearchShardsAction extends TransportMasterNodeReadA
}
Set nodeIds = new HashSet<>();
- GroupShardsIterator groupShardsIterator = clusterService.operationRouting()
+ List groupShardsIterator = clusterService.operationRouting()
.searchShards(clusterState, concreteIndices, routingMap, request.preference());
ShardRouting shard;
ClusterSearchShardsGroup[] groupResponses = new ClusterSearchShardsGroup[groupShardsIterator.size()];
diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/diskusage/TransportAnalyzeIndexDiskUsageAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/diskusage/TransportAnalyzeIndexDiskUsageAction.java
index 6c0eb7971409..4a3f398656c1 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/indices/diskusage/TransportAnalyzeIndexDiskUsageAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/indices/diskusage/TransportAnalyzeIndexDiskUsageAction.java
@@ -21,7 +21,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@@ -214,13 +213,8 @@ public class TransportAnalyzeIndexDiskUsageAction extends TransportBroadcastActi
}
@Override
- protected GroupShardsIterator shards(
- ClusterState clusterState,
- AnalyzeIndexDiskUsageRequest request,
- String[] concreteIndices
- ) {
- final GroupShardsIterator groups = clusterService.operationRouting()
- .searchShards(clusterState, concreteIndices, null, null);
+ protected List shards(ClusterState clusterState, AnalyzeIndexDiskUsageRequest request, String[] concreteIndices) {
+ final List groups = clusterService.operationRouting().searchShards(clusterState, concreteIndices, null, null);
for (ShardIterator group : groups) {
// fails fast if any non-active groups
if (group.size() == 0) {
diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java
index e01f36471267..10d755c75b1c 100644
--- a/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java
+++ b/server/src/main/java/org/elasticsearch/action/admin/indices/validate/query/TransportValidateQueryAction.java
@@ -22,7 +22,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@@ -145,7 +144,7 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<
}
@Override
- protected GroupShardsIterator shards(ClusterState clusterState, ValidateQueryRequest request, String[] concreteIndices) {
+ protected List shards(ClusterState clusterState, ValidateQueryRequest request, String[] concreteIndices) {
final String routing;
if (request.allShards()) {
routing = null;
diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java
index fce925d86853..15577632176f 100644
--- a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java
+++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java
@@ -18,7 +18,6 @@ import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@@ -93,7 +92,7 @@ final class RequestDispatcher {
this.onComplete = new RunOnce(onComplete);
this.indexSelectors = ConcurrentCollections.newConcurrentMap();
for (String index : indices) {
- final GroupShardsIterator shardIts;
+ final List shardIts;
try {
shardIts = clusterService.operationRouting().searchShards(clusterState, new String[] { index }, null, null);
} catch (Exception e) {
@@ -250,7 +249,7 @@ final class RequestDispatcher {
private final Set unmatchedShardIds = new HashSet<>();
private final Map failures = new HashMap<>();
- IndexSelector(GroupShardsIterator shardIts) {
+ IndexSelector(List shardIts) {
for (ShardIterator shardIt : shardIts) {
for (ShardRouting shard : shardIt) {
nodeToShards.computeIfAbsent(shard.currentNodeId(), node -> new ArrayList<>()).add(shard);
diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
index 9f47e1f9773a..44752d6f3360 100644
--- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java
@@ -22,7 +22,6 @@ import org.elasticsearch.action.search.TransportSearchAction.SearchTimeProvider;
import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.util.Maps;
@@ -60,9 +59,9 @@ import java.util.stream.Collectors;
import static org.elasticsearch.core.Strings.format;
/**
- * This is an abstract base class that encapsulates the logic to fan out to all shards in provided {@link GroupShardsIterator}
+ * This is an abstract base class that encapsulates the logic to fan out to all shards in provided {@link List}
* and collect the results. If a shard request returns a failure this class handles the advance to the next replica of the shard until
- * the shards replica iterator is exhausted. Each shard is referenced by position in the {@link GroupShardsIterator} which is later
+ * the shards replica iterator is exhausted. Each shard is referenced by position in the {@link List} which is later
* referred to as the {@code shardIndex}.
* The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection of
* distributed frequencies
@@ -93,8 +92,8 @@ abstract class AbstractSearchAsyncAction exten
private final SearchTimeProvider timeProvider;
private final SearchResponse.Clusters clusters;
- protected final GroupShardsIterator toSkipShardsIts;
- protected final GroupShardsIterator shardsIts;
+ protected final List toSkipShardsIts;
+ protected final List shardsIts;
private final SearchShardIterator[] shardIterators;
private final AtomicInteger outstandingShards;
private final int maxConcurrentRequestsPerNode;
@@ -116,7 +115,7 @@ abstract class AbstractSearchAsyncAction exten
Executor executor,
SearchRequest request,
ActionListener listener,
- GroupShardsIterator shardsIts,
+ List shardsIts,
SearchTimeProvider timeProvider,
ClusterState clusterState,
SearchTask task,
@@ -135,8 +134,8 @@ abstract class AbstractSearchAsyncAction exten
iterators.add(iterator);
}
}
- this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators);
- this.shardsIts = new GroupShardsIterator<>(iterators);
+ this.toSkipShardsIts = toSkipIterators;
+ this.shardsIts = iterators;
outstandingShards = new AtomicInteger(shardsIts.size());
this.shardIterators = iterators.toArray(new SearchShardIterator[0]);
// we later compute the shard index based on the natural order of the shards
@@ -171,8 +170,8 @@ abstract class AbstractSearchAsyncAction exten
SearchSourceBuilder sourceBuilder
) {
progressListener.notifyListShards(
- SearchProgressListener.buildSearchShards(this.shardsIts),
- SearchProgressListener.buildSearchShards(toSkipShardsIts),
+ SearchProgressListener.buildSearchShardsFromIter(this.shardsIts),
+ SearchProgressListener.buildSearchShardsFromIter(toSkipShardsIts),
clusters,
sourceBuilder == null || sourceBuilder.size() > 0,
timeProvider
diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java
index d45a8a6f01cd..f7b258a9f6b7 100644
--- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java
+++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java
@@ -12,7 +12,6 @@ package org.elasticsearch.action.search;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.FixedBitSet;
import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.common.util.Maps;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.CountDown;
@@ -61,8 +60,8 @@ final class CanMatchPreFilterSearchPhase {
private final Logger logger;
private final SearchRequest request;
- private final GroupShardsIterator shardsIts;
- private final ActionListener> listener;
+ private final List shardsIts;
+ private final ActionListener> listener;
private final TransportSearchAction.SearchTimeProvider timeProvider;
private final BiFunction nodeIdToConnection;
private final SearchTransportService searchTransportService;
@@ -86,12 +85,12 @@ final class CanMatchPreFilterSearchPhase {
Map concreteIndexBoosts,
Executor executor,
SearchRequest request,
- GroupShardsIterator shardsIts,
+ List shardsIts,
TransportSearchAction.SearchTimeProvider timeProvider,
SearchTask task,
boolean requireAtLeastOneMatch,
CoordinatorRewriteContextProvider coordinatorRewriteContextProvider,
- ActionListener> listener
+ ActionListener> listener
) {
this.logger = logger;
this.searchTransportService = searchTransportService;
@@ -169,10 +168,9 @@ final class CanMatchPreFilterSearchPhase {
if (matchedShardLevelRequests.isEmpty()) {
finishPhase();
} else {
- GroupShardsIterator matchingShards = new GroupShardsIterator<>(matchedShardLevelRequests);
// verify missing shards only for the shards that we hit for the query
- checkNoMissingShards(matchingShards);
- new Round(matchingShards).run();
+ checkNoMissingShards(matchedShardLevelRequests);
+ new Round(matchedShardLevelRequests).run();
}
}
@@ -202,12 +200,12 @@ final class CanMatchPreFilterSearchPhase {
minAndMaxes[shardIndex] = minAndMax;
}
- private void checkNoMissingShards(GroupShardsIterator shards) {
+ private void checkNoMissingShards(List shards) {
assert assertSearchCoordinationThread();
SearchPhase.doCheckNoMissingShards("can_match", request, shards, SearchPhase::makeMissingShardsError);
}
- private Map> groupByNode(GroupShardsIterator shards) {
+ private Map> groupByNode(List shards) {
Map> requests = new HashMap<>();
for (int i = 0; i < shards.size(); i++) {
final SearchShardIterator shardRoutings = shards.get(i);
@@ -230,11 +228,11 @@ final class CanMatchPreFilterSearchPhase {
* to retry on other available shard copies.
*/
class Round extends AbstractRunnable {
- private final GroupShardsIterator shards;
+ private final List shards;
private final CountDown countDown;
private final AtomicReferenceArray failedResponses;
- Round(GroupShardsIterator shards) {
+ Round(List shards) {
this.shards = shards;
this.countDown = new CountDown(shards.size());
this.failedResponses = new AtomicReferenceArray<>(shardsIts.size());
@@ -328,7 +326,7 @@ final class CanMatchPreFilterSearchPhase {
finishPhase();
} else {
// trigger another round, forcing execution
- executor.execute(new Round(new GroupShardsIterator<>(remainingShards)) {
+ executor.execute(new Round(remainingShards) {
@Override
public boolean isForceExecution() {
return true;
@@ -419,7 +417,7 @@ final class CanMatchPreFilterSearchPhase {
listener.onFailure(new SearchPhaseExecutionException("can_match", msg, cause, ShardSearchFailure.EMPTY_ARRAY));
}
- private synchronized GroupShardsIterator getIterator(GroupShardsIterator shardsIts) {
+ private synchronized List getIterator(List shardsIts) {
// TODO: pick the local shard when possible
if (requireAtLeastOneMatch && numPossibleMatches == 0) {
// this is a special case where we have no hit but we need to get at least one search response in order
@@ -452,14 +450,10 @@ final class CanMatchPreFilterSearchPhase {
return shardsIts;
}
FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source());
- return new GroupShardsIterator<>(sortShards(shardsIts, minAndMaxes, fieldSort.order()));
+ return sortShards(shardsIts, minAndMaxes, fieldSort.order());
}
- private static List sortShards(
- GroupShardsIterator shardsIts,
- MinAndMax>[] minAndMaxes,
- SortOrder order
- ) {
+ private static List sortShards(List shardsIts, MinAndMax>[] minAndMaxes, SortOrder order) {
int bound = shardsIts.size();
List toSort = new ArrayList<>(bound);
for (int i = 0; i < bound; i++) {
diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java
index 5c5c47b5fcc4..056806fbb0b0 100644
--- a/server/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java
@@ -20,7 +20,6 @@ import org.apache.lucene.util.SetOnce;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
@@ -56,7 +55,7 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
SearchPhaseResults queryPhaseResultConsumer,
SearchRequest request,
ActionListener listener,
- GroupShardsIterator shardsIts,
+ List shardsIts,
TransportSearchAction.SearchTimeProvider timeProvider,
ClusterState clusterState,
SearchTask task,
diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java
index 702369dc3839..1308a2fb61cf 100644
--- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java
+++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java
@@ -8,11 +8,11 @@
*/
package org.elasticsearch.action.search;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.transport.Transport;
+import java.util.List;
import java.util.Objects;
import java.util.function.Function;
@@ -45,14 +45,14 @@ abstract class SearchPhase {
+ "]. Consider using `allow_partial_search_results` setting to bypass this error.";
}
- protected void doCheckNoMissingShards(String phaseName, SearchRequest request, GroupShardsIterator shardsIts) {
+ protected void doCheckNoMissingShards(String phaseName, SearchRequest request, List shardsIts) {
doCheckNoMissingShards(phaseName, request, shardsIts, this::missingShardsErrorMessage);
}
protected static void doCheckNoMissingShards(
String phaseName,
SearchRequest request,
- GroupShardsIterator shardsIts,
+ List shardsIts,
Function makeErrorMessage
) {
assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults";
diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java b/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java
index a7f92700435a..6016a0c7a1eb 100644
--- a/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java
+++ b/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java
@@ -13,7 +13,6 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.search.TotalHits;
import org.elasticsearch.action.search.SearchResponse.Clusters;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.aggregations.InternalAggregations;
@@ -21,7 +20,6 @@ import org.elasticsearch.search.query.QuerySearchResult;
import java.util.List;
import java.util.Objects;
-import java.util.stream.StreamSupport;
/**
* A listener that allows to track progress of the {@link TransportSearchAction}.
@@ -225,7 +223,7 @@ public abstract class SearchProgressListener {
.toList();
}
- static List buildSearchShards(GroupShardsIterator its) {
- return StreamSupport.stream(its.spliterator(), false).map(e -> new SearchShard(e.getClusterAlias(), e.shardId())).toList();
+ static List buildSearchShardsFromIter(List its) {
+ return its.stream().map(e -> new SearchShard(e.getClusterAlias(), e.shardId())).toList();
}
}
diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java
index f75b84abc2f0..088a16deb76d 100644
--- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java
@@ -14,7 +14,6 @@ import org.apache.lucene.search.TopFieldDocs;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
@@ -25,6 +24,7 @@ import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.Transport;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.function.BiFunction;
@@ -52,7 +52,7 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction resultConsumer,
SearchRequest request,
ActionListener listener,
- GroupShardsIterator shardsIts,
+ List shardsIts,
TransportSearchAction.SearchTimeProvider timeProvider,
ClusterState clusterState,
SearchTask task,
diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java
index 6c95a3c8fd43..b8d0a928e05a 100644
--- a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java
@@ -23,7 +23,6 @@ import org.elasticsearch.action.support.ChannelActionListener;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterState;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
@@ -49,6 +48,7 @@ import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
+import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;
import java.util.function.BiFunction;
@@ -150,7 +150,7 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction shardIterators,
+ List shardIterators,
TransportSearchAction.SearchTimeProvider timeProvider,
BiFunction connectionLookup,
ClusterState clusterState,
@@ -212,7 +212,7 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction shardIterators,
+ List shardIterators,
TransportSearchAction.SearchTimeProvider timeProvider,
BiFunction connectionLookup,
ClusterState clusterState,
diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java
index 4b13d1642b60..6f075c6f3500 100644
--- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java
@@ -11,6 +11,7 @@ package org.elasticsearch.action.search;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionListener;
@@ -41,7 +42,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.OperationRouting;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
@@ -1286,7 +1286,7 @@ public class TransportSearchAction extends HandledTransportAction shardIterators = mergeShardsIterators(localShardIterators, remoteShardIterators);
+ final List shardIterators = mergeShardsIterators(localShardIterators, remoteShardIterators);
failIfOverShardCountLimit(clusterService, shardIterators.size());
@@ -1420,7 +1420,7 @@ public class TransportSearchAction extends HandledTransportAction mergeShardsIterators(
+ static List mergeShardsIterators(
List localShardIterators,
List remoteShardIterators
) {
@@ -1430,7 +1430,8 @@ public class TransportSearchAction extends HandledTransportAction shardIterators,
+ List shardIterators,
SearchTimeProvider timeProvider,
BiFunction connectionLookup,
ClusterState clusterState,
@@ -1462,7 +1463,7 @@ public class TransportSearchAction extends HandledTransportAction shardIterators,
+ List shardIterators,
SearchTimeProvider timeProvider,
BiFunction connectionLookup,
ClusterState clusterState,
@@ -1855,7 +1856,7 @@ public class TransportSearchAction extends HandledTransportAction shardRoutings = clusterService.operationRouting()
+ List shardRoutings = clusterService.operationRouting()
.searchShards(
clusterState,
concreteIndices,
diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java
index 614a3e9cf22a..83889b7cf752 100644
--- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java
+++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java
@@ -9,6 +9,7 @@
package org.elasticsearch.action.search;
+import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.RemoteClusterActionType;
@@ -18,7 +19,6 @@ import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.query.Rewriteable;
@@ -138,15 +138,14 @@ public class TransportSearchShardsAction extends HandledTransportAction shardIts = GroupShardsIterator.sortAndCreate(
- transportSearchAction.getLocalShardsIterator(
- clusterState,
- searchRequest,
- searchShardsRequest.clusterAlias(),
- indicesAndAliases,
- concreteIndexNames
- )
+ List shardIts = transportSearchAction.getLocalShardsIterator(
+ clusterState,
+ searchRequest,
+ searchShardsRequest.clusterAlias(),
+ indicesAndAliases,
+ concreteIndexNames
);
+ CollectionUtil.timSort(shardIts);
if (SearchService.canRewriteToMatchNone(searchRequest.source()) == false) {
delegate.onResponse(new SearchShardsResponse(toGroups(shardIts), clusterState.nodes().getAllNodes(), aliasFilters));
} else {
@@ -170,7 +169,7 @@ public class TransportSearchShardsAction extends HandledTransportAction toGroups(GroupShardsIterator shardIts) {
+ private static List toGroups(List shardIts) {
List groups = new ArrayList<>(shardIts.size());
for (SearchShardIterator shardIt : shardIts) {
boolean skip = shardIt.skip();
diff --git a/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java b/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java
index 836eedd960f1..6fabf515ac75 100644
--- a/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java
+++ b/server/src/main/java/org/elasticsearch/action/support/broadcast/TransportBroadcastAction.java
@@ -22,7 +22,6 @@ import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
@@ -35,6 +34,7 @@ import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.Transports;
import java.io.IOException;
+import java.util.List;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReferenceArray;
@@ -108,7 +108,7 @@ public abstract class TransportBroadcastAction<
* Determines the shards this operation will be executed on. The operation is executed once per shard iterator, typically
* on the first shard in it. If the operation fails, it will be retried on the next shard in the iterator.
*/
- protected abstract GroupShardsIterator shards(ClusterState clusterState, Request request, String[] concreteIndices);
+ protected abstract List shards(ClusterState clusterState, Request request, String[] concreteIndices);
protected abstract ClusterBlockException checkGlobalBlock(ClusterState state, Request request);
@@ -121,7 +121,7 @@ public abstract class TransportBroadcastAction<
final ActionListener listener;
final ClusterState clusterState;
final DiscoveryNodes nodes;
- final GroupShardsIterator shardsIts;
+ final List shardsIts;
final int expectedOps;
final AtomicInteger counterOps = new AtomicInteger();
// ShardResponse or Exception
diff --git a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java
index 02479a9f8d14..8a2e9168c3bd 100644
--- a/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java
+++ b/server/src/main/java/org/elasticsearch/action/termvectors/TransportTermVectorsAction.java
@@ -14,7 +14,6 @@ import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
-import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.Writeable;
@@ -65,13 +64,8 @@ public class TransportTermVectorsAction extends TransportSingleShardAction groupShardsIter = operationRouting.searchShards(
- state,
- new String[] { request.concreteIndex() },
- null,
- request.request().preference()
- );
- return groupShardsIter.iterator().next();
+ return operationRouting.searchShards(state, new String[] { request.concreteIndex() }, null, request.request().preference())
+ .getFirst();
}
return operationRouting.useOnlyPromotableShardsForStateless(
diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java b/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
deleted file mode 100644
index 590a1bbb1692..000000000000
--- a/server/src/main/java/org/elasticsearch/cluster/routing/GroupShardsIterator.java
+++ /dev/null
@@ -1,60 +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", the "GNU Affero General Public License v3.0 only", and the "Server Side
- * Public License v 1"; you may not use this file except in compliance with, at
- * your election, the "Elastic License 2.0", the "GNU Affero General Public
- * License v3.0 only", or the "Server Side Public License, v 1".
- */
-
-package org.elasticsearch.cluster.routing;
-
-import org.apache.lucene.util.CollectionUtil;
-import org.elasticsearch.common.util.Countable;
-
-import java.util.Iterator;
-import java.util.List;
-
-/**
- * This class implements a compilation of {@link ShardIterator}s. Each {@link ShardIterator}
- * iterated by this {@link Iterable} represents a group of shards.
- * ShardsIterators are always returned in ascending order independently of their order at construction
- * time. The incoming iterators are sorted to ensure consistent iteration behavior across Nodes / JVMs.
-*/
-public final class GroupShardsIterator & Countable> implements Iterable {
-
- private final List iterators;
-
- /**
- * Constructs a new sorted GroupShardsIterator from the given list. Items are sorted based on their natural ordering.
- * @see PlainShardIterator#compareTo(ShardIterator)
- */
- public static & Countable> GroupShardsIterator sortAndCreate(List iterators) {
- CollectionUtil.timSort(iterators);
- return new GroupShardsIterator<>(iterators);
- }
-
- /**
- * Constructs a new GroupShardsIterator from the given list.
- */
- public GroupShardsIterator(List iterators) {
- this.iterators = iterators;
- }
-
- /**
- * Return the number of groups
- * @return number of groups
- */
- public int size() {
- return iterators.size();
- }
-
- @Override
- public Iterator iterator() {
- return iterators.iterator();
- }
-
- public ShardIt get(int index) {
- return iterators.get(index);
- }
-}
diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java b/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java
index 49da00eae8a5..1c0dafb799ea 100644
--- a/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java
+++ b/server/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java
@@ -9,6 +9,7 @@
package org.elasticsearch.cluster.routing;
+import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -27,6 +28,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
@@ -95,7 +97,7 @@ public class OperationRouting {
}
}
- public GroupShardsIterator searchShards(
+ public List searchShards(
ClusterState clusterState,
String[] concreteIndices,
@Nullable Map