diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java index 7d79e00406637..05535323bfeeb 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/_nightly/esql/QueryPlanningBenchmark.java @@ -14,8 +14,13 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexMode; import org.elasticsearch.license.XPackLicenseState; -import org.elasticsearch.xpack.esql.analysis.*; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.AnalyzerSettings; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.Verifier; import org.elasticsearch.xpack.esql.core.expression.FoldContext; +import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.core.util.DateUtils; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; @@ -26,6 +31,7 @@ import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.parser.QueryParams; +import org.elasticsearch.xpack.esql.plan.IndexPattern; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plugin.QueryPragmas; import org.elasticsearch.xpack.esql.session.Configuration; @@ -108,7 +114,7 @@ public void setup() { new AnalyzerContext( config, functionRegistry, - IndexResolution.valid(esIndex), + Map.of(new IndexPattern(Source.EMPTY, esIndex.name()), IndexResolution.valid(esIndex)), Map.of(), new EnrichResolution(), InferenceResolution.EMPTY, diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java index 2ecd089dedd2f..bee67b1486192 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java @@ -51,6 +51,7 @@ import java.util.Locale; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.Callable; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.LongStream; @@ -122,23 +123,59 @@ protected EsqlSpecTestCase( this.mode = randomFrom(Mode.values()); } - private static boolean dataLoaded = false; + private static class Protected { + private volatile boolean completed = false; + private volatile boolean started = false; + private volatile Throwable failure = null; + + private void protectedBlock(Callable callable) { + if (completed) { + return; + } + // In case tests get run in parallel, we ensure only one setup is run, and other tests wait for this + synchronized (this) { + if (completed) { + return; + } + if (started) { + // Should only happen if a previous test setup failed, possibly with partial setup, let's fail fast the current test + if (failure != null) { + fail(failure, "Previous test setup failed: " + failure.getMessage()); + } + fail("Previous test setup failed with unknown error"); + } + started = true; + try { + callable.call(); + completed = true; + } catch (Throwable t) { + failure = t; + fail(failure, "Current test setup failed: " + failure.getMessage()); + } + } + } + + private synchronized void reset() { + completed = false; + started = false; + failure = null; + } + } + + private static final Protected INGEST = new Protected(); protected static boolean testClustersOk = true; @Before - public void setup() throws IOException { + public void setup() { assumeTrue("test clusters were broken", testClustersOk); - boolean supportsLookup = supportsIndexModeLookup(); - boolean supportsSourceMapping = supportsSourceFieldMapping(); - boolean supportsInferenceTestService = supportsInferenceTestService(); - if (dataLoaded == false) { - if (supportsInferenceTestService) { + INGEST.protectedBlock(() -> { + // Inference endpoints must be created before ingesting any datasets that rely on them (mapping of inference_id) + if (supportsInferenceTestService()) { createInferenceEndpoints(adminClient()); } - - loadDataSetIntoEs(client(), supportsLookup, supportsSourceMapping, supportsInferenceTestService); - dataLoaded = true; - } + loadDataSetIntoEs(client(), supportsIndexModeLookup(), supportsSourceFieldMapping(), supportsInferenceTestService()); + return null; + }); } @AfterClass @@ -147,7 +184,6 @@ public static void wipeTestData() throws IOException { return; } try { - dataLoaded = false; adminClient().performRequest(new Request("DELETE", "/*")); } catch (ResponseException e) { // 404 here just means we had no indexes @@ -155,7 +191,7 @@ public static void wipeTestData() throws IOException { throw e; } } - + INGEST.reset(); deleteInferenceEndpoints(adminClient()); } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java index 96f8ffdcccbbe..ef3514308a436 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java @@ -78,8 +78,9 @@ public class CsvTestsDataLoader { private static final TestDataset LANGUAGES = new TestDataset("languages"); private static final TestDataset LANGUAGES_LOOKUP = LANGUAGES.withIndex("languages_lookup").withSetting("lookup-settings.json"); private static final TestDataset LANGUAGES_NON_UNIQUE_KEY = new TestDataset("languages_non_unique_key"); - private static final TestDataset LANGUAGES_LOOKUP_NON_UNIQUE_KEY = LANGUAGES_NON_UNIQUE_KEY.withIndex("languages_lookup_non_unique_key") - .withSetting("lookup-settings.json"); + private static final TestDataset LANGUAGES_LOOKUP_NON_UNIQUE_KEY = LANGUAGES_LOOKUP.withIndex("languages_lookup_non_unique_key") + .withData("languages_non_unique_key.csv") + .withDynamicTypeMapping(Map.of("country", "text")); private static final TestDataset LANGUAGES_NESTED_FIELDS = new TestDataset( "languages_nested_fields", "mapping-languages_nested_fields.json", @@ -423,11 +424,13 @@ private static void loadDataSetIntoEs( Logger logger = LogManager.getLogger(CsvTestsDataLoader.class); Set loadedDatasets = new HashSet<>(); + logger.info("Loading test datasets"); for (var dataset : availableDatasetsForEs(supportsIndexModeLookup, supportsSourceFieldMapping, inferenceEnabled, timeSeriesOnly)) { load(client, dataset, logger, indexCreator); loadedDatasets.add(dataset.indexName); } forceMerge(client, loadedDatasets, logger); + logger.info("Loading enrich policies"); for (var policy : ENRICH_POLICIES) { loadEnrichPolicy(client, policy.policyName, policy.policyFileName, logger); } @@ -569,6 +572,7 @@ private static void deleteInferenceEndpoint(RestClient client, String inferenceI } private static void loadEnrichPolicy(RestClient client, String policyName, String policyFileName, Logger logger) throws IOException { + logger.info("Loading enrich policy [{}] from file [{}]", policyName, policyFileName); URL policyMapping = getResource("/" + policyFileName); String entity = readTextFile(policyMapping); Request request = new Request("PUT", "/_enrich/policy/" + policyName); @@ -588,6 +592,7 @@ private static URL getResource(String name) { } private static void load(RestClient client, TestDataset dataset, Logger logger, IndexCreator indexCreator) throws IOException { + logger.info("Loading dataset [{}] into ES index [{}]", dataset.dataFileName, dataset.indexName); URL mapping = getResource("/" + dataset.mappingFileName); Settings indexSettings = dataset.readSettingsFile(); indexCreator.createIndex(client, dataset.indexName, readMappingFile(mapping, dataset.typeMapping), indexSettings); @@ -854,15 +859,16 @@ public record TestDataset( String dataFileName, String settingFileName, boolean allowSubFields, - @Nullable Map typeMapping, + @Nullable Map typeMapping, // Override mappings read from mappings file + @Nullable Map dynamicTypeMapping, // Define mappings not in the mapping files, but available from field-caps boolean requiresInferenceEndpoint ) { public TestDataset(String indexName, String mappingFileName, String dataFileName) { - this(indexName, mappingFileName, dataFileName, null, true, null, false); + this(indexName, mappingFileName, dataFileName, null, true, null, null, false); } public TestDataset(String indexName) { - this(indexName, "mapping-" + indexName + ".json", indexName + ".csv", null, true, null, false); + this(indexName, "mapping-" + indexName + ".json", indexName + ".csv", null, true, null, null, false); } public TestDataset withIndex(String indexName) { @@ -873,6 +879,7 @@ public TestDataset withIndex(String indexName) { settingFileName, allowSubFields, typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } @@ -885,6 +892,7 @@ public TestDataset withData(String dataFileName) { settingFileName, allowSubFields, typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } @@ -897,6 +905,7 @@ public TestDataset noData() { settingFileName, allowSubFields, typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } @@ -909,6 +918,7 @@ public TestDataset withSetting(String settingFileName) { settingFileName, allowSubFields, typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } @@ -921,6 +931,7 @@ public TestDataset noSubfields() { settingFileName, false, typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } @@ -933,12 +944,35 @@ public TestDataset withTypeMapping(Map typeMapping) { settingFileName, allowSubFields, typeMapping, + dynamicTypeMapping, + requiresInferenceEndpoint + ); + } + + public TestDataset withDynamicTypeMapping(Map dynamicTypeMapping) { + return new TestDataset( + indexName, + mappingFileName, + dataFileName, + settingFileName, + allowSubFields, + typeMapping, + dynamicTypeMapping, requiresInferenceEndpoint ); } public TestDataset withInferenceEndpoint(boolean needsInference) { - return new TestDataset(indexName, mappingFileName, dataFileName, settingFileName, allowSubFields, typeMapping, needsInference); + return new TestDataset( + indexName, + mappingFileName, + dataFileName, + settingFileName, + allowSubFields, + typeMapping, + dynamicTypeMapping, + needsInference + ); } private Settings readSettingsFile() throws IOException { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index acf4492bcc1aa..a0f40813bb99f 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -102,6 +102,7 @@ import org.elasticsearch.xpack.esql.inference.InferenceService; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.parser.QueryParam; +import org.elasticsearch.xpack.esql.plan.IndexPattern; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.EsRelation; import org.elasticsearch.xpack.esql.plan.logical.Explain; @@ -451,11 +452,11 @@ public static TransportVersion randomMinimumVersion() { public static AnalyzerContext testAnalyzerContext( Configuration configuration, EsqlFunctionRegistry functionRegistry, - IndexResolution indexResolution, + Map indexResolutions, EnrichResolution enrichResolution, InferenceResolution inferenceResolution ) { - return testAnalyzerContext(configuration, functionRegistry, indexResolution, Map.of(), enrichResolution, inferenceResolution); + return testAnalyzerContext(configuration, functionRegistry, indexResolutions, Map.of(), enrichResolution, inferenceResolution); } /** @@ -464,7 +465,7 @@ public static AnalyzerContext testAnalyzerContext( public static AnalyzerContext testAnalyzerContext( Configuration configuration, EsqlFunctionRegistry functionRegistry, - IndexResolution indexResolution, + Map indexResolutions, Map lookupResolution, EnrichResolution enrichResolution, InferenceResolution inferenceResolution @@ -472,7 +473,7 @@ public static AnalyzerContext testAnalyzerContext( return new AnalyzerContext( configuration, functionRegistry, - indexResolution, + indexResolutions, lookupResolution, enrichResolution, inferenceResolution, diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/inlinestats.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/inlinestats.csv-spec index 7ec65467e7e98..586a0d87cee71 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/inlinestats.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/inlinestats.csv-spec @@ -4150,6 +4150,7 @@ required_capability: inline_stats required_capability: fix_join_output_merging FROM languages_lookup_non_unique_key +| EVAL country = MV_SORT(country) | KEEP country, language_name | EVAL language_code = null::integer | INLINE STATS MAX(language_code) BY language_code @@ -4157,7 +4158,7 @@ FROM languages_lookup_non_unique_key | LIMIT 5 ; -country:text |language_name:keyword |MAX(language_code):integer |language_code:integer +country:keyword |language_name:keyword |MAX(language_code):integer |language_code:integer Atlantis |null |null |null [Austria, Germany]|German |null |null Canada |English |null |null diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec index 610269f63b4be..b4318ee4687da 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec @@ -247,11 +247,12 @@ FROM employees | EVAL language_code = emp_no % 10 | LOOKUP JOIN languages_lookup_non_unique_key ON language_code | WHERE emp_no > 10090 AND emp_no < 10096 +| EVAL country = MV_SORT(country) | SORT emp_no, country | KEEP emp_no, language_code, language_name, country ; -emp_no:integer | language_code:integer | language_name:keyword | country:text +emp_no:integer | language_code:integer | language_name:keyword | country:keyword 10091 | 1 | English | Canada 10091 | 1 | null | United Kingdom 10091 | 1 | English | United States of America @@ -272,11 +273,12 @@ FROM employees | LIMIT 5 | EVAL language_code = emp_no % 10 | LOOKUP JOIN languages_lookup_non_unique_key ON language_code +| EVAL country = MV_SORT(country) | KEEP emp_no, language_code, language_name, country ; ignoreOrder:true -emp_no:integer | language_code:integer | language_name:keyword | country:text +emp_no:integer | language_code:integer | language_name:keyword | country:keyword 10001 | 1 | English | Canada 10001 | 1 | English | null 10001 | 1 | null | United Kingdom @@ -324,7 +326,7 @@ ROW language_code = 2 ignoreOrder:true language_code:integer | country:text | language_name:keyword -2 | [Austria, Germany] | German +2 | [Germany, Austria] | German 2 | Switzerland | German 2 | null | German ; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java index beb9015097415..a1774c5ddcf77 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java @@ -247,12 +247,10 @@ private static class ResolveTable extends ParameterizedAnalyzerRule resolveUsingColumns(List cols, List childrenOutput, IndexResolution indexResolution) { + private LogicalPlan resolveInsist(Insist insist, List childrenOutput, AnalyzerContext context) { List list = new ArrayList<>(); + List resolutions = collectIndexResolutions(insist, context); for (Attribute a : insist.insistedAttributes()) { - list.add(resolveInsistAttribute(a, childrenOutput, indexResolution)); + list.add(resolveInsistAttribute(a, childrenOutput, resolutions)); } return insist.withAttributes(list); } - private Attribute resolveInsistAttribute(Attribute attribute, List childrenOutput, IndexResolution indexResolution) { + private List collectIndexResolutions(LogicalPlan plan, AnalyzerContext context) { + List resolutions = new ArrayList<>(); + plan.forEachDown(EsRelation.class, e -> { + var resolution = context.indexResolution().get(new IndexPattern(e.source(), e.indexPattern())); + if (resolution != null) { + resolutions.add(resolution); + } + }); + return resolutions; + } + + private Attribute resolveInsistAttribute(Attribute attribute, List childrenOutput, List indices) { Attribute resolvedCol = maybeResolveAttribute((UnresolvedAttribute) attribute, childrenOutput); // Field isn't mapped anywhere. if (resolvedCol instanceof UnresolvedAttribute) { @@ -974,7 +985,8 @@ private Attribute resolveInsistAttribute(Attribute attribute, List ch } // Field is partially unmapped. - if (resolvedCol instanceof FieldAttribute fa && indexResolution.get().isPartiallyUnmappedField(fa.name())) { + // TODO: Should the check for partially unmapped fields be done specific to each sub-query in a fork? + if (resolvedCol instanceof FieldAttribute fa && indices.stream().anyMatch(r -> r.get().isPartiallyUnmappedField(fa.name()))) { return fa.dataType() == KEYWORD ? insistKeyword(fa) : invalidInsistAttribute(fa); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java index da74cd2bd779c..adebb69407e15 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java @@ -11,6 +11,7 @@ import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.inference.InferenceResolution; +import org.elasticsearch.xpack.esql.plan.IndexPattern; import org.elasticsearch.xpack.esql.session.Configuration; import org.elasticsearch.xpack.esql.session.EsqlSession; @@ -19,7 +20,7 @@ public record AnalyzerContext( Configuration configuration, EsqlFunctionRegistry functionRegistry, - IndexResolution indexResolution, + Map indexResolution, Map lookupResolution, EnrichResolution enrichResolution, InferenceResolution inferenceResolution, @@ -29,7 +30,7 @@ public record AnalyzerContext( public AnalyzerContext( Configuration configuration, EsqlFunctionRegistry functionRegistry, - IndexResolution indexResolution, + Map indexResolution, Map lookupResolution, EnrichResolution enrichResolution, InferenceResolution inferenceResolution, @@ -52,7 +53,7 @@ public AnalyzerContext(Configuration configuration, EsqlFunctionRegistry functio this( configuration, functionRegistry, - result.indices(), + result.indexResolution(), result.lookupIndices(), result.enrichResolution(), result.inferenceResolution(), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java index 2488172cb184a..baa9a9519b231 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java @@ -16,7 +16,9 @@ import org.elasticsearch.xpack.esql.plan.logical.UnresolvedRelation; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** * This class is part of the planner. Acts somewhat like a linker, to find the indices and enrich policies referenced by the query. @@ -24,14 +26,13 @@ public class PreAnalyzer { public record PreAnalysis( - IndexMode indexMode, - IndexPattern indexPattern, + Map indexes, List enriches, List lookupIndices, boolean supportsAggregateMetricDouble, boolean supportsDenseVector ) { - public static final PreAnalysis EMPTY = new PreAnalysis(null, null, List.of(), List.of(), false, false); + public static final PreAnalysis EMPTY = new PreAnalysis(Map.of(), List.of(), List.of(), false, false); } public PreAnalysis preAnalyze(LogicalPlan plan) { @@ -43,17 +44,19 @@ public PreAnalysis preAnalyze(LogicalPlan plan) { } protected PreAnalysis doPreAnalyze(LogicalPlan plan) { - Holder indexMode = new Holder<>(); - Holder indexPattern = new Holder<>(); + Map indexes = new HashMap<>(); List lookupIndices = new ArrayList<>(); plan.forEachUp(UnresolvedRelation.class, p -> { if (p.indexMode() == IndexMode.LOOKUP) { lookupIndices.add(p.indexPattern()); - } else if (indexMode.get() == null || indexMode.get() == p.indexMode()) { - indexMode.set(p.indexMode()); - indexPattern.set(p.indexPattern()); + } else if (indexes.containsKey(p.indexPattern()) == false || indexes.get(p.indexPattern()) == p.indexMode()) { + indexes.put(p.indexPattern(), p.indexMode()); } else { - throw new IllegalStateException("index mode is already set"); + IndexMode m1 = p.indexMode(); + IndexMode m2 = indexes.get(p.indexPattern()); + throw new IllegalStateException( + "index pattern '" + p.indexPattern() + "' found with with different index mode: " + m2 + " != " + m1 + ); } }); @@ -71,6 +74,11 @@ protected PreAnalysis doPreAnalyze(LogicalPlan plan) { */ Holder supportsAggregateMetricDouble = new Holder<>(false); Holder supportsDenseVector = new Holder<>(false); + indexes.forEach((ip, mode) -> { + if (mode == IndexMode.TIME_SERIES) { + supportsAggregateMetricDouble.set(true); + } + }); plan.forEachDown(p -> p.forEachExpression(UnresolvedFunction.class, fn -> { if (fn.name().equalsIgnoreCase("knn") || fn.name().equalsIgnoreCase("to_dense_vector") @@ -90,13 +98,6 @@ protected PreAnalysis doPreAnalyze(LogicalPlan plan) { // mark plan as preAnalyzed (if it were marked, there would be no analysis) plan.forEachUp(LogicalPlan::setPreAnalyzed); - return new PreAnalysis( - indexMode.get(), - indexPattern.get(), - unresolvedEnriches, - lookupIndices, - indexMode.get() == IndexMode.TIME_SERIES || supportsAggregateMetricDouble.get(), - supportsDenseVector.get() - ); + return new PreAnalysis(indexes, unresolvedEnriches, lookupIndices, supportsAggregateMetricDouble.get(), supportsDenseVector.get()); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java index 4f74561495c58..a97b13a61a71e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtils.java @@ -33,6 +33,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -186,7 +187,7 @@ static void updateExecutionInfoWithUnavailableClusters( static void updateExecutionInfoWithClustersWithNoMatchingIndices( EsqlExecutionInfo executionInfo, - IndexResolution indexResolution, + Collection indexResolutions, boolean usedFilter ) { if (executionInfo.clusterInfo.isEmpty()) { @@ -195,8 +196,10 @@ static void updateExecutionInfoWithClustersWithNoMatchingIndices( // Get the clusters which are still running, and we will check whether they have any matching indices. // NOTE: we assume that updateExecutionInfoWithUnavailableClusters() was already run and took care of unavailable clusters. final Set clustersWithNoMatchingIndices = executionInfo.getRunningClusterAliases().collect(toSet()); - for (String indexName : indexResolution.resolvedIndices()) { - clustersWithNoMatchingIndices.remove(RemoteClusterAware.parseClusterAlias(indexName)); + for (IndexResolution indexResolution : indexResolutions) { + for (String indexName : indexResolution.resolvedIndices()) { + clustersWithNoMatchingIndices.remove(RemoteClusterAware.parseClusterAlias(indexName)); + } } /* * Rules enforced at planning time around non-matching indices @@ -234,20 +237,22 @@ static void updateExecutionInfoWithClustersWithNoMatchingIndices( } } else { // We check for the valid resolution because if we have empty resolution it's still an error. - if (indexResolution.isValid()) { - List failures = indexResolution.failures().getOrDefault(c, List.of()); - // No matching indices, no concrete index requested, and no error in field-caps; just mark as done. - if (failures.isEmpty()) { - markClusterWithFinalStateAndNoShards(executionInfo, c, Cluster.Status.SUCCESSFUL, null); - } else { - // skip reporting index_not_found exceptions to avoid spamming users with such errors - // when queries use a remote cluster wildcard, e.g., `*:my-logs*`. - Exception nonIndexNotFound = failures.stream() - .map(FieldCapabilitiesFailure::getException) - .filter(ex -> ExceptionsHelper.unwrap(ex, IndexNotFoundException.class) == null) - .findAny() - .orElse(null); - markClusterWithFinalStateAndNoShards(executionInfo, c, Cluster.Status.SKIPPED, nonIndexNotFound); + for (IndexResolution indexResolution : indexResolutions) { + if (indexResolution.isValid()) { + List failures = indexResolution.failures().getOrDefault(c, List.of()); + // No matching indices, no concrete index requested, and no error in field-caps; just mark as done. + if (failures.isEmpty()) { + markClusterWithFinalStateAndNoShards(executionInfo, c, Cluster.Status.SUCCESSFUL, null); + } else { + // skip reporting index_not_found exceptions to avoid spamming users with such errors + // when queries use a remote cluster wildcard, e.g., `*:my-logs*`. + Exception nonIndexNotFound = failures.stream() + .map(FieldCapabilitiesFailure::getException) + .filter(ex -> ExceptionsHelper.unwrap(ex, IndexNotFoundException.class) == null) + .findAny() + .orElse(null); + markClusterWithFinalStateAndNoShards(executionInfo, c, Cluster.Status.SKIPPED, nonIndexNotFound); + } } } } @@ -258,8 +263,11 @@ static void updateExecutionInfoWithClustersWithNoMatchingIndices( } // Filter-less version, mainly for testing where we don't need filter support - static void updateExecutionInfoWithClustersWithNoMatchingIndices(EsqlExecutionInfo executionInfo, IndexResolution indexResolution) { - updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution, false); + static void updateExecutionInfoWithClustersWithNoMatchingIndices( + EsqlExecutionInfo executionInfo, + Set indexResolutions + ) { + updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolutions, false); } // visible for testing @@ -304,24 +312,31 @@ static void updateExecutionInfoAtEndOfPlanning(EsqlExecutionInfo execInfo) { /** * Checks the index expression for the presence of remote clusters. * If found, it will ensure that the caller has a valid Enterprise (or Trial) license on the querying cluster - * as well as initialize corresponding cluster state in execution info. + * as well as initialize the corresponding cluster state in execution info. * @throws org.elasticsearch.ElasticsearchStatusException if the license is not valid (or present) for ES|QL CCS search. */ public static void initCrossClusterState( IndicesExpressionGrouper indicesGrouper, XPackLicenseState licenseState, - IndexPattern indexPattern, + Set indexPatterns, EsqlExecutionInfo executionInfo ) throws ElasticsearchStatusException { - if (indexPattern == null) { + if (indexPatterns.isEmpty()) { return; } try { - var groupedIndices = indicesGrouper.groupIndices( - IndicesOptions.DEFAULT, - Strings.splitStringByCommaToArray(indexPattern.indexPattern()), - false - ); + // TODO it is not safe to concat multiple index patterns in case any of them contains exclusion. + // This is going to be resolved in #136804 + String[] indexExpressions = indexPatterns.stream() + .map(indexPattern -> Strings.splitStringByCommaToArray(indexPattern.indexPattern())) + .reduce((a, b) -> { + String[] combined = new String[a.length + b.length]; + System.arraycopy(a, 0, combined, 0, a.length); + System.arraycopy(b, 0, combined, a.length, b.length); + return combined; + }) + .get(); + var groupedIndices = indicesGrouper.groupIndices(IndicesOptions.DEFAULT, indexExpressions, false); executionInfo.clusterInfoInitializing(true); // initialize the cluster entries in EsqlExecutionInfo before throwing the invalid license error diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index d694588bf18c4..ed88903dd5e0f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -459,9 +459,24 @@ private EsqlStatement parse(String query, QueryParams params) { static void handleFieldCapsFailures( boolean allowPartialResults, EsqlExecutionInfo executionInfo, - Map> failures + Map indexResolutions ) throws Exception { FailureCollector failureCollector = new FailureCollector(); + for (IndexResolution indexResolution : indexResolutions.values()) { + handleFieldCapsFailures(allowPartialResults, executionInfo, indexResolution.failures(), failureCollector); + } + Exception failure = failureCollector.getFailure(); + if (failure != null) { + throw failure; + } + } + + static void handleFieldCapsFailures( + boolean allowPartialResults, + EsqlExecutionInfo executionInfo, + Map> failures, + FailureCollector failureCollector + ) throws Exception { for (var e : failures.entrySet()) { String clusterAlias = e.getKey(); EsqlExecutionInfo.Cluster cluster = executionInfo.getCluster(clusterAlias); @@ -491,10 +506,6 @@ static void handleFieldCapsFailures( ); } } - Exception failure = failureCollector.getFailure(); - if (failure != null) { - throw failure; - } } public void analyzedPlan( @@ -532,14 +543,19 @@ private void resolveIndicesAndAnalyze( PreAnalysisResult result, ActionListener> logicalPlanListener ) { - EsqlCCSUtils.initCrossClusterState(indicesExpressionGrouper, verifier.licenseState(), preAnalysis.indexPattern(), executionInfo); + EsqlCCSUtils.initCrossClusterState( + indicesExpressionGrouper, + verifier.licenseState(), + preAnalysis.indexes().keySet(), + executionInfo + ); // The main index pattern dictates on which nodes the query can be executed, so we use the minimum transport version from this field // caps request. SubscribableListener.newForked( - l -> preAnalyzeMainIndicesAndRetrieveMinTransportVersion(preAnalysis, executionInfo, result, requestFilter, l) + l -> preAnalyzeMainIndices(preAnalysis.indexes().entrySet().iterator(), preAnalysis, executionInfo, result, requestFilter, l) ).andThenApply(r -> { - if (r.indices.isValid() + if (r.indexResolution.isEmpty() == false // Rule out ROW case with no FROM clauses && executionInfo.isCrossClusterSearch() && executionInfo.getRunningClusterAliases().findAny().isEmpty()) { LOGGER.debug("No more clusters to search, ending analysis stage"); @@ -779,7 +795,35 @@ private void validateRemoteVersions(EsqlExecutionInfo executionInfo) { }); } - private void preAnalyzeMainIndicesAndRetrieveMinTransportVersion( + private void preAnalyzeMainIndices( + Iterator> indexPatterns, + PreAnalyzer.PreAnalysis preAnalysis, + EsqlExecutionInfo executionInfo, + PreAnalysisResult result, + QueryBuilder requestFilter, + ActionListener listener + ) { + if (indexPatterns.hasNext()) { + var index = indexPatterns.next(); + preAnalyzeMainIndices( + index.getKey(), + index.getValue(), + preAnalysis, + executionInfo, + result, + requestFilter, + listener.delegateFailureAndWrap((l, r) -> { + preAnalyzeMainIndices(indexPatterns, preAnalysis, executionInfo, r, requestFilter, l); + }) + ); + } else { + listener.onResponse(result); + } + } + + private void preAnalyzeMainIndices( + IndexPattern indexPattern, + IndexMode indexMode, PreAnalyzer.PreAnalysis preAnalysis, EsqlExecutionInfo executionInfo, PreAnalysisResult result, @@ -791,42 +835,36 @@ private void preAnalyzeMainIndicesAndRetrieveMinTransportVersion( ThreadPool.Names.SEARCH_COORDINATION, ThreadPool.Names.SYSTEM_READ ); - if (preAnalysis.indexPattern() != null) { - if (executionInfo.clusterAliases().isEmpty()) { - // return empty resolution if the expression is pure CCS and resolved no remote clusters (like no-such-cluster*:index) - listener.onResponse( - result.withIndices(IndexResolution.valid(new EsIndex(preAnalysis.indexPattern().indexPattern(), Map.of(), Map.of()))) - .withMinimumTransportVersion(TransportVersion.current()) - ); - } else { - indexResolver.resolveAsMergedMappingAndRetrieveMinimumVersion( - preAnalysis.indexPattern().indexPattern(), - result.fieldNames, - // Maybe if no indices are returned, retry without index mode and provide a clearer error message. - switch (preAnalysis.indexMode()) { - case IndexMode.TIME_SERIES -> { - var indexModeFilter = new TermQueryBuilder(IndexModeFieldMapper.NAME, IndexMode.TIME_SERIES.getName()); - yield requestFilter != null - ? new BoolQueryBuilder().filter(requestFilter).filter(indexModeFilter) - : indexModeFilter; - } - default -> requestFilter; - }, - preAnalysis.indexMode() == IndexMode.TIME_SERIES, - preAnalysis.supportsAggregateMetricDouble(), - preAnalysis.supportsDenseVector(), - listener.delegateFailureAndWrap((l, indexResolution) -> { - EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, indexResolution.inner().failures()); - l.onResponse( - result.withIndices(indexResolution.inner()).withMinimumTransportVersion(indexResolution.minimumVersion()) - ); - }) - ); - } - } else { - // occurs when dealing with local relations (row a = 1) + // TODO: This is not yet index specific, but that will not matter as soon as #136804 is dealt with + if (executionInfo.clusterAliases().isEmpty()) { + // return empty resolution if the expression is pure CCS and resolved no remote clusters (like no-such-cluster*:index) listener.onResponse( - result.withIndices(IndexResolution.invalid("[none specified]")).withMinimumTransportVersion(TransportVersion.current()) + result.withIndices(indexPattern, IndexResolution.valid(new EsIndex(indexPattern.indexPattern(), Map.of(), Map.of()))) + ); + } else { + indexResolver.resolveAsMergedMappingAndRetrieveMinimumVersion( + indexPattern.indexPattern(), + result.fieldNames, + // Maybe if no indices are returned, retry without index mode and provide a clearer error message. + switch (indexMode) { + case IndexMode.TIME_SERIES -> { + var indexModeFilter = new TermQueryBuilder(IndexModeFieldMapper.NAME, IndexMode.TIME_SERIES.getName()); + yield requestFilter != null + ? new BoolQueryBuilder().filter(requestFilter).filter(indexModeFilter) + : indexModeFilter; + } + default -> requestFilter; + }, + indexMode == IndexMode.TIME_SERIES, + preAnalysis.supportsAggregateMetricDouble(), + preAnalysis.supportsDenseVector(), + listener.delegateFailureAndWrap((l, indexResolution) -> { + EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, indexResolution.inner().failures()); + l.onResponse( + result.withIndices(indexPattern, indexResolution.inner()) + .withMinimumTransportVersion(indexResolution.minimumVersion()) + ); + }) ); } } @@ -843,10 +881,14 @@ private void analyzeWithRetry( ) { LOGGER.debug("Analyzing the plan ({})", description); try { - if (result.indices.isValid() || requestFilter != null) { + if (result.indexResolution.values().stream().anyMatch(IndexResolution::isValid) || requestFilter != null) { // We won't run this check with no filter and no valid indices since this may lead to false positive - missing index report // when the resolution result is not valid for a different reason. - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, result.indices, requestFilter != null); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices( + executionInfo, + result.indexResolution.values(), + requestFilter != null + ); } LogicalPlan plan = analyzedPlan(parsed, configuration, result, executionInfo); LOGGER.debug("Analyzed plan ({}):\n{}", description, plan); @@ -900,7 +942,7 @@ private PhysicalPlan logicalPlanToPhysicalPlan( private LogicalPlan analyzedPlan(LogicalPlan parsed, Configuration configuration, PreAnalysisResult r, EsqlExecutionInfo executionInfo) throws Exception { - handleFieldCapsFailures(configuration.allowPartialResults(), executionInfo, r.indices.failures()); + handleFieldCapsFailures(configuration.allowPartialResults(), executionInfo, r.indexResolution()); Analyzer analyzer = new Analyzer(new AnalyzerContext(configuration, functionRegistry, r), verifier); LogicalPlan plan = analyzer.analyze(parsed); plan.setAnalyzed(); @@ -946,7 +988,7 @@ private PhysicalPlan optimizedPhysicalPlan(LogicalPlan optimizedPlan, PhysicalPl public record PreAnalysisResult( Set fieldNames, Set wildcardJoinIndices, - IndexResolution indices, + Map indexResolution, Map lookupIndices, EnrichResolution enrichResolution, InferenceResolution inferenceResolution, @@ -954,21 +996,22 @@ public record PreAnalysisResult( ) { public PreAnalysisResult(Set fieldNames, Set wildcardJoinIndices) { - this(fieldNames, wildcardJoinIndices, null, new HashMap<>(), null, InferenceResolution.EMPTY, null); - } - - PreAnalysisResult withIndices(IndexResolution indices) { - return new PreAnalysisResult( + this( fieldNames, wildcardJoinIndices, - indices, - lookupIndices, - enrichResolution, - inferenceResolution, - minimumTransportVersion + new HashMap<>(), + new HashMap<>(), + null, + InferenceResolution.EMPTY, + TransportVersion.current() ); } + PreAnalysisResult withIndices(IndexPattern indexPattern, IndexResolution indices) { + indexResolution.put(indexPattern, indices); + return this; + } + PreAnalysisResult addLookupIndexResolution(String index, IndexResolution indexResolution) { lookupIndices.put(index, indexResolution); return this; @@ -978,7 +1021,7 @@ PreAnalysisResult withEnrichResolution(EnrichResolution enrichResolution) { return new PreAnalysisResult( fieldNames, wildcardJoinIndices, - indices, + indexResolution, lookupIndices, enrichResolution, inferenceResolution, @@ -990,7 +1033,7 @@ PreAnalysisResult withInferenceResolution(InferenceResolution inferenceResolutio return new PreAnalysisResult( fieldNames, wildcardJoinIndices, - indices, + indexResolution, lookupIndices, enrichResolution, inferenceResolution, @@ -999,10 +1042,16 @@ PreAnalysisResult withInferenceResolution(InferenceResolution inferenceResolutio } PreAnalysisResult withMinimumTransportVersion(TransportVersion minimumTransportVersion) { + if (this.minimumTransportVersion != null) { + if (this.minimumTransportVersion.equals(minimumTransportVersion)) { + return this; + } + minimumTransportVersion = TransportVersion.min(this.minimumTransportVersion, minimumTransportVersion); + } return new PreAnalysisResult( fieldNames, wildcardJoinIndices, - indices, + indexResolution, lookupIndices, enrichResolution, inferenceResolution, diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index f90547f57c0ff..e87f21bcdc7d0 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -60,6 +60,7 @@ import org.elasticsearch.xpack.esql.analysis.PreAnalyzer; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.FoldContext; +import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.core.type.InvalidMappedField; @@ -79,6 +80,7 @@ import org.elasticsearch.xpack.esql.optimizer.LogicalPreOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.TestLocalPhysicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.IndexPattern; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.physical.ChangePointExec; @@ -414,6 +416,16 @@ protected void assertResults(ExpectedResults expected, ActualResults actual, boo CsvAssert.assertResults(expected, actual, ignoreOrder, logger); } + private static Map loadIndexResolution( + Map datasets + ) { + Map indexResolutions = new HashMap<>(); + for (var entry : datasets.entrySet()) { + indexResolutions.put(entry.getKey(), loadIndexResolution(entry.getValue())); + } + return indexResolutions; + } + private static IndexResolution loadIndexResolution(CsvTestsDataLoader.MultiIndexTestDataset datasets) { var indexNames = datasets.datasets().stream().map(CsvTestsDataLoader.TestDataset::indexName); Map indexModes = indexNames.collect(Collectors.toMap(x -> x, x -> IndexMode.STANDARD)); @@ -429,21 +441,30 @@ private static IndexResolution loadIndexResolution(CsvTestsDataLoader.MultiIndex private static Map createMappingForIndex(CsvTestsDataLoader.TestDataset dataset) { var mapping = new TreeMap<>(loadMapping(dataset.mappingFileName())); - if (dataset.typeMapping() == null) { - return mapping; + if (dataset.typeMapping() != null) { + for (var entry : dataset.typeMapping().entrySet()) { + if (mapping.containsKey(entry.getKey())) { + DataType dataType = DataType.fromTypeName(entry.getValue()); + EsField field = mapping.get(entry.getKey()); + EsField editedField = new EsField( + field.getName(), + dataType, + field.getProperties(), + field.isAggregatable(), + field.getTimeSeriesFieldType() + ); + mapping.put(entry.getKey(), editedField); + } + } } - for (var entry : dataset.typeMapping().entrySet()) { - if (mapping.containsKey(entry.getKey())) { - DataType dataType = DataType.fromTypeName(entry.getValue()); - EsField field = mapping.get(entry.getKey()); - EsField editedField = new EsField( - field.getName(), - dataType, - field.getProperties(), - field.isAggregatable(), - field.getTimeSeriesFieldType() - ); - mapping.put(entry.getKey(), editedField); + // Add dynamic mappings, but only if they are not already mapped + if (dataset.dynamicTypeMapping() != null) { + for (var entry : dataset.dynamicTypeMapping().entrySet()) { + if (mapping.containsKey(entry.getKey()) == false) { + DataType dataType = DataType.fromTypeName(entry.getValue()); + EsField editedField = new EsField(entry.getKey(), dataType, Map.of(), false, EsField.TimeSeriesFieldType.NONE); + mapping.put(entry.getKey(), editedField); + } } } return mapping; @@ -526,7 +547,7 @@ private static EnrichPolicy loadEnrichPolicyMapping(String policyFileName) { private LogicalPlan analyzedPlan( LogicalPlan parsed, - CsvTestsDataLoader.MultiIndexTestDataset datasets, + Map datasets, TransportVersion minimumVersion ) { var indexResolution = loadIndexResolution(datasets); @@ -549,46 +570,65 @@ private LogicalPlan analyzedPlan( return plan; } - private static CsvTestsDataLoader.MultiIndexTestDataset testDatasets(LogicalPlan parsed) { + private Map testDatasets(LogicalPlan parsed) { var preAnalysis = new PreAnalyzer().preAnalyze(parsed); - if (preAnalysis.indexPattern() == null) { + if (preAnalysis.indexes().isEmpty()) { // If the data set doesn't matter we'll just grab one we know works. Employees is fine. - return CsvTestsDataLoader.MultiIndexTestDataset.of(CSV_DATASET_MAP.get("employees")); + return Map.of( + new IndexPattern(Source.EMPTY, "employees"), + CsvTestsDataLoader.MultiIndexTestDataset.of(CSV_DATASET_MAP.get("employees")) + ); } - String indexName = preAnalysis.indexPattern().indexPattern(); - List datasets = new ArrayList<>(); - if (indexName.endsWith("*")) { - String indexPrefix = indexName.substring(0, indexName.length() - 1); - for (var entry : CSV_DATASET_MAP.entrySet()) { - if (entry.getKey().startsWith(indexPrefix)) { - datasets.add(entry.getValue()); + List missing = new ArrayList<>(); + Map all = new HashMap<>(); + for (IndexPattern indexPattern : preAnalysis.indexes().keySet()) { + List datasets = new ArrayList<>(); + String indexName = indexPattern.indexPattern(); + if (indexName.endsWith("*")) { + String indexPrefix = indexName.substring(0, indexName.length() - 1); + for (var entry : CSV_DATASET_MAP.entrySet()) { + if (entry.getKey().startsWith(indexPrefix)) { + datasets.add(entry.getValue()); + } } - } - } else { - for (String index : indexName.split(",")) { - var dataset = CSV_DATASET_MAP.get(index); - if (dataset == null) { - throw new IllegalArgumentException("unknown CSV dataset for table [" + index + "]"); + } else { + for (String index : indexName.split(",")) { + var dataset = CSV_DATASET_MAP.get(index); + if (dataset == null) { + throw new IllegalArgumentException("unknown CSV dataset for table [" + index + "]"); + } + datasets.add(dataset); } - datasets.add(dataset); } + if (datasets.isEmpty() == false) { + all.put(indexPattern, new CsvTestsDataLoader.MultiIndexTestDataset(indexName, datasets)); + } else { + missing.add(indexName); + } + } + if (all.isEmpty()) { + throw new IllegalArgumentException("Found no CSV datasets for table [" + preAnalysis.indexes() + "]"); } - if (datasets.isEmpty()) { - throw new IllegalArgumentException("unknown CSV dataset for table [" + indexName + "]"); + if (missing.isEmpty() == false) { + throw new IllegalArgumentException("Did not find datasets for tables: " + missing); } - return new CsvTestsDataLoader.MultiIndexTestDataset(indexName, datasets); + return all; } private static TestPhysicalOperationProviders testOperationProviders( FoldContext foldCtx, - CsvTestsDataLoader.MultiIndexTestDataset datasets + Map allDatasets ) throws Exception { var indexPages = new ArrayList(); - for (CsvTestsDataLoader.TestDataset dataset : datasets.datasets()) { - var testData = loadPageFromCsv(CsvTests.class.getResource("/data/" + dataset.dataFileName()), dataset.typeMapping()); - Set mappedFields = loadMapping(dataset.mappingFileName()).keySet(); - indexPages.add(new TestPhysicalOperationProviders.IndexPage(dataset.indexName(), testData.v1(), testData.v2(), mappedFields)); + for (CsvTestsDataLoader.MultiIndexTestDataset datasets : allDatasets.values()) { + for (CsvTestsDataLoader.TestDataset dataset : datasets.datasets()) { + var testData = loadPageFromCsv(CsvTests.class.getResource("/data/" + dataset.dataFileName()), dataset.typeMapping()); + Set mappedFields = loadMapping(dataset.mappingFileName()).keySet(); + indexPages.add( + new TestPhysicalOperationProviders.IndexPage(dataset.indexName(), testData.v1(), testData.v2(), mappedFields) + ); + } } return TestPhysicalOperationProviders.create(foldCtx, indexPages); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java index d6cd6ee3ead84..4b46c26f56f35 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -12,6 +12,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.enrich.EnrichPolicy; import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.core.type.InvalidMappedField; import org.elasticsearch.xpack.esql.enrich.ResolvedEnrichPolicy; @@ -22,18 +23,23 @@ import org.elasticsearch.xpack.esql.inference.ResolvedInference; import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.parser.QueryParams; +import org.elasticsearch.xpack.esql.plan.IndexPattern; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.plan.logical.UnresolvedRelation; import org.elasticsearch.xpack.esql.session.Configuration; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import static org.elasticsearch.xpack.core.enrich.EnrichPolicy.GEO_MATCH_TYPE; import static org.elasticsearch.xpack.core.enrich.EnrichPolicy.MATCH_TYPE; @@ -55,10 +61,16 @@ public static Analyzer expandedDefaultAnalyzer() { return analyzer(expandedDefaultIndexResolution()); } + /** Simplest analyzer with a single index, which must be valid */ public static Analyzer analyzer(IndexResolution indexResolution) { return analyzer(indexResolution, TEST_VERIFIER); } + /** Simple analyzer with multiple indexes, which may also be invalid */ + public static Analyzer analyzer(Map indexResolutions) { + return analyzer(indexResolutions, defaultLookupResolution(), defaultEnrichResolution(), TEST_VERIFIER, TEST_CFG); + } + public static Analyzer analyzer(IndexResolution indexResolution, Map lookupResolution) { return analyzer(indexResolution, lookupResolution, TEST_VERIFIER); } @@ -77,11 +89,11 @@ public static Analyzer analyzer( EnrichResolution enrichResolution, Verifier verifier ) { - return analyzer(indexResolution, lookupResolution, enrichResolution, verifier, TEST_CFG); + return analyzer(indexResolutions(indexResolution), lookupResolution, enrichResolution, verifier, TEST_CFG); } public static Analyzer analyzer( - IndexResolution indexResolution, + Map indexResolutions, Map lookupResolution, EnrichResolution enrichResolution, Verifier verifier, @@ -91,7 +103,7 @@ public static Analyzer analyzer( testAnalyzerContext( config, new EsqlFunctionRegistry(), - indexResolution, + mergeIndexResolutions(indexResolutions, defaultSubqueryResolution()), lookupResolution, enrichResolution, defaultInferenceResolution() @@ -100,22 +112,25 @@ public static Analyzer analyzer( ); } - public static Analyzer analyzer(IndexResolution indexResolution, Verifier verifier, Configuration config) { - return analyzer(indexResolution, defaultLookupResolution(), defaultEnrichResolution(), verifier, config); + private static Map mergeIndexResolutions( + Map indexResolutions, + Map more + ) { + Map combined = new HashMap<>(indexResolutions); + combined.putAll(more); + return combined; + } + + public static Analyzer analyzer(Map indexResolutions, Verifier verifier, Configuration config) { + return analyzer(indexResolutions, defaultLookupResolution(), defaultEnrichResolution(), verifier, config); } public static Analyzer analyzer(Verifier verifier) { - return new Analyzer( - testAnalyzerContext( - EsqlTestUtils.TEST_CFG, - new EsqlFunctionRegistry(), - analyzerDefaultMapping(), - defaultLookupResolution(), - defaultEnrichResolution(), - defaultInferenceResolution() - ), - verifier - ); + return analyzer(analyzerDefaultMapping(), defaultLookupResolution(), defaultEnrichResolution(), verifier, EsqlTestUtils.TEST_CFG); + } + + public static Analyzer analyzer(Map indexResolutions, Verifier verifier) { + return analyzer(indexResolutions, defaultLookupResolution(), defaultEnrichResolution(), verifier, EsqlTestUtils.TEST_CFG); } public static LogicalPlan analyze(String query) { @@ -123,11 +138,14 @@ public static LogicalPlan analyze(String query) { } public static LogicalPlan analyze(String query, String mapping) { - return analyze(query, "test", mapping); + return analyze(query, indexFromQuery(query), mapping); } public static LogicalPlan analyze(String query, String index, String mapping) { - return analyze(query, analyzer(loadMapping(mapping, index), TEST_VERIFIER, configuration(query))); + Map indexResolutions = index == null + ? Map.of() + : Map.of(new IndexPattern(Source.EMPTY, index), loadMapping(mapping, index)); + return analyze(query, analyzer(indexResolutions, TEST_VERIFIER, configuration(query))); } public static LogicalPlan analyze(String query, Analyzer analyzer) { @@ -138,12 +156,40 @@ public static LogicalPlan analyze(String query, Analyzer analyzer) { return analyzed; } + private static final Pattern indexFromPattern = Pattern.compile("(?i)FROM\\s+([\\w-]+)"); + + private static String indexFromQuery(String query) { + // Extract the index name from the FROM clause of the query using regexp + Matcher matcher = indexFromPattern.matcher(query); + if (matcher.find()) { + return matcher.group(1); + } + return null; + } + public static LogicalPlan analyze(String query, String mapping, QueryParams params) { + return analyze(query, indexFromQuery(query), mapping, params); + } + + public static LogicalPlan analyze(String query, String index, String mapping, QueryParams params) { var plan = new EsqlParser().createStatement(query, params); - var analyzer = analyzer(loadMapping(mapping, "test"), TEST_VERIFIER, configuration(query)); + var indexResolutions = Map.of(new IndexPattern(Source.EMPTY, index), loadMapping(mapping, index)); + var analyzer = analyzer(indexResolutions, TEST_VERIFIER, configuration(query)); return analyzer.analyze(plan); } + public static UnresolvedRelation unresolvedRelation(String index) { + return new UnresolvedRelation( + Source.EMPTY, + new IndexPattern(Source.EMPTY, index), + false, + List.of(), + IndexMode.STANDARD, + null, + "FROM" + ); + } + public static IndexResolution loadMapping(String resource, String indexName, IndexMode indexMode) { EsIndex test = new EsIndex(indexName, EsqlTestUtils.loadMapping(resource), Map.of(indexName, indexMode)); return IndexResolution.valid(test); @@ -154,8 +200,30 @@ public static IndexResolution loadMapping(String resource, String indexName) { return IndexResolution.valid(test); } - public static IndexResolution analyzerDefaultMapping() { - return loadMapping("mapping-basic.json", "test"); + public static Map analyzerDefaultMapping() { + // Most tests use either "test" or "employees" as the index name, but for the same mapping + return Map.of( + new IndexPattern(Source.EMPTY, "test"), + loadMapping("mapping-basic.json", "test"), + new IndexPattern(Source.EMPTY, "employees"), + loadMapping("mapping-basic.json", "employees") + ); + } + + public static Map indexResolutions(EsIndex... indexes) { + Map map = new HashMap<>(); + for (EsIndex index : indexes) { + map.put(new IndexPattern(Source.EMPTY, index.name()), IndexResolution.valid(index)); + } + return map; + } + + public static Map indexResolutions(IndexResolution... indexes) { + Map map = new HashMap<>(); + for (IndexResolution index : indexes) { + map.put(new IndexPattern(Source.EMPTY, index.get().name()), index); + } + return map; } public static IndexResolution expandedDefaultIndexResolution() { @@ -232,6 +300,17 @@ public static InferenceResolution defaultInferenceResolution() { .build(); } + public static Map defaultSubqueryResolution() { + return Map.of( + new IndexPattern(Source.EMPTY, "languages"), + loadMapping("mapping-languages.json", "languages"), + new IndexPattern(Source.EMPTY, "sample_data"), + loadMapping("mapping-sample_data.json", "sample_data"), + new IndexPattern(Source.EMPTY, "test_mixed_types"), + loadMapping("mapping-default-incompatible.json", "test_mixed_types") + ); + } + public static String randomInferenceId() { return ESTestCase.randomFrom(VALID_INFERENCE_IDS); } @@ -308,7 +387,7 @@ public static IndexResolution indexWithDateDateNanosUnionType() { EsField dateDateNanosField = new InvalidMappedField(dateDateNanos, typesToIndices1); EsField dateDateNanosLongField = new InvalidMappedField(dateDateNanosLong, typesToIndices2); EsIndex index = new EsIndex( - "test*", + "index*", Map.of(dateDateNanos, dateDateNanosField, dateDateNanosLong, dateDateNanosLongField), Map.of("index1", IndexMode.STANDARD, "index2", IndexMode.STANDARD, "index3", IndexMode.STANDARD) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index 62cd898386870..46c8b73213abe 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -131,10 +131,12 @@ import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyzerDefaultMapping; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultEnrichResolution; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultInferenceResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexWithDateDateNanosUnionType; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.randomInferenceIdOtherThan; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.tsdbIndexResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.unresolvedRelation; 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; @@ -142,6 +144,7 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_PERIOD; import static org.elasticsearch.xpack.esql.core.type.DataType.DENSE_VECTOR; import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; +import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; import static org.elasticsearch.xpack.esql.core.type.DataType.LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.UNSUPPORTED; @@ -162,16 +165,7 @@ //@TestLogging(value = "org.elasticsearch.xpack.esql.analysis:TRACE", reason = "debug") public class AnalyzerTests extends ESTestCase { - private static final UnresolvedRelation UNRESOLVED_RELATION = new UnresolvedRelation( - EMPTY, - new IndexPattern(EMPTY, "idx"), - false, - List.of(), - IndexMode.STANDARD, - null, - "FROM" - ); - + private static final UnresolvedRelation UNRESOLVED_RELATION = unresolvedRelation("idx"); private static final int MAX_LIMIT = AnalyzerSettings.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(Settings.EMPTY); private static final int DEFAULT_LIMIT = AnalyzerSettings.QUERY_RESULT_TRUNCATION_DEFAULT_SIZE.getDefault(Settings.EMPTY); private static final int DEFAULT_TIMESERIES_LIMIT = AnalyzerSettings.QUERY_TIMESERIES_RESULT_TRUNCATION_DEFAULT_SIZE.getDefault( @@ -188,7 +182,7 @@ public void testIndexResolution() { } public void testFailOnUnresolvedIndex() { - Analyzer analyzer = analyzer(IndexResolution.invalid("Unknown index [idx]")); + Analyzer analyzer = analyzer(Map.of(new IndexPattern(Source.EMPTY, "idx"), IndexResolution.invalid("Unknown index [idx]"))); VerificationException e = expectThrows(VerificationException.class, () -> analyzer.analyze(UNRESOLVED_RELATION)); @@ -199,7 +193,7 @@ public void testIndexWithClusterResolution() { EsIndex idx = new EsIndex("cluster:idx", Map.of()); Analyzer analyzer = analyzer(IndexResolution.valid(idx)); - var plan = analyzer.analyze(UNRESOLVED_RELATION); + var plan = analyzer.analyze(unresolvedRelation("cluster:idx")); var limit = as(plan, Limit.class); assertEquals(new EsRelation(EMPTY, idx.name(), IndexMode.STANDARD, idx.indexNameWithModes(), NO_FIELDS), limit.child()); @@ -268,7 +262,7 @@ public void testRowAttributeResolution() { var plan = analyzer.analyze( new Eval( EMPTY, - new Row(EMPTY, List.of(new Alias(EMPTY, "emp_no", new Literal(EMPTY, 1, DataType.INTEGER)))), + new Row(EMPTY, List.of(new Alias(EMPTY, "emp_no", new Literal(EMPTY, 1, INTEGER)))), List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no"))) ) ); @@ -410,16 +404,16 @@ public void testNoProjection() { from test """, DataType.KEYWORD, - DataType.INTEGER, + INTEGER, DataType.KEYWORD, DataType.TEXT, DATETIME, DataType.TEXT, DataType.KEYWORD, - DataType.INTEGER, + INTEGER, DataType.KEYWORD, DataType.LONG, - DataType.INTEGER + INTEGER ); } @@ -1687,7 +1681,7 @@ public void testEnrichPolicyWithError() { AnalyzerContext context = testAnalyzerContext( configuration("from test"), new EsqlFunctionRegistry(), - testIndex, + indexResolutions(testIndex), enrichResolution, emptyInferenceResolution() ); @@ -1843,7 +1837,7 @@ public void testEnrichFieldsIncludeMatchField() { AnalyzerContext context = testAnalyzerContext( configuration(query), new EsqlFunctionRegistry(), - testIndex, + indexResolutions(testIndex), enrichResolution, emptyInferenceResolution() ); @@ -1924,7 +1918,7 @@ public void testUnresolvedMvExpand() { public void testRegularStats() { var plan = analyze(""" - from tests + from test | stats by salary """); @@ -2644,7 +2638,7 @@ private void validateConditionalFunctions(LogicalPlan plan) { assertEquals(projection.dataType(), DataType.DOUBLE); projection = as(projections.get(1), ReferenceAttribute.class); assertEquals(projection.name(), "y"); - assertEquals(projection.dataType(), DataType.INTEGER); + assertEquals(projection.dataType(), INTEGER); projection = as(projections.get(2), ReferenceAttribute.class); assertEquals(projection.name(), "z"); assertEquals(projection.dataType(), DataType.LONG); @@ -3053,7 +3047,7 @@ public void testFromEnrichAndMatchColonUsage() { | EVAL x = to_string(languages) | ENRICH _any:languages ON x | WHERE first_name: "Anna" - """, "mapping-default.json"); + """, "*:test", "mapping-default.json"); var limit = as(plan, Limit.class); var filter = as(limit.child(), Filter.class); var match = as(filter.condition(), MatchOperator.class); @@ -3062,7 +3056,7 @@ public void testFromEnrichAndMatchColonUsage() { assertEquals(enrich.policy().getMatchField(), "language_code"); var eval = as(enrich.child(), Eval.class); var esRelation = as(eval.child(), EsRelation.class); - assertEquals(esRelation.indexPattern(), "test"); + assertEquals(esRelation.indexPattern(), "*:test"); // This tests nothing, as whatever appears here comes from the test itself } public void testFunctionNamedParamsAsFunctionArgument() { @@ -3120,7 +3114,7 @@ public void testResolveInsist_fieldExists_insistedOutputContainsNoUnmappedFields Attribute last = plan.output().getLast(); assertThat(last.name(), is("emp_no")); - assertThat(last.dataType(), is(DataType.INTEGER)); + assertThat(last.dataType(), is(INTEGER)); assertThat( plan.output() .stream() @@ -3157,7 +3151,7 @@ public void testResolveInsist_multiIndexFieldPartiallyMappedWithSingleKeywordTyp assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3172,7 +3166,7 @@ public void testResolveInsist_multiIndexFieldPartiallyMappedWithSingleKeywordTyp ); String query = "FROM foo, bar | INSIST_🐔 message"; - var plan = analyze(query, analyzer(resolution, TEST_VERIFIER, configuration(query))); + var plan = analyze(query, analyzer(indexResolutions(resolution), TEST_VERIFIER, configuration(query))); var limit = as(plan, Limit.class); var insist = as(limit.child(), Insist.class); var attribute = (FieldAttribute) EsqlTestUtils.singleValue(insist.output()); @@ -3184,7 +3178,7 @@ public void testResolveInsist_multiIndexFieldExistsWithSingleTypeButIsNotKeyword assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3212,7 +3206,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesNoKeyw assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3240,7 +3234,7 @@ public void testResolveInsist_multiIndexSameMapping_fieldIsMapped() { assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3265,7 +3259,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithKe assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3294,7 +3288,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithCa assumeTrue("Requires UNMAPPED FIELDS", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); IndexResolution resolution = IndexResolver.mergedMappings( - "foo, bar", + "foo,bar", new IndexResolver.FieldsInfo( new FieldCapabilitiesResponse( List.of( @@ -3310,7 +3304,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithCa ); VerificationException e = expectThrows( VerificationException.class, - () -> analyze("FROM multi_index | INSIST_🐔 message | EVAL message = message :: keyword", analyzer(resolution, TEST_VERIFIER)) + () -> analyze("FROM foo, bar | INSIST_🐔 message | EVAL message = message :: keyword", analyzer(resolution, TEST_VERIFIER)) ); // This isn't the most informative error, but it'll do for now. assertThat( @@ -3810,7 +3804,7 @@ private static LogicalPlan analyzeWithEmptyFieldCapsResponse(String query) throw ); 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)); + var analyzer = analyzer(indexResolutions(resolution), TEST_VERIFIER, configuration(query)); return analyze(query, analyzer); } @@ -4427,7 +4421,7 @@ public void testImplicitCastingForDateAndDateNanosFields() { // Validate if a union typed field is cast to a type explicitly, implicit casting won't be applied again, and include some cases of // nested casting as well. LogicalPlan plan = analyze(""" - FROM tests + FROM index* | Eval a = date_and_date_nanos, b = date_and_date_nanos::datetime, c = date_and_date_nanos::date_nanos, d = date_and_date_nanos::datetime::datetime, e = date_and_date_nanos::datetime::date_nanos, f = date_and_date_nanos::date_nanos::datetime, g = date_and_date_nanos::date_nanos::date_nanos, @@ -4535,7 +4529,7 @@ public void testImplicitCastingForDateAndDateNanosFields() { fa = as(toDateNanos.field(), FieldAttribute.class); verifyNameAndTypeAndMultiTypeEsField(fa.name(), fa.dataType(), "$$date_and_date_nanos$converted_to$long", LONG, fa); EsRelation esRelation = as(eval.child(), EsRelation.class); - assertEquals("test*", esRelation.indexPattern()); + assertEquals("index*", esRelation.indexPattern()); } public void testGroupingOverridesInStats() { @@ -4628,12 +4622,11 @@ public void testImplicitCastingForAggregateMetricDouble() { Map.of("k8s", IndexMode.TIME_SERIES, "k8s-downsampled", IndexMode.TIME_SERIES), Set.of() ); - var indexResolution = IndexResolution.valid(esIndex); var analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - indexResolution, + indexResolutions(esIndex), defaultEnrichResolution(), defaultInferenceResolution() ), @@ -4697,6 +4690,6 @@ static Literal string(String value) { } static Literal literal(int value) { - return new Literal(EMPTY, value, DataType.INTEGER); + return new Literal(EMPTY, value, INTEGER); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java index 606db6fb8fb2c..ae3426178e36b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java @@ -45,6 +45,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyInferenceResolution; import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyPolicyResolution; import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; @@ -56,7 +57,13 @@ public class ParsingTests extends ESTestCase { private final IndexResolution defaultIndex = loadIndexResolution("mapping-basic.json"); private final Analyzer defaultAnalyzer = new Analyzer( - testAnalyzerContext(TEST_CFG, new EsqlFunctionRegistry(), defaultIndex, emptyPolicyResolution(), emptyInferenceResolution()), + testAnalyzerContext( + TEST_CFG, + new EsqlFunctionRegistry(), + indexResolutions(defaultIndex), + emptyPolicyResolution(), + emptyInferenceResolution() + ), TEST_VERIFIER ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index b35258978b72c..0992489b6086d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1006,10 +1006,10 @@ public void testFilterNullField() { query("from test | where null::boolean"); // Provide `NULL` type in `EVAL` - query("from t | EVAL x = null | where x"); + query("from test | EVAL x = null | where x"); // `to_string(null)` is of `KEYWORD` type null, resulting in `to_string(null) == "abc"` being of `BOOLEAN` - query("from t | where to_string(null) == \"abc\""); + query("from test | where to_string(null) == \"abc\""); // Other DataTypes can contain null values assertEquals("1:19: Condition expression needs to be boolean, found [KEYWORD]", error("from test | where null::string")); @@ -1114,27 +1114,27 @@ public void testInlineImpossibleConvert() { public void testAggregateOnCounter() { assertThat( - error("FROM tests | STATS min(network.bytes_in)", tsdb), + error("FROM test | STATS min(network.bytes_in)", tsdb), equalTo( - "1:20: argument of [min(network.bytes_in)] must be" + "1:19: argument of [min(network.bytes_in)] must be" + " [boolean, date, ip, string, version, aggregate_metric_double or numeric except counter types]," + " found value [network.bytes_in] type [counter_long]" ) ); assertThat( - error("FROM tests | STATS max(network.bytes_in)", tsdb), + error("FROM test | STATS max(network.bytes_in)", tsdb), equalTo( - "1:20: argument of [max(network.bytes_in)] must be" + "1:19: argument of [max(network.bytes_in)] must be" + " [boolean, date, ip, string, version, aggregate_metric_double or numeric except counter types]," + " found value [network.bytes_in] type [counter_long]" ) ); assertThat( - error("FROM tests | STATS count(network.bytes_out)", tsdb), + error("FROM test | STATS count(network.bytes_out)", tsdb), equalTo( - "1:20: argument of [count(network.bytes_out)] must be [any type except counter types or dense_vector]," + "1:19: argument of [count(network.bytes_out)] must be [any type except counter types or dense_vector]," + " found value [network.bytes_out] type [counter_long]" ) ); @@ -1157,91 +1157,88 @@ public void testAggsResolutionWithUnresolvedGroupings() { new String[] { "avg", "count", "count_distinct", "min", "max", "median", "median_absolute_deviation", "sum", "values" } ); - assertThat(error("FROM tests | STATS " + agg_func + "(emp_no) by foobar"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); + assertThat(error("FROM test | STATS " + agg_func + "(emp_no) by foobar"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); + assertThat(error("FROM test | STATS " + agg_func + "(x) by foobar, x = emp_no"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); + assertThat(error("FROM test | STATS " + agg_func + "(foobar) by foobar"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); assertThat( - error("FROM tests | STATS " + agg_func + "(x) by foobar, x = emp_no"), - matchesRegex("1:\\d+: Unknown column \\[foobar]") - ); - assertThat(error("FROM tests | STATS " + agg_func + "(foobar) by foobar"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); - assertThat( - error("FROM tests | STATS " + agg_func + "(foobar) by BUCKET(hire_date, 10)"), + error("FROM test | STATS " + agg_func + "(foobar) by BUCKET(hire_date, 10)"), matchesRegex( "1:\\d+: function expects exactly four arguments when the first one is of type \\[DATETIME]" + " and the second of type \\[INTEGER]\n" + "line 1:\\d+: Unknown column \\[foobar]" ) ); - assertThat(error("FROM tests | STATS " + agg_func + "(foobar) by emp_no"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); + assertThat(error("FROM test | STATS " + agg_func + "(foobar) by emp_no"), matchesRegex("1:\\d+: Unknown column \\[foobar]")); // TODO: Ideally, we'd detect that count_distinct(x) doesn't require an error message. assertThat( - error("FROM tests | STATS " + agg_func + "(x) by x = foobar"), + error("FROM test | STATS " + agg_func + "(x) by x = foobar"), matchesRegex("1:\\d+: Unknown column \\[foobar]\n" + "line 1:\\d+: Unknown column \\[x]") ); } public void testNotAllowRateOutsideMetrics() { assertThat( - error("FROM tests | STATS avg(rate(network.bytes_in))", tsdb), + error("FROM test | STATS avg(rate(network.bytes_in))", tsdb), equalTo("1:24: time_series aggregate[rate(network.bytes_in)] can only be used with the TS command") ); assertThat( - error("FROM tests | STATS rate(network.bytes_in)", tsdb), + error("FROM test | STATS rate(network.bytes_in)", tsdb), equalTo("1:20: time_series aggregate[rate(network.bytes_in)] can only be used with the TS command") ); assertThat( - error("FROM tests | STATS max_over_time(network.connections)", tsdb), + error("FROM test | STATS max_over_time(network.connections)", tsdb), equalTo("1:20: time_series aggregate[max_over_time(network.connections)] can only be used with the TS command") ); assertThat( - error("FROM tests | EVAL r = rate(network.bytes_in)", tsdb), + error("FROM test | EVAL r = rate(network.bytes_in)", tsdb), equalTo("1:23: aggregate function [rate(network.bytes_in)] not allowed outside STATS command") ); } public void testTimeseriesAggregate() { assertThat( - error("TS tests | STATS rate(network.bytes_in)", tsdb), + error("TS test | STATS rate(network.bytes_in)", tsdb), equalTo( "1:18: time-series aggregate function [rate(network.bytes_in)] can only be used with the TS command " + "and inside another aggregate function" ) ); assertThat( - error("TS tests | STATS avg_over_time(network.connections)", tsdb), + error("TS test | STATS avg_over_time(network.connections)", tsdb), equalTo( "1:18: time-series aggregate function [avg_over_time(network.connections)] can only be used " + "with the TS command and inside another aggregate function" ) ); assertThat( - error("TS tests | STATS avg(rate(network.bytes_in)), rate(network.bytes_in)", tsdb), + error("TS test | STATS avg(rate(network.bytes_in)), rate(network.bytes_in)", tsdb), equalTo( "1:47: time-series aggregate function [rate(network.bytes_in)] can only be used " + "with the TS command and inside another aggregate function" ) ); - assertThat(error("TS tests | STATS max(avg(rate(network.bytes_in)))", tsdb), equalTo(""" + assertThat(error("TS test | STATS max(avg(rate(network.bytes_in)))", tsdb), equalTo(""" 1:22: nested aggregations [avg(rate(network.bytes_in))] \ not allowed inside other aggregations [max(avg(rate(network.bytes_in)))] line 1:12: cannot use aggregate function [avg(rate(network.bytes_in))] \ inside over-time aggregation function [rate(network.bytes_in)]""")); - assertThat(error("TS tests | STATS max(avg(rate(network.bytes_in)))", tsdb), equalTo(""" + assertThat(error("TS test | STATS max(avg(rate(network.bytes_in)))", tsdb), equalTo(""" 1:22: nested aggregations [avg(rate(network.bytes_in))] \ not allowed inside other aggregations [max(avg(rate(network.bytes_in)))] line 1:12: cannot use aggregate function [avg(rate(network.bytes_in))] \ inside over-time aggregation function [rate(network.bytes_in)]""")); assertThat( - error("TS tests | STATS rate(network.bytes_in) BY bucket(@timestamp, 1 hour)", tsdb), + error("TS test | STATS rate(network.bytes_in) BY bucket(@timestamp, 1 hour)", tsdb), equalTo( "1:18: time-series aggregate function [rate(network.bytes_in)] can only be used " + "with the TS command and inside another aggregate function" ) ); assertThat( - error("TS tests | STATS COUNT(*)", tsdb), + error("TS test | STATS COUNT(*)", tsdb), equalTo("1:18: count_star [COUNT(*)] can't be used with TS command; use count on a field instead") ); } @@ -1733,45 +1730,45 @@ public void testConditionalFunctionsWithMixedNumericTypes() { // counter assertEquals( - "1:23: second argument of [" + "1:22: second argument of [" + functionName + "(network.bytes_in, 0)] must be [counter_long], found value [0] type [integer]", - error("FROM tests | eval x = " + functionName + "(network.bytes_in, 0)", tsdb) + error("FROM test | eval x = " + functionName + "(network.bytes_in, 0)", tsdb) ); assertEquals( - "1:23: second argument of [" + "1:22: second argument of [" + functionName + "(network.bytes_in, to_long(0))] must be [counter_long], " + "found value [to_long(0)] type [long]", - error("FROM tests | eval x = " + functionName + "(network.bytes_in, to_long(0))", tsdb) + error("FROM test | eval x = " + functionName + "(network.bytes_in, to_long(0))", tsdb) ); assertEquals( - "1:23: second argument of [" + "1:22: second argument of [" + functionName + "(network.bytes_in, 0.0)] must be [counter_long], found value [0.0] type [double]", - error("FROM tests | eval x = " + functionName + "(network.bytes_in, 0.0)", tsdb) + error("FROM test | eval x = " + functionName + "(network.bytes_in, 0.0)", tsdb) ); assertEquals( - "1:23: third argument of [" + "1:22: third argument of [" + functionName + "(null, network.bytes_in, 0)] must be [counter_long], found value [0] type [integer]", - error("FROM tests | eval x = " + functionName + "(null, network.bytes_in, 0)", tsdb) + error("FROM test | eval x = " + functionName + "(null, network.bytes_in, 0)", tsdb) ); assertEquals( - "1:23: third argument of [" + "1:22: third argument of [" + functionName + "(null, network.bytes_in, to_long(0))] must be [counter_long], " + "found value [to_long(0)] type [long]", - error("FROM tests | eval x = " + functionName + "(null, network.bytes_in, to_long(0))", tsdb) + error("FROM test | eval x = " + functionName + "(null, network.bytes_in, to_long(0))", tsdb) ); assertEquals( - "1:23: third argument of [" + "1:22: third argument of [" + functionName + "(null, network.bytes_in, 0.0)] must be [counter_long], found value [0.0] type [double]", - error("FROM tests | eval x = " + functionName + "(null, network.bytes_in, 0.0)", tsdb) + error("FROM test | eval x = " + functionName + "(null, network.bytes_in, 0.0)", tsdb) ); } @@ -1781,8 +1778,8 @@ public void testConditionalFunctionsWithMixedNumericTypes() { error("from test | eval x = case(languages == 1, salary, height)") ); assertEquals( - "1:23: third argument of [case(name == \"a\", network.bytes_in, 0)] must be [counter_long], found value [0] type [integer]", - error("FROM tests | eval x = case(name == \"a\", network.bytes_in, 0)", tsdb) + "1:22: third argument of [case(name == \"a\", network.bytes_in, 0)] must be [counter_long], found value [0] type [integer]", + error("FROM test | eval x = case(name == \"a\", network.bytes_in, 0)", tsdb) ); } @@ -1843,35 +1840,35 @@ public void testToDatePeriodTimeDurationInInvalidPosition() { public void testToDatePeriodToTimeDurationWithInvalidType() { assertEquals( "1:36: argument of [1.5::date_period] must be [date_period or string], found value [1.5] type [double]", - error("from types | EVAL x = birth_date + 1.5::date_period") + error("from test | EVAL x = birth_date + 1.5::date_period") ); assertEquals( "1:37: argument of [to_timeduration(1)] must be [time_duration or string], found value [1] type [integer]", - error("from types | EVAL x = birth_date - to_timeduration(1)") + error("from test | EVAL x = birth_date - to_timeduration(1)") ); assertEquals( "1:45: argument of [x::date_period] must be [date_period or string], found value [x] type [double]", - error("from types | EVAL x = 1.5, y = birth_date + x::date_period") + error("from test | EVAL x = 1.5, y = birth_date + x::date_period") ); assertEquals( "1:44: argument of [to_timeduration(x)] must be [time_duration or string], found value [x] type [integer]", - error("from types | EVAL x = 1, y = birth_date - to_timeduration(x)") + error("from test | EVAL x = 1, y = birth_date - to_timeduration(x)") ); assertEquals( "1:64: argument of [x::date_period] must be [date_period or string], found value [x] type [datetime]", - error("from types | EVAL x = \"2024-09-08\"::datetime, y = birth_date + x::date_period") + error("from test | EVAL x = \"2024-09-08\"::datetime, y = birth_date + x::date_period") ); assertEquals( "1:65: argument of [to_timeduration(x)] must be [time_duration or string], found value [x] type [datetime]", - error("from types | EVAL x = \"2024-09-08\"::datetime, y = birth_date - to_timeduration(x)") + error("from test | EVAL x = \"2024-09-08\"::datetime, y = birth_date - to_timeduration(x)") ); assertEquals( "1:58: argument of [x::date_period] must be [date_period or string], found value [x] type [ip]", - error("from types | EVAL x = \"2024-09-08\"::ip, y = birth_date + x::date_period") + error("from test | EVAL x = \"2024-09-08\"::ip, y = birth_date + x::date_period") ); assertEquals( "1:59: argument of [to_timeduration(x)] must be [time_duration or string], found value [x] type [ip]", - error("from types | EVAL x = \"2024-09-08\"::ip, y = birth_date - to_timeduration(x)") + error("from test | EVAL x = \"2024-09-08\"::ip, y = birth_date - to_timeduration(x)") ); } @@ -1879,17 +1876,17 @@ public void testIntervalAsString() { // DateTrunc for (String interval : List.of("1 minu", "1 dy", "1.5 minutes", "0.5 days", "minutes 1", "day 5")) { assertThat( - error("from types | EVAL x = date_trunc(\"" + interval + "\", \"1991-06-26T00:00:00.000Z\")"), + error("from test | EVAL x = date_trunc(\"" + interval + "\", \"1991-06-26T00:00:00.000Z\")"), containsString("1:35: Cannot convert string [" + interval + "] to [DATE_PERIOD or TIME_DURATION]") ); assertThat( - error("from types | EVAL x = \"1991-06-26T00:00:00.000Z\", y = date_trunc(\"" + interval + "\", x::datetime)"), + error("from test | EVAL x = \"1991-06-26T00:00:00.000Z\", y = date_trunc(\"" + interval + "\", x::datetime)"), containsString("1:67: Cannot convert string [" + interval + "] to [DATE_PERIOD or TIME_DURATION]") ); } for (String interval : List.of("1", "0.5", "invalid")) { assertThat( - error("from types | EVAL x = date_trunc(\"" + interval + "\", \"1991-06-26T00:00:00.000Z\")"), + error("from test | EVAL x = date_trunc(\"" + interval + "\", \"1991-06-26T00:00:00.000Z\")"), containsString( "1:24: first argument of [date_trunc(\"" + interval @@ -1899,7 +1896,7 @@ public void testIntervalAsString() { ) ); assertThat( - error("from types | EVAL x = \"1991-06-26T00:00:00.000Z\", y = date_trunc(\"" + interval + "\", x::datetime)"), + error("from test | EVAL x = \"1991-06-26T00:00:00.000Z\", y = date_trunc(\"" + interval + "\", x::datetime)"), containsString( "1:56: first argument of [date_trunc(\"" + interval @@ -2191,8 +2188,8 @@ public void testFilterByAggregate() { ); assertEquals("1:23: aggregate function [max(a)] not allowed outside STATS command", error("ROW a = 1 | WHERE 1 + max(a) > 0")); assertEquals( - "1:24: aggregate function [min(languages)] not allowed outside STATS command", - error("FROM employees | WHERE min(languages) > 2") + "1:19: aggregate function [min(languages)] not allowed outside STATS command", + error("FROM test | WHERE min(languages) > 2") ); assertEquals( "1:19: aggregate function [present(gender)] not allowed outside STATS command", @@ -2848,7 +2845,7 @@ public void testLimitBeforeInlineStats_WithFork() { assertThat( error( - "FROM employees\n" + "FROM test\n" + "| KEEP emp_no, languages, gender\n" + "| FORK (WHERE emp_no == 10048 OR emp_no == 10081)\n" + " (WHERE emp_no == 10081 OR emp_no == 10087)\n" @@ -2864,7 +2861,7 @@ public void testLimitBeforeInlineStats_WithFork() { assertThat( error( - "FROM employees\n" + "FROM test\n" + "| KEEP emp_no, languages, gender\n" + "| FORK (WHERE emp_no == 10048 OR emp_no == 10081)\n" + " (WHERE emp_no == 10081 OR emp_no == 10087)\n" diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java index 733c23af12966..e70d428e16beb 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/CheckLicenseTests.java @@ -42,7 +42,7 @@ public class CheckLicenseTests extends ESTestCase { private final EsqlParser parser = new EsqlParser(); - private final String esql = "from tests | eval license() | LIMIT 10"; + private final String esql = "from test | eval license() | LIMIT 10"; public void testLicense() { for (License.OperationMode functionLicense : License.OperationMode.values()) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLocalPhysicalPlanOptimizerTests.java index 8c887aca9dcce..d7fc5b9a6009e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLocalPhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLocalPhysicalPlanOptimizerTests.java @@ -47,6 +47,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; public class AbstractLocalPhysicalPlanOptimizerTests extends MapperServiceTestCase { protected final Configuration config; @@ -101,7 +102,7 @@ public void init() { testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - timeSeriesIndex, + indexResolutions(timeSeriesIndex), enrichResolution, emptyInferenceResolution() ), @@ -123,7 +124,7 @@ private Analyzer makeAnalyzer(String mappingFileName, EnrichResolution enrichRes testAnalyzerContext( config, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), defaultLookupResolution(), enrichResolution, emptyInferenceResolution() @@ -138,7 +139,13 @@ protected Analyzer makeAnalyzer(String mappingFileName) { protected Analyzer makeAnalyzer(IndexResolution indexResolution) { return new Analyzer( - testAnalyzerContext(config, new EsqlFunctionRegistry(), indexResolution, new EnrichResolution(), emptyInferenceResolution()), + testAnalyzerContext( + config, + new EsqlFunctionRegistry(), + indexResolutions(indexResolution), + new EnrichResolution(), + emptyInferenceResolution() + ), new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java index 93e987decb9c4..498c138bb92bc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/AbstractLogicalPlanOptimizerTests.java @@ -18,7 +18,6 @@ import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; @@ -38,6 +37,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultInferenceResolution; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; import static org.hamcrest.Matchers.containsString; @@ -105,15 +105,15 @@ public static void init() { "mapping-languages.json" ); - // Most tests used data from the test index, so we load it here, and use it in the plan() function. + // Most tests use either "test" or "employees" as the index name, but for the same mapping mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); + EsIndex employees = new EsIndex("employees", mapping, Map.of("employees", IndexMode.STANDARD)); analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test, employees), defaultLookupResolution(), enrichResolution, emptyInferenceResolution() @@ -121,15 +121,14 @@ public static void init() { TEST_VERIFIER ); - // Some tests use data from the airports index, so we load it here, and use it in the plan_airports() function. + // Some tests use data from the airports index, so we load it here, and use it in the planAirports() function. mappingAirports = loadMapping("mapping-airports.json"); EsIndex airports = new EsIndex("airports", mappingAirports, Map.of("airports", IndexMode.STANDARD)); - IndexResolution getIndexResultAirports = IndexResolution.valid(airports); analyzerAirports = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResultAirports, + indexResolutions(airports), defaultLookupResolution(), enrichResolution, emptyInferenceResolution() @@ -140,12 +139,11 @@ public static void init() { // Some tests need additional types, so we load that index here and use it in the plan_types() function. mappingTypes = loadMapping("mapping-all-types.json"); EsIndex types = new EsIndex("types", mappingTypes, Map.of("types", IndexMode.STANDARD)); - IndexResolution getIndexResultTypes = IndexResolution.valid(types); analyzerTypes = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResultTypes, + indexResolutions(types), enrichResolution, defaultInferenceResolution() ), @@ -155,12 +153,11 @@ public static void init() { // Some tests use mappings from mapping-extra.json to be able to test more types so we load it here mappingExtra = loadMapping("mapping-extra.json"); EsIndex extra = new EsIndex("extra", mappingExtra, Map.of("extra", IndexMode.STANDARD)); - IndexResolution getIndexResultExtra = IndexResolution.valid(extra); analyzerExtra = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResultExtra, + indexResolutions(extra), enrichResolution, emptyInferenceResolution() ), @@ -168,12 +165,12 @@ public static void init() { ); metricMapping = loadMapping("k8s-mappings.json"); - var metricsIndex = IndexResolution.valid(new EsIndex("k8s", metricMapping, Map.of("k8s", IndexMode.TIME_SERIES))); + var metricsIndex = new EsIndex("k8s", metricMapping, Map.of("k8s", IndexMode.TIME_SERIES)); metricsAnalyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - metricsIndex, + indexResolutions(metricsIndex), enrichResolution, emptyInferenceResolution() ), @@ -185,19 +182,17 @@ public static void init() { "partial_type_keyword", new EsField("partial_type_keyword", KEYWORD, emptyMap(), true, EsField.TimeSeriesFieldType.NONE) ); - var multiIndex = IndexResolution.valid( - new EsIndex( - "multi_index", - multiIndexMapping, - Map.of("test1", IndexMode.STANDARD, "test2", IndexMode.STANDARD), - Set.of("partial_type_keyword") - ) + var multiIndex = new EsIndex( + "multi_index", + multiIndexMapping, + Map.of("test1", IndexMode.STANDARD, "test2", IndexMode.STANDARD), + Set.of("partial_type_keyword") ); multiIndexAnalyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - multiIndex, + indexResolutions(multiIndex), enrichResolution, emptyInferenceResolution() ), @@ -205,14 +200,12 @@ public static void init() { ); var sampleDataMapping = loadMapping("mapping-sample_data.json"); - var sampleDataIndex = IndexResolution.valid( - new EsIndex("sample_data", sampleDataMapping, Map.of("sample_data", IndexMode.STANDARD)) - ); + var sampleDataIndex = new EsIndex("sample_data", sampleDataMapping, Map.of("sample_data", IndexMode.STANDARD)); sampleDataIndexAnalyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - sampleDataIndex, + indexResolutions(sampleDataIndex), enrichResolution, emptyInferenceResolution() ), @@ -230,25 +223,19 @@ protected LogicalPlan plan(String query) { protected LogicalPlan plan(String query, LogicalPlanOptimizer optimizer) { var analyzed = analyzer.analyze(parser.createStatement(query)); - // System.out.println(analyzed); var optimized = optimizer.optimize(analyzed); - // System.out.println(optimized); return optimized; } protected LogicalPlan planAirports(String query) { var analyzed = analyzerAirports.analyze(parser.createStatement(query)); - // System.out.println(analyzed); var optimized = logicalOptimizer.optimize(analyzed); - // System.out.println(optimized); return optimized; } protected LogicalPlan planExtra(String query) { var analyzed = analyzerExtra.analyze(parser.createStatement(query)); - // System.out.println(analyzed); var optimized = logicalOptimizer.optimize(analyzed); - // System.out.println(optimized); return optimized; } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java index 1ef512a070d14..df7e7189dce0d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java @@ -45,7 +45,6 @@ import org.elasticsearch.xpack.esql.expression.predicate.nulls.IsNotNull; import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules; import org.elasticsearch.xpack.esql.optimizer.rules.logical.local.InferIsNotNull; import org.elasticsearch.xpack.esql.parser.EsqlParser; @@ -97,6 +96,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY; import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; @@ -124,14 +124,13 @@ public static void init() { mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), emptyPolicyResolution(), emptyInferenceResolution() ), @@ -518,14 +517,13 @@ public void testSparseDocument() throws Exception { SearchStats searchStats = statsForExistingField("field000", "field001", "field002", "field003", "field004"); EsIndex index = new EsIndex("large", large, Map.of("large", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(index); var logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); var analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(index), emptyPolicyResolution(), emptyInferenceResolution() ), @@ -1114,13 +1112,12 @@ private static Analyzer analyzerWithUnionTypeMapping() { Map.of("integer_long_field", unionTypeField), Map.of("test1", IndexMode.STANDARD, "test2", IndexMode.STANDARD) ); - IndexResolution getIndexResult = IndexResolution.valid(test); return new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), emptyPolicyResolution(), emptyInferenceResolution() ), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java index d45e178b8ff32..8c6cb43b86539 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java @@ -2278,7 +2278,7 @@ public void testToDateNanosPushDown() { plannerOptimizerDateDateNanosUnionTypes = new TestPlannerOptimizer(EsqlTestUtils.TEST_CFG, makeAnalyzer(indexWithUnionTypedFields)); var stats = EsqlTestUtils.statsForExistingField("date_and_date_nanos", "date_and_date_nanos_and_long"); String query = """ - from test* + from index* | where date_and_date_nanos < "2025-01-01" and date_and_date_nanos_and_long::date_nanos >= "2024-01-01\""""; var plan = plannerOptimizerDateDateNanosUnionTypes.plan(query, stats); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index 73492becdcb41..c9819c2730fca 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -97,7 +97,6 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.rules.logical.LiteralsOnTheRight; import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneRedundantOrderBy; @@ -180,6 +179,7 @@ import static org.elasticsearch.xpack.esql.analysis.Analyzer.NO_FIELDS; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultAnalyzer; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.core.expression.Literal.NULL; import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY; import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; @@ -4298,7 +4298,7 @@ public void testIsNotNullConstraintForAliasedExpressions() { */ public void testSpatialTypesAndStatsUseDocValues() { var plan = planAirports(""" - from test + from airports | stats centroid = st_centroid_agg(location) """); @@ -4325,7 +4325,7 @@ public void testSpatialTypesAndStatsUseDocValues() { */ public void testSpatialTypesAndStatsUseDocValuesWithEval() { var plan = planAirports(""" - from test + from airports | stats centroid = st_centroid_agg(to_geopoint(location)) """); @@ -4361,7 +4361,7 @@ public void testTrivialTypeConversionWrittenAway() { default -> "to_" + type; }; var field = "types." + type; - var plan = planExtra("from test | eval new_" + field + " = " + func + "(" + field + ")"); + var plan = planExtra("from extra | eval new_" + field + " = " + func + "(" + field + ")"); var eval = as(plan, Eval.class); var alias = as(eval.fields().get(0), Alias.class); assertThat(func + "(" + field + ")", alias.name(), equalTo("new_" + field)); @@ -5318,12 +5318,11 @@ private static boolean oneLeaveIsNull(Expression e) { public void testEmptyMappingIndex() { EsIndex empty = new EsIndex("empty_test", emptyMap(), Map.of()); - IndexResolution getIndexResultAirports = IndexResolution.valid(empty); var analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResultAirports, + indexResolutions(empty), enrichResolution, emptyInferenceResolution() ), @@ -8571,7 +8570,7 @@ public void testSampleMerged() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); var query = """ - FROM TEST + FROM test | SAMPLE .3 | EVAL irrelevant1 = 1 | SAMPLE .5 @@ -8601,7 +8600,7 @@ public void testSamplePushDown() { "GROK first_name \"%{WORD:bar}\"", "DISSECT first_name \"%{z}\"" )) { - var query = "FROM TEST | " + command + " | SAMPLE .5"; + var query = "FROM test | " + command + " | SAMPLE .5"; var optimized = optimizedPlan(query); var unary = as(optimized, UnaryPlan.class); @@ -8616,7 +8615,7 @@ public void testSamplePushDown() { public void testSamplePushDown_sort() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); - var query = "FROM TEST | WHERE emp_no > 0 | SAMPLE 0.5 | LIMIT 100"; + var query = "FROM test | WHERE emp_no > 0 | SAMPLE 0.5 | LIMIT 100"; var optimized = optimizedPlan(query); var limit = as(optimized, Limit.class); @@ -8630,7 +8629,7 @@ public void testSamplePushDown_sort() { public void testSamplePushDown_where() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); - var query = "FROM TEST | SORT emp_no | SAMPLE 0.5 | LIMIT 100"; + var query = "FROM test | SORT emp_no | SAMPLE 0.5 | LIMIT 100"; var optimized = optimizedPlan(query); var topN = as(optimized, TopN.class); @@ -8644,7 +8643,7 @@ public void testSampleNoPushDown() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); for (var command : List.of("LIMIT 100", "MV_EXPAND languages", "STATS COUNT()")) { - var query = "FROM TEST | " + command + " | SAMPLE .5"; + var query = "FROM test | " + command + " | SAMPLE .5"; var optimized = optimizedPlan(query); var limit = as(optimized, Limit.class); @@ -8668,7 +8667,7 @@ public void testSampleNoPushDownLookupJoin() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); var query = """ - FROM TEST + FROM test | EVAL language_code = emp_no | LOOKUP JOIN languages_lookup ON language_code | SAMPLE .5 @@ -8696,7 +8695,7 @@ public void testSampleNoPushDownChangePoint() { assumeTrue("sample must be enabled", EsqlCapabilities.Cap.SAMPLE_V3.isEnabled()); var query = """ - FROM TEST + FROM test | CHANGE_POINT emp_no ON hire_date | SAMPLE .5 """; @@ -8712,7 +8711,7 @@ public void testSampleNoPushDownChangePoint() { public void testPushDownConjunctionsToKnnPrefilter() { var query = """ - from test + from types | where knn(dense_vector, [0, 1, 2]) and integer > 10 """; var optimized = planTypes(query); @@ -8730,7 +8729,7 @@ public void testPushDownConjunctionsToKnnPrefilter() { public void testPushDownMultipleFiltersToKnnPrefilter() { var query = """ - from test + from types | where knn(dense_vector, [0, 1, 2]) | where integer > 10 | where keyword == "test" @@ -8751,7 +8750,7 @@ public void testPushDownMultipleFiltersToKnnPrefilter() { public void testNotPushDownDisjunctionsToKnnPrefilter() { var query = """ - from test + from types | where knn(dense_vector, [0, 1, 2]) or integer > 10 """; var optimized = planTypes(query); @@ -8778,7 +8777,7 @@ public void testPushDownConjunctionsAndNotDisjunctionsToKnnPrefilter() { */ // Both conjunctions are pushed down to knn prefilters, disjunctions are not var query = """ - from test + from types | where ((knn(dense_vector, [0, 1, 2]) or integer > 10) and keyword == "test") and ((short < 5) or (double > 5.0)) """; @@ -8811,7 +8810,7 @@ public void testMorePushDownConjunctionsAndNotDisjunctionsToKnnPrefilter() { */ // Just the conjunction is pushed down to knn prefilters, disjunctions are not var query = """ - from test + from types | where ((knn(dense_vector, [0, 1, 2]) and integer > 10) or keyword == "test") or ((short < 5) and (double > 5.0)) """; @@ -8838,7 +8837,7 @@ public void testMultipleKnnQueriesInPrefilters() { knn(dense_vector, [4, 5, 6], 10) */ var query = """ - from test + from types | where ((knn(dense_vector, [0, 1, 2]) or integer > 10) and ((keyword == "test") or knn(dense_vector, [4, 5, 6]))) """; var optimized = planTypes(query); @@ -8870,7 +8869,7 @@ public void testMultipleKnnQueriesInPrefilters() { public void testKnnImplicitLimit() { var query = """ - from test + from types | where knn(dense_vector, [0, 1, 2]) """; var optimized = planTypes(query); @@ -8883,7 +8882,7 @@ public void testKnnImplicitLimit() { public void testKnnWithLimit() { var query = """ - from test + from types | where knn(dense_vector, [0, 1, 2]) | limit 10 """; @@ -8897,7 +8896,7 @@ public void testKnnWithLimit() { public void testKnnWithTopN() { var query = """ - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) | sort _score desc | limit 10 @@ -8912,7 +8911,7 @@ public void testKnnWithTopN() { public void testKnnWithMultipleLimitsAfterTopN() { var query = """ - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) | limit 20 | sort _score desc @@ -8930,7 +8929,7 @@ public void testKnnWithMultipleLimitsAfterTopN() { public void testKnnWithMultipleLimitsCombined() { var query = """ - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) | limit 20 | limit 10 @@ -8946,7 +8945,7 @@ public void testKnnWithMultipleLimitsCombined() { public void testKnnWithMultipleClauses() { var query = """ - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) and match(keyword, "test") | where knn(dense_vector, [1, 2, 3]) | sort _score @@ -8967,14 +8966,14 @@ public void testKnnWithMultipleClauses() { public void testKnnWithStats() { assertThat( - typesError("from test | where knn(dense_vector, [0, 1, 2]) | stats c = count(*)"), + typesError("from types | where knn(dense_vector, [0, 1, 2]) | stats c = count(*)"), containsString("Knn function must be used with a LIMIT clause") ); } public void testKnnWithRerankAmdTopN() { assertThat(typesError(""" - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) | rerank "some text" on text with { "inference_id" : "reranking-inference-id" } | sort _score desc @@ -8984,7 +8983,7 @@ public void testKnnWithRerankAmdTopN() { public void testKnnWithRerankAmdLimit() { var query = """ - from test metadata _score + from types metadata _score | where knn(dense_vector, [0, 1, 2]) | rerank "some text" on text with { "inference_id" : "reranking-inference-id" } | limit 100 diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 40acba1df02af..250a77765a769 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -189,6 +189,7 @@ import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.core.expression.Expressions.name; import static org.elasticsearch.xpack.esql.core.expression.Expressions.names; import static org.elasticsearch.xpack.esql.core.expression.function.scalar.FunctionTestUtils.l; @@ -393,13 +394,23 @@ TestDataSource makeTestDataSource( SearchStats stats ) { Map mapping = loadMapping(mappingFileName); - EsIndex index = new EsIndex(indexName, mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(index); + EsIndex[] indexes = new EsIndex[1 + lookupResolution.size()]; + indexes[0] = new EsIndex(indexName, mapping, Map.of(indexName, IndexMode.STANDARD)); + for (int i = 0; i < lookupResolution.size(); i++) { + indexes[i + 1] = lookupResolution.values().toArray(new IndexResolution[0])[i].get(); + } Analyzer analyzer = new Analyzer( - testAnalyzerContext(config, functionRegistry, getIndexResult, lookupResolution, enrichResolution, emptyInferenceResolution()), + testAnalyzerContext( + config, + functionRegistry, + indexResolutions(indexes), + lookupResolution, + enrichResolution, + emptyInferenceResolution() + ), TEST_VERIFIER ); - return new TestDataSource(mapping, index, analyzer, stats); + return new TestDataSource(mapping, indexes[0], analyzer, stats); } TestDataSource makeTestDataSource( @@ -1440,7 +1451,7 @@ public void testPushMultipleBinaryLogicFilters() { */ public void testPushMultipleFunctions() { var plan = physicalPlan(""" - from airports + from test | where starts_with(first_name, "*Firs") or ends_with(first_name, "irst*") | where ends_with(last_name, "ast") """); @@ -3700,7 +3711,7 @@ public void testSpatialTypesAndStatsCentroidUseDocValues() { for (boolean withDocValues : new boolean[] { false, true }) { var testData = withDocValues ? airports : airportsNoDocValues; var fieldExtractPreference = withDocValues ? FieldExtractPreference.DOC_VALUES : FieldExtractPreference.NONE; - var plan = physicalPlan(query, testData); + var plan = physicalPlan(query.replace("airports", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -3763,7 +3774,7 @@ public void testSpatialTypesAndStatsExtentUseDocValues() { for (boolean withDocValues : new boolean[] { false, true }) { var fieldExtractPreference = withDocValues ? FieldExtractPreference.DOC_VALUES : FieldExtractPreference.NONE; var testData = withDocValues ? airports : airportsNoDocValues; - var plan = physicalPlan(query, testData); + var plan = physicalPlan(query.replace("airports", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -3826,7 +3837,7 @@ public void testSpatialTypesAndStatsExtentAndCentroidUseDocValues() { for (boolean withDocValues : new boolean[] { false, true }) { var fieldExtractPreference = withDocValues ? FieldExtractPreference.DOC_VALUES : FieldExtractPreference.NONE; var testData = withDocValues ? airports : airportsNoDocValues; - var plan = physicalPlan(query, testData); + var plan = physicalPlan(query.replace("airports", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -3875,7 +3886,7 @@ public void testSpatialTypesAndStatsExtentOfGeoShapeUsesBinaryExtraction() { var query = "FROM airports_city_boundaries | STATS extent = ST_EXTENT_AGG(city_boundary)"; for (boolean useDocValues : new Boolean[] { true, false }) { var testData = useDocValues ? airportsCityBoundaries : airportsCityBoundariesNoDocValues; - var plan = physicalPlan(query, testData); + var plan = physicalPlan(query.replace("airports_city_boundaries", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -3938,11 +3949,14 @@ public void testSpatialTypesAndStatsExtentOfShapesNegativeCases() { */ public void testSpatialTypesAndStatsExtentOfCartesianShapesWithAndWithoutDocValues() { for (boolean hasDocValues : new boolean[] { true, false }) { - var query = """ - FROM cartesian_multipolygons \ - | STATS extent = ST_EXTENT_AGG(shape)"""; - var testData = hasDocValues ? cartesianMultipolygons : cartesianMultipolygonsNoDocValues; - var fieldExtractPreference = hasDocValues ? FieldExtractPreference.EXTRACT_SPATIAL_BOUNDS : FieldExtractPreference.NONE; + var query = "FROM cartesian_multipolygons | STATS extent = ST_EXTENT_AGG(shape)"; + var testData = cartesianMultipolygons; + var fieldExtractPreference = FieldExtractPreference.EXTRACT_SPATIAL_BOUNDS; + if (hasDocValues == false) { + query = "FROM cartesian_multipolygons_no_doc_values | STATS extent = ST_EXTENT_AGG(shape)"; + testData = cartesianMultipolygonsNoDocValues; + fieldExtractPreference = FieldExtractPreference.NONE; + } var plan = physicalPlan(query, testData); var limit = as(plan, LimitExec.class); @@ -3990,7 +4004,7 @@ public void testSpatialTypesAndStatsExtentOfCartesianShapesWithAndWithoutDocValu */ public void testMixedSpatialBoundsAndPointsExtracted() { var query = """ - FROM airports_city_boundaries \ + FROM INDEX \ | STATS extent = ST_EXTENT_AGG(city_boundary), centroid = ST_CENTROID_AGG(city_location)"""; for (boolean pointDocValues : new Boolean[] { true, false }) { for (boolean shapeDocValues : new Boolean[] { true, false }) { @@ -3998,7 +4012,7 @@ public void testMixedSpatialBoundsAndPointsExtracted() { ? (shapeDocValues ? airportsCityBoundaries : airportsCityBoundariesNoShapeDocValues) : (shapeDocValues ? airportsCityBoundariesNoPointDocValues : airportsCityBoundariesNoDocValues); var msg = "DocValues[point:" + pointDocValues + ", shape:" + shapeDocValues + "]"; - var plan = physicalPlan(query, testData); + var plan = physicalPlan(query.replace("INDEX", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -4289,7 +4303,7 @@ public void testSpatialTypesAndStatsUseDocValuesMultiAggregationsGrouped() { var plan = this.physicalPlan(""" FROM airports | STATS centroid=ST_CENTROID_AGG(location), count=COUNT() BY scalerank - """, testData); + """.replace("airports", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); @@ -4856,7 +4870,7 @@ public void testPushSpatialIntersectsStringToSourceAndUseDocValuesForCentroid() var testData = useDocValues ? (isIndexed ? airports : airportsNotIndexed) : (isIndexed ? airportsNoDocValues : airportsNotIndexedNorDocValues); - var plan = this.physicalPlan(query, testData); + var plan = this.physicalPlan(query.replace("airports", testData.index.name()), testData); var limit = as(plan, LimitExec.class); var agg = as(limit.child(), AggregateExec.class); assertThat("No groupings in aggregation", agg.groupings().size(), equalTo(0)); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/HoistRemoteEnrichTopNTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/HoistRemoteEnrichTopNTests.java index 8032f3d469dc1..247fd80cf392b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/HoistRemoteEnrichTopNTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/HoistRemoteEnrichTopNTests.java @@ -17,7 +17,6 @@ import org.elasticsearch.xpack.esql.expression.Order; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.AbstractLogicalPlanOptimizerTests; import org.elasticsearch.xpack.esql.plan.logical.Enrich; import org.elasticsearch.xpack.esql.plan.logical.Eval; @@ -34,6 +33,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.startsWith; @@ -127,12 +127,11 @@ private LogicalPlan planWithPolicyOverride(String query) { ); var mapping = loadMapping("mapping-host_inventory.json"); EsIndex inventory = new EsIndex("host_inventory", mapping, Map.of("host_inventory", IndexMode.STANDARD)); - IndexResolution resolution = IndexResolution.valid(inventory); var analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - resolution, + indexResolutions(inventory), defaultLookupResolution(), enrichResolution, emptyInferenceResolution() diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java index eedad20dcc42f..2f41ace68b8c3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateInlineEvalsTests.java @@ -18,7 +18,6 @@ import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.AbstractLogicalPlanOptimizerTests; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; @@ -42,6 +41,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultInferenceResolution; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; @@ -57,12 +57,11 @@ public static void init() { parser = new EsqlParser(); mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), defaultLookupResolution(), new EnrichResolution(), defaultInferenceResolution() diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java index 260feb5e9b8b2..549ac3987c8b6 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java @@ -174,7 +174,7 @@ public void testShadowingAfterPushdown() { // \_EsRelation[test_lookup][LOOKUP][emp_no{f}#37, languages{f}#40, salary{f}#42] public void testShadowingAfterPushdown2() { String query = """ - FROM test_lookup + FROM test | RENAME emp_no AS x, salary AS salary2 | EVAL y = x, gender = last_name | RENAME y AS languages, gender AS ln @@ -247,7 +247,7 @@ public void testShadowingAfterPushdownExpressionJoin() { ); String query = """ - FROM test_lookup + FROM test | RENAME languages as lang2 | EVAL y = emp_no | RENAME y AS lang @@ -295,7 +295,7 @@ public void testShadowingAfterPushdownExpressionJoinKeepOrig() { ); String query = """ - FROM test_lookup + FROM test | RENAME languages as lang2 | EVAL y = emp_no | RENAME y AS lang @@ -347,7 +347,7 @@ public void testShadowingAfterPushdownRenameExpressionJoin() { ); String query = """ - FROM test_lookup + FROM test | RENAME languages AS lang | LOOKUP JOIN test_lookup ON lang == languages | KEEP languages, emp_no, salary @@ -393,7 +393,7 @@ public void testShadowingAfterPushdownEvalExpressionJoin() { ); String query = """ - FROM test_lookup + FROM test | EVAL lang = languages + 0 | DROP languages | LOOKUP JOIN test_lookup ON lang == languages diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/IgnoreNullMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/IgnoreNullMetricsTests.java index 6f927dae662d8..bbf18569ede0b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/IgnoreNullMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/IgnoreNullMetricsTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.esql.expression.predicate.logical.Or; import org.elasticsearch.xpack.esql.expression.predicate.nulls.IsNotNull; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.LocalLogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; @@ -49,6 +48,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.defaultLookupResolution; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; @@ -84,12 +84,11 @@ private static void init() { new EsField("_tsid", DataType.TSID_DATA_TYPE, Map.of(), true, EsField.TimeSeriesFieldType.NONE) ); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.TIME_SERIES)); - IndexResolution getIndexResult = IndexResolution.valid(test); analyzer = new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), defaultLookupResolution(), enrichResolution, emptyInferenceResolution() diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/AbstractStatementParserTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/AbstractStatementParserTests.java index 86b594c027a4a..a14612a56fb85 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/AbstractStatementParserTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/AbstractStatementParserTests.java @@ -34,7 +34,7 @@ import static org.elasticsearch.xpack.esql.expression.function.FunctionResolutionStrategy.DEFAULT; import static org.hamcrest.Matchers.containsString; -abstract class AbstractStatementParserTests extends ESTestCase { +public abstract class AbstractStatementParserTests extends ESTestCase { EsqlParser parser = new EsqlParser(); @@ -52,7 +52,7 @@ LogicalPlan statement(String query, String arg) { return statement(LoggerMessageFormat.format(null, query, arg), new QueryParams()); } - LogicalPlan statement(String e) { + protected LogicalPlan statement(String e) { return statement(e, new QueryParams()); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java index 96507bf9615a1..9e1084ebe2d07 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/FilterTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.xpack.esql.core.util.Queries; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.io.stream.ExpressionQueryBuilder; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; @@ -61,6 +60,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.core.querydsl.query.Query.unscore; import static org.elasticsearch.xpack.esql.core.util.Queries.Clause.FILTER; import static org.elasticsearch.xpack.esql.core.util.Queries.Clause.MUST; @@ -86,7 +86,6 @@ public static void init() { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); logicalOptimizer = new LogicalPlanOptimizer(unboundLogicalOptimizerContext()); TransportVersion minimumVersion = logicalOptimizer.context().minimumVersion(); physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(EsqlTestUtils.TEST_CFG, minimumVersion)); @@ -96,7 +95,7 @@ public static void init() { new AnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), Map.of(), EsqlTestUtils.emptyPolicyResolution(), emptyInferenceResolution(), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java index 5212abf4270f7..027e7d8b7fc89 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/PlanConcurrencyCalculatorTests.java @@ -36,32 +36,32 @@ public class PlanConcurrencyCalculatorTests extends ESTestCase { public void testSimpleLimit() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 """, 9); } public void testLimitZero() { - assertConcurrency("FROM x | LIMIT 0", null); + assertConcurrency("FROM test | LIMIT 0", null); } public void testBiggestPragmaOverride() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 """, Integer.MAX_VALUE, Integer.MAX_VALUE); } public void testSmallestPragmaOverride() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 """, 1, 1); } public void testPragmaOverrideWithUnsupportedCommands() { assertConcurrency(""" - FROM x + FROM test | WHERE salary * 2 > 5 | LIMIT 512 """, 1, 1); @@ -69,20 +69,20 @@ public void testPragmaOverrideWithUnsupportedCommands() { public void testImplicitLimit() { assertConcurrency(""" - FROM x + FROM test """, 9); } public void testStats() { assertConcurrency(""" - FROM x + FROM test | STATS COUNT(salary) """, null); } public void testStatsWithLimit() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 | STATS COUNT(salary) """, 9); @@ -90,14 +90,14 @@ public void testStatsWithLimit() { public void testSortBeforeLimit() { assertConcurrency(""" - FROM x + FROM test | SORT salary """, null); } public void testSortAfterLimit() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 | SORT salary """, 9); @@ -105,7 +105,7 @@ public void testSortAfterLimit() { public void testStatsWithSortBeforeLimit() { assertConcurrency(""" - FROM x + FROM test | SORT salary | LIMIT 512 | STATS COUNT(salary) @@ -114,7 +114,7 @@ public void testStatsWithSortBeforeLimit() { public void testStatsWithSortAfterLimit() { assertConcurrency(""" - FROM x + FROM test | SORT salary | LIMIT 512 | STATS COUNT(salary) @@ -123,7 +123,7 @@ public void testStatsWithSortAfterLimit() { public void testWhereBeforeLimit() { assertConcurrency(""" - FROM x + FROM test | WHERE salary * 2 > 5 | LIMIT 512 """, null); @@ -131,7 +131,7 @@ public void testWhereBeforeLimit() { public void testWhereAfterLimit() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 | WHERE salary * 2 > 5 """, 9); @@ -139,7 +139,7 @@ public void testWhereAfterLimit() { public void testWherePushedToLuceneQueryBeforeLimit() { assertConcurrency(""" - FROM x + FROM test | WHERE first_name LIKE "A%" | LIMIT 512 """, null); @@ -147,7 +147,7 @@ public void testWherePushedToLuceneQueryBeforeLimit() { public void testWherePushedToLuceneQueryAfterLimit() { assertConcurrency(""" - FROM x + FROM test | LIMIT 512 | WHERE first_name LIKE "A%" """, 9); @@ -155,7 +155,7 @@ public void testWherePushedToLuceneQueryAfterLimit() { public void testExpand() { assertConcurrency(""" - FROM x + FROM test | LIMIT 2048 | MV_EXPAND salary | LIMIT 512 @@ -164,7 +164,7 @@ public void testExpand() { public void testEval() { assertConcurrency(""" - FROM x + FROM test | EVAL x=salary*2 | LIMIT 512 """, 9); @@ -172,7 +172,7 @@ public void testEval() { public void testRename() { assertConcurrency(""" - FROM x + FROM test | RENAME salary as x | LIMIT 512 """, 9); @@ -180,7 +180,7 @@ public void testRename() { public void testKeep() { assertConcurrency(""" - FROM x + FROM test | KEEP salary | LIMIT 512 """, 9); @@ -188,7 +188,7 @@ public void testKeep() { public void testDrop() { assertConcurrency(""" - FROM x + FROM test | DROP salary | LIMIT 512 """, 9); @@ -196,7 +196,7 @@ public void testDrop() { public void testDissect() { assertConcurrency(""" - FROM x + FROM test | DISSECT first_name "%{a} %{b}" | LIMIT 512 """, 9); @@ -204,7 +204,7 @@ public void testDissect() { public void testGrok() { assertConcurrency(""" - FROM x + FROM test | GROK first_name "%{EMAILADDRESS:email}" | LIMIT 512 """, 9); @@ -212,7 +212,7 @@ public void testGrok() { public void testEnrich() { assertConcurrency(""" - FROM x + FROM test | ENRICH languages ON first_name | LIMIT 512 """, 9); @@ -220,7 +220,7 @@ public void testEnrich() { public void testLookup() { assertConcurrency(""" - FROM x + FROM test | RENAME salary as language_code | LOOKUP JOIN languages_lookup on language_code | LIMIT 512 diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java index 4fd28e8897ee7..cca78dcc8b466 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java @@ -32,6 +32,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexWithDateDateNanosUnionType; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.matchesRegex; @@ -45,16 +46,15 @@ public class QueryTranslatorTests extends ESTestCase { private static TestPlannerOptimizer plannerOptimizerDateDateNanosUnionTypes; - private static Analyzer makeAnalyzer(String mappingFileName) { + private static Analyzer makeAnalyzer(String indexName, String mappingFileName) { var mapping = loadMapping(mappingFileName); - EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); + EsIndex test = new EsIndex(indexName, mapping, Map.of(indexName, IndexMode.STANDARD)); return new Analyzer( testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), emptyPolicyResolution(), emptyInferenceResolution() ), @@ -67,7 +67,7 @@ public static Analyzer makeAnalyzer(IndexResolution indexResolution) { testAnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - indexResolution, + indexResolutions(indexResolution), emptyPolicyResolution(), emptyInferenceResolution() ), @@ -77,8 +77,8 @@ public static Analyzer makeAnalyzer(IndexResolution indexResolution) { @BeforeClass public static void init() { - plannerOptimizer = new TestPlannerOptimizer(EsqlTestUtils.TEST_CFG, makeAnalyzer("mapping-all-types.json")); - plannerOptimizerIPs = new TestPlannerOptimizer(EsqlTestUtils.TEST_CFG, makeAnalyzer("mapping-hosts.json")); + plannerOptimizer = new TestPlannerOptimizer(EsqlTestUtils.TEST_CFG, makeAnalyzer("test", "mapping-all-types.json")); + plannerOptimizerIPs = new TestPlannerOptimizer(EsqlTestUtils.TEST_CFG, makeAnalyzer("hosts", "mapping-hosts.json")); } @Override @@ -331,27 +331,27 @@ public void testToDateNanos() { // == term assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos == "2025-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos == "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"term":{"date_and_date_nanos":{"value":"2025-01-01T00:00:00.000Z","boost":0.0}}}""")); // != term assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos != "2025-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos != "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"bool":{"must_not":[{"term":{"date_and_date_nanos":{"value":"2025-01-01T00:00:00.000Z","boost":0.0}}}],\ "boost":0.0}}""")); // > range assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos > "2025-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos > "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"range":{"date_and_date_nanos":{"gt":"2025-01-01T00:00:00.000Z","time_zone":"Z",\ "format":"strict_date_optional_time_nanos","boost":0.0}}}""")); // >= range assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos >= "2025-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos >= "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"range":{"date_and_date_nanos":{"gte":"2025-01-01T00:00:00.000Z","time_zone":"Z",\ "format":"strict_date_optional_time_nanos","boost":0.0}}}""")); @@ -359,7 +359,7 @@ public void testToDateNanos() { // < range assertQueryTranslationDateDateNanosUnionTypes( """ - FROM test* | WHERE date_and_date_nanos < "2025-01-01" and date_and_date_nanos_and_long::date_nanos > "2025-01-01\"""", + FROM index* | WHERE date_and_date_nanos < "2025-01-01" and date_and_date_nanos_and_long::date_nanos > "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ @@ -369,21 +369,21 @@ public void testToDateNanos() { // <= range assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos <= "2025-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos <= "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"range":{"date_and_date_nanos":{"lte":"2025-01-01T00:00:00.000Z","time_zone":"Z",\ "format":"strict_date_optional_time_nanos","boost":0.0}}}""")); // <= and >= assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos <= "2025-01-01" and date_and_date_nanos > "2020-01-01\"""", stats, containsString(""" + FROM index* | WHERE date_and_date_nanos <= "2025-01-01" and date_and_date_nanos > "2020-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ "next":{"range":{"date_and_date_nanos":{"gt":"2020-01-01T00:00:00.000Z","lte":"2025-01-01T00:00:00.000Z","time_zone":"Z",\ "format":"strict_date_optional_time_nanos","boost":0.0}}}""")); // >= or < assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos >= "2025-01-01" or date_and_date_nanos < "2020-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos >= "2025-01-01" or date_and_date_nanos < "2020-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"range":\\{"date_and_date_nanos":\\{"gte":"2025-01-01T00:00:00.000Z",\ "time_zone":"Z","format":"strict_date_optional_time_nanos","boost":0.0.*""" + """ @@ -392,7 +392,7 @@ public void testToDateNanos() { // > or = assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos > "2025-01-01" or date_and_date_nanos == "2020-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos > "2025-01-01" or date_and_date_nanos == "2020-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"range":\\{"date_and_date_nanos":\\{"gt":"2025-01-01T00:00:00.000Z",\ "time_zone":"Z","format":"strict_date_optional_time_nanos","boost":0.0.*""" + """ @@ -401,7 +401,7 @@ public void testToDateNanos() { // < or != assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos < "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos < "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"range":\\{"date_and_date_nanos":\\{"lt":"2020-01-01T00:00:00.000Z",\ "time_zone":"Z","format":"strict_date_optional_time_nanos","boost":0.0.*""" + """ @@ -410,7 +410,7 @@ public void testToDateNanos() { // == or == assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos == "2020-01-01" or date_and_date_nanos == "2025-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos == "2020-01-01" or date_and_date_nanos == "2025-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"term":\\{"date_and_date_nanos":\\{"value":"2020-01-01T00:00:00.000Z",\ "boost":0.0.*""" + """ @@ -419,7 +419,7 @@ public void testToDateNanos() { // != or != assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos != "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos != "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"must_not".*"term":\\{"date_and_date_nanos":\\{"value":\ "2020-01-01T00:00:00.000Z","boost":0.0.*""" + """ @@ -428,7 +428,7 @@ public void testToDateNanos() { // = or != assertQueryTranslationDateDateNanosUnionTypes(""" - FROM test* | WHERE date_and_date_nanos == "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" + FROM index* | WHERE date_and_date_nanos == "2020-01-01" or date_and_date_nanos != "2025-01-01\"""", stats, matchesRegex(""" .*bool.*should.*""" + """ esql_single_value":\\{"field":"date_and_date_nanos".*"term":\\{"date_and_date_nanos":\\{"value":\ "2020-01-01T00:00:00.000Z","boost":0.0.*""" + """ @@ -438,7 +438,7 @@ public void testToDateNanos() { // explicit casting assertQueryTranslationDateDateNanosUnionTypes( """ - FROM test* | WHERE date_and_date_nanos::datetime < "2025-12-31" and date_and_date_nanos > "2025-01-01\"""", + FROM index* | WHERE date_and_date_nanos::datetime < "2025-12-31" and date_and_date_nanos > "2025-01-01\"""", stats, containsString(""" "esql_single_value":{"field":"date_and_date_nanos",\ diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java index 544934f44e0b6..0371b6511fba6 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.parser.EsqlParser; @@ -45,6 +44,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; public class ClusterRequestTests extends AbstractWireSerializingTestCase { @@ -171,7 +171,6 @@ private static String randomQuery() { static Versioned parse(String query) { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); LogicalOptimizerContext context = unboundLogicalOptimizerContext(); TransportVersion minimumVersion = context.minimumVersion(); var logicalOptimizer = new LogicalPlanOptimizer(context); @@ -179,7 +178,7 @@ static Versioned parse(String query) { new AnalyzerContext( EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), - getIndexResult, + indexResolutions(test), Map.of(), emptyPolicyResolution(), emptyInferenceResolution(), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java index dd4a0538988c6..4b6ede9da2ae2 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.index.EsIndex; -import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; @@ -50,6 +49,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.EsqlTestUtils.testAnalyzerContext; import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; public class DataNodeRequestSerializationTests extends AbstractWireSerializingTestCase { @Override @@ -302,9 +302,14 @@ protected DataNodeRequest mutateInstance(DataNodeRequest in) throws IOException static Versioned parse(String query) { Map mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); - IndexResolution getIndexResult = IndexResolution.valid(test); var analyzer = new Analyzer( - testAnalyzerContext(TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution(), emptyInferenceResolution()), + testAnalyzerContext( + TEST_CFG, + new EsqlFunctionRegistry(), + indexResolutions(test), + emptyPolicyResolution(), + emptyInferenceResolution() + ), TEST_VERIFIER ); TransportVersion minimumVersion = analyzer.context().minimumVersion(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtilsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtilsTests.java index 3a2eaa544c6ab..4504e2fdec96b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtilsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlCCSUtilsTests.java @@ -224,7 +224,7 @@ public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() { IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), Map.of()); - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, Set.of(indexResolution)); EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS); assertThat(localCluster.getIndexExpression(), equalTo("logs*")); @@ -267,7 +267,7 @@ public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() { ); IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), Map.of()); - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, Set.of(indexResolution)); EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS); assertThat(localCluster.getIndexExpression(), equalTo("logs*")); @@ -309,7 +309,7 @@ public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() { var failures = Map.of(REMOTE1_ALIAS, List.of(failure)); IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), failures); - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, Set.of(indexResolution)); EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS); assertThat(localCluster.getIndexExpression(), equalTo("logs*")); @@ -350,7 +350,7 @@ public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() { var failure = new FieldCapabilitiesFailure(new String[] { "logs-a" }, new NoSeedNodeLeftException("unable to connect")); var failures = Map.of(REMOTE1_ALIAS, List.of(failure)); IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), failures); - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, Set.of(indexResolution)); EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS); assertThat(localCluster.getIndexExpression(), equalTo("logs*")); @@ -399,7 +399,7 @@ public void testUpdateExecutionInfoWithClustersWithNoMatchingIndices() { var failures = Map.of(REMOTE1_ALIAS, List.of(failure)); IndexResolution indexResolution = IndexResolution.valid(esIndex, esIndex.concreteIndices(), failures); - EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); + EsqlCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, Set.of(indexResolution)); EsqlExecutionInfo.Cluster localCluster = executionInfo.getCluster(LOCAL_CLUSTER_ALIAS); assertThat(localCluster.getIndexExpression(), equalTo("logs*")); @@ -713,7 +713,7 @@ private void assertLicenseCheckPasses( String... expectedRemotes ) { var executionInfo = new EsqlExecutionInfo(true); - initCrossClusterState(indicesGrouper, createLicenseState(status), pattern, executionInfo); + initCrossClusterState(indicesGrouper, createLicenseState(status), Set.of(pattern), executionInfo); assertThat(executionInfo.clusterAliases(), containsInAnyOrder(expectedRemotes)); } @@ -728,7 +728,7 @@ private void assertLicenseCheckFails( equalTo( "A valid Enterprise license is required to run ES|QL cross-cluster searches. License found: " + expectedErrorMessageSuffix ), - () -> initCrossClusterState(indicesGrouper, createLicenseState(licenseStatus), pattern, new EsqlExecutionInfo(true)) + () -> initCrossClusterState(indicesGrouper, createLicenseState(licenseStatus), Set.of(pattern), new EsqlExecutionInfo(true)) ); assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java index 7013cc6bc9ea0..94a1db9bcb676 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/VerifierMetricsTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.esql.telemetry; +import org.elasticsearch.index.IndexMode; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.watcher.common.stats.Counters; @@ -14,6 +15,7 @@ import org.elasticsearch.xpack.esql.analysis.Verifier; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; import org.elasticsearch.xpack.esql.expression.function.FunctionDefinition; +import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.parser.EsqlParser; import java.util.HashMap; @@ -23,6 +25,8 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.withDefaultLimitWarning; import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyzer; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.indexResolutions; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.loadMapping; import static org.elasticsearch.xpack.esql.telemetry.FeatureMetric.DISSECT; import static org.elasticsearch.xpack.esql.telemetry.FeatureMetric.DROP; import static org.elasticsearch.xpack.esql.telemetry.FeatureMetric.ENRICH; @@ -833,7 +837,9 @@ private Counters esql(String esql, Verifier v) { metrics = new Metrics(new EsqlFunctionRegistry()); verifier = new Verifier(metrics, new XPackLicenseState(() -> 0L)); } - analyzer(verifier).analyze(parser.createStatement(esql)); + IndexResolution metricsIndex = loadMapping("mapping-basic.json", "metrics", IndexMode.TIME_SERIES); + IndexResolution employees = loadMapping("mapping-basic.json", "employees"); + analyzer(indexResolutions(metricsIndex, employees), verifier).analyze(parser.createStatement(esql)); return metrics == null ? null : metrics.stats(); }