1
0
Эх сурвалжийг харах

ESQL: Opt in to support for new `aggregate_metric_double` and `dense_vector` using query constructs (#135215)

Works around a problem with ESQL planning around new types. Without this
if you have either `aggregate_metric_double` or `dense_vector` fields
running on a node running < 9.2.0 ESQL will detect those for queries like
`FROM *` and request those fields. The old nodes don't know about the data
type. So they'll claim they can't parse the request and fail. That's bad
because in pure 9.1.x cluster `FROM *` would have returned `null` for
those fields rather than failing. And, come on, `FROM *` shouldn't fail
during an upgrade.

And it's not *just* during an upgrade. A mixed remote cluster being sent
`FROM remote:*` over cross cluster search will have the same problem if
it is 9.1 and the coordinating node is 9.2.

The fix we'd *like* to do is get the version of all nodes that'll have
the request and only enable the type if all versions support it. That's
complex, but it easy to think about and explain and handle in the planner.

We thought about a different fix - downgrading these fields to
`unsupported` on write to an old version - but that's difficult to reason
about and quite likely to fail in a long tail of weird ways.

This is something a simpler version of the "get all the versions and
disable unsupported fields" that uses a cute hack to not have to fetch the
version, which is the complex part. Instead, it scrapes the query for the
`TS` command or for functions like `KNN` or `TO_AGGREGATE_METRIC_DOUBLE`
and enables these fields if those are there. This works because folks who
want `dense_vector` will always be using `KNN` in 9.2.0. And folks who want `aggregate_metric_double` will use `TS`. These things will fail if the
entire cluster and all remote nodes aren't on 9.2.0 because they don't
exist on those versions. So users will tell us, by writing things that
require 9.2.0, if all the nodes are on 9.2.0.

We'll implement the transport version based approach in a follow up. But,
for now, we'll use this trick.

Fix https://github.com/elastic/elasticsearch/issues/135193
Nik Everett 2 долоо хоног өмнө
parent
commit
94c1ba4822
26 өөрчлөгдсөн 1259 нэмэгдсэн , 171 устгасан
  1. 13 0
      x-pack/plugin/build.gradle
  2. 43 0
      x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/CreatedVersion.java
  3. 56 3
      x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java
  4. 35 0
      x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/AllSupportedFieldsIT.java
  5. 104 0
      x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/AllSupportedFieldsIT.java
  6. 35 0
      x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/AllSupportedFieldsIT.java
  7. 563 0
      x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java
  8. 1 1
      x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java
  9. 7 2
      x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector-bit.csv-spec
  10. 8 2
      x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector-byte.csv-spec
  11. 7 1
      x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector.csv-spec
  12. 1 5
      x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich.csv-spec
  13. 1 5
      x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec
  14. 4 0
      x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/DenseVectorFieldTypeIT.java
  15. 2 1
      x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupJoinTypesIT.java
  16. 6 4
      x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/KnnFunctionIT.java
  17. 5 1
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
  18. 46 7
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java
  19. 25 16
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java
  20. 5 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java
  21. 28 13
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java
  22. 108 33
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java
  23. 1 1
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java
  24. 86 29
      x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/40_tsdb.yml
  25. 12 18
      x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/40_unsupported_types.yml
  26. 57 29
      x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/46_downsample.yml

+ 13 - 0
x-pack/plugin/build.gradle

@@ -142,6 +142,19 @@ tasks.named("yamlRestCompatTestTransform").configure({ task ->
   task.skipTest("ml/sparse_vector_search/Search on a sparse_vector field with dots in the field names", "Vectors are no longer returned by default")
   task.skipTest("ml/sparse_vector_search/Search on a nested sparse_vector field with dots in the field names and conflicting child fields", "Vectors are no longer returned by default")
   task.skipTest("esql/190_lookup_join/lookup-no-key-only-key", "Requires the fix")
+  task.skipTest("esql/40_tsdb/aggregate_metric_double unsortable", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/avg of aggregate_metric_double", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/grouping stats on aggregate_metric_double", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/render aggregate_metric_double when missing min and max", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/render aggregate_metric_double when missing value", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/sorting with aggregate_metric_double with partial submetrics", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/stats on aggregate_metric_double missing min and max", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/to_string aggregate_metric_double", "Extra function required to enable the field type")
+  task.skipTest("esql/40_tsdb/stats on aggregate_metric_double with partial submetrics", "Extra function required to enable the field type")
+  task.skipTest("esql/46_downsample/MV_EXPAND on non-MV aggregate metric double", "Extra function required to enable the field type")
+  task.skipTest("esql/46_downsample/Query stats on downsampled index", "Extra function required to enable the field type")
+  task.skipTest("esql/46_downsample/Render stats from downsampled index", "Extra function required to enable the field type")
+  task.skipTest("esql/46_downsample/Sort from multiple indices one with aggregate metric double", "Extra function required to enable the field type")
 })
 
 tasks.named('yamlRestCompatTest').configure {

+ 43 - 0
x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/CreatedVersion.java

@@ -0,0 +1,43 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.core.type;
+
+import org.elasticsearch.TransportVersion;
+
+/**
+ * Version that supports a {@link DataType}.
+ */
+public interface CreatedVersion {
+    boolean supports(TransportVersion version);
+
+    CreatedVersion SUPPORTED_ON_ALL_NODES = new CreatedVersion() {
+        @Override
+        public boolean supports(TransportVersion version) {
+            return true;
+        }
+
+        @Override
+        public String toString() {
+            return "SupportedOnAllVersions";
+        }
+    };
+
+    static CreatedVersion supportedOn(TransportVersion createdVersion) {
+        return new CreatedVersion() {
+            @Override
+            public boolean supports(TransportVersion version) {
+                return version.supports(createdVersion);
+            }
+
+            @Override
+            public String toString() {
+                return "SupportedOn[" + createdVersion + "]";
+            }
+        };
+    }
+}

+ 56 - 3
x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java

@@ -7,8 +7,10 @@
 package org.elasticsearch.xpack.esql.core.type;
 
 import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.TransportVersion;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.util.FeatureFlag;
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.index.mapper.SourceFieldMapper;
@@ -32,6 +34,8 @@ import java.util.Set;
 import java.util.function.Function;
 
 import static java.util.stream.Collectors.toMap;
+import static org.elasticsearch.TransportVersions.INFERENCE_REQUEST_ADAPTIVE_RATE_LIMITING;
+import static org.elasticsearch.TransportVersions.ML_INFERENCE_SAGEMAKER_CHAT_COMPLETION;
 
 /**
  * This enum represents data types the ES|QL query processing layer is able to
@@ -140,7 +144,7 @@ import static java.util.stream.Collectors.toMap;
  *         unsupported types.</li>
  * </ul>
  */
-public enum DataType {
+public enum DataType implements Writeable {
     /**
      * Fields of this type are unsupported by any functions and are always
      * rendered as {@code null} in the response.
@@ -306,12 +310,26 @@ public enum DataType {
      */
     PARTIAL_AGG(builder().esType("partial_agg").estimatedSize(1024)),
 
-    AGGREGATE_METRIC_DOUBLE(builder().esType("aggregate_metric_double").estimatedSize(Double.BYTES * 3 + Integer.BYTES)),
+    AGGREGATE_METRIC_DOUBLE(
+        builder().esType("aggregate_metric_double")
+            .estimatedSize(Double.BYTES * 3 + Integer.BYTES)
+            .createdVersion(
+                // Version created just *after* we committed support for aggregate_metric_double
+                INFERENCE_REQUEST_ADAPTIVE_RATE_LIMITING
+            )
+    ),
 
     /**
      * Fields with this type are dense vectors, represented as an array of double values.
      */
-    DENSE_VECTOR(builder().esType("dense_vector").estimatedSize(4096));
+    DENSE_VECTOR(
+        builder().esType("dense_vector")
+            .estimatedSize(4096)
+            .createdVersion(
+                // Version created just *after* we committed support for dense_vector
+                ML_INFERENCE_SAGEMAKER_CHAT_COMPLETION
+            )
+    );
 
     /**
      * Types that are actively being built. These types are
@@ -375,6 +393,11 @@ public enum DataType {
      */
     private final DataType counter;
 
+    /**
+     * Version that first created this data type.
+     */
+    private final CreatedVersion createdVersion;
+
     DataType(Builder builder) {
         String typeString = builder.typeName != null ? builder.typeName : builder.esType;
         this.typeName = typeString.toLowerCase(Locale.ROOT);
@@ -387,6 +410,7 @@ public enum DataType {
         this.isCounter = builder.isCounter;
         this.widenSmallNumeric = builder.widenSmallNumeric;
         this.counter = builder.counter;
+        this.createdVersion = builder.createdVersion;
     }
 
     private static final Collection<DataType> TYPES = Arrays.stream(values())
@@ -727,7 +751,20 @@ public enum DataType {
         return counter;
     }
 
+    @Override
     public void writeTo(StreamOutput out) throws IOException {
+        if (createdVersion.supports(out.getTransportVersion()) == false) {
+            /*
+             * TODO when we implement version aware planning flip this to an IllegalStateException
+             * so we throw a 500 error. It'll be our bug then. Right now it's a sign that the user
+             * tried to do something like `KNN(dense_vector_field, [1, 2])` against an old node.
+             * Like, during the rolling upgrade that enables KNN or to a remote cluster that has
+             * not yet been upgraded.
+             */
+            throw new IllegalArgumentException(
+                "remote node at version [" + out.getTransportVersion() + "] doesn't understand data type [" + this + "]"
+            );
+        }
         ((PlanStreamOutput) out).writeCachedString(typeName);
     }
 
@@ -779,6 +816,10 @@ public enum DataType {
         };
     }
 
+    public CreatedVersion createdVersion() {
+        return createdVersion;
+    }
+
     public static DataType suggestedCast(Set<DataType> originalTypes) {
         if (originalTypes.isEmpty() || originalTypes.contains(UNSUPPORTED)) {
             return null;
@@ -846,6 +887,13 @@ public enum DataType {
          */
         private DataType counter;
 
+        /**
+         * The version when this data type was created. We default to the first
+         * version for which we maintain wire compatibility, which is pretty
+         * much {@code 8.18.0}.
+         */
+        private CreatedVersion createdVersion = CreatedVersion.SUPPORTED_ON_ALL_NODES;
+
         Builder() {}
 
         Builder esType(String esType) {
@@ -901,5 +949,10 @@ public enum DataType {
             this.counter = counter;
             return this;
         }
+
+        Builder createdVersion(TransportVersion createdVersion) {
+            this.createdVersion = CreatedVersion.supportedOn(createdVersion);
+            return this;
+        }
     }
 }

+ 35 - 0
x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/AllSupportedFieldsIT.java

@@ -0,0 +1,35 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.qa.mixed;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.test.TestClustersThreadFilter;
+import org.elasticsearch.test.cluster.ElasticsearchCluster;
+import org.elasticsearch.xpack.esql.qa.rest.AllSupportedFieldsTestCase;
+import org.junit.ClassRule;
+
+/**
+ * Fetch all field types in a mixed version cluster, simulating a rolling upgrade.
+ */
+@ThreadLeakFilters(filters = TestClustersThreadFilter.class)
+public class AllSupportedFieldsIT extends AllSupportedFieldsTestCase {
+    @ClassRule
+    public static ElasticsearchCluster cluster = Clusters.mixedVersionCluster();
+
+    public AllSupportedFieldsIT(MappedFieldType.FieldExtractPreference extractPreference, IndexMode indexMode) {
+        super(extractPreference, indexMode);
+    }
+
+    @Override
+    protected String getTestRestCluster() {
+        return cluster.getHttpAddresses();
+    }
+}

+ 104 - 0
x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/AllSupportedFieldsIT.java

@@ -0,0 +1,104 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.ccq;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+
+import org.apache.http.HttpHost;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.core.IOUtils;
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.test.TestClustersThreadFilter;
+import org.elasticsearch.test.cluster.ElasticsearchCluster;
+import org.elasticsearch.xpack.esql.qa.rest.AllSupportedFieldsTestCase;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * Fetch all field types via cross cluster search, possible on a different version.
+ */
+@ThreadLeakFilters(filters = TestClustersThreadFilter.class)
+public class AllSupportedFieldsIT extends AllSupportedFieldsTestCase {
+    static ElasticsearchCluster remoteCluster = Clusters.remoteCluster();
+    static ElasticsearchCluster localCluster = Clusters.localCluster(remoteCluster);
+
+    @ClassRule
+    public static TestRule clusterRule = RuleChain.outerRule(remoteCluster).around(localCluster);
+
+    private static RestClient remoteClient;
+    private static Map<String, NodeInfo> remoteNodeToInfo;
+
+    public AllSupportedFieldsIT(MappedFieldType.FieldExtractPreference extractPreference, IndexMode indexMode) {
+        super(extractPreference, indexMode);
+    }
+
+    @Before
+    public void createRemoteIndices() throws IOException {
+        if (supportsNodeAssignment()) {
+            for (Map.Entry<String, NodeInfo> e : remoteNodeToInfo().entrySet()) {
+                createIndexForNode(remoteClient(), e.getKey(), e.getValue().id());
+            }
+        } else {
+            createIndexForNode(remoteClient(), null, null);
+        }
+    }
+
+    private Map<String, NodeInfo> remoteNodeToInfo() throws IOException {
+        if (remoteNodeToInfo == null) {
+            remoteNodeToInfo = fetchNodeToInfo(remoteClient(), "remote_cluster");
+        }
+        return remoteNodeToInfo;
+    }
+
+    @Override
+    protected Map<String, NodeInfo> allNodeToInfo() throws IOException {
+        Map<String, NodeInfo> all = new TreeMap<>();
+        all.putAll(super.allNodeToInfo());
+        all.putAll(remoteNodeToInfo());
+        return all;
+    }
+
+    private RestClient remoteClient() throws IOException {
+        if (remoteClient == null) {
+            var clusterHosts = parseClusterHosts(remoteCluster.getHttpAddresses());
+            remoteClient = buildClient(restClientSettings(), clusterHosts.toArray(new HttpHost[0]));
+        }
+        return remoteClient;
+    }
+
+    @Override
+    protected String getTestRestCluster() {
+        return localCluster.getHttpAddresses();
+    }
+
+    @AfterClass
+    public static void closeRemoteClient() throws IOException {
+        try {
+            IOUtils.close(remoteClient);
+        } finally {
+            remoteClient = null;
+        }
+    }
+
+    @Override
+    protected boolean fetchDenseVectorAggMetricDoubleIfFns() throws IOException {
+        return super.fetchDenseVectorAggMetricDoubleIfFns()
+            && clusterHasCapability(remoteClient(), "GET", "/_query", List.of(), List.of("DENSE_VECTOR_AGG_METRIC_DOUBLE_IF_FNS")).orElse(
+                false
+            );
+    }
+}

+ 35 - 0
x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/AllSupportedFieldsIT.java

@@ -0,0 +1,35 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.qa.single_node;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
+
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.test.TestClustersThreadFilter;
+import org.elasticsearch.test.cluster.ElasticsearchCluster;
+import org.elasticsearch.xpack.esql.qa.rest.AllSupportedFieldsTestCase;
+import org.junit.ClassRule;
+
+/**
+ * Simple test for fetching all supported field types.
+ */
+@ThreadLeakFilters(filters = TestClustersThreadFilter.class)
+public class AllSupportedFieldsIT extends AllSupportedFieldsTestCase {
+    @ClassRule
+    public static ElasticsearchCluster cluster = Clusters.testCluster(c -> {});
+
+    public AllSupportedFieldsIT(MappedFieldType.FieldExtractPreference extractPreference, IndexMode indexMode) {
+        super(extractPreference, indexMode);
+    }
+
+    @Override
+    protected String getTestRestCluster() {
+        return cluster.getHttpAddresses();
+    }
+}

+ 563 - 0
x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java

@@ -0,0 +1,563 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.qa.rest;
+
+import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
+
+import org.apache.http.util.EntityUtils;
+import org.elasticsearch.TransportVersion;
+import org.elasticsearch.client.Request;
+import org.elasticsearch.client.ResponseException;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.index.IndexMode;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.logging.LogManager;
+import org.elasticsearch.logging.Logger;
+import org.elasticsearch.test.MapMatcher;
+import org.elasticsearch.test.rest.ESRestTestCase;
+import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xcontent.json.JsonXContent;
+import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.Rule;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import static org.elasticsearch.TransportVersions.INDEX_SOURCE;
+import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue;
+import static org.elasticsearch.test.ListMatcher.matchesList;
+import static org.elasticsearch.test.MapMatcher.assertMap;
+import static org.elasticsearch.test.MapMatcher.matchesMap;
+import static org.hamcrest.Matchers.any;
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.either;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.nullValue;
+
+/**
+ * Creates indices with all supported fields and fetches values from them.
+ * <p>
+ *     In a single cluster where all nodes are on a single version this is
+ *     just an "is it plugged in" style smoke test. In a mixed version cluster
+ *     this is testing the behavior of fetching potentially unsupported field
+ *     types. The same is true multi-cluster cases.
+ * </p>
+ * <p>
+ *     This isn't trying to test complex interactions with field loading so we
+ *     load constant field values and have simple mappings.
+ * </p>
+ */
+public class AllSupportedFieldsTestCase extends ESRestTestCase {
+    private static final Logger logger = LogManager.getLogger(FieldExtractorTestCase.class);
+
+    @Rule(order = Integer.MIN_VALUE)
+    public ProfileLogger profileLogger = new ProfileLogger();
+
+    @ParametersFactory(argumentFormatting = "pref=%s mode=%s")
+    public static List<Object[]> args() {
+        List<Object[]> args = new ArrayList<>();
+        for (MappedFieldType.FieldExtractPreference extractPreference : Arrays.asList(
+            null,
+            MappedFieldType.FieldExtractPreference.NONE,
+            MappedFieldType.FieldExtractPreference.STORED
+        )) {
+            for (IndexMode indexMode : IndexMode.values()) {
+                args.add(new Object[] { extractPreference, indexMode });
+            }
+        }
+        return args;
+    }
+
+    private final MappedFieldType.FieldExtractPreference extractPreference;
+    private final IndexMode indexMode;
+
+    protected AllSupportedFieldsTestCase(MappedFieldType.FieldExtractPreference extractPreference, IndexMode indexMode) {
+        this.extractPreference = extractPreference;
+        this.indexMode = indexMode;
+    }
+
+    protected record NodeInfo(String cluster, String id, TransportVersion version, Set<String> roles) {}
+
+    private static Map<String, NodeInfo> nodeToInfo;
+
+    private Map<String, NodeInfo> nodeToInfo() throws IOException {
+        if (nodeToInfo == null) {
+            nodeToInfo = fetchNodeToInfo(client(), null);
+        }
+        return nodeToInfo;
+    }
+
+    private static Boolean denseVectorAggMetricDoubleIfFns;
+
+    private boolean denseVectorAggMetricDoubleIfFns() throws IOException {
+        if (denseVectorAggMetricDoubleIfFns == null) {
+            denseVectorAggMetricDoubleIfFns = fetchDenseVectorAggMetricDoubleIfFns();
+        }
+        return denseVectorAggMetricDoubleIfFns;
+    }
+
+    protected boolean fetchDenseVectorAggMetricDoubleIfFns() throws IOException {
+        return clusterHasCapability("GET", "/_query", List.of(), List.of("DENSE_VECTOR_AGG_METRIC_DOUBLE_IF_FNS")).orElse(false);
+    }
+
+    private static Boolean supportsNodeAssignment;
+
+    protected boolean supportsNodeAssignment() throws IOException {
+        if (supportsNodeAssignment == null) {
+            for (NodeInfo i : allNodeToInfo().values()) {
+                logger.error("NOCOMMIT {}", i);
+            }
+            supportsNodeAssignment = allNodeToInfo().values()
+                .stream()
+                .allMatch(i -> (i.roles.contains("index") && i.roles.contains("search")) || (i.roles.contains("data")));
+        }
+        return supportsNodeAssignment;
+    }
+
+    /**
+     * Map from node name to information about the node.
+     */
+    protected Map<String, NodeInfo> allNodeToInfo() throws IOException {
+        return nodeToInfo();
+    }
+
+    protected static Map<String, NodeInfo> fetchNodeToInfo(RestClient client, String cluster) throws IOException {
+        Map<String, NodeInfo> nodeToInfo = new TreeMap<>();
+        Request request = new Request("GET", "/_nodes");
+        Map<String, Object> response = responseAsMap(client.performRequest(request));
+        Map<?, ?> nodes = (Map<?, ?>) extractValue(response, "nodes");
+        for (Map.Entry<?, ?> n : nodes.entrySet()) {
+            String id = (String) n.getKey();
+            Map<?, ?> nodeInfo = (Map<?, ?>) n.getValue();
+            String nodeName = (String) extractValue(nodeInfo, "name");
+            TransportVersion transportVersion = TransportVersion.fromId((Integer) extractValue(nodeInfo, "transport_version"));
+            List<?> roles = (List<?>) nodeInfo.get("roles");
+            nodeToInfo.put(
+                nodeName,
+                new NodeInfo(cluster, id, transportVersion, roles.stream().map(Object::toString).collect(Collectors.toSet()))
+            );
+        }
+
+        return nodeToInfo;
+    }
+
+    @Before
+    public void createIndices() throws IOException {
+        if (supportsNodeAssignment()) {
+            for (Map.Entry<String, NodeInfo> e : nodeToInfo().entrySet()) {
+                createIndexForNode(client(), e.getKey(), e.getValue().id());
+            }
+        } else {
+            createIndexForNode(client(), null, null);
+        }
+    }
+
+    public final void testFetchAll() throws IOException {
+        Map<String, Object> response = esql("""
+            , _id, _ignored, _index_mode, _score, _source, _version
+            | LIMIT 1000
+            """);
+        if ((Boolean) response.get("is_partial")) {
+            throw new AssertionError("partial results: " + response);
+        }
+        List<?> columns = (List<?>) response.get("columns");
+        List<?> values = (List<?>) response.get("values");
+
+        MapMatcher expectedColumns = matchesMap();
+        for (DataType type : DataType.values()) {
+            if (supportedInIndex(type) == false) {
+                continue;
+            }
+            expectedColumns = expectedColumns.entry(fieldName(type), expectedType(type));
+        }
+        expectedColumns = expectedColumns.entry("_id", "keyword")
+            .entry("_ignored", "keyword")
+            .entry("_index", "keyword")
+            .entry("_index_mode", "keyword")
+            .entry("_score", "double")
+            .entry("_source", "_source")
+            .entry("_version", "long");
+        assertMap(nameToType(columns), expectedColumns);
+
+        MapMatcher expectedAllValues = matchesMap();
+        for (Map.Entry<String, NodeInfo> e : expectedIndices().entrySet()) {
+            String indexName = e.getKey();
+            NodeInfo nodeInfo = e.getValue();
+            MapMatcher expectedValues = matchesMap();
+            for (DataType type : DataType.values()) {
+                if (supportedInIndex(type) == false) {
+                    continue;
+                }
+                expectedValues = expectedValues.entry(fieldName(type), expectedValue(nodeInfo.version, type));
+            }
+            expectedValues = expectedValues.entry("_id", any(String.class))
+                .entry("_ignored", nullValue())
+                .entry("_index", indexName)
+                .entry("_index_mode", indexMode.toString())
+                .entry("_score", 0.0)
+                .entry("_source", matchesMap().extraOk())
+                .entry("_version", 1);
+            expectedAllValues = expectedAllValues.entry(indexName, expectedValues);
+        }
+        assertMap(indexToRow(columns, values), expectedAllValues);
+        profileLogger.clearProfile();
+    }
+
+    public final void testFetchDenseVector() throws IOException {
+        Map<String, Object> response;
+        try {
+            response = esql("""
+                | EVAL k = v_l2_norm(f_dense_vector, [1])  // workaround to enable fetching dense_vector
+                | KEEP _index, f_dense_vector
+                | LIMIT 1000
+                """);
+            if ((Boolean) response.get("is_partial")) {
+                Map<?, ?> clusters = (Map<?, ?>) response.get("_clusters");
+                Map<?, ?> details = (Map<?, ?>) clusters.get("details");
+
+                boolean foundError = false;
+                for (Map.Entry<?, ?> cluster : details.entrySet()) {
+                    String failures = cluster.getValue().toString();
+                    if (denseVectorAggMetricDoubleIfFns()) {
+                        throw new AssertionError("should correctly fetch the dense_vector: " + failures);
+                    }
+                    foundError |= failures.contains("doesn't understand data type [DENSE_VECTOR]");
+                }
+                assertTrue("didn't find errors: " + details, foundError);
+                return;
+            }
+        } catch (ResponseException e) {
+            if (denseVectorAggMetricDoubleIfFns()) {
+                throw new AssertionError("should correctly fetch the dense_vector", e);
+            }
+            assertThat(
+                "old version should fail with this error",
+                EntityUtils.toString(e.getResponse().getEntity()),
+                anyOf(
+                    containsString("Unknown function [v_l2_norm]"),
+                    containsString("Cannot use field [f_dense_vector] with unsupported type"),
+                    containsString("doesn't understand data type [DENSE_VECTOR]")
+                )
+            );
+            // Failure is expected and fine
+            return;
+        }
+        List<?> columns = (List<?>) response.get("columns");
+        List<?> values = (List<?>) response.get("values");
+
+        MapMatcher expectedColumns = matchesMap().entry("f_dense_vector", "dense_vector").entry("_index", "keyword");
+        assertMap(nameToType(columns), expectedColumns);
+
+        MapMatcher expectedAllValues = matchesMap();
+        for (Map.Entry<String, NodeInfo> e : expectedIndices().entrySet()) {
+            String indexName = e.getKey();
+            NodeInfo nodeInfo = e.getValue();
+            MapMatcher expectedValues = matchesMap();
+            expectedValues = expectedValues.entry("f_dense_vector", expectedDenseVector(nodeInfo.version));
+            expectedValues = expectedValues.entry("_index", indexName);
+            expectedAllValues = expectedAllValues.entry(indexName, expectedValues);
+        }
+        assertMap(indexToRow(columns, values), expectedAllValues);
+    }
+
+    private Map<String, Object> esql(String query) throws IOException {
+        Request request = new Request("POST", "_query");
+        XContentBuilder body = JsonXContent.contentBuilder().startObject();
+        body.field("query", "FROM *:%mode%*,%mode%* METADATA _index".replace("%mode%", indexMode.toString()) + query);
+        {
+            body.startObject("pragma");
+            if (extractPreference != null) {
+                body.field("field_extract_preference", extractPreference);
+            }
+            body.endObject();
+        }
+        body.field("accept_pragma_risks", "true");
+        body.field("profile", true);
+        body.field("include_ccs_metadata", true);
+        body.endObject();
+        request.setJsonEntity(Strings.toString(body));
+
+        Map<String, Object> response = responseAsMap(client().performRequest(request));
+        profileLogger.extractProfile(response, true);
+        return response;
+    }
+
+    protected void createIndexForNode(RestClient client, String nodeName, String nodeId) throws IOException {
+        String indexName = indexMode.toString();
+        if (nodeName != null) {
+            indexName += "_" + nodeName.toLowerCase(Locale.ROOT);
+        }
+        if (false == indexExists(client, indexName)) {
+            createAllTypesIndex(client, indexName, nodeId);
+            createAllTypesDoc(client, indexName);
+        }
+    }
+
+    private void createAllTypesIndex(RestClient client, String indexName, String nodeId) throws IOException {
+        XContentBuilder config = JsonXContent.contentBuilder().startObject();
+        {
+            config.startObject("settings");
+            config.startObject("index");
+            config.field("mode", indexMode);
+            if (indexMode == IndexMode.TIME_SERIES) {
+                config.field("routing_path", "f_keyword");
+            }
+            if (nodeId != null) {
+                config.field("routing.allocation.include._id", nodeId);
+            }
+            config.endObject();
+            config.endObject();
+        }
+        {
+            config.startObject("mappings").startObject("properties");
+            for (DataType type : DataType.values()) {
+                if (supportedInIndex(type) == false) {
+                    continue;
+                }
+                config.startObject(fieldName(type));
+                typeMapping(indexMode, config, type);
+                config.endObject();
+            }
+            config.endObject().endObject().endObject();
+        }
+        Request request = new Request("PUT", indexName);
+        request.setJsonEntity(Strings.toString(config));
+        client.performRequest(request);
+    }
+
+    private String fieldName(DataType type) {
+        return type == DataType.DATETIME ? "@timestamp" : "f_" + type.esType();
+    }
+
+    private void typeMapping(IndexMode indexMode, XContentBuilder config, DataType type) throws IOException {
+        switch (type) {
+            case COUNTER_DOUBLE, COUNTER_INTEGER, COUNTER_LONG -> config.field("type", type.esType().replace("counter_", ""))
+                .field("time_series_metric", "counter");
+            case SCALED_FLOAT -> config.field("type", type.esType()).field("scaling_factor", 1);
+            case AGGREGATE_METRIC_DOUBLE -> config.field("type", type.esType())
+                .field("metrics", List.of("min", "max", "sum", "value_count"))
+                .field("default_metric", "max");
+            case NULL -> config.field("type", "keyword");
+            case KEYWORD -> {
+                config.field("type", type.esType());
+                if (indexMode == IndexMode.TIME_SERIES) {
+                    config.field("time_series_dimension", true);
+                }
+            }
+            default -> config.field("type", type.esType());
+        }
+    }
+
+    private void createAllTypesDoc(RestClient client, String indexName) throws IOException {
+        XContentBuilder doc = JsonXContent.contentBuilder().startObject();
+        for (DataType type : DataType.values()) {
+            if (supportedInIndex(type) == false) {
+                continue;
+            }
+            doc.field(fieldName(type));
+            switch (type) {
+                case BOOLEAN -> doc.value(true);
+                case COUNTER_LONG, LONG, COUNTER_INTEGER, INTEGER, UNSIGNED_LONG, SHORT, BYTE -> doc.value(1);
+                case COUNTER_DOUBLE, DOUBLE, FLOAT, HALF_FLOAT, SCALED_FLOAT -> doc.value(1.1);
+                case KEYWORD, TEXT -> doc.value("foo");
+                case DATETIME, DATE_NANOS -> doc.value("2025-01-01T01:00:00Z");
+                case IP -> doc.value("192.168.0.1");
+                case VERSION -> doc.value("1.0.0-SNAPSHOT");
+                case GEO_POINT, GEO_SHAPE -> doc.value("POINT (-71.34 41.12)");
+                case NULL -> doc.nullValue();
+                case AGGREGATE_METRIC_DOUBLE -> {
+                    doc.startObject();
+                    doc.field("min", -302.50);
+                    doc.field("max", 702.30);
+                    doc.field("sum", 200.0);
+                    doc.field("value_count", 25);
+                    doc.endObject();
+                }
+                case DENSE_VECTOR -> doc.value(List.of(0.5, 10, 6));
+                default -> throw new AssertionError("unsupported field type [" + type + "]");
+            }
+        }
+        doc.endObject();
+        Request request = new Request("POST", indexName + "/_doc");
+        request.addParameter("refresh", "");
+        request.setJsonEntity(Strings.toString(doc));
+        client.performRequest(request);
+    }
+
+    private Matcher<?> expectedValue(TransportVersion version, DataType type) throws IOException {
+        return switch (type) {
+            case BOOLEAN -> equalTo(true);
+            case COUNTER_LONG, LONG, COUNTER_INTEGER, INTEGER, UNSIGNED_LONG, SHORT, BYTE -> equalTo(1);
+            case COUNTER_DOUBLE, DOUBLE -> equalTo(1.1);
+            case FLOAT -> equalTo(1.100000023841858);
+            case HALF_FLOAT -> equalTo(1.099609375);
+            case SCALED_FLOAT -> equalTo(1.0);
+            // TODO what about the extra types and ES supports and ESQL flattens away like semantic_text and wildcard?
+            case KEYWORD, TEXT -> equalTo("foo");
+            case DATETIME, DATE_NANOS -> equalTo("2025-01-01T01:00:00.000Z");
+            case IP -> equalTo("192.168.0.1");
+            case VERSION -> equalTo("1.0.0-SNAPSHOT");
+            case GEO_POINT -> extractPreference == MappedFieldType.FieldExtractPreference.DOC_VALUES || syntheticSourceByDefault()
+                ? equalTo("POINT (-71.34000004269183 41.1199999647215)")
+                : equalTo("POINT (-71.34 41.12)");
+            case GEO_SHAPE -> equalTo("POINT (-71.34 41.12)");
+            case NULL -> nullValue();
+            case AGGREGATE_METRIC_DOUBLE -> {
+                if (denseVectorAggMetricDoubleIfFns()) {
+                    // If all versions are new we get null. If any are old, some *might* support aggregate_metric_double
+                    yield nullValue();
+                }
+                Matcher<String> expected = equalTo("{\"min\":-302.5,\"max\":702.3,\"sum\":200.0,\"value_count\":25}");
+                yield anyOf(nullValue(), expected);
+            }
+            case DENSE_VECTOR -> {
+                if (denseVectorAggMetricDoubleIfFns()) {
+                    // If all versions are new we get null. If any are old, some *might* support dense_vector
+                    yield nullValue();
+                }
+                yield anyOf(nullValue(), expectedDenseVector(version));
+            }
+            default -> throw new AssertionError("unsupported field type [" + type + "]");
+        };
+    }
+
+    private Matcher<List<?>> expectedDenseVector(TransportVersion version) {
+        return version.onOrAfter(INDEX_SOURCE) // *after* 9.1
+            ? matchesList().item(0.5).item(10.0).item(5.9999995)
+            : matchesList().item(0.04283529).item(0.85670584).item(0.5140235);
+    }
+
+    /**
+     * Is the type supported in indices?
+     */
+    private static boolean supportedInIndex(DataType t) {
+        return switch (t) {
+            // These are supported but implied by the index process.
+            case OBJECT, SOURCE, DOC_DATA_TYPE, TSID_DATA_TYPE,
+                // Internal only
+                UNSUPPORTED, PARTIAL_AGG,
+                // You can't index these - they are just constants.
+                DATE_PERIOD, TIME_DURATION, GEOTILE, GEOHASH, GEOHEX,
+                // TODO fix geo
+                CARTESIAN_POINT, CARTESIAN_SHAPE -> false;
+            default -> true;
+        };
+    }
+
+    private Map<String, Object> nameToType(List<?> columns) {
+        Map<String, Object> result = new TreeMap<>();
+        for (Object c : columns) {
+            Map<?, ?> map = (Map<?, ?>) c;
+            result.put(map.get("name").toString(), map.get("type"));
+        }
+        return result;
+    }
+
+    private List<String> names(List<?> columns) {
+        List<String> result = new ArrayList<>();
+        for (Object c : columns) {
+            Map<?, ?> map = (Map<?, ?>) c;
+            result.add(map.get("name").toString());
+        }
+        return result;
+    }
+
+    private Map<String, Map<String, Object>> indexToRow(List<?> columns, List<?> values) {
+        List<String> names = names(columns);
+        int timestampIdx = names.indexOf("_index");
+        if (timestampIdx < 0) {
+            throw new IllegalStateException("query didn't return _index");
+        }
+        Map<String, Map<String, Object>> result = new TreeMap<>();
+        for (Object r : values) {
+            List<?> row = (List<?>) r;
+            result.put(row.get(timestampIdx).toString(), nameToValue(names, row));
+        }
+        return result;
+    }
+
+    private Map<String, Object> nameToValue(List<String> names, List<?> values) {
+        Map<String, Object> result = new TreeMap<>();
+        for (int i = 0; i < values.size(); i++) {
+            result.put(names.get(i), values.get(i));
+        }
+        return result;
+    }
+
+    private Matcher<String> expectedType(DataType type) throws IOException {
+        return switch (type) {
+            case COUNTER_DOUBLE, COUNTER_LONG, COUNTER_INTEGER -> {
+                if (indexMode == IndexMode.TIME_SERIES) {
+                    yield equalTo(type.esType());
+                }
+                yield equalTo(type.esType().replace("counter_", ""));
+            }
+            case BYTE, SHORT -> equalTo("integer");
+            case HALF_FLOAT, SCALED_FLOAT, FLOAT -> equalTo("double");
+            case NULL -> equalTo("keyword");
+            // Currently unsupported without TS command or KNN function
+            case AGGREGATE_METRIC_DOUBLE, DENSE_VECTOR -> {
+                if (denseVectorAggMetricDoubleIfFns()) {
+                    // If all versions are new we get null. If any are old, some *might* support dense_vector
+                    yield equalTo("unsupported");
+                }
+                yield either(equalTo("unsupported")).or(equalTo(type.esType()));
+            }
+            default -> equalTo(type.esType());
+        };
+    }
+
+    @Override
+    protected boolean preserveClusterUponCompletion() {
+        return true;
+    }
+
+    private boolean syntheticSourceByDefault() {
+        return switch (indexMode) {
+            case TIME_SERIES, LOGSDB -> true;
+            case STANDARD, LOOKUP -> false;
+        };
+    }
+
+    private Map<String, NodeInfo> expectedIndices() throws IOException {
+        logger.error("ADFADF NOCOMMIT");
+        Map<String, NodeInfo> result = new TreeMap<>();
+        if (supportsNodeAssignment()) {
+            logger.error("supports {}", allNodeToInfo());
+            for (Map.Entry<String, NodeInfo> e : allNodeToInfo().entrySet()) {
+                String name = indexMode + "_" + e.getKey();
+                if (e.getValue().cluster != null) {
+                    name = e.getValue().cluster + ":" + name;
+                }
+                result.put(name, e.getValue());
+            }
+        } else {
+            logger.error("one per {}", allNodeToInfo());
+            for (Map.Entry<String, NodeInfo> e : allNodeToInfo().entrySet()) {
+                String name = indexMode.toString();
+                if (e.getValue().cluster != null) {
+                    name = e.getValue().cluster + ":" + name;
+                }
+                // We should only end up with one per cluster
+                result.put(name, new NodeInfo(e.getValue().cluster, null, e.getValue().version(), null));
+            }
+        }
+        return result;
+    }
+}

+ 1 - 1
x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/FieldExtractorTestCase.java

@@ -73,7 +73,7 @@ public abstract class FieldExtractorTestCase extends ESRestTestCase {
     public ProfileLogger profileLogger = new ProfileLogger();
 
     @ParametersFactory(argumentFormatting = "%s")
-    public static List<Object[]> args() throws Exception {
+    public static List<Object[]> args() {
         return List.of(
             new Object[] { null },
             new Object[] { MappedFieldType.FieldExtractPreference.NONE },

+ 7 - 2
x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector-bit.csv-spec

@@ -1,7 +1,9 @@
 retrieveBitVectorData
 required_capability: dense_vector_field_type_bit_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
+| EVAL k = v_l2_norm(bit_vector, [1])  // workaround to enable fetching dense_vector
 | KEEP id, bit_vector
 | SORT id
 ;
@@ -15,9 +17,10 @@ id:l | bit_vector:dense_vector
 
 denseBitVectorWithEval
 required_capability: dense_vector_field_type_bit_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
-| EVAL v = bit_vector
+| EVAL v = bit_vector, k = v_l2_norm(bit_vector, [1])  // workaround to enable fetching dense_vector
 | KEEP id, v
 | SORT id
 ;
@@ -31,11 +34,13 @@ id:l | v:dense_vector
 
 denseBitVectorWithRenameAndDrop
 required_capability: dense_vector_field_type_bit_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
 | EVAL v = bit_vector
+| EVAL k = v_l2_norm(bit_vector, [1])  // workaround to enable fetching dense_vector
 | RENAME v AS new_vector
-| DROP float_vector, byte_vector, bit_vector
+| DROP float_vector, byte_vector, bit_vector, k
 | SORT id
 ;
 

+ 8 - 2
x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector-byte.csv-spec

@@ -1,7 +1,9 @@
 retrieveByteVectorData
 required_capability: dense_vector_field_type_byte_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
+| EVAL k = v_l2_norm(byte_vector, [1]) // workaround to enable fetching dense_vector
 | KEEP id, byte_vector
 | SORT id
 ;
@@ -15,9 +17,11 @@ id:l | byte_vector:dense_vector
 
 denseByteVectorWithEval
 required_capability: dense_vector_field_type_byte_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
 | EVAL v = byte_vector
+| EVAL k = v_l2_norm(byte_vector, [1])  // workaround to enable fetching dense_vector
 | KEEP id, v
 | SORT id
 ;
@@ -31,11 +35,13 @@ id:l | v:dense_vector
 
 denseByteVectorWithRenameAndDrop
 required_capability: dense_vector_field_type_byte_elements
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector 
-| EVAL v = byte_vector 
+| EVAL v = byte_vector
+| EVAL k = v_l2_norm(byte_vector, [1])  // workaround to enable fetching dense_vector
 | RENAME v AS new_vector 
-| DROP float_vector, byte_vector, bit_vector
+| DROP float_vector, byte_vector, bit_vector, k
 | SORT id
 ;
 

+ 7 - 1
x-pack/plugin/esql/qa/testFixtures/src/main/resources/dense_vector.csv-spec

@@ -1,7 +1,9 @@
 retrieveDenseVectorData
 required_capability: dense_vector_field_type
+required_capability: dense_vector_agg_metric_double_if_fns
 
 FROM dense_vector
+| EVAL k = v_l2_norm(float_vector, [1])  // workaround to enable fetching dense_vector
 | KEEP id, float_vector
 | SORT id
 ;
@@ -15,9 +17,11 @@ id:l | float_vector:dense_vector
 
 denseVectorWithEval
 required_capability: dense_vector_field_type
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector
 | EVAL v = float_vector
+| EVAL k = v_l2_norm(float_vector, [1])  // workaround to enable fetching dense_vector
 | KEEP id, v
 | SORT id
 ;
@@ -31,11 +35,13 @@ id:l | v:dense_vector
 
 denseVectorWithRenameAndDrop
 required_capability: dense_vector_field_type
+required_capability: l2_norm_vector_similarity_function
 
 FROM dense_vector 
 | EVAL v = float_vector 
+| EVAL k = v_l2_norm(float_vector, [1])  // workaround to enable fetching dense_vector
 | RENAME v AS new_vector 
-| DROP float_vector, byte_vector, bit_vector
+| DROP float_vector, byte_vector, bit_vector, k
 | SORT id
 ;
 

+ 1 - 5
x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich.csv-spec

@@ -650,11 +650,7 @@ IDR       |  Indore       |  POINT(75.8472 22.7167)  |  India      |  POINT(75.8
 fieldsInOtherIndicesBug
 required_capability: enrich_load
 required_capability: fix_replace_missing_field_with_null_duplicate_name_id_in_layout
-
-// from * accidentally selects columns with dense_vector field type.
-// This is not properly handled when the query is planned by newer node and executed by an older one.
-// see https://github.com/elastic/elasticsearch/issues/135193
-required_capability: dense_vector_field_type
+required_capability: dense_vector_agg_metric_double_if_fns
 
 from *
 | keep author.keyword, book_no, scalerank, street, bytes_in, @timestamp, abbrev, city_location, distance, description, birth_date, language_code, intersects, client_ip, event_duration, version

+ 1 - 5
x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec

@@ -1673,11 +1673,7 @@ null      |1952-02-27T00:00:00.000Z
 enrichLookupStatsBug
 required_capability: join_lookup_v12
 required_capability: fix_replace_missing_field_with_null_duplicate_name_id_in_layout
-
-// from * accidentally selects columns with dense_vector field type.
-// This is not properly handled when the query is planned by newer node and executed by an older one.
-// see https://github.com/elastic/elasticsearch/issues/135193
-required_capability: dense_vector_field_type
+required_capability: dense_vector_agg_metric_double_if_fns
 
 from *
 | enrich languages_policy on cluster

+ 4 - 0
x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/DenseVectorFieldTypeIT.java

@@ -93,6 +93,8 @@ public class DenseVectorFieldTypeIT extends AbstractEsqlIntegTestCase {
     public void testRetrieveFieldType() {
         var query = """
             FROM test
+            | EVAL k = v_l2_norm(vector, [1])  // workaround to enable fetching dense_vector
+            | DROP k
             """;
 
         try (var resp = run(query)) {
@@ -105,6 +107,7 @@ public class DenseVectorFieldTypeIT extends AbstractEsqlIntegTestCase {
     public void testRetrieveTopNDenseVectorFieldData() {
         var query = """
                 FROM test
+                | EVAL k = v_l2_norm(vector, [1])  // workaround to enable fetching dense_vector
                 | KEEP id, vector
                 | SORT id ASC
             """;
@@ -132,6 +135,7 @@ public class DenseVectorFieldTypeIT extends AbstractEsqlIntegTestCase {
     public void testRetrieveDenseVectorFieldData() {
         var query = """
             FROM test
+            | EVAL k = v_l2_norm(vector, [1])  // workaround to enable fetching dense_vector
             | KEEP id, vector
             """;
 

+ 2 - 1
x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupJoinTypesIT.java

@@ -244,7 +244,8 @@ public class LookupJoinTypesIT extends ESIntegTestCase {
                     if (type == NULL
                         || type == DOC_DATA_TYPE
                         || type == TSID_DATA_TYPE
-                        || type == AGGREGATE_METRIC_DOUBLE
+                        || type == AGGREGATE_METRIC_DOUBLE  // need special handling for loads at the moment
+                        || type == DENSE_VECTOR  // need special handling for loads at the moment
                         || type == GEOHASH
                         || type == GEOTILE
                         || type == GEOHEX

+ 6 - 4
x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/KnnFunctionIT.java

@@ -220,10 +220,12 @@ public class KnnFunctionIT extends AbstractEsqlIntegTestCase {
         var error = expectThrows(VerificationException.class, () -> run(query));
         assertThat(
             error.getMessage(),
-            containsString(
-                "line 3:13: [KNN] function cannot operate on [lookup_vector], supplied by an index [test_lookup] in non-STANDARD "
-                    + "mode [lookup]"
-            )
+            // TODO revert this when we have proper versioned type resolutions
+            // containsString(
+            // "line 3:13: [KNN] function cannot operate on [lookup_vector], supplied by an index [test_lookup] in non-STANDARD "
+            // + "mode [lookup]"
+            // )
+            containsString("line 3:13: Cannot use field [lookup_vector] with unsupported type [dense_vector]")
         );
     }
 

+ 5 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java

@@ -1551,7 +1551,11 @@ public class EsqlCapabilities {
          */
         INLINE_STATS_FIX_PRUNING_NULL_FILTER(INLINESTATS_V11.enabled),
 
-        INLINE_STATS_FIX_OPTIMIZED_AS_LOCAL_RELATION(INLINESTATS_V11.enabled);
+        INLINE_STATS_FIX_OPTIMIZED_AS_LOCAL_RELATION(INLINESTATS_V11.enabled),
+
+        DENSE_VECTOR_AGG_METRIC_DOUBLE_IF_FNS
+
+        ;
 
         private final boolean enabled;
 

+ 46 - 7
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java

@@ -9,6 +9,7 @@ package org.elasticsearch.xpack.esql.analysis;
 
 import org.elasticsearch.index.IndexMode;
 import org.elasticsearch.xpack.esql.core.util.Holder;
+import org.elasticsearch.xpack.esql.expression.function.UnresolvedFunction;
 import org.elasticsearch.xpack.esql.plan.IndexPattern;
 import org.elasticsearch.xpack.esql.plan.logical.Enrich;
 import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
@@ -22,8 +23,15 @@ import java.util.List;
  */
 public class PreAnalyzer {
 
-    public record PreAnalysis(IndexMode indexMode, IndexPattern indexPattern, List<Enrich> enriches, List<IndexPattern> lookupIndices) {
-        public static final PreAnalysis EMPTY = new PreAnalysis(null, null, List.of(), List.of());
+    public record PreAnalysis(
+        IndexMode indexMode,
+        IndexPattern indexPattern,
+        List<Enrich> enriches,
+        List<IndexPattern> lookupIndices,
+        boolean supportsAggregateMetricDouble,
+        boolean supportsDenseVector
+    ) {
+        public static final PreAnalysis EMPTY = new PreAnalysis(null, null, List.of(), List.of(), false, false);
     }
 
     public PreAnalysis preAnalyze(LogicalPlan plan) {
@@ -35,13 +43,9 @@ public class PreAnalyzer {
     }
 
     protected PreAnalysis doPreAnalyze(LogicalPlan plan) {
-
         Holder<IndexMode> indexMode = new Holder<>();
         Holder<IndexPattern> index = new Holder<>();
-
-        List<Enrich> unresolvedEnriches = new ArrayList<>();
         List<IndexPattern> lookupIndices = new ArrayList<>();
-
         plan.forEachUp(UnresolvedRelation.class, p -> {
             if (p.indexMode() == IndexMode.LOOKUP) {
                 lookupIndices.add(p.indexPattern());
@@ -53,11 +57,46 @@ public class PreAnalyzer {
             }
         });
 
+        List<Enrich> unresolvedEnriches = new ArrayList<>();
         plan.forEachUp(Enrich.class, unresolvedEnriches::add);
 
+        /*
+         * Enable aggregate_metric_double and dense_vector when we see certain function
+         * or the TS command. This allows us to release these when not all nodes understand
+         * these types. These functions are only supported on newer nodes, so we use them
+         * as a signal that the query is only for nodes that support these types.
+         *
+         * This work around is temporary until we flow the minimum transport version
+         * back through a cross cluster search field caps call.
+         */
+        Holder<Boolean> supportsAggregateMetricDouble = new Holder<>(false);
+        Holder<Boolean> supportsDenseVector = new Holder<>(false);
+        plan.forEachDown(p -> p.forEachExpression(UnresolvedFunction.class, fn -> {
+            if (fn.name().equalsIgnoreCase("knn")
+                || fn.name().equalsIgnoreCase("to_dense_vector")
+                || fn.name().equalsIgnoreCase("v_cosine")
+                || fn.name().equalsIgnoreCase("v_hamming")
+                || fn.name().equalsIgnoreCase("v_l1_norm")
+                || fn.name().equalsIgnoreCase("v_l2_norm")
+                || fn.name().equalsIgnoreCase("v_dot_product")
+                || fn.name().equalsIgnoreCase("v_magnitude")) {
+                supportsDenseVector.set(true);
+            }
+            if (fn.name().equalsIgnoreCase("to_aggregate_metric_double")) {
+                supportsAggregateMetricDouble.set(true);
+            }
+        }));
+
         // mark plan as preAnalyzed (if it were marked, there would be no analysis)
         plan.forEachUp(LogicalPlan::setPreAnalyzed);
 
-        return new PreAnalysis(indexMode.get(), index.get(), unresolvedEnriches, lookupIndices);
+        return new PreAnalysis(
+            indexMode.get(),
+            index.get(),
+            unresolvedEnriches,
+            lookupIndices,
+            indexMode.get() == IndexMode.TIME_SERIES || supportsAggregateMetricDouble.get(),
+            supportsDenseVector.get()
+        );
     }
 }

+ 25 - 16
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java

@@ -445,22 +445,31 @@ public class EnrichPolicyResolver {
                     }
                     try (ThreadContext.StoredContext ignored = threadContext.stashWithOrigin(ClientHelper.ENRICH_ORIGIN)) {
                         String indexName = EnrichPolicy.getBaseName(policyName);
-                        indexResolver.resolveAsMergedMapping(indexName, IndexResolver.ALL_FIELDS, null, false, refs.acquire(indexResult -> {
-                            if (indexResult.isValid() && indexResult.get().concreteIndices().size() == 1) {
-                                EsIndex esIndex = indexResult.get();
-                                var concreteIndices = Map.of(request.clusterAlias, Iterables.get(esIndex.concreteIndices(), 0));
-                                var resolved = new ResolvedEnrichPolicy(
-                                    p.getMatchField(),
-                                    p.getType(),
-                                    p.getEnrichFields(),
-                                    concreteIndices,
-                                    esIndex.mapping()
-                                );
-                                resolvedPolices.put(policyName, resolved);
-                            } else {
-                                failures.put(policyName, indexResult.toString());
-                            }
-                        }));
+                        indexResolver.resolveAsMergedMapping(
+                            indexName,
+                            IndexResolver.ALL_FIELDS,
+                            null,
+                            false,
+                            // Disable aggregate_metric_double and dense_vector until we get version checks in planning
+                            false,
+                            false,
+                            refs.acquire(indexResult -> {
+                                if (indexResult.isValid() && indexResult.get().concreteIndices().size() == 1) {
+                                    EsIndex esIndex = indexResult.get();
+                                    var concreteIndices = Map.of(request.clusterAlias, Iterables.get(esIndex.concreteIndices(), 0));
+                                    var resolved = new ResolvedEnrichPolicy(
+                                        p.getMatchField(),
+                                        p.getType(),
+                                        p.getEnrichFields(),
+                                        concreteIndices,
+                                        esIndex.mapping()
+                                    );
+                                    resolvedPolices.put(policyName, resolved);
+                                } else {
+                                    failures.put(policyName, indexResult.toString());
+                                }
+                            })
+                        );
                     }
                 }
             }

+ 5 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java

@@ -510,6 +510,9 @@ public class EsqlSession {
             result.wildcardJoinIndices().contains(localPattern) ? IndexResolver.ALL_FIELDS : result.fieldNames,
             null,
             false,
+            // Disable aggregate_metric_double and dense_vector until we get version checks in planning
+            false,
+            false,
             listener.map(indexResolution -> receiveLookupIndexResolution(result, localPattern, executionInfo, indexResolution))
         );
     }
@@ -725,6 +728,8 @@ public class EsqlSession {
                         default -> requestFilter;
                     },
                     preAnalysis.indexMode() == IndexMode.TIME_SERIES,
+                    preAnalysis.supportsAggregateMetricDouble(),
+                    preAnalysis.supportsDenseVector(),
                     listener.delegateFailureAndWrap((l, indexResolution) -> {
                         EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, indexResolution.failures());
                         l.onResponse(result.withIndices(indexResolution));

+ 28 - 13
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java

@@ -42,7 +42,9 @@ import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import static org.elasticsearch.xpack.esql.core.type.DataType.AGGREGATE_METRIC_DOUBLE;
 import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME;
+import static org.elasticsearch.xpack.esql.core.type.DataType.DENSE_VECTOR;
 import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD;
 import static org.elasticsearch.xpack.esql.core.type.DataType.OBJECT;
 import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT;
@@ -82,25 +84,33 @@ public class IndexResolver {
         Set<String> fieldNames,
         QueryBuilder requestFilter,
         boolean includeAllDimensions,
+        boolean supportsAggregateMetricDouble,
+        boolean supportsDenseVector,
         ActionListener<IndexResolution> listener
     ) {
         client.execute(
             EsqlResolveFieldsAction.TYPE,
             createFieldCapsRequest(indexWildcard, fieldNames, requestFilter, includeAllDimensions),
-            listener.delegateFailureAndWrap((l, response) -> l.onResponse(mergedMappings(indexWildcard, response)))
+            listener.delegateFailureAndWrap(
+                (l, response) -> l.onResponse(
+                    mergedMappings(indexWildcard, new FieldsInfo(response, supportsAggregateMetricDouble, supportsDenseVector))
+                )
+            )
         );
     }
 
+    public record FieldsInfo(FieldCapabilitiesResponse caps, boolean supportAggregateMetricDouble, boolean supportDenseVector) {}
+
     // public for testing only
-    public static IndexResolution mergedMappings(String indexPattern, FieldCapabilitiesResponse fieldCapsResponse) {
+    public static IndexResolution mergedMappings(String indexPattern, FieldsInfo fieldsInfo) {
         assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION); // too expensive to run this on a transport worker
-        var numberOfIndices = fieldCapsResponse.getIndexResponses().size();
-        if (fieldCapsResponse.getIndexResponses().isEmpty()) {
+        int numberOfIndices = fieldsInfo.caps.getIndexResponses().size();
+        if (numberOfIndices == 0) {
             return IndexResolution.notFound(indexPattern);
         }
 
         // For each field name, store a list of the field caps responses from each index
-        var collectedFieldCaps = collectFieldCaps(fieldCapsResponse);
+        var collectedFieldCaps = collectFieldCaps(fieldsInfo.caps);
         Map<String, IndexFieldCapabilitiesWithSourceHash> fieldsCaps = collectedFieldCaps.fieldsCaps;
         Map<String, Integer> indexMappingHashDuplicates = collectedFieldCaps.indexMappingHashDuplicates;
 
@@ -138,7 +148,7 @@ public class IndexResolver {
             var fieldCap = fieldsCaps.get(fullName);
             List<IndexFieldCapabilities> fcs = fieldCap.fieldCapabilities;
             EsField field = firstUnsupportedParent == null
-                ? createField(fieldCapsResponse, name, fullName, fcs, isAlias)
+                ? createField(fieldsInfo, name, fullName, fcs, isAlias)
                 : new UnsupportedEsField(
                     fullName,
                     firstUnsupportedParent.getOriginalTypes(),
@@ -152,13 +162,13 @@ public class IndexResolver {
             }
         }
 
-        Map<String, IndexMode> concreteIndices = Maps.newMapWithExpectedSize(fieldCapsResponse.getIndexResponses().size());
-        for (FieldCapabilitiesIndexResponse ir : fieldCapsResponse.getIndexResponses()) {
+        Map<String, IndexMode> concreteIndices = Maps.newMapWithExpectedSize(fieldsInfo.caps.getIndexResponses().size());
+        for (FieldCapabilitiesIndexResponse ir : fieldsInfo.caps.getIndexResponses()) {
             concreteIndices.put(ir.getIndexName(), ir.getIndexMode());
         }
 
         boolean allEmpty = true;
-        for (FieldCapabilitiesIndexResponse ir : fieldCapsResponse.getIndexResponses()) {
+        for (FieldCapabilitiesIndexResponse ir : fieldsInfo.caps.getIndexResponses()) {
             allEmpty &= ir.get().isEmpty();
         }
         // If all the mappings are empty we return an empty set of resolved indices to line up with QL
@@ -168,7 +178,7 @@ public class IndexResolver {
         // for fields that do not exist in the index (but the index has a mapping) will result in "VerificationException Unknown column"
         // errors.
         var index = new EsIndex(indexPattern, rootFields, allEmpty ? Map.of() : concreteIndices, partiallyUnmappedFields);
-        var failures = EsqlCCSUtils.groupFailuresPerCluster(fieldCapsResponse.getFailures());
+        var failures = EsqlCCSUtils.groupFailuresPerCluster(fieldsInfo.caps.getFailures());
         return IndexResolution.valid(index, concreteIndices.keySet(), failures);
     }
 
@@ -215,7 +225,7 @@ public class IndexResolver {
     }
 
     private static EsField createField(
-        FieldCapabilitiesResponse fieldCapsResponse,
+        FieldsInfo fieldsInfo,
         String name,
         String fullName,
         List<IndexFieldCapabilities> fcs,
@@ -224,12 +234,17 @@ public class IndexResolver {
         IndexFieldCapabilities first = fcs.get(0);
         List<IndexFieldCapabilities> rest = fcs.subList(1, fcs.size());
         DataType type = EsqlDataTypeRegistry.INSTANCE.fromEs(first.type(), first.metricType());
+        type = switch (type) {
+            case AGGREGATE_METRIC_DOUBLE -> fieldsInfo.supportAggregateMetricDouble ? AGGREGATE_METRIC_DOUBLE : UNSUPPORTED;
+            case DENSE_VECTOR -> fieldsInfo.supportDenseVector ? DENSE_VECTOR : UNSUPPORTED;
+            default -> type;
+        };
         boolean aggregatable = first.isAggregatable();
         EsField.TimeSeriesFieldType timeSeriesFieldType = EsField.TimeSeriesFieldType.fromIndexFieldCapabilities(first);
         if (rest.isEmpty() == false) {
             for (IndexFieldCapabilities fc : rest) {
                 if (first.metricType() != fc.metricType()) {
-                    return conflictingMetricTypes(name, fullName, fieldCapsResponse);
+                    return conflictingMetricTypes(name, fullName, fieldsInfo.caps);
                 }
                 try {
                     timeSeriesFieldType = timeSeriesFieldType.merge(EsField.TimeSeriesFieldType.fromIndexFieldCapabilities(fc));
@@ -239,7 +254,7 @@ public class IndexResolver {
             }
             for (IndexFieldCapabilities fc : rest) {
                 if (type != EsqlDataTypeRegistry.INSTANCE.fromEs(fc.type(), fc.metricType())) {
-                    return conflictingTypes(name, fullName, fieldCapsResponse);
+                    return conflictingTypes(name, fullName, fieldsInfo.caps);
                 }
             }
             for (IndexFieldCapabilities fc : rest) {

+ 108 - 33
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java

@@ -37,6 +37,7 @@ import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
 import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
 import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
 import org.elasticsearch.xpack.esql.core.expression.UnresolvedAttribute;
+import org.elasticsearch.xpack.esql.core.plugin.EsqlCorePlugin;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.type.EsField;
@@ -136,6 +137,7 @@ import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.randomInfe
 import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.tsdbIndexResolution;
 import static org.elasticsearch.xpack.esql.core.plugin.EsqlCorePlugin.DENSE_VECTOR_FEATURE_FLAG;
 import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY;
+import static org.elasticsearch.xpack.esql.core.type.DataType.AGGREGATE_METRIC_DOUBLE;
 import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME;
 import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS;
 import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_PERIOD;
@@ -3175,12 +3177,16 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(
-                    fieldCapabilitiesIndexResponse("foo", messageResponseMap("keyword")),
-                    fieldCapabilitiesIndexResponse("bar", Map.of())
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("keyword")),
+                        fieldCapabilitiesIndexResponse("bar", Map.of())
+                    ),
+                    List.of()
                 ),
-                List.of()
+                true,
+                true
             )
         );
 
@@ -3198,9 +3204,16 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), fieldCapabilitiesIndexResponse("bar", Map.of())),
-                List.of()
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
+                        fieldCapabilitiesIndexResponse("bar", Map.of())
+                    ),
+                    List.of()
+                ),
+                true,
+                true
             )
         );
         var plan = analyze("FROM foo, bar | INSIST_🐔 message", analyzer(resolution, TEST_VERIFIER));
@@ -3219,13 +3232,17 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(
-                    fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
-                    fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
-                    fieldCapabilitiesIndexResponse("bazz", Map.of())
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
+                        fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
+                        fieldCapabilitiesIndexResponse("bazz", Map.of())
+                    ),
+                    List.of()
                 ),
-                List.of()
+                true,
+                true
             )
         );
         var plan = analyze("FROM foo, bar | INSIST_🐔 message", analyzer(resolution, TEST_VERIFIER));
@@ -3243,12 +3260,16 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(
-                    fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
-                    fieldCapabilitiesIndexResponse("bar", messageResponseMap("long"))
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
+                        fieldCapabilitiesIndexResponse("bar", messageResponseMap("long"))
+                    ),
+                    List.of()
                 ),
-                List.of()
+                true,
+                true
             )
         );
         var plan = analyze("FROM foo, bar | INSIST_🐔 message", analyzer(resolution, TEST_VERIFIER));
@@ -3264,14 +3285,18 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(
-                    fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
-                    fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
-                    fieldCapabilitiesIndexResponse("bazz", messageResponseMap("keyword")),
-                    fieldCapabilitiesIndexResponse("qux", Map.of())
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
+                        fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
+                        fieldCapabilitiesIndexResponse("bazz", messageResponseMap("keyword")),
+                        fieldCapabilitiesIndexResponse("qux", Map.of())
+                    ),
+                    List.of()
                 ),
-                List.of()
+                true,
+                true
             )
         );
         var plan = analyze("FROM foo, bar | INSIST_🐔 message", analyzer(resolution, TEST_VERIFIER));
@@ -3289,13 +3314,17 @@ public class AnalyzerTests extends ESTestCase {
 
         IndexResolution resolution = IndexResolver.mergedMappings(
             "foo, bar",
-            new FieldCapabilitiesResponse(
-                List.of(
-                    fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
-                    fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
-                    fieldCapabilitiesIndexResponse("bazz", Map.of())
+            new IndexResolver.FieldsInfo(
+                new FieldCapabilitiesResponse(
+                    List.of(
+                        fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")),
+                        fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")),
+                        fieldCapabilitiesIndexResponse("bazz", Map.of())
+                    ),
+                    List.of()
                 ),
-                List.of()
+                true,
+                true
             )
         );
         VerificationException e = expectThrows(
@@ -3309,6 +3338,52 @@ public class AnalyzerTests extends ESTestCase {
         );
     }
 
+    public void testResolveDenseVector() {
+        FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(
+            List.of(fieldCapabilitiesIndexResponse("foo", Map.of("v", new IndexFieldCapabilitiesBuilder("v", "dense_vector").build()))),
+            List.of()
+        );
+        {
+            IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true));
+            var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER));
+            assertThat(plan.output(), hasSize(1));
+            assertThat(plan.output().getFirst().dataType(), equalTo(DENSE_VECTOR_FEATURE_FLAG.isEnabled() ? DENSE_VECTOR : UNSUPPORTED));
+        }
+        {
+            IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, false));
+            var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER));
+            assertThat(plan.output(), hasSize(1));
+            assertThat(plan.output().getFirst().dataType(), equalTo(UNSUPPORTED));
+        }
+    }
+
+    public void testResolveAggregateMetricDouble() {
+        FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(
+            List.of(
+                fieldCapabilitiesIndexResponse(
+                    "foo",
+                    Map.of("v", new IndexFieldCapabilitiesBuilder("v", "aggregate_metric_double").build())
+                )
+            ),
+            List.of()
+        );
+        {
+            IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true));
+            var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER));
+            assertThat(plan.output(), hasSize(1));
+            assertThat(
+                plan.output().getFirst().dataType(),
+                equalTo(EsqlCorePlugin.AGGREGATE_METRIC_DOUBLE_FEATURE_FLAG.isEnabled() ? AGGREGATE_METRIC_DOUBLE : UNSUPPORTED)
+            );
+        }
+        {
+            IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, false, true));
+            var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER));
+            assertThat(plan.output(), hasSize(1));
+            assertThat(plan.output().getFirst().dataType(), equalTo(UNSUPPORTED));
+        }
+    }
+
     public void testBasicFork() {
         LogicalPlan plan = analyze("""
             from test
@@ -3756,7 +3831,7 @@ public class AnalyzerTests extends ESTestCase {
         List<FieldCapabilitiesIndexResponse> idxResponses = List.of(
             new FieldCapabilitiesIndexResponse("idx", "idx", Map.of(), true, IndexMode.STANDARD)
         );
-        FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(idxResponses, List.of());
+        IndexResolver.FieldsInfo caps = new IndexResolver.FieldsInfo(new FieldCapabilitiesResponse(idxResponses, List.of()), true, true);
         IndexResolution resolution = IndexResolver.mergedMappings("test*", caps);
         var analyzer = analyzer(resolution, TEST_VERIFIER, configuration(query));
         return analyze(query, analyzer);

+ 1 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java

@@ -54,7 +54,7 @@ public class EsqlDataTypeRegistryTests extends ESTestCase {
 
         FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(idxResponses, List.of());
         // IndexResolver uses EsqlDataTypeRegistry directly
-        IndexResolution resolution = IndexResolver.mergedMappings("idx-*", caps);
+        IndexResolution resolution = IndexResolver.mergedMappings("idx-*", new IndexResolver.FieldsInfo(caps, true, true));
         EsField f = resolution.get().mapping().get(field);
         assertThat(f.getDataType(), equalTo(expected));
     }

+ 86 - 29
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/40_tsdb.yml

@@ -231,10 +231,21 @@ filter on counter without cast:
       catch: bad_request
       esql.query:
         body:
-          query: 'from test | where k8s.pod.network.tx == 1434577921'
+          query: |
+            FROM test
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | WHERE k8s.pod.network.tx == 1434577921
 
 ---
 cast counter then filter:
+  - requires:
+      test_runner_features: [ capabilities ]
+      capabilities:
+        - method: POST
+          path: /_query
+          parameters: [ ]
+          capabilities: [ aggregate_metric_double_convert_to ]
+      reason: "Uses TO_AGGREGATE_METRIC_DOUBLE"
   - do:
       esql.query:
         body:
@@ -257,13 +268,18 @@ sort on counter without cast:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [sorting_on_source_and_counters_forbidden]
+          capabilities: [sorting_on_source_and_counters_forbidden, aggregate_metric_double_convert_to]
       reason: "Sorting on counters shouldn't have been possible"
   - do:
       catch: /cannot sort on counter_long/
       esql.query:
         body:
-          query: 'from test |  KEEP k8s.pod.network.tx | sort k8s.pod.network.tx | limit 1'
+          query: |
+            FROM test
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | KEEP k8s.pod.network.tx
+            | SORT k8s.pod.network.tx
+            | LIMIT 1
 
 ---
 cast then sort on counter:
@@ -282,14 +298,17 @@ from doc with aggregate_metric_double:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double]
+          capabilities: [aggregate_metric_double, aggregate_metric_double_convert_to]
       reason: "Support for aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'from test2'
+          query: |
+            FROM test2
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
 
   - match: {columns.0.name: "@timestamp"}
   - match: {columns.0.type: "date"}
@@ -311,14 +330,17 @@ stats on aggregate_metric_double:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double]
+          capabilities: [aggregate_metric_double, aggregate_metric_double_convert_to]
       reason: "Support for aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test2 | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric)'
+          query: |
+            FROM test2
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric)
   - length: {values: 1}
   - length: {values.0: 4}
   - match: {columns.0.name: "max(agg_metric)"}
@@ -342,16 +364,18 @@ grouping stats on aggregate_metric_double:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double]
+          capabilities: [aggregate_metric_double, aggregate_metric_double_convert_to]
       reason: "Support for aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: "FROM test2
-          | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric) BY dim
-          | SORT dim"
+          query: |
+            FROM test2
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric) BY dim
+            | SORT dim
   - length: {values: 2}
   - length: {values.0: 5}
   - match: {columns.0.name: "max(agg_metric)"}
@@ -383,14 +407,18 @@ sorting with aggregate_metric_double with partial submetrics:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double_sorting]
+          capabilities: [aggregate_metric_double_sorting, aggregate_metric_double_convert_to]
       reason: "Support for sorting when aggregate_metric_double present"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test3 | SORT @timestamp | KEEP @timestamp, agg_metric'
+          query: |
+            FROM test3
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | SORT @timestamp
+            | KEEP @timestamp, agg_metric
 
   - length: {values: 4}
   - length: {values.0: 2}
@@ -415,13 +443,17 @@ aggregate_metric_double unsortable:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double_sorting]
+          capabilities: [aggregate_metric_double_sorting, aggregate_metric_double_convert_to]
       reason: "Support for sorting when aggregate_metric_double present"
   - do:
       catch: /cannot sort on aggregate_metric_double/
       esql.query:
         body:
-          query: 'FROM test2 | sort agg_metric'
+          query: |
+            FROM test2
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
+            | SORT agg_metric
 
 ---
 stats on aggregate_metric_double with partial submetrics:
@@ -431,14 +463,18 @@ stats on aggregate_metric_double with partial submetrics:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double_partial_submetrics]
+          capabilities: [aggregate_metric_double_partial_submetrics, aggregate_metric_double_convert_to]
       reason: "Support for partial submetrics in aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test3 | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric) BY k8s.pod.uid | SORT k8s.pod.uid'
+          query: |
+            FROM test3
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric) BY k8s.pod.uid
+            | SORT k8s.pod.uid
 
   - length: {values: 2}
   - length: {values.0: 5}
@@ -471,14 +507,17 @@ stats on aggregate_metric_double missing min and max:
         - method: POST
           path: /_query
           parameters: [ ]
-          capabilities: [ aggregate_metric_double_partial_submetrics ]
+          capabilities: [ aggregate_metric_double_partial_submetrics, aggregate_metric_double_convert_to ]
       reason: "Support for partial submetrics in aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test4 | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric)'
+          query: |
+            FROM test4
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | STATS max(agg_metric), min(agg_metric), sum(agg_metric), count(agg_metric)
 
   - length: {values: 1}
   - length: {values.0: 4}
@@ -503,14 +542,17 @@ render aggregate_metric_double when missing min and max:
         - method: POST
           path: /_query
           parameters: [ ]
-          capabilities: [ aggregate_metric_double_rendering ]
+          capabilities: [ aggregate_metric_double_rendering, aggregate_metric_double_convert_to ]
       reason: "Support for rendering aggregate_metric_doubles"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test4 | KEEP agg_metric'
+          query: |
+            FROM test4
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | KEEP agg_metric
 
   - length: {values: 1}
   - length: {values.0: 1}
@@ -527,14 +569,18 @@ render aggregate_metric_double when missing value:
         - method: POST
           path: /_query
           parameters: [ ]
-          capabilities: [ aggregate_metric_double_rendering ]
+          capabilities: [ aggregate_metric_double_rendering, aggregate_metric_double_convert_to ]
       reason: "Support for rendering aggregate_metric_doubles"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test3 | WHERE @timestamp == "2021-04-28T19:51:04.467Z" | KEEP agg_metric'
+          query: |
+            FROM test3
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | WHERE @timestamp == "2021-04-28T19:51:04.467Z"
+            | KEEP agg_metric
 
   - length: {values: 1}
   - length: {values.0: 1}
@@ -551,14 +597,18 @@ to_string aggregate_metric_double:
         - method: POST
           path: /_query
           parameters: [ ]
-          capabilities: [ aggregate_metric_double_rendering ]
+          capabilities: [ aggregate_metric_double_rendering, aggregate_metric_double_convert_to ]
       reason: "Support for rendering aggregate_metric_doubles"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test4 | EVAL agg = to_string(agg_metric) | KEEP agg'
+          query: |
+            FROM test4
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | EVAL agg = to_string(agg_metric)
+            | KEEP agg
 
   - length: {values: 1}
   - length: {values.0: 1}
@@ -574,14 +624,17 @@ from index pattern unsupported counter:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double_partial_submetrics]
+          capabilities: [aggregate_metric_double_partial_submetrics, aggregate_metric_double_convert_to]
       reason: "Support for partial submetrics in aggregate_metric_double"
   - do:
       allowed_warnings_regex:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test*'
+          query: |
+            FROM test*
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
 
   - match: {columns.0.name: "@timestamp"}
   - match: {columns.0.type: "date"}
@@ -716,7 +769,7 @@ avg of aggregate_metric_double:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double_avg]
+          capabilities: [aggregate_metric_double_avg, aggregate_metric_double_convert_to]
       reason: "support avg aggregations with aggregate metric double"
 
   - do:
@@ -724,7 +777,11 @@ avg of aggregate_metric_double:
         - "No limit defined, adding default limit of \\[.*\\]"
       esql.query:
         body:
-          query: 'FROM test2 | STATS avg = avg(agg_metric) | KEEP avg'
+          query: |
+            FROM test2
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | STATS avg = avg(agg_metric)
+            | KEEP avg
 
   - length: {values: 1}
   - length: {values.0: 1}

+ 12 - 18
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/40_unsupported_types.yml

@@ -145,7 +145,7 @@ unsupported:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [dense_vector_field_type]
+          capabilities: [dense_vector_field_type, dense_vector_agg_metric_double_if_fns]
       reason: "uses original_type"
 
   - do:
@@ -157,8 +157,7 @@ unsupported:
           query: 'from test'
 
   - match: { columns.0.name: aggregate_metric_double }
-  - match: { columns.0.type: aggregate_metric_double }
-  - is_false: columns.0.original_types
+  - match: { columns.0.type: unsupported }
   - match: { columns.1.name: binary }
   - match: { columns.1.type: unsupported }
   - match: { columns.1.original_types: [binary] }
@@ -170,7 +169,7 @@ unsupported:
   - match: { columns.4.name: date_range }
   - match: { columns.4.type: unsupported }
   - match: { columns.5.name: dense_vector }
-  - match: { columns.5.type: dense_vector }
+  - match: { columns.5.type: unsupported }
   - match: { columns.6.name: double_range }
   - match: { columns.6.type: unsupported }
   - match: { columns.7.name: float_range }
@@ -219,14 +218,12 @@ unsupported:
   - match: { columns.28.type: integer }
 
   - length: { values: 1 }
-  - match: { values.0.0: '{"min":1.0,"max":3.0,"sum":10.1,"value_count":5}' }
+  - match: { values.0.0: null }
   - match: { values.0.1: null }
   - match: { values.0.2: null }
   - match: { values.0.3: "2015-01-01T12:10:30.123456789Z" }
   - match: { values.0.4: null }
-  - match: { values.0.5.0: 0.5 }
-  - match: { values.0.5.1: 10.0 }
-  - match: { values.0.5.2: 6.0 }
+  - match: { values.0.5: null }
   - match: { values.0.6: null }
   - match: { values.0.7: null }
   - match: { values.0.8: "POINT (10.0 12.0)" }
@@ -258,8 +255,7 @@ unsupported:
         body:
           query: 'from test | limit 0'
   - match: { columns.0.name: aggregate_metric_double }
-  - match: { columns.0.type: aggregate_metric_double }
-  - is_false: columns.0.original_types
+  - match: { columns.0.type: unsupported }
   - match: { columns.1.name: binary }
   - match: { columns.1.type: unsupported }
   - match: { columns.1.original_types: [binary] }
@@ -271,7 +267,7 @@ unsupported:
   - match: { columns.4.name: date_range }
   - match: { columns.4.type: unsupported }
   - match: { columns.5.name: dense_vector }
-  - match: { columns.5.type: dense_vector }
+  - match: { columns.5.type: unsupported }
   - match: { columns.6.name: double_range }
   - match: { columns.6.type: unsupported }
   - match: { columns.7.name: float_range }
@@ -342,7 +338,7 @@ unsupported with sort:
         - method: POST
           path: /_query
           parameters: [ ]
-          capabilities: [ dense_vector_field_type ]
+          capabilities: [ dense_vector_field_type, dense_vector_agg_metric_double_if_fns ]
       reason: "support for sorting when dense_vector_field_type present"
 
   - do:
@@ -354,7 +350,7 @@ unsupported with sort:
           query: 'from test | sort some_doc.bar'
 
   - match: { columns.0.name: aggregate_metric_double }
-  - match: { columns.0.type: aggregate_metric_double }
+  - match: { columns.0.type: unsupported }
   - match: { columns.1.name: binary }
   - match: { columns.1.type: unsupported }
   - match: { columns.2.name: completion }
@@ -364,7 +360,7 @@ unsupported with sort:
   - match: { columns.4.name: date_range }
   - match: { columns.4.type: unsupported }
   - match: { columns.5.name: dense_vector }
-  - match: { columns.5.type: dense_vector }
+  - match: { columns.5.type: unsupported }
   - match: { columns.6.name: double_range }
   - match: { columns.6.type: unsupported }
   - match: { columns.7.name: float_range }
@@ -413,14 +409,12 @@ unsupported with sort:
   - match: { columns.28.type: integer }
 
   - length: { values: 1 }
-  - match: { values.0.0: '{"min":1.0,"max":3.0,"sum":10.1,"value_count":5}' }
+  - match: { values.0.0: null }
   - match: { values.0.1: null }
   - match: { values.0.2: null }
   - match: { values.0.3: "2015-01-01T12:10:30.123456789Z" }
   - match: { values.0.4: null }
-  - match: { values.0.5.0: 0.5 }
-  - match: { values.0.5.1: 10.0 }
-  - match: { values.0.5.2: 6.0 }
+  - match: { values.0.5: null }
   - match: { values.0.6: null }
   - match: { values.0.7: null }
   - match: { values.0.8: "POINT (10.0 12.0)" }

+ 57 - 29
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/46_downsample.yml

@@ -83,7 +83,7 @@ setup:
         - method: POST
           path: /_query
           parameters: []
-          capabilities: [aggregate_metric_double]
+          capabilities: [aggregate_metric_double, dense_vector_agg_metric_double_if_fns]
       reason: "Support for aggregate_metric_double"
   - do:
       indices.downsample:
@@ -98,9 +98,12 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-downsample |
-          STATS max(k8s.pod.network.rx), min(k8s.pod.network.rx), sum(k8s.pod.network.rx), count(k8s.pod.network.rx)
-          | LIMIT 100"
+          query: |
+            FROM test-downsample
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
+            | STATS max(k8s.pod.network.rx), min(k8s.pod.network.rx), sum(k8s.pod.network.rx), count(k8s.pod.network.rx)
+            | LIMIT 100
 
   - length: {values: 1}
   - length: {values.0: 4}
@@ -140,7 +143,13 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-downsample | WHERE @timestamp == \"2021-04-28T19:00:00.000Z\" | KEEP k8s.pod.network.rx | LIMIT 100"
+          query: |
+            FROM test-downsample
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
+            | WHERE @timestamp == "2021-04-28T19:00:00.000Z"
+            | KEEP k8s.pod.network.rx
+            | LIMIT 100
   - length: {values: 1}
   - length: {values.0: 1}
   - match: {columns.0.name: "k8s.pod.network.rx"}
@@ -231,11 +240,12 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-* |
-          WHERE k8s.pod.uid == \"947e4ced-1786-4e53-9e0c-5c447e959507\" |
-          EVAL rx = to_aggregate_metric_double(k8s.pod.network.rx) |
-          STATS max(rx), min(rx), sum(rx), count(rx) |
-          LIMIT 100"
+          query: |
+            FROM test-*
+            | WHERE k8s.pod.uid == "947e4ced-1786-4e53-9e0c-5c447e959507"
+            | EVAL rx = to_aggregate_metric_double(k8s.pod.network.rx)
+            | STATS max(rx), min(rx), sum(rx), count(rx)
+            | LIMIT 100
 
   - length: {values: 1}
   - length: {values.0: 4}
@@ -336,10 +346,12 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-* |
-          WHERE k8s.pod.uid == \"947e4ced-1786-4e53-9e0c-5c447e959507\" |
-          STATS max(k8s.pod.network.rx), min(k8s.pod.network.rx), sum(k8s.pod.network.rx), count(k8s.pod.network.rx), avg(k8s.pod.network.rx) |
-          LIMIT 100"
+          query: |
+            FROM test-*
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | WHERE k8s.pod.uid == \"947e4ced-1786-4e53-9e0c-5c447e959507\"
+            | STATS max(k8s.pod.network.rx), min(k8s.pod.network.rx), sum(k8s.pod.network.rx), count(k8s.pod.network.rx), avg(k8s.pod.network.rx)
+            | LIMIT 100
 
   - length: {values: 1}
   - length: {values.0: 5}
@@ -457,12 +469,14 @@ setup:
   - do:
       esql.query:
         body:
-          query: "TS test-* |
-          STATS avg = sum(avg_over_time(k8s.pod.network.rx)),
-                count = sum(count_over_time(k8s.pod.network.rx)),
-                sum = sum(sum_over_time(k8s.pod.network.rx))
-                BY time_bucket = bucket(@timestamp, 1 hour) |
-          SORT time_bucket | LIMIT 10"
+          query: |
+            TS test-*
+            | STATS avg = sum(avg_over_time(k8s.pod.network.rx)),
+                    count = sum(count_over_time(k8s.pod.network.rx)),
+                    sum = sum(sum_over_time(k8s.pod.network.rx))
+                 BY time_bucket = bucket(@timestamp, 1 hour)
+            | SORT time_bucket
+            | LIMIT 10
 
   - length: {values: 4}
   - length: {values.0: 4}
@@ -579,13 +593,14 @@ setup:
   - do:
       esql.query:
         body:
-          query: "TS test-* |
-          STATS avg = sum(avg_over_time(k8s.pod.network.rx)),
-                count = sum(count_over_time(k8s.pod.network.rx)),
-                sum = sum(sum_over_time(k8s.pod.network.rx))
-          BY k8s.pod.name, time_bucket = bucket(@timestamp, 1 hour) |
-          SORT time_bucket, k8s.pod.name |
-          LIMIT 10"
+          query: |
+            TS test-*
+            | STATS avg = sum(avg_over_time(k8s.pod.network.rx)),
+                    count = sum(count_over_time(k8s.pod.network.rx)),
+                    sum = sum(sum_over_time(k8s.pod.network.rx))
+                 BY k8s.pod.name, time_bucket = bucket(@timestamp, 1 hour)
+            | SORT time_bucket, k8s.pod.name
+            |LIMIT 10
 
   - length: {values: 6}
   - length: {values.0: 5}
@@ -671,7 +686,13 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-* | SORT some_field, @timestamp, k8s.pod.uid | KEEP k8s.pod.network.rx, some_field, @timestamp | LIMIT 10"
+          query: |
+            FROM test-*
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
+            | SORT some_field, @timestamp, k8s.pod.uid
+            | KEEP k8s.pod.network.rx, some_field, @timestamp
+            | LIMIT 10
 
   - length: {values: 5}
   - length: {values.0: 3}
@@ -721,7 +742,14 @@ setup:
   - do:
       esql.query:
         body:
-          query: "FROM test-downsample | MV_EXPAND k8s.pod.network.rx | SORT @timestamp, k8s.pod.uid | KEEP k8s.pod.network.rx, @timestamp | LIMIT 10"
+          query: |
+            FROM test-downsample
+            | EVAL a = TO_AGGREGATE_METRIC_DOUBLE(1)  // Temporary workaround to enable aggregate_metric_double
+            | DROP a
+            | MV_EXPAND k8s.pod.network.rx
+            | SORT @timestamp, k8s.pod.uid
+            | KEEP k8s.pod.network.rx, @timestamp
+            | LIMIT 10
 
   - length: {values: 4}
   - length: {values.0: 2}