diff --git a/docs/changelog/133405.yaml b/docs/changelog/133405.yaml new file mode 100644 index 0000000000000..30a86b310e050 --- /dev/null +++ b/docs/changelog/133405.yaml @@ -0,0 +1,5 @@ +pr: 133405 +summary: Fix wrong pruning of plans with no output columns +area: ES|QL +type: bug +issues: [] diff --git a/server/src/main/resources/transport/definitions/referable/esql_plan_with_no_columns.csv b/server/src/main/resources/transport/definitions/referable/esql_plan_with_no_columns.csv new file mode 100644 index 0000000000000..085d1343edd9e --- /dev/null +++ b/server/src/main/resources/transport/definitions/referable/esql_plan_with_no_columns.csv @@ -0,0 +1 @@ +9186000 diff --git a/server/src/main/resources/transport/upper_bounds/9.3.csv b/server/src/main/resources/transport/upper_bounds/9.3.csv index 2147eab66c207..dfb000bd20c3d 100644 --- a/server/src/main/resources/transport/upper_bounds/9.3.csv +++ b/server/src/main/resources/transport/upper_bounds/9.3.csv @@ -1 +1 @@ -initial_9.2.0,9185000 +esql_plan_with_no_columns,9186000 diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicPageTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicPageTests.java index 55b39766ca350..809f9332f888e 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicPageTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicPageTests.java @@ -51,6 +51,15 @@ public void testEqualityAndHashCodeSmallInput() { ); in.releaseBlocks(); + in = new Page(10); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + in, + page -> new Page(10), + page -> new Page(8, blockFactory.newConstantIntBlockWith(1, 8)), + Page::releaseBlocks + ); + in.releaseBlocks(); + in = new Page(blockFactory.newIntArrayVector(new int[] {}, 0).asBlock()); EqualsHashCodeTestUtils.checkEqualsAndHashCode( in, @@ -133,8 +142,8 @@ public void testEqualityAndHashCode() throws IOException { return new Page(blocks); }; - int positions = randomIntBetween(1, 512); - int blockCount = randomIntBetween(1, 256); + int positions = randomIntBetween(0, 512); + int blockCount = randomIntBetween(0, 256); Block[] blocks = new Block[blockCount]; for (int blockIndex = 0; blockIndex < blockCount; blockIndex++) { blocks[blockIndex] = switch (randomInt(9)) { @@ -198,6 +207,16 @@ public void testAppend() { page2.releaseBlocks(); } + public void testAppendToEmpty() { + Page page1 = new Page(10); + Page page2 = page1.appendBlock(blockFactory.newLongArrayVector(LongStream.range(0, 10).toArray(), 10).asBlock()); + assertThat(0, is(page1.getBlockCount())); + assertThat(1, is(page2.getBlockCount())); + LongBlock block1 = page2.getBlock(0); + IntStream.range(0, 10).forEach(i -> assertThat((long) i, is(block1.getLong(i)))); + page2.releaseBlocks(); + } + public void testPageSerializationSimple() throws IOException { IntVector toFilter = blockFactory.newIntArrayVector(IntStream.range(0, 20).toArray(), 20); Page origPage = new Page( @@ -248,6 +267,22 @@ public void testPageSerializationSimple() throws IOException { } } + public void testPageSerializationEmpty() throws IOException { + Page origPage = new Page(10); + try { + Page deserPage = serializeDeserializePage(origPage); + try { + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origPage, unused -> deserPage); + assertEquals(origPage.getBlockCount(), deserPage.getBlockCount()); + assertEquals(origPage.getPositionCount(), deserPage.getPositionCount()); + } finally { + deserPage.releaseBlocks(); + } + } finally { + origPage.releaseBlocks(); + } + } + public void testSerializationListPages() throws IOException { final int positions = randomIntBetween(1, 64); List origPages = List.of( @@ -265,7 +300,8 @@ public void testSerializationListPages() throws IOException { positions ) ), - new Page(blockFactory.newConstantBytesRefBlockWith(new BytesRef("Hello World"), positions)) + new Page(blockFactory.newConstantBytesRefBlockWith(new BytesRef("Hello World"), positions)), + new Page(10) ); try { EqualsHashCodeTestUtils.checkEqualsAndHashCode(origPages, page -> { 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 badf7b1f1d62f..bb64309140ac0 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 @@ -36,6 +36,7 @@ import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.lucene.DataPartitioning; import org.elasticsearch.core.PathUtils; import org.elasticsearch.core.SuppressForbidden; @@ -498,7 +499,11 @@ public static LogicalPlan emptySource() { } public static LogicalPlan localSource(BlockFactory blockFactory, List fields, List row) { - return new LocalRelation(Source.EMPTY, fields, LocalSupplier.of(BlockUtils.fromListRow(blockFactory, row))); + return new LocalRelation( + Source.EMPTY, + fields, + LocalSupplier.of(row.isEmpty() ? new Page(0) : new Page(BlockUtils.fromListRow(blockFactory, row))) + ); } public static T as(Object node, Class type) { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/EsqlQueryGenerator.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/EsqlQueryGenerator.java index a235422cf005b..8b3d12772b53f 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/EsqlQueryGenerator.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/EsqlQueryGenerator.java @@ -11,6 +11,7 @@ import org.elasticsearch.xpack.esql.generator.command.CommandGenerator; import org.elasticsearch.xpack.esql.generator.command.pipe.ChangePointGenerator; import org.elasticsearch.xpack.esql.generator.command.pipe.DissectGenerator; +import org.elasticsearch.xpack.esql.generator.command.pipe.DropAllGenerator; import org.elasticsearch.xpack.esql.generator.command.pipe.DropGenerator; import org.elasticsearch.xpack.esql.generator.command.pipe.EnrichGenerator; import org.elasticsearch.xpack.esql.generator.command.pipe.EvalGenerator; @@ -63,6 +64,7 @@ public class EsqlQueryGenerator { ChangePointGenerator.INSTANCE, DissectGenerator.INSTANCE, DropGenerator.INSTANCE, + DropAllGenerator.INSTANCE, EnrichGenerator.INSTANCE, EvalGenerator.INSTANCE, ForkGenerator.INSTANCE, diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/command/pipe/DropAllGenerator.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/command/pipe/DropAllGenerator.java new file mode 100644 index 0000000000000..622c77572fc9c --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/generator/command/pipe/DropAllGenerator.java @@ -0,0 +1,69 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.generator.command.pipe; + +import org.elasticsearch.xpack.esql.generator.Column; +import org.elasticsearch.xpack.esql.generator.EsqlQueryGenerator; +import org.elasticsearch.xpack.esql.generator.QueryExecutor; +import org.elasticsearch.xpack.esql.generator.command.CommandGenerator; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class DropAllGenerator implements CommandGenerator { + + public static final String DROP_ALL = "drop_all"; + public static final String DROPPED_COLUMNS = "dropped_columns"; + + public static final CommandGenerator INSTANCE = new DropAllGenerator(); + + @Override + public CommandDescription generate( + List previousCommands, + List previousOutput, + QuerySchema schema, + QueryExecutor executor + ) { + Set droppedColumns = new HashSet<>(); + String name = EsqlQueryGenerator.randomStringField(previousOutput); + if (name == null || name.isEmpty()) { + return CommandGenerator.EMPTY_DESCRIPTION; + } + + String cmdString = " | keep " + name + " | drop " + name; + return new CommandDescription(DROP_ALL, this, cmdString, Map.ofEntries(Map.entry(DROPPED_COLUMNS, droppedColumns))); + } + + @Override + @SuppressWarnings("unchecked") + public ValidationResult validateOutput( + List previousCommands, + CommandDescription commandDescription, + List previousColumns, + List> previousOutput, + List columns, + List> output + ) { + if (commandDescription == EMPTY_DESCRIPTION) { + return VALIDATION_OK; + } + + if (columns.size() > 0) { + return new ValidationResult( + false, + "Expecting no columns, got [" + columns.stream().map(Column::name).collect(Collectors.joining(", ")) + "]" + ); + } + + return VALIDATION_OK; + } + +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/drop.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/drop.csv-spec index eec8e073e3eec..3134233f371d3 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/drop.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/drop.csv-spec @@ -49,29 +49,86 @@ b:integer | x:integer ; dropAllColumns -from employees | keep height | drop height | eval x = 1; +required_capability: fix_no_columns +from employees | limit 4 | keep height | drop height | eval x = 1; x:integer +1 +1 +1 +1 +; + +dropEvalKeep +required_capability: fix_no_columns +from employees | drop salary | eval salary = 1 | keep salary | limit 4; + +salary:integer +1 +1 +1 +1 ; + +dropEvalStats +required_capability: fix_no_columns +from mv_sample* | drop `client_ip`, message | eval `event_duration` = "foo", @timestamp = 1 | stats max(@timestamp) by event_duration; + +max(@timestamp):integer | event_duration:keyword +1 | foo +; + + + +dropAllColumnsIndexPattern +required_capability: fix_no_columns +from emp* | drop languages | eval languages = 123 | keep languages | limit 4; + +languages:integer +123 +123 +123 +123 +; + + +dropAllColumnsWithMetadata +required_capability: fix_no_columns +from employees metadata _index | drop languages | eval languages = 123 | keep languages | limit 4; + +languages:integer +123 +123 +123 +123 +; + + dropAllColumns_WithLimit +required_capability: fix_no_columns from employees | keep height | drop height | eval x = 1 | limit 3; x:integer +1 +1 +1 ; dropAllColumns_WithCount +required_capability: fix_no_columns from employees | keep height | drop height | eval x = 1 | stats c=count(x); c:long -0 +100 ; dropAllColumns_WithStats +required_capability: fix_no_columns from employees | keep height | drop height | eval x = 1 | stats c=count(x), mi=min(x), s=sum(x); -c:l|mi:i|s:l -0 |null|null +c:l | mi:i | s:l +100 | 1 | 100 ; 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 b0d6f1099ec1f..6c46e558c49c8 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 @@ -5287,6 +5287,21 @@ null | null | corge | null | null null | null | fred | null | null ; +lookupAfterDropAllColumns +required_capability: fix_no_columns +required_capability: join_lookup_v12 +FROM languages +| DROP language_code +| EVAL language_code = 3 +| LOOKUP JOIN languages_lookup ON language_code +| LIMIT 3 +; + + language_code:integer | language_name:keyword +3 |Spanish +3 |Spanish +3 |Spanish +; lookupJoinWithPushableFilterOnRight required_capability: join_lookup_v12 diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/sample.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/sample.csv-spec index 8ecf1b7d374de..405e618b91707 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/sample.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/sample.csv-spec @@ -246,3 +246,13 @@ emp_no:integer 10081 // end::sampleForDocs-result[] ; + + +sampleStatsEval +required_capability: fix_no_columns +required_capability: sample_v3 +FROM employees | SAMPLE 0.5 | LIMIT 10 | STATS count = COUNT() | EVAL is_expected = count > 0; + +count:long | is_expected:boolean +10 | true +; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterLookupJoinIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterLookupJoinIT.java index 8485e515a60d7..fa2a4e255332e 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterLookupJoinIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/CrossClusterLookupJoinIT.java @@ -445,7 +445,7 @@ public void testLookupJoinEmptyIndex() throws IOException { setSkipUnavailable(REMOTE_CLUSTER_1, randomBoolean()); Exception ex; - for (String index : List.of("values_lookup", "values_lookup_map", "values_lookup_map_lookup")) { + for (String index : List.of("values_lookup", "values_lookup_map_lookup")) { ex = expectThrows( VerificationException.class, () -> runQuery("FROM logs-* | LOOKUP JOIN " + index + " ON v | KEEP v", randomBoolean()) @@ -457,6 +457,18 @@ public void testLookupJoinEmptyIndex() throws IOException { ); assertThat(ex.getMessage(), containsString("Unknown column [v] in right side of join")); } + + ex = expectThrows( + VerificationException.class, + () -> runQuery("FROM logs-* | LOOKUP JOIN values_lookup_map ON v | KEEP v", randomBoolean()) + ); + assertThat( + ex.getMessage(), + containsString( + "Lookup Join requires a single lookup mode index; " + + "[values_lookup_map] resolves to [values_lookup_map] in [standard] mode" + ) + ); } public void testLookupJoinIndexMode() throws IOException { diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java index 1c409a6b47820..2f8245107488c 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java @@ -959,7 +959,7 @@ public void testDropAllColumns() { logger.info(results); assertThat(results.columns(), hasSize(1)); assertThat(results.columns(), contains(new ColumnInfoImpl("a", "integer", null))); - assertThat(getValuesList(results), is(empty())); + assertThat(getValuesList(results).size(), is(40)); } } @@ -967,7 +967,7 @@ public void testDropAllColumnsWithStats() { try (EsqlQueryResponse results = run("from test | stats g = count(data) | drop g")) { logger.info(results); assertThat(results.columns(), is(empty())); - assertThat(getValuesList(results), is(empty())); + assertThat(getValuesList(results).size(), is(1)); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 81bc6a5d89a0f..6f694adb611fa 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -1587,7 +1587,13 @@ public enum Cap { /** * Support for requesting the "_tsid" metadata field. */ - METADATA_TSID_FIELD; + METADATA_TSID_FIELD, + + /** + * Fix management of plans with no columns + * https://github.com/elastic/elasticsearch/issues/120272 + */ + FIX_NO_COLUMNS; private final boolean enabled; 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 7057ca3d36a45..ea4268bd0e5e9 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 @@ -12,6 +12,7 @@ import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Strings; import org.elasticsearch.index.IndexMode; import org.elasticsearch.logging.Logger; @@ -472,7 +473,7 @@ private LocalRelation tableMapAsRelation(Source source, Map mapT // prepare the block for the supplier blocks[i++] = column.values(); } - LocalSupplier supplier = LocalSupplier.of(blocks); + LocalSupplier supplier = LocalSupplier.of(blocks.length > 0 ? new Page(blocks) : new Page(0)); return new LocalRelation(source, attributes, supplier); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/Predicates.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/Predicates.java index 64fd63a844957..b9a58e82a2349 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/Predicates.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/Predicates.java @@ -145,6 +145,9 @@ public static Tuple> extractCommon(List } splitAnds.add(split); } + if (common == null) { + common = List.of(); + } List trimmed = new ArrayList<>(expressions.size()); final List finalCommon = common; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java index 5cc1864aa2c5c..ae346e4d52449 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java @@ -29,7 +29,7 @@ import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropagateNullable; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PropgateUnmappedFields; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneColumns; -import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneEmptyPlans; +import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneEmptyAggregates; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneFilters; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneLiteralsInOrderBy; import org.elasticsearch.xpack.esql.optimizer.rules.logical.PruneRedundantOrderBy; @@ -168,7 +168,6 @@ protected static Batch operators(boolean local) { "Operator Optimization", new CombineProjections(local), new CombineEvals(), - new PruneEmptyPlans(), new PropagateEmptyRelation(), new FoldNull(), new SplitInWithFoldableValue(), @@ -207,7 +206,8 @@ protected static Batch operators(boolean local) { new PruneRedundantOrderBy(), new PruneRedundantSortClauses(), new PruneLeftJoinOnNullMatchingField(), - new PruneInlineJoinOnEmptyRightSide() + new PruneInlineJoinOnEmptyRightSide(), + new PruneEmptyAggregates() ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java index b79bcfaefeb8f..818b507d6ba67 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PropagateEmptyRelation.java @@ -10,6 +10,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.FoldContext; @@ -40,7 +41,10 @@ protected LogicalPlan rule(UnaryPlan plan, LogicalOptimizerContext ctx) { // only care about non-grouped aggs might return something (count) if (plan instanceof Aggregate agg && agg.groupings().isEmpty()) { List emptyBlocks = aggsFromEmpty(ctx.foldCtx(), agg.aggregates()); - p = replacePlanByRelation(plan, LocalSupplier.of(emptyBlocks.toArray(Block[]::new))); + p = replacePlanByRelation( + plan, + LocalSupplier.of(emptyBlocks.isEmpty() ? new Page(0) : new Page(emptyBlocks.toArray(Block[]::new))) + ); } else { p = PruneEmptyPlans.skipPlan(plan); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java index 96fea83dc05f1..78b7a4d439680 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneColumns.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical; -import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.index.IndexMode; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Attribute; @@ -111,7 +111,7 @@ private static LogicalPlan pruneColumnsInAggregate(Aggregate aggregate, Attribut p = new LocalRelation( aggregate.source(), List.of(Expressions.attribute(aggregate.aggregates().getFirst())), - LocalSupplier.of(new Block[] { BlockUtils.constantBlock(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, null, 1) }) + LocalSupplier.of(new Page(BlockUtils.constantBlock(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, null, 1))) ); } else { // Aggs cannot produce pages with 0 columns, so retain one grouping. diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyAggregates.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyAggregates.java new file mode 100644 index 0000000000000..116c723fd4de2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneEmptyAggregates.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.optimizer.rules.logical; + +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.xpack.esql.plan.logical.Aggregate; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; + +import java.util.List; + +/** + * STATS with no aggregates and no groupings can be replaced with a single, empty row. + * This can happen due to expression pruning at optimization time, after all them are dropped, + * eg. + * + * STATS a = count(*) by b | drop a, b + */ +public final class PruneEmptyAggregates extends OptimizerRules.OptimizerRule { + @Override + protected LogicalPlan rule(Aggregate agg) { + if (agg.aggregates().isEmpty() && agg.groupings().isEmpty()) { + return new LocalRelation(agg.source(), List.of(), LocalSupplier.of(new Page(1))); + } + return agg; + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java index a5ea01c53aa4e..cbc2e267754b4 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRowAsLocalRelation.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.Row; @@ -29,6 +30,6 @@ protected LogicalPlan rule(Row row, LogicalOptimizerContext context) { List values = new ArrayList<>(fields.size()); fields.forEach(f -> values.add(f.child().fold(context.foldCtx()))); var blocks = BlockUtils.fromListRow(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, values); - return new LocalRelation(row.source(), row.output(), new CopyingLocalSupplier(blocks)); + return new LocalRelation(row.source(), row.output(), new CopyingLocalSupplier(blocks.length == 0 ? new Page(0) : new Page(blocks))); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java index 62ee81f04c892..9a6176fa5628d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEval.java @@ -9,6 +9,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.Literal; @@ -136,6 +137,6 @@ private static LocalRelation localRelation(Source source, List newEvals) attributes.add(alias.toAttribute()); blocks[i] = BlockUtils.constantBlock(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, ((Literal) alias.child()).value(), 1); } - return new LocalRelation(source, attributes, LocalSupplier.of(blocks)); + return new LocalRelation(source, attributes, LocalSupplier.of(new Page(blocks))); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java index c762015dc597e..c48320a0dcf2c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SubstituteSurrogateAggregations.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical; -import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.EmptyAttribute; @@ -114,7 +114,7 @@ protected LogicalPlan rule(Aggregate aggregate) { plan = new LocalRelation( source, List.of(new EmptyAttribute(source)), - LocalSupplier.of(new Block[] { BlockUtils.constantBlock(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, null, 1) }) + LocalSupplier.of(new Page(BlockUtils.constantBlock(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, null, 1))) ); } // 5. force the initial projection in place diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java index 0fff9e233e956..c16be81a405b2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushStatsToSource.java @@ -51,7 +51,7 @@ protected PhysicalPlan rule(AggregateExec aggregateExec, LocalPhysicalOptimizerC // for the moment support pushing count just for one field List stats = tuple.v2(); - if (stats.size() > 1) { + if (stats.size() != 1) { return aggregateExec; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java index d7cd0a558e1f5..d0bdf627db2c2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java @@ -10,8 +10,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.Literal; @@ -65,11 +65,13 @@ public static LogicalPlan stubSource(UnaryPlan sourcePlan, LogicalPlan target) { public static LogicalPlan inlineData(InlineJoin target, LocalRelation data) { if (target.config().leftFields().isEmpty()) { List schema = data.output(); - Block[] blocks = data.supplier().get(); + Page page = data.supplier().get(); List aliases = new ArrayList<>(schema.size()); for (int i = 0; i < schema.size(); i++) { Attribute attr = schema.get(i); - aliases.add(new Alias(attr.source(), attr.name(), Literal.of(attr, BlockUtils.toJavaObject(blocks[i], 0)), attr.id())); + aliases.add( + new Alias(attr.source(), attr.name(), Literal.of(attr, BlockUtils.toJavaObject(page.getBlock(i), 0)), attr.id()) + ); } return new Eval(target.source(), target.left(), aliases); } else { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplier.java index 5f8e0e50731d5..8234605c53f5d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplier.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplier.java @@ -11,27 +11,27 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.optimizer.rules.logical.ReplaceRowAsLocalRelation; import org.elasticsearch.xpack.esql.plan.logical.InlineStats; import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.session.EsqlSession; import java.io.IOException; -import java.util.Arrays; /** - * A {@link LocalSupplier} that allways creates a new copy of the {@link Block}s initially provided at creation time. + * A {@link LocalSupplier} that allways creates a new copy of the {@link Page} initially provided at creation time. * This is created specifically for {@link InlineStats} usage in {@link EsqlSession} for queries that use ROW command. * - * The ROW which gets replaced by {@link ReplaceRowAsLocalRelation} with a {@link LocalRelation} will have its blocks + * The ROW which gets replaced by {@link ReplaceRowAsLocalRelation} with a {@link LocalRelation} will have its page * used (and released) at least twice: * - the {@link LocalRelation} from the left-hand side is used as a source for the right-hand side * - the same {@link LocalRelation} is then used to continue the execution of the query on the left-hand side * * It delegates all its operations to {@link ImmediateLocalSupplier} and, to prevent the double release, it will always - * create a deep copy of the blocks received in the constructor initially. + * create a deep copy of the page received in the constructor initially. * - * Example with the flow and the blocks reuse for a query like "row x = 1 | inline stats y = max(x)" + * Example with the flow and the page reuse for a query like "row x = 1 | inline stats y = max(x)" * Step 1: * Limit[1000[INTEGER],true] * \_InlineJoin[LEFT,[],[],[]] @@ -61,8 +61,8 @@ public class CopyingLocalSupplier implements LocalSupplier { private final ImmediateLocalSupplier delegate; - public CopyingLocalSupplier(Block[] blocks) { - delegate = new ImmediateLocalSupplier(blocks); + public CopyingLocalSupplier(Page page) { + delegate = new ImmediateLocalSupplier(page); } public CopyingLocalSupplier(StreamInput in) throws IOException { @@ -70,12 +70,12 @@ public CopyingLocalSupplier(StreamInput in) throws IOException { } @Override - public Block[] get() { - Block[] blockCopies = new Block[delegate.blocks.length]; + public Page get() { + Block[] blockCopies = new Block[delegate.page.getBlockCount()]; for (int i = 0; i < blockCopies.length; i++) { - blockCopies[i] = delegate.blocks[i].deepCopy(PlannerUtils.NON_BREAKING_BLOCK_FACTORY); + blockCopies[i] = delegate.page.getBlock(i).deepCopy(PlannerUtils.NON_BREAKING_BLOCK_FACTORY); } - return blockCopies; + return new Page(delegate.page.getPositionCount(), blockCopies); } @Override @@ -94,7 +94,7 @@ public boolean equals(Object obj) { return false; } CopyingLocalSupplier other = (CopyingLocalSupplier) obj; - return Arrays.equals(delegate.blocks, other.delegate.blocks); + return delegate.page.equals(other.delegate.page); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EmptyLocalSupplier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EmptyLocalSupplier.java index 265b1d6c434f1..ae8465b7bcd53 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EmptyLocalSupplier.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EmptyLocalSupplier.java @@ -9,8 +9,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import java.io.IOException; @@ -28,8 +28,8 @@ public String getWriteableName() { } @Override - public Block[] get() { - return BlockUtils.NO_BLOCKS; + public Page get() { + return new Page(0, BlockUtils.NO_BLOCKS); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplier.java index 8c2a51459e7dd..bc32588f838ff 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplier.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplier.java @@ -7,49 +7,73 @@ package org.elasticsearch.xpack.esql.plan.logical.local; +import org.elasticsearch.TransportVersion; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; import java.io.IOException; -import java.util.Arrays; /** * A {@link LocalSupplier} that contains already filled {@link Block}s. */ public class ImmediateLocalSupplier implements LocalSupplier { + private static final TransportVersion ESQL_PLAN_WITH_NO_COLUMNS = TransportVersion.fromName("esql_plan_with_no_columns"); + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( LocalSupplier.class, "ImmediateSupplier", ImmediateLocalSupplier::new ); - final Block[] blocks; + protected Page page; - ImmediateLocalSupplier(Block[] blocks) { - this.blocks = blocks; + ImmediateLocalSupplier(Page page) { + this.page = page; } ImmediateLocalSupplier(StreamInput in) throws IOException { - this(((PlanStreamInput) in).readCachedBlockArray()); + this( + in.getTransportVersion().supports(ESQL_PLAN_WITH_NO_COLUMNS) + ? new Page(in.readInt(), ((PlanStreamInput) in).readCachedBlockArray()) + : legacyPage((PlanStreamInput) in) + ); + } + + private static Page legacyPage(PlanStreamInput in) throws IOException { + Block[] blocks = in.readCachedBlockArray(); + if (blocks.length == 0) { + // the page can't determine the position count from an empty array + return new Page(0, blocks); + } + return new Page(blocks); } @Override - public Block[] get() { - return blocks; + public Page get() { + return page; } @Override public String toString() { - return Arrays.toString(blocks); + return page.toString(); } @Override public void writeTo(StreamOutput out) throws IOException { + if (out.getTransportVersion().supports(ESQL_PLAN_WITH_NO_COLUMNS)) { + out.writeInt(page.getPositionCount()); + } + + Block[] blocks = new Block[page.getBlockCount()]; + for (int i = 0; i < blocks.length; i++) { + blocks[i] = page.getBlock(i); + } out.writeArray((o, v) -> ((PlanStreamOutput) o).writeCachedBlock(v), blocks); } @@ -59,12 +83,12 @@ public boolean equals(Object obj) { return false; } ImmediateLocalSupplier other = (ImmediateLocalSupplier) obj; - return Arrays.equals(blocks, other.blocks); + return page.equals(other.page); } @Override public int hashCode() { - return Arrays.hashCode(blocks); + return page.hashCode(); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplier.java index fe626d84b2902..4431ab067b3bd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplier.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplier.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import java.util.function.Supplier; @@ -22,10 +23,10 @@ * {@link UnsupportedOperationException}. *

*/ -public interface LocalSupplier extends Supplier, NamedWriteable { +public interface LocalSupplier extends Supplier, NamedWriteable { - static LocalSupplier of(Block[] blocks) { - return new ImmediateLocalSupplier(blocks); + static LocalSupplier of(Page page) { + return new ImmediateLocalSupplier(page); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LocalSourceExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LocalSourceExec.java index 48677db3f4325..609e9ea7f9232 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LocalSourceExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LocalSourceExec.java @@ -12,6 +12,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; @@ -64,7 +65,7 @@ public LocalSourceExec(StreamInput in) throws IOException { */ public static LocalSupplier readLegacyLocalSupplierFrom(PlanStreamInput in) throws IOException { Block[] blocks = in.readCachedBlockArray(); - return blocks.length == 0 ? EmptyLocalSupplier.EMPTY : LocalSupplier.of(blocks); + return blocks.length == 0 ? EmptyLocalSupplier.EMPTY : LocalSupplier.of(new Page(blocks)); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java index 7778ef601eeb3..926c2912c25bf 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java @@ -656,7 +656,7 @@ private PhysicalOperation planHashJoin(HashJoinExec join, LocalExecutionPlannerC } Layout layout = layoutBuilder.build(); LocalSourceExec localSourceExec = (LocalSourceExec) join.joinData(); - Block[] localData = localSourceExec.supplier().get(); + Page localData = localSourceExec.supplier().get(); RowInTableLookupOperator.Key[] keys = new RowInTableLookupOperator.Key[join.leftFields().size()]; int[] blockMapping = new int[join.leftFields().size()]; @@ -667,7 +667,7 @@ private PhysicalOperation planHashJoin(HashJoinExec join, LocalExecutionPlannerC List output = join.joinData().output(); for (int l = 0; l < output.size(); l++) { if (output.get(l).name().equals(right.name())) { - localField = localData[l]; + localField = localData.getBlock(l); } } if (localField == null) { @@ -688,7 +688,7 @@ private PhysicalOperation planHashJoin(HashJoinExec join, LocalExecutionPlannerC Block localField = null; for (int l = 0; l < joinDataOutput.size(); l++) { if (joinDataOutput.get(l).name().equals(f.name())) { - localField = localData[l]; + localField = localData.getBlock(l); } } if (localField == null) { @@ -813,7 +813,7 @@ private static EsRelation findEsRelation(PhysicalPlan node) { private PhysicalOperation planLocal(LocalSourceExec localSourceExec, LocalExecutionPlannerContext context) { Layout.Builder layout = new Layout.Builder(); layout.append(localSourceExec.output()); - LocalSourceOperator.BlockSupplier supplier = () -> localSourceExec.supplier().get(); + LocalSourceOperator.PageSupplier supplier = () -> localSourceExec.supplier().get(); var operator = new LocalSourceOperator(supplier); return PhysicalOperation.fromSource(new LocalSourceFactory(() -> operator), layout.build()); } 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 a4bfa329d8ac5..293bbb89dc121 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 @@ -234,7 +234,7 @@ public void executeOptimizedPlan( values.add(List.of("coordinator", "optimizedLogicalPlan", optimizedLogicalPlanString)); values.add(List.of("coordinator", "optimizedPhysicalPlan", physicalPlanString)); var blocks = BlockUtils.fromList(PlannerUtils.NON_BREAKING_BLOCK_FACTORY, values); - physicalPlan = new LocalSourceExec(Source.EMPTY, fields, LocalSupplier.of(blocks)); + physicalPlan = new LocalSourceExec(Source.EMPTY, fields, LocalSupplier.of(new Page(blocks))); planRunner.run(physicalPlan, listener); } else { // TODO: this could be snuck into the underlying listener @@ -292,13 +292,13 @@ private void executeSubPlan( executionInfo.startSubPlans(); runner.run(physicalSubPlan, listener.delegateFailureAndWrap((next, result) -> { - AtomicReference localRelationBlocks = new AtomicReference<>(); + AtomicReference localRelationPage = new AtomicReference<>(); try { // Translate the subquery into a separate, coordinator based plan and the results 'broadcasted' as a local relation completionInfoAccumulator.accumulate(result.completionInfo()); LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan().source(), result); - localRelationBlocks.set(resultWrapper.supplier().get()); - var releasingNext = ActionListener.runAfter(next, () -> releaseLocalRelationBlocks(localRelationBlocks)); + localRelationPage.set(resultWrapper.supplier().get()); + var releasingNext = ActionListener.runAfter(next, () -> releaseLocalRelationBlocks(localRelationPage)); subPlansResults.add(resultWrapper); // replace the original logical plan with the backing result @@ -340,7 +340,7 @@ private void executeSubPlan( } catch (Exception e) { // safely release the blocks in case an exception occurs either before, but also after the "final" runner.run() forks off // the current thread, but with the blocks still referenced - releaseLocalRelationBlocks(localRelationBlocks); + releaseLocalRelationBlocks(localRelationPage); throw e; } finally { Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks))); @@ -356,14 +356,15 @@ private LocalRelation resultToPlan(Source planSource, Result result) { actual -> "sub-plan execution results too large [" + ByteSizeValue.ofBytes(actual) + "] > " + intermediateLocalRelationMaxSize ); List schema = result.schema(); + Block[] blocks = SessionUtils.fromPages(schema, pages, blockFactory); - return new LocalRelation(planSource, schema, LocalSupplier.of(blocks)); + return new LocalRelation(planSource, schema, LocalSupplier.of(blocks.length == 0 ? new Page(0) : new Page(blocks))); } - private static void releaseLocalRelationBlocks(AtomicReference localRelationBlocks) { - Block[] relationBlocks = localRelationBlocks.getAndSet(null); - if (relationBlocks != null) { - Releasables.closeExpectNoException(relationBlocks); + private static void releaseLocalRelationBlocks(AtomicReference localRelationPage) { + Page relationPage = localRelationPage.getAndSet(null); + if (relationPage != null) { + Releasables.closeExpectNoException(relationPage); } } @@ -632,8 +633,9 @@ private ElasticsearchException findFailure(Map allFields = new HashSet<>(fieldNames.stream().flatMap(FieldNameUtils::withSubfields).collect(toSet())); + allFields.add(MetadataAttribute.INDEX); + return new PreAnalysisResult(allFields, wildcardJoinIndices); } } 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 1ea527db909f1..6f1f060b466d2 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 @@ -12,6 +12,7 @@ import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.compute.aggregation.QuantileStates; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Tuple; @@ -190,7 +191,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.hasItem; @@ -211,9 +211,10 @@ public void testEmptyProjections() { | drop salary """); - var relation = as(plan, LocalRelation.class); - assertThat(relation.output(), is(empty())); - assertThat(relation.supplier().get(), emptyArray()); + var project = as(plan, EsqlProject.class); + assertThat(project.expressions(), is(empty())); + var limit = as(project.child(), Limit.class); + as(limit.child(), EsRelation.class); } public void testEmptyProjectionInStat() { @@ -222,10 +223,12 @@ public void testEmptyProjectionInStat() { | stats c = count(salary) | drop c """); - - var relation = as(plan, LocalRelation.class); + var limit = as(plan, Limit.class); + var relation = as(limit.child(), LocalRelation.class); assertThat(relation.output(), is(empty())); - assertThat(relation.supplier().get(), emptyArray()); + Page page = relation.supplier().get(); + assertThat(page.getBlockCount(), is(0)); + assertThat(page.getPositionCount(), is(1)); } /** @@ -252,8 +255,8 @@ public void testEmptyProjectInStatWithEval() { var limit = as(eval.child(), Limit.class); var singleRowRelation = as(limit.child(), LocalRelation.class); var singleRow = singleRowRelation.supplier().get(); - assertThat(singleRow.length, equalTo(1)); - assertThat(singleRow[0].getPositionCount(), equalTo(1)); + assertThat(singleRow.getBlockCount(), equalTo(1)); + assertThat(singleRow.getBlock(0).getPositionCount(), equalTo(1)); var exprs = eval.fields(); assertThat(exprs.size(), equalTo(1)); @@ -5169,8 +5172,8 @@ public void testAggOfLiteral() { var eval = as(project.child(), Eval.class); var singleRowRelation = as(eval.child(), LocalRelation.class); var singleRow = singleRowRelation.supplier().get(); - assertThat(singleRow.length, equalTo(1)); - assertThat(singleRow[0].getPositionCount(), equalTo(1)); + assertThat(singleRow.getBlockCount(), equalTo(1)); + assertThat(singleRow.getBlock(0).getPositionCount(), equalTo(1)); assertAggOfConstExprs(testCase, eval.fields()); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEvalTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEvalTests.java index 3cf35d2c609c8..656c3659083a9 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEvalTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceStatsFilteredAggWithEvalTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical; -import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.LongVectorBlock; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Expressions; import org.elasticsearch.xpack.esql.core.expression.FoldContext; @@ -56,10 +56,10 @@ public void testReplaceStatsFilteredAggWithEvalSingleAgg() { var project = as(plan, Limit.class); var source = as(project.child(), LocalRelation.class); assertThat(Expressions.names(source.output()), contains("sum(salary) where false")); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - assertThat(blocks[0].getPositionCount(), is(1)); - assertTrue(blocks[0].areAllValuesNull()); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + assertThat(page.getBlock(0).getPositionCount(), is(1)); + assertTrue(page.getBlock(0).areAllValuesNull()); } /** @@ -90,10 +90,10 @@ public void testReplaceStatsFilteredAggWithEvalSingleAggWithExpression() { var limit = as(eval.child(), Limit.class); var source = as(limit.child(), LocalRelation.class); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - assertThat(blocks[0].getPositionCount(), is(1)); - assertTrue(blocks[0].areAllValuesNull()); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + assertThat(page.getBlock(0).getPositionCount(), is(1)); + assertTrue(page.getBlock(0).areAllValuesNull()); } /** @@ -200,9 +200,9 @@ public void testReplaceStatsFilteredAggWithEvalNotTrue() { var limit = as(plan, Limit.class); var source = as(limit.child(), LocalRelation.class); assertThat(Expressions.names(source.output()), contains("count(salary) where not true")); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - var block = as(blocks[0], LongVectorBlock.class); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + var block = as(page.getBlock(0), LongVectorBlock.class); assertThat(block.getPositionCount(), is(1)); assertThat(block.asVector().getLong(0), is(0L)); } @@ -242,9 +242,9 @@ public void testReplaceStatsFilteredAggWithEvalCount() { var limit = as(plan, Limit.class); var source = as(limit.child(), LocalRelation.class); assertThat(Expressions.names(source.output()), contains("count(salary) where false")); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - var block = as(blocks[0], LongVectorBlock.class); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + var block = as(page.getBlock(0), LongVectorBlock.class); assertThat(block.getPositionCount(), is(1)); assertThat(block.asVector().getLong(0), is(0L)); } @@ -277,9 +277,9 @@ public void testReplaceStatsFilteredAggWithEvalCountDistinctInExpression() { var limit = as(eval.child(), Limit.class); var source = as(limit.child(), LocalRelation.class); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - var block = as(blocks[0], LongVectorBlock.class); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + var block = as(page.getBlock(0), LongVectorBlock.class); assertThat(block.getPositionCount(), is(1)); assertThat(block.asVector().getLong(0), is(0L)); } @@ -344,9 +344,9 @@ public void testReplaceStatsFilteredAggWithEvalFilterUsingEvaledValue() { var limit = as(plan, Limit.class); var source = as(limit.child(), LocalRelation.class); assertThat(Expressions.names(source.output()), contains("count")); - Block[] blocks = source.supplier().get(); - assertThat(blocks.length, is(1)); - var block = as(blocks[0], LongVectorBlock.class); + Page page = source.supplier().get(); + assertThat(page.getBlockCount(), is(1)); + var block = as(page.getBlock(0), LongVectorBlock.class); assertThat(block.getPositionCount(), is(1)); assertThat(block.asVector().getLong(0), is(0L)); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplierTests.java index d9a9e1481b793..15c8eb075bbe1 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/CopyingLocalSupplierTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import static org.hamcrest.Matchers.equalTo; @@ -20,8 +21,9 @@ public class CopyingLocalSupplierTests extends LocalSupplierTests { @Override protected LocalSupplier createTestInstance() { - Block[] blocks = randomList(1, 10, LocalSupplierTests::randomBlock).toArray(Block[]::new); - return new CopyingLocalSupplier(blocks); + int blockSize = randomInt(1000); + Block[] blocks = randomList(1, 10, () -> LocalSupplierTests.randomBlock(blockSize)).toArray(Block[]::new); + return new CopyingLocalSupplier(new Page(blocks)); } protected void assertOnBWCObject(LocalSupplier testInstance, LocalSupplier bwcDeserializedObject, TransportVersion version) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplierTests.java index 1de9581f4dbc0..165116af0ba5f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/ImmediateLocalSupplierTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; import static org.hamcrest.Matchers.equalTo; @@ -16,8 +17,9 @@ public class ImmediateLocalSupplierTests extends LocalSupplierTests { @Override protected LocalSupplier createTestInstance() { - Block[] blocks = randomList(1, 10, LocalSupplierTests::randomBlock).toArray(Block[]::new); - return new ImmediateLocalSupplier(blocks); + int blockSize = randomInt(1000); + Block[] blocks = randomList(1, 10, () -> LocalSupplierTests.randomBlock(blockSize)).toArray(Block[]::new); + return new ImmediateLocalSupplier(new Page(blocks)); } protected void assertOnBWCObject(LocalSupplier testInstance, LocalSupplier bwcDeserializedObject, TransportVersion version) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelationSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelationSerializationTests.java index cf59a35799ad1..bf02afa13128c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelationSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelationSerializationTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.tree.Source; @@ -36,7 +37,7 @@ private static LocalSupplier randomLocalSupplier(List attributes) { 1 ); } - return LocalSupplier.of(blocks); + return LocalSupplier.of(new Page(blocks)); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplierTests.java index 9a9ca3af6bb1a..6c89318a16330 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplierTests.java @@ -16,6 +16,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.test.AbstractWireTestCase; import org.elasticsearch.test.TransportVersionUtils; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; @@ -97,22 +98,27 @@ public static LocalSupplier randomLocalSupplier() { } public static LocalSupplier randomNonEmpty() { - Block[] blocks = randomList(1, 10, LocalSupplierTests::randomBlock).toArray(Block[]::new); - return randomBoolean() ? LocalSupplier.of(blocks) : new CopyingLocalSupplier(blocks); + int blockSize = randomInt(1000); + Block[] blocks = randomList(1, 10, () -> LocalSupplierTests.randomBlock(blockSize)).toArray(Block[]::new); + return randomBoolean() ? LocalSupplier.of(new Page(blocks)) : new CopyingLocalSupplier(new Page(blocks)); } @Override protected LocalSupplier mutateInstance(LocalSupplier instance) throws IOException { - Block[] blocks = instance.get(); + Page page = instance.get(); + Block[] blocks = new Block[page.getBlockCount()]; + for (int i = 0; i < page.getBlockCount(); i++) { + blocks[i] = page.getBlock(i); + } if (blocks.length > 0 && randomBoolean()) { if (randomBoolean()) { return EmptyLocalSupplier.EMPTY; } - return LocalSupplier.of(Arrays.copyOf(blocks, blocks.length - 1, Block[].class)); + return LocalSupplier.of(new Page(page.getPositionCount(), Arrays.copyOf(blocks, blocks.length - 1, Block[].class))); } blocks = Arrays.copyOf(blocks, blocks.length + 1, Block[].class); - blocks[blocks.length - 1] = randomBlock(); - return LocalSupplier.of(blocks); + blocks[blocks.length - 1] = randomBlock(page.getPositionCount()); + return LocalSupplier.of(new Page(blocks)); } @Override @@ -120,10 +126,9 @@ protected NamedWriteableRegistry getNamedWriteableRegistry() { return new NamedWriteableRegistry(PlanWritables.others()); } - static Block randomBlock() { - int len = between(1, 1000); - try (IntBlock.Builder ints = BLOCK_FACTORY.newIntBlockBuilder(len)) { - for (int i = 0; i < len; i++) { + static Block randomBlock(int blockSize) { + try (IntBlock.Builder ints = BLOCK_FACTORY.newIntBlockBuilder(blockSize)) { + for (int i = 0; i < blockSize; i++) { ints.appendInt(randomInt()); } return ints.build(); @@ -132,6 +137,6 @@ static Block randomBlock() { @Override protected boolean shouldBeSame(LocalSupplier newInstance) { - return newInstance.get().length == 0; + return newInstance.get().getBlockCount() == 0; } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/FieldNameUtilsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/FieldNameUtilsTests.java index c59fe53714b46..16d3665db10e0 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/FieldNameUtilsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/FieldNameUtilsTests.java @@ -47,7 +47,7 @@ public void testBasicEvalAndDrop() { public void testSimple1() { assertFieldNames( "from employees | sort emp_no | keep emp_no, still_hired | limit 3", - Set.of("emp_no", "emp_no.*", "still_hired", "still_hired.*") + Set.of("_index", "emp_no", "emp_no.*", "still_hired", "still_hired.*") ); } @@ -61,7 +61,7 @@ public void testSimple2() { public void testDirectFilter() { assertFieldNames( "from employees | sort emp_no | where still_hired | keep emp_no | limit 3", - Set.of("emp_no", "emp_no.*", "still_hired", "still_hired.*") + Set.of("_index", "emp_no", "emp_no.*", "still_hired", "still_hired.*") ); } @@ -72,14 +72,14 @@ public void testForkEval() { public void testSort1() { assertFieldNames( "from employees | sort still_hired, emp_no | keep emp_no, still_hired | limit 3", - Set.of("emp_no", "emp_no.*", "still_hired", "still_hired.*") + Set.of("_index", "emp_no", "emp_no.*", "still_hired", "still_hired.*") ); } public void testStatsBy() { assertFieldNames( "from employees | stats avg(salary) by still_hired | sort still_hired", - Set.of("salary", "salary.*", "still_hired", "still_hired.*") + Set.of("_index", "salary", "salary.*", "still_hired", "still_hired.*") ); } @@ -87,7 +87,7 @@ public void testStatsByAlwaysTrue() { assertFieldNames( "from employees | where first_name is not null | eval always_true = starts_with(first_name, \"\") " + "| stats avg(salary) by always_true", - Set.of("first_name", "first_name.*", "salary", "salary.*") + Set.of("_index", "first_name", "first_name.*", "salary", "salary.*") ); } @@ -96,7 +96,7 @@ public void testStatsByAlwaysFalse() { "from employees | where first_name is not null " + "| eval always_false = starts_with(first_name, \"nonestartwiththis\") " + "| stats avg(salary) by always_false", - Set.of("first_name", "first_name.*", "salary", "salary.*") + Set.of("_index", "first_name", "first_name.*", "salary", "salary.*") ); } @@ -104,7 +104,7 @@ public void testIn1() { assertFieldNames( "from employees | keep emp_no, is_rehired, still_hired " + "| where is_rehired in (still_hired, true) | where is_rehired != still_hired", - Set.of("emp_no", "emp_no.*", "is_rehired", "is_rehired.*", "still_hired", "still_hired.*") + Set.of("_index", "emp_no", "emp_no.*", "is_rehired", "is_rehired.*", "still_hired", "still_hired.*") ); } @@ -116,17 +116,20 @@ public void testConvertFromString1() { | eval rehired_bool = to_boolean(rehired_str) | eval all_false = to_boolean(first_name) | drop first_name - | limit 5""", Set.of("emp_no", "emp_no.*", "is_rehired", "is_rehired.*", "first_name", "first_name.*")); + | limit 5""", Set.of("_index", "emp_no", "emp_no.*", "is_rehired", "is_rehired.*", "first_name", "first_name.*")); } public void testConvertFromDouble1() { - assertFieldNames(""" - from employees - | eval h_2 = height - 2.0, double2bool = to_boolean(h_2) - | where emp_no in (10036, 10037, 10038) - | keep emp_no, height, *2bool""", Set.of("height", "height.*", "emp_no", "emp_no.*", "h_2", "h_2.*", "*2bool.*", "*2bool")); + assertFieldNames( + """ + from employees + | eval h_2 = height - 2.0, double2bool = to_boolean(h_2) + | where emp_no in (10036, 10037, 10038) + | keep emp_no, height, *2bool""", + Set.of("_index", "height", "height.*", "emp_no", "emp_no.*", "h_2", "h_2.*", "*2bool.*", "*2bool") + ); // TODO asking for more shouldn't hurt. Can we do better? ("h_2" shouldn't be in the list of fields) - // Set.of("height", "height.*", "emp_no", "emp_no.*", "*2bool.*", "*2bool")); + // Set.of("_index", "height", "height.*", "emp_no", "emp_no.*", "*2bool.*", "*2bool")); } public void testConvertFromIntAndLong() { @@ -134,6 +137,7 @@ public void testConvertFromIntAndLong() { "from employees | keep emp_no, salary_change*" + "| eval int2bool = to_boolean(salary_change.int), long2bool = to_boolean(salary_change.long) | limit 10", Set.of( + "_index", "emp_no", "emp_no.*", "salary_change*", @@ -149,7 +153,7 @@ public void testIntToInt() { assertFieldNames(""" from employees | where emp_no < 10002 - | keep emp_no""", Set.of("emp_no", "emp_no.*")); + | keep emp_no""", Set.of("_index", "emp_no", "emp_no.*")); } public void testLongToLong() { @@ -159,7 +163,7 @@ public void testLongToLong() { | where languages.long < avg_worked_seconds | limit 1 | keep emp_no""", - Set.of("emp_no", "emp_no.*", "languages.long", "languages.long.*", "avg_worked_seconds", "avg_worked_seconds.*") + Set.of("_index", "emp_no", "emp_no.*", "languages.long", "languages.long.*", "avg_worked_seconds", "avg_worked_seconds.*") ); } @@ -169,7 +173,7 @@ public void testDateToDate() { | where birth_date < hire_date | keep emp_no | sort emp_no - | limit 1""", Set.of("birth_date", "birth_date.*", "emp_no", "emp_no.*", "hire_date", "hire_date.*")); + | limit 1""", Set.of("_index", "birth_date", "birth_date.*", "emp_no", "emp_no.*", "hire_date", "hire_date.*")); } public void testTwoConditionsWithDefault() { @@ -177,7 +181,7 @@ public void testTwoConditionsWithDefault() { from employees | eval type = case(languages <= 1, "monolingual", languages <= 2, "bilingual", "polyglot") | keep emp_no, type - | limit 10""", Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + | limit 10""", Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testSingleCondition() { @@ -185,7 +189,7 @@ public void testSingleCondition() { from employees | eval g = case(gender == "F", true) | keep gender, g - | limit 10""", Set.of("gender", "gender.*")); + | limit 10""", Set.of("_index", "gender", "gender.*")); } public void testConditionIsNull() { @@ -193,24 +197,24 @@ public void testConditionIsNull() { from employees | eval g = case(gender == "F", 1, languages > 1, 2, 3) | keep gender, languages, g - | limit 25""", Set.of("gender", "gender.*", "languages", "languages.*")); + | limit 25""", Set.of("_index", "gender", "gender.*", "languages", "languages.*")); } public void testEvalAssign() { assertFieldNames( "from employees | sort hire_date | eval x = hire_date | keep emp_no, x | limit 5", - Set.of("hire_date", "hire_date.*", "emp_no", "emp_no.*") + Set.of("_index", "hire_date", "hire_date.*", "emp_no", "emp_no.*") ); } public void testMinMax() { - assertFieldNames("from employees | stats min = min(hire_date), max = max(hire_date)", Set.of("hire_date", "hire_date.*")); + assertFieldNames("from employees | stats min = min(hire_date), max = max(hire_date)", Set.of("_index", "hire_date", "hire_date.*")); } public void testEvalDateTruncIntervalExpressionPeriod() { assertFieldNames( "from employees | sort hire_date | eval x = date_trunc(hire_date, 1 month) | keep emp_no, hire_date, x | limit 5", - Set.of("hire_date", "hire_date.*", "emp_no", "emp_no.*") + Set.of("_index", "hire_date", "hire_date.*", "emp_no", "emp_no.*") ); } @@ -221,7 +225,7 @@ public void testEvalDateTruncGrouping() { | stats count(emp_no) by y | sort y | keep y, `count(emp_no)` - | limit 5""", Set.of("hire_date", "hire_date.*", "emp_no", "emp_no.*")); + | limit 5""", Set.of("_index", "hire_date", "hire_date.*", "emp_no", "emp_no.*")); } public void testIn2() { @@ -231,7 +235,7 @@ public void testIn2() { | where birth_date not in (x, hire_date) | keep x, hire_date | sort x desc - | limit 4""", Set.of("hire_date", "hire_date.*", "birth_date", "birth_date.*")); + | limit 4""", Set.of("_index", "hire_date", "hire_date.*", "birth_date", "birth_date.*")); } public void testBucketMonth() { @@ -240,13 +244,13 @@ public void testBucketMonth() { | where hire_date >= "1985-01-01T00:00:00Z" and hire_date < "1986-01-01T00:00:00Z" | eval hd = bucket(hire_date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") | sort hire_date - | keep hire_date, hd""", Set.of("hire_date", "hire_date.*")); + | keep hire_date, hd""", Set.of("_index", "hire_date", "hire_date.*")); } public void testBorn_before_today() { assertFieldNames( "from employees | where birth_date < now() | sort emp_no asc | keep emp_no, birth_date| limit 1", - Set.of("birth_date", "birth_date.*", "emp_no", "emp_no.*") + Set.of("_index", "birth_date", "birth_date.*", "emp_no", "emp_no.*") ); } @@ -256,7 +260,7 @@ public void testBucketMonthInAgg() { | WHERE hire_date >= "1985-01-01T00:00:00Z" AND hire_date < "1986-01-01T00:00:00Z" | EVAL bucket = BUCKET(hire_date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") | STATS AVG(salary) BY bucket - | SORT bucket""", Set.of("salary", "salary.*", "hire_date", "hire_date.*")); + | SORT bucket""", Set.of("_index", "salary", "salary.*", "hire_date", "hire_date.*")); } public void testEvalDateParseDynamic() { @@ -267,7 +271,7 @@ public void testEvalDateParseDynamic() { | eval birth_date_string = date_format("yyyy-MM-dd", birth_date) | eval new_date = date_parse("yyyy-MM-dd", birth_date_string) | eval bool = new_date == birth_date - | keep emp_no, new_date, birth_date, bool""", Set.of("emp_no", "emp_no.*", "birth_date", "birth_date.*")); + | keep emp_no, new_date, birth_date, bool""", Set.of("_index", "emp_no", "emp_no.*", "birth_date", "birth_date.*")); } public void testDateFields() { @@ -275,7 +279,7 @@ public void testDateFields() { from employees | where emp_no == 10049 or emp_no == 10050 | eval year = date_extract("year", birth_date), month = date_extract("month_of_year", birth_date) - | keep emp_no, year, month""", Set.of("emp_no", "emp_no.*", "birth_date", "birth_date.*")); + | keep emp_no, year, month""", Set.of("_index", "emp_no", "emp_no.*", "birth_date", "birth_date.*")); } public void testEvalDissect() { @@ -285,7 +289,7 @@ public void testEvalDissect() { | dissect full_name "%{a} %{b}" | sort emp_no asc | keep full_name, a, b - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); } public void testDissectExpression() { @@ -294,7 +298,7 @@ public void testDissectExpression() { | dissect concat(first_name, " ", last_name) "%{a} %{b}" | sort emp_no asc | keep a, b - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); } public void testMultivalueInput1() { @@ -303,7 +307,7 @@ public void testMultivalueInput1() { | where emp_no <= 10006 | dissect job_positions "%{a} %{b} %{c}" | sort emp_no - | keep emp_no, a, b, c""", Set.of("emp_no", "emp_no.*", "job_positions", "job_positions.*")); + | keep emp_no, a, b, c""", Set.of("_index", "emp_no", "emp_no.*", "job_positions", "job_positions.*")); } public void testLimitZero() { @@ -358,14 +362,14 @@ public void testDocsEval() { | KEEP first_name, last_name, height | EVAL height_feet = height * 3.281, height_cm = height * 100 | WHERE first_name == "Georgi" - | LIMIT 1""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "height", "height.*")); + | LIMIT 1""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "height", "height.*")); } public void testDocsKeepWildcard() { assertFieldNames(""" FROM employees | KEEP h* - | LIMIT 0""", Set.of("h*")); + | LIMIT 0""", Set.of("_index", "h*")); } public void testDocsKeepDoubleWildcard() { @@ -380,7 +384,7 @@ public void testDocsRename() { FROM employees | KEEP first_name, last_name, still_hired | RENAME still_hired AS employed - | LIMIT 0""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "still_hired", "still_hired.*")); + | LIMIT 0""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "still_hired", "still_hired.*")); } public void testDocsRenameMultipleColumns() { @@ -388,14 +392,14 @@ public void testDocsRenameMultipleColumns() { FROM employees | KEEP first_name, last_name | RENAME first_name AS fn, last_name AS ln - | LIMIT 0""", Set.of("first_name", "first_name.*", "last_name", "last_name.*")); + | LIMIT 0""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*")); } public void testDocsStats() { assertFieldNames(""" FROM employees | STATS count = COUNT(emp_no) BY languages - | SORT languages""", Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + | SORT languages""", Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testEvalStats() { @@ -420,13 +424,13 @@ public void testEvalStats() { assertFieldNames(""" FROM employees | STATS count = COUNT(*) BY first_name - | SORT first_name""", Set.of("first_name", "first_name.*")); + | SORT first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees | EVAL y = "a" | STATS count = COUNT(*) BY x = y - | SORT x, first_name""", Set.of("first_name", "first_name.*")); + | SORT x, first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees @@ -444,26 +448,26 @@ public void testEvalStats() { FROM employees | EVAL y = to_upper(first_name), z = "z" | STATS count = COUNT(*) BY first_name = to_lower(y), z - | SORT first_name""", Set.of("first_name", "first_name.*")); + | SORT first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees | EVAL y = "a" | STATS count = COUNT(*) BY x = y, z = first_name - | SORT x, z""", Set.of("first_name", "first_name.*")); + | SORT x, z""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees | EVAL y = "a" | STATS count = COUNT(*) BY x = y, first_name - | SORT x, first_name""", Set.of("first_name", "first_name.*")); + | SORT x, first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees | EVAL y = "a" | STATS count = COUNT(first_name) BY x = y | SORT x - | DROP first_name""", Set.of("first_name", "first_name.*")); + | DROP first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees @@ -475,14 +479,14 @@ public void testEvalStats() { FROM employees | EVAL y = "a" | STATS count = COUNT(*) BY first_name, y - | MV_EXPAND first_name""", Set.of("first_name", "first_name.*")); + | MV_EXPAND first_name""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees | MV_EXPAND first_name | EVAL y = "a" | STATS count = COUNT(*) BY first_name, y - | SORT y""", Set.of("first_name", "first_name.*")); + | SORT y""", Set.of("_index", "first_name", "first_name.*")); assertFieldNames(""" FROM employees @@ -503,7 +507,7 @@ public void testEvalStats() { | EVAL y = "a" | STATS count = COUNT(*) BY first_name, y | STATS count = COUNT(count) by x = y - | SORT x""", Set.of("first_name", "first_name.*")); + | SORT x""", Set.of("_index", "first_name", "first_name.*")); } public void testSortWithLimitOne_DropHeight() { @@ -516,13 +520,13 @@ public void testSortWithLimitOne_DropHeight_WithInlineStats() { } public void testDropAllColumns() { - assertFieldNames("from employees | keep height | drop height | eval x = 1", Set.of("height", "height.*")); + assertFieldNames("from employees | keep height | drop height | eval x = 1", Set.of("_index", "height", "height.*")); } public void testDropAllColumns_WithStats() { assertFieldNames( "from employees | keep height | drop height | eval x = 1 | stats c=count(x), mi=min(x), s=sum(x)", - Set.of("height", "height.*") + Set.of("_index", "height", "height.*") ); } @@ -535,18 +539,21 @@ public void testEnrichOn() { | eval x = to_string(languages) | enrich languages_policy on x | keep emp_no, language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } public void testEnrichOn2() { - assertFieldNames(""" - from employees - | eval x = to_string(languages) - | enrich languages_policy on x - | keep emp_no, language_name - | sort emp_no - | limit 1""", Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*")); + assertFieldNames( + """ + from employees + | eval x = to_string(languages) + | enrich languages_policy on x + | keep emp_no, language_name + | sort emp_no + | limit 1""", + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + ); } public void testUselessEnrich() { @@ -558,13 +565,16 @@ public void testUselessEnrich() { } public void testSimpleSortLimit() { - assertFieldNames(""" - from employees - | eval x = to_string(languages) - | enrich languages_policy on x - | keep emp_no, language_name - | sort emp_no - | limit 1""", Set.of("languages", "languages.*", "emp_no", "emp_no.*", "language_name", "language_name.*", "x", "x.*")); + assertFieldNames( + """ + from employees + | eval x = to_string(languages) + | enrich languages_policy on x + | keep emp_no, language_name + | sort emp_no + | limit 1""", + Set.of("_index", "languages", "languages.*", "emp_no", "emp_no.*", "language_name", "language_name.*", "x", "x.*") + ); } public void testWith() { @@ -572,7 +582,7 @@ public void testWith() { """ from employees | eval x = to_string(languages) | keep emp_no, x | sort emp_no | limit 1 | enrich languages_policy on x with language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -581,7 +591,7 @@ public void testWithAlias() { """ from employees | sort emp_no | limit 3 | eval x = to_string(languages) | keep emp_no, x | enrich languages_policy on x with lang = language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -590,7 +600,7 @@ public void testWithAliasSort() { """ from employees | eval x = to_string(languages) | keep emp_no, x | sort emp_no | limit 3 | enrich languages_policy on x with lang = language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -599,7 +609,7 @@ public void testWithAliasAndPlain() { """ from employees | sort emp_no desc | limit 3 | eval x = to_string(languages) | keep emp_no, x | enrich languages_policy on x with lang = language_name, language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -608,7 +618,7 @@ public void testWithTwoAliasesSameProp() { """ from employees | sort emp_no | limit 1 | eval x = to_string(languages) | keep emp_no, x | enrich languages_policy on x with lang = language_name, lang2 = language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -617,7 +627,7 @@ public void testRedundantWith() { """ from employees | sort emp_no | limit 1 | eval x = to_string(languages) | keep emp_no, x | enrich languages_policy on x with language_name, language_name""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -628,7 +638,7 @@ public void testNullInput() { | where emp_no == 10017 | keep emp_no, gender | enrich languages_policy on gender with language_name, language_name""", - Set.of("gender", "gender.*", "emp_no", "emp_no.*", "language_name", "language_name.*") + Set.of("_index", "gender", "gender.*", "emp_no", "emp_no.*", "language_name", "language_name.*") ); } @@ -640,7 +650,7 @@ public void testConstantNullInput() { | eval x = to_string(languages) | keep emp_no, x | enrich languages_policy on x with language_name, language_name""", - Set.of("languages", "languages.*", "emp_no", "emp_no.*", "language_name", "language_name.*", "x", "x.*") + Set.of("_index", "languages", "languages.*", "emp_no", "emp_no.*", "language_name", "language_name.*", "x", "x.*") ); } @@ -654,6 +664,7 @@ public void testEnrichEval() { | keep emp_no, x, lang, language | sort emp_no desc | limit 3""", Set.of( + "_index", "emp_no", "x", "lang", @@ -679,14 +690,14 @@ public void testSimple() { | where x > 1 | keep emp_no, language_name | limit 1""", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "y", "x.*", "y.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "y", "x.*", "y.*") ); } public void testEvalNullSort() { assertFieldNames( "from employees | eval x = null | sort x asc, emp_no desc | keep emp_no, x, last_name | limit 2", - Set.of("last_name", "last_name.*", "emp_no", "emp_no.*") + Set.of("_index", "last_name", "last_name.*", "emp_no", "emp_no.*") ); } @@ -697,7 +708,7 @@ public void testFilterEvalFilter() { | eval name_len = length(first_name) | where name_len < 4 | keep first_name - | sort first_name""", Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + | sort first_name""", Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testEvalWithIsNullIsNotNull() { @@ -708,7 +719,7 @@ public void testEvalWithIsNullIsNotNull() { | sort emp_no | limit 1 | keep *true*, *false*, first_name, last_name""", - Set.of("emp_no", "emp_no.*", "first_name", "first_name.*", "last_name", "last_name.*", "*true*", "*false*") + Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*", "last_name", "last_name.*", "*true*", "*false*") ); } @@ -716,6 +727,7 @@ public void testInDouble() { assertFieldNames( "from employees | keep emp_no, height, height.float, height.half_float, height.scaled_float | where height in (2.03)", Set.of( + "_index", "emp_no", "emp_no.*", "height", @@ -733,7 +745,7 @@ public void testInDouble() { public void testConvertFromDatetime() { assertFieldNames( "from employees | sort emp_no | eval hire_double = to_double(hire_date) | keep emp_no, hire_date, hire_double | limit 3", - Set.of("emp_no", "emp_no.*", "hire_date", "hire_date.*") + Set.of("_index", "emp_no", "emp_no.*", "hire_date", "hire_date.*") ); } @@ -743,7 +755,7 @@ public void testBucket() { | WHERE hire_date >= "1985-01-01T00:00:00Z" AND hire_date < "1986-01-01T00:00:00Z" | EVAL bh = bucket(height, 20, 1.41, 2.10) | SORT hire_date - | KEEP hire_date, height, bh""", Set.of("hire_date", "hire_date.*", "height", "height.*")); + | KEEP hire_date, height, bh""", Set.of("_index", "hire_date", "hire_date.*", "height", "height.*")); } public void testEvalGrok() { @@ -753,7 +765,7 @@ public void testEvalGrok() { | grok full_name "%{WORD:a} %{WORD:b}" | sort emp_no asc | keep full_name, a, b - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); } public void testGrokExpression() { @@ -762,7 +774,7 @@ public void testGrokExpression() { | grok concat(first_name, " ", last_name) "%{WORD:a} %{WORD:b}" | sort emp_no asc | keep a, b - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); } public void testEvalGrokSort() { @@ -772,7 +784,7 @@ public void testEvalGrokSort() { | grok full_name "%{WORD:a} %{WORD:b}" | sort a asc | keep full_name, a, b - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*")); } public void testGrokStats() { @@ -782,7 +794,7 @@ public void testGrokStats() { | grok x "%{WORD:a} %{WORD:b}" | stats n = max(emp_no) by a | keep a, n - | sort a asc""", Set.of("gender", "gender.*", "emp_no", "emp_no.*")); + | sort a asc""", Set.of("_index", "gender", "gender.*", "emp_no", "emp_no.*")); } public void testNullOnePattern() { @@ -790,7 +802,7 @@ public void testNullOnePattern() { from employees | where emp_no == 10030 | grok first_name "%{WORD:a}" - | keep first_name, a""", Set.of("first_name", "first_name.*", "emp_no", "emp_no.*")); + | keep first_name, a""", Set.of("_index", "first_name", "first_name.*", "emp_no", "emp_no.*")); } public void testMultivalueInput() { @@ -799,7 +811,7 @@ public void testMultivalueInput() { | where emp_no <= 10006 | grok job_positions "%{WORD:a} %{WORD:b} %{WORD:c}" | sort emp_no - | keep emp_no, a, b, c, job_positions""", Set.of("job_positions", "job_positions.*", "emp_no", "emp_no.*")); + | keep emp_no, a, b, c, job_positions""", Set.of("_index", "job_positions", "job_positions.*", "emp_no", "emp_no.*")); } public void testSelectAll() { @@ -816,7 +828,7 @@ public void testFilterById_WithInlineStats() { } public void testKeepId() { - assertFieldNames("FROM apps metadata _id | WHERE id == 3 | KEEP _id", Set.of("id", "id.*")); + assertFieldNames("FROM apps metadata _id | WHERE id == 3 | KEEP _id", Set.of("_index", "id", "id.*")); } public void testIdRangeAndSort() { @@ -824,19 +836,19 @@ public void testIdRangeAndSort() { FROM apps metadata _id | WHERE _id >= "2" AND _id <= "7" | SORT _id - | keep id, name, _id""", Set.of("id", "id.*", "name", "name.*")); + | keep id, name, _id""", Set.of("_index", "id", "id.*", "name", "name.*")); } public void testOrderById() { - assertFieldNames("FROM apps metadata _id | KEEP _id, name | SORT _id", Set.of("name", "name.*")); + assertFieldNames("FROM apps metadata _id | KEEP _id, name | SORT _id", Set.of("_index", "name", "name.*")); } public void testOrderByIdDesc() { - assertFieldNames("FROM apps metadata _id | KEEP _id, name | SORT _id DESC", Set.of("name", "name.*")); + assertFieldNames("FROM apps metadata _id | KEEP _id, name | SORT _id DESC", Set.of("_index", "name", "name.*")); } public void testConcatId() { - assertFieldNames("FROM apps metadata _id | eval c = concat(_id, name) | SORT _id | KEEP c", Set.of("name", "name.*")); + assertFieldNames("FROM apps metadata _id | eval c = concat(_id, name) | SORT _id | KEEP c", Set.of("_index", "name", "name.*")); } public void testStatsOnId() { @@ -844,45 +856,51 @@ public void testStatsOnId() { } public void testStatsOnIdByGroup() { - assertFieldNames("FROM apps metadata _id | stats c = count(_id) by name | sort c desc, name | limit 5", Set.of("name", "name.*")); + assertFieldNames( + "FROM apps metadata _id | stats c = count(_id) by name | sort c desc, name | limit 5", + Set.of("_index", "name", "name.*") + ); } public void testSimpleProject() { assertFieldNames( "from hosts | keep card, host, ip0, ip1", - Set.of("card", "card.*", "host", "host.*", "ip0", "ip0.*", "ip1", "ip1.*") + Set.of("_index", "card", "card.*", "host", "host.*", "ip0", "ip0.*", "ip1", "ip1.*") ); } public void testEquals() { assertFieldNames( "from hosts | sort host, card | where ip0 == ip1 | keep card, host", - Set.of("card", "card.*", "host", "host.*", "ip0", "ip0.*", "ip1", "ip1.*") + Set.of("_index", "card", "card.*", "host", "host.*", "ip0", "ip0.*", "ip1", "ip1.*") ); } public void testConditional() { - assertFieldNames("from hosts | eval eq=case(ip0==ip1, ip0, ip1) | keep eq, ip0, ip1", Set.of("ip1", "ip1.*", "ip0", "ip0.*")); + assertFieldNames( + "from hosts | eval eq=case(ip0==ip1, ip0, ip1) | keep eq, ip0, ip1", + Set.of("_index", "ip1", "ip1.*", "ip0", "ip0.*") + ); } public void testWhereWithAverageBySubField() { assertFieldNames( "from employees | where languages + 1 == 6 | stats avg(avg_worked_seconds) by languages.long", - Set.of("languages", "languages.*", "avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*") + Set.of("_index", "languages", "languages.*", "avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*") ); } public void testAverageOfEvalValue() { assertFieldNames( "from employees | eval ratio = salary / height | stats avg(ratio)", - Set.of("salary", "salary.*", "height", "height.*") + Set.of("_index", "salary", "salary.*", "height", "height.*") ); } public void testTopNProjectEvalProject() { assertFieldNames( "from employees | sort salary | limit 1 | keep languages, salary | eval x = languages + 1 | keep x", - Set.of("salary", "salary.*", "languages", "languages.*") + Set.of("_index", "salary", "salary.*", "languages", "languages.*") ); } @@ -893,11 +911,14 @@ public void testMvSum() { | eval salary_change = mv_sum(salary_change.int) | sort emp_no | keep emp_no, salary_change.int, salary_change - | limit 7""", Set.of("emp_no", "emp_no.*", "salary_change.int", "salary_change.int.*")); + | limit 7""", Set.of("_index", "emp_no", "emp_no.*", "salary_change.int", "salary_change.int.*")); } public void testMetaIndexAliasedInAggs() { - assertFieldNames("from employees metadata _index | eval _i = _index | stats max = max(emp_no) by _i", Set.of("emp_no", "emp_no.*")); + assertFieldNames( + "from employees metadata _index | eval _i = _index | stats max = max(emp_no) by _i", + Set.of("_index", "emp_no", "emp_no.*") + ); } public void testCoalesceFolding() { @@ -906,13 +927,13 @@ public void testCoalesceFolding() { | EVAL foo=COALESCE(true, false, null) | SORT emp_no ASC | KEEP emp_no, first_name, foo - | limit 3""", Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + | limit 3""", Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testRenameEvalProject() { assertFieldNames( "from employees | rename languages as x | keep x | eval z = 2 * x | keep x, z | limit 3", - Set.of("languages", "languages.*") + Set.of("_index", "languages", "languages.*") ); } @@ -924,27 +945,27 @@ public void testRenameProjectEval() { | keep x, y | eval x2 = x + 1 | eval y2 = y + 2 - | limit 3""", Set.of("languages", "languages.*")); + | limit 3""", Set.of("_index", "languages", "languages.*")); } public void testRenameWithFilterPushedToES() { assertFieldNames( "from employees | rename emp_no as x | keep languages, first_name, last_name, x | where x > 10030 and x < 10040 | limit 5", - Set.of("emp_no", "emp_no.*", "languages", "languages.*", "first_name", "first_name.*", "last_name", "last_name.*") + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "first_name", "first_name.*", "last_name", "last_name.*") ); } public void testRenameOverride() { assertFieldNames( "from employees | rename emp_no as languages | keep languages, last_name | limit 3", - Set.of("emp_no", "emp_no.*", "last_name", "last_name.*") + Set.of("_index", "emp_no", "emp_no.*", "last_name", "last_name.*") ); } public void testProjectRenameDate() { assertFieldNames( "from employees | sort hire_date | rename hire_date as x | keep emp_no, x | limit 5", - Set.of("hire_date", "hire_date.*", "emp_no", "emp_no.*") + Set.of("_index", "hire_date", "hire_date.*", "emp_no", "emp_no.*") ); } @@ -959,13 +980,13 @@ public void testRenameDrop() { } public void testMaxOfLong() { - assertFieldNames("from employees | stats l = max(languages.long)", Set.of("languages.long", "languages.long.*")); + assertFieldNames("from employees | stats l = max(languages.long)", Set.of("_index", "languages.long", "languages.long.*")); } public void testGroupByAlias() { assertFieldNames( "from employees | rename languages as l | keep l, height | stats m = min(height) by l | sort l", - Set.of("languages", "languages.*", "height", "height.*") + Set.of("_index", "languages", "languages.*", "height", "height.*") ); } @@ -974,7 +995,7 @@ public void testByStringAndLong() { from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(gender) by gender, trunk_worked_seconds - | sort c desc""", Set.of("avg_worked_seconds", "avg_worked_seconds.*", "gender", "gender.*")); + | sort c desc""", Set.of("_index", "avg_worked_seconds", "avg_worked_seconds.*", "gender", "gender.*")); } public void testByStringAndLongWithAlias() { @@ -984,7 +1005,7 @@ public void testByStringAndLongWithAlias() { | rename gender as g, trunk_worked_seconds as tws | keep g, tws | stats c = count(g) by g, tws - | sort c desc""", Set.of("avg_worked_seconds", "avg_worked_seconds.*", "gender", "gender.*")); + | sort c desc""", Set.of("_index", "avg_worked_seconds", "avg_worked_seconds.*", "gender", "gender.*")); } public void testByStringAndString() { @@ -993,7 +1014,7 @@ public void testByStringAndString() { | eval hire_year_str = date_format("yyyy", hire_date) | stats c = count(gender) by gender, hire_year_str | sort c desc, gender, hire_year_str - | where c >= 5""", Set.of("hire_date", "hire_date.*", "gender", "gender.*")); + | where c >= 5""", Set.of("_index", "hire_date", "hire_date.*", "gender", "gender.*")); } public void testByLongAndLong() { @@ -1001,18 +1022,21 @@ public void testByLongAndLong() { from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(languages.long) by languages.long, trunk_worked_seconds - | sort c desc""", Set.of("avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*")); + | sort c desc""", Set.of("_index", "avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*")); } public void testByDateAndKeywordAndIntWithAlias() { - assertFieldNames(""" - from employees - | eval d = date_trunc(hire_date, 1 year) - | rename gender as g, languages as l, emp_no as e - | keep d, g, l, e - | stats c = count(e) by d, g, l - | sort c desc, d, l desc - | limit 10""", Set.of("hire_date", "hire_date.*", "gender", "gender.*", "languages", "languages.*", "emp_no", "emp_no.*")); + assertFieldNames( + """ + from employees + | eval d = date_trunc(hire_date, 1 year) + | rename gender as g, languages as l, emp_no as e + | keep d, g, l, e + | stats c = count(e) by d, g, l + | sort c desc, d, l desc + | limit 10""", + Set.of("_index", "hire_date", "hire_date.*", "gender", "gender.*", "languages", "languages.*", "emp_no", "emp_no.*") + ); } public void testCountDistinctOfKeywords() { @@ -1021,14 +1045,14 @@ public void testCountDistinctOfKeywords() { from employees | eval hire_year_str = date_format("yyyy", hire_date) | stats g = count_distinct(gender), h = count_distinct(hire_year_str)""", - Set.of("hire_date", "hire_date.*", "gender", "gender.*") + Set.of("_index", "hire_date", "hire_date.*", "gender", "gender.*") ); } public void testCountDistinctOfIpPrecision() { assertFieldNames(""" FROM hosts - | STATS COUNT_DISTINCT(ip0, 80000), COUNT_DISTINCT(ip1, 5)""", Set.of("ip0", "ip0.*", "ip1", "ip1.*")); + | STATS COUNT_DISTINCT(ip0, 80000), COUNT_DISTINCT(ip1, 5)""", Set.of("_index", "ip0", "ip0.*", "ip1", "ip1.*")); } public void testPercentileOfLong() { @@ -1036,20 +1060,20 @@ public void testPercentileOfLong() { """ from employees | stats p0 = percentile(salary_change.long, 0), p50 = percentile(salary_change.long, 50)""", - Set.of("salary_change.long", "salary_change.long.*") + Set.of("_index", "salary_change.long", "salary_change.long.*") ); } public void testMedianOfInteger() { assertFieldNames(""" FROM employees - | STATS MEDIAN(salary), PERCENTILE(salary, 50)""", Set.of("salary", "salary.*")); + | STATS MEDIAN(salary), PERCENTILE(salary, 50)""", Set.of("_index", "salary", "salary.*")); } public void testMedianAbsoluteDeviation() { assertFieldNames(""" FROM employees - | STATS MEDIAN(salary), MEDIAN_ABSOLUTE_DEVIATION(salary)""", Set.of("salary", "salary.*")); + | STATS MEDIAN(salary), MEDIAN_ABSOLUTE_DEVIATION(salary)""", Set.of("_index", "salary", "salary.*")); } public void testIn3VLWithComputedNull() { @@ -1062,7 +1086,7 @@ public void testIn3VLWithComputedNull() { | keep emp_no, job_positions | eval nil = concat("", null) | eval is_in = job_positions in ("Accountant", "Internship", nil)""", - Set.of("job_positions", "job_positions.*", "emp_no", "emp_no.*") + Set.of("_index", "job_positions", "job_positions.*", "emp_no", "emp_no.*") ); } @@ -1080,21 +1104,21 @@ version > TO_VER("1.1"), "high", version IS NULL, "none", "low") | SORT version DESC NULLS LAST, id DESC - | KEEP v, version, version_text, id, m, g, i, c""", Set.of("version", "version.*", "id", "id.*")); + | KEEP v, version, version_text, id, m, g, i, c""", Set.of("_index", "version", "version.*", "id", "id.*")); } public void testLikePrefix() { assertFieldNames(""" from employees | where first_name like "Eberhar*" - | keep emp_no, first_name""", Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + | keep emp_no, first_name""", Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testRLikePrefix() { assertFieldNames(""" from employees | where first_name rlike "Aleja.*" - | keep emp_no""", Set.of("first_name", "first_name.*", "emp_no", "emp_no.*")); + | keep emp_no""", Set.of("_index", "first_name", "first_name.*", "emp_no", "emp_no.*")); } public void testByUnmentionedLongAndLong() { @@ -1104,7 +1128,7 @@ public void testByUnmentionedLongAndLong() { | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(gender) by languages.long, trunk_worked_seconds | sort c desc""", - Set.of("avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*", "gender", "gender.*") + Set.of("_index", "avg_worked_seconds", "avg_worked_seconds.*", "languages.long", "languages.long.*", "gender", "gender.*") ); } @@ -1113,7 +1137,7 @@ public void testRenameNopProject() { from employees | rename emp_no as emp_no | keep emp_no, last_name - | limit 3""", Set.of("emp_no", "emp_no.*", "last_name", "last_name.*")); + | limit 3""", Set.of("_index", "emp_no", "emp_no.*", "last_name", "last_name.*")); } public void testRename() { @@ -1121,7 +1145,7 @@ public void testRename() { from test | rename emp_no as e | keep first_name, e - """, Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testChainedRename() { @@ -1129,9 +1153,12 @@ public void testChainedRename() { from test | rename emp_no as r1, r1 as r2, r2 as r3 | keep first_name, r3 - """, Set.of("emp_no", "emp_no.*", "first_name", "first_name.*", "r1", "r1.*", "r2", "r2.*"));// TODO asking for more shouldn't - // hurt. Can we do better? - // Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*", "r1", "r1.*", "r2", "r2.*"));// TODO + // asking for + // more + // shouldn't + // hurt. Can we do better? + // Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testChainedRenameReuse() { @@ -1139,9 +1166,12 @@ public void testChainedRenameReuse() { from test | rename emp_no as r1, r1 as r2, r2 as r3, first_name as r1 | keep r1, r3 - """, Set.of("emp_no", "emp_no.*", "first_name", "first_name.*", "r1", "r1.*", "r2", "r2.*"));// TODO asking for more shouldn't - // hurt. Can we do better? - // Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*", "r1", "r1.*", "r2", "r2.*"));// TODO + // asking for + // more + // shouldn't + // hurt. Can we do better? + // Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } public void testRenameBackAndForth() { @@ -1149,8 +1179,8 @@ public void testRenameBackAndForth() { from test | rename emp_no as r1, r1 as emp_no | keep emp_no - """, Set.of("emp_no", "emp_no.*", "r1", "r1.*"));// TODO asking for more shouldn't hurt. Can we do better? - // Set.of("emp_no", "emp_no.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "r1", "r1.*"));// TODO asking for more shouldn't hurt. Can we do better? + // Set.of("_index", "emp_no", "emp_no.*")); } public void testRenameReuseAlias() { @@ -1163,7 +1193,7 @@ public void testRenameReuseAlias() { public void testIfDuplicateNamesGroupingHasPriority() { assertFieldNames( "from employees | stats languages = avg(height), languages = min(height) by languages | sort languages", - Set.of("height", "height.*", "languages", "languages.*") + Set.of("_index", "height", "height.*", "languages", "languages.*") ); } @@ -1173,7 +1203,7 @@ public void testCoalesce() { | EVAL first_name = COALESCE(first_name, "X") | SORT first_name DESC, emp_no ASC | KEEP emp_no, first_name - | limit 10""", Set.of("first_name", "first_name.*", "emp_no", "emp_no.*")); + | limit 10""", Set.of("_index", "first_name", "first_name.*", "emp_no", "emp_no.*")); } public void testCoalesceBackwards() { @@ -1182,7 +1212,7 @@ public void testCoalesceBackwards() { | EVAL first_name = COALESCE("X", first_name) | SORT first_name DESC, emp_no ASC | KEEP emp_no, first_name - | limit 10""", Set.of("first_name", "first_name.*", "emp_no", "emp_no.*")); + | limit 10""", Set.of("_index", "first_name", "first_name.*", "emp_no", "emp_no.*")); } public void testGroupByVersionCast() { @@ -1191,7 +1221,7 @@ public void testGroupByVersionCast() { | EVAL g = TO_VER(CONCAT("1.", TO_STR(version))) | STATS id = MAX(id) BY g | SORT id - | DROP g""", Set.of("version", "version.*", "id", "id.*")); + | DROP g""", Set.of("_index", "version", "version.*", "id", "id.*")); } public void testCoalesceEndsInNull() { @@ -1200,17 +1230,20 @@ public void testCoalesceEndsInNull() { | EVAL first_name = COALESCE(first_name, last_name, null) | SORT first_name DESC, emp_no ASC | KEEP emp_no, first_name - | limit 3""", Set.of("first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); + | limit 3""", Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "emp_no", "emp_no.*")); } public void testMvAvg() { - assertFieldNames(""" - from employees - | where emp_no > 10008 - | eval salary_change = mv_avg(salary_change) - | sort emp_no - | keep emp_no, salary_change.int, salary_change - | limit 7""", Set.of("emp_no", "emp_no.*", "salary_change", "salary_change.*", "salary_change.int", "salary_change.int.*")); + assertFieldNames( + """ + from employees + | where emp_no > 10008 + | eval salary_change = mv_avg(salary_change) + | sort emp_no + | keep emp_no, salary_change.int, salary_change + | limit 7""", + Set.of("_index", "emp_no", "emp_no.*", "salary_change", "salary_change.*", "salary_change.int", "salary_change.int.*") + ); } public void testEvalOverride() { @@ -1219,8 +1252,9 @@ public void testEvalOverride() { | eval languages = languages + 1 | eval languages = languages + 1 | limit 5 - | keep l*""", Set.of("languages", "languages.*", "l*"));// subtlety here. Keeping only "languages*" can remove any other "l*" - // named fields + | keep l*""", Set.of("_index", "languages", "languages.*", "l*"));// subtlety here. Keeping only "languages*" can + // remove any other "l*" + // named fields } public void testBasicWildcardKeep() { @@ -1231,7 +1265,7 @@ public void testBasicWildcardKeep2() { assertFieldNames(""" from test | keep un* - """, Set.of("un*")); + """, Set.of("_index", "un*")); } public void testWildcardKeep() { @@ -1246,7 +1280,7 @@ public void testProjectThenDropName() { from test | keep *name | drop first_name - """, Set.of("*name", "*name.*", "first_name", "first_name.*")); + """, Set.of("_index", "*name", "*name.*", "first_name", "first_name.*")); } public void testProjectAfterDropName() { @@ -1254,7 +1288,7 @@ public void testProjectAfterDropName() { from test | drop first_name | keep *name - """, Set.of("*name.*", "*name", "first_name", "first_name.*")); + """, Set.of("_index", "*name.*", "*name", "first_name", "first_name.*")); } public void testProjectWithMixedQuoting() { @@ -1262,7 +1296,7 @@ public void testProjectWithMixedQuoting() { from test | drop first_name | keep *`name` - """, Set.of("*name.*", "*name", "first_name", "first_name.*")); + """, Set.of("_index", "*name.*", "*name", "first_name", "first_name.*")); } public void testProjectKeepAndDropName() { @@ -1270,7 +1304,7 @@ public void testProjectKeepAndDropName() { from test | drop first_name | keep last_name - """, Set.of("last_name", "last_name.*", "first_name", "first_name.*")); + """, Set.of("_index", "last_name", "last_name.*", "first_name", "first_name.*")); } public void testProjectDropPattern() { @@ -1319,7 +1353,7 @@ public void testProjectDropPatternAndKeepOthers() { from test | drop l* | keep first_name, salary - """, Set.of("l*", "first_name", "first_name.*", "salary", "salary.*")); + """, Set.of("_index", "l*", "first_name", "first_name.*", "salary", "salary.*")); } public void testProjectDropWithQuotedAndUnquotedPatternAndKeepOthers() { @@ -1327,7 +1361,7 @@ public void testProjectDropWithQuotedAndUnquotedPatternAndKeepOthers() { from test | drop `l`* | keep first_name, salary - """, Set.of("l*", "first_name", "first_name.*", "salary", "salary.*")); + """, Set.of("_index", "l*", "first_name", "first_name.*", "salary", "salary.*")); } public void testAliasesThatGetDropped() { @@ -1353,7 +1387,7 @@ public void testCountAllGrouped() { | stats c = count(*) by languages | rename languages as l | sort l DESC - """, Set.of("languages", "languages.*")); + """, Set.of("_index", "languages", "languages.*")); } public void testCountAllAndOtherStatGrouped() { @@ -1361,7 +1395,7 @@ public void testCountAllAndOtherStatGrouped() { from test | stats c = count(*), min = min(emp_no) by languages | sort languages - """, Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testCountAllAndOtherStatGrouped_WithInlineStats() { @@ -1371,7 +1405,7 @@ public void testCountAllAndOtherStatGrouped_WithInlineStats() { | inline stats c = count(*), min = min(emp_no) by languages | stats c = count(*), min = min(emp_no) by languages | sort languages - """, Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testCountAllWithImplicitNameOtherStatGrouped() { @@ -1380,7 +1414,7 @@ public void testCountAllWithImplicitNameOtherStatGrouped() { | stats count(*), min = min(emp_no) by languages | drop `count(*)` | sort languages - """, Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testDropWithQuotedAndUnquotedName() { @@ -1389,7 +1423,7 @@ public void testDropWithQuotedAndUnquotedName() { | stats count(*), min = min(emp_no) by languages | drop count`(*)` | sort languages - """, Set.of("emp_no", "emp_no.*", "languages", "languages.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*")); } public void testCountAllWithEval() { @@ -1400,7 +1434,7 @@ public void testCountAllWithEval() { | eval x = min + 1 | stats ca = count(*), cx = count(x) by l | sort l - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testCountAllWithEval_AndInlineStats() { @@ -1413,7 +1447,7 @@ public void testCountAllWithEval_AndInlineStats() { | eval x = min + 1 | stats ca = count(*), cx = count(x) by l | sort l - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testKeepAfterEval_AndInlineStats() { @@ -1426,7 +1460,7 @@ public void testKeepAfterEval_AndInlineStats() { | eval x = min + 1 | keep x, l | sort l - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testKeepBeforeEval_AndInlineStats() { @@ -1439,7 +1473,7 @@ public void testKeepBeforeEval_AndInlineStats() { | eval x = `max(salary)` + 1 | stats min = min(salary) by l | sort l - """, Set.of("languages", "languages.*", "salary", "salary.*", "emp_no", "emp_no.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*", "emp_no", "emp_no.*")); } public void testStatsBeforeEval_AndInlineStats() { @@ -1451,7 +1485,7 @@ public void testStatsBeforeEval_AndInlineStats() { | eval salary = min + 1 | inline stats max(salary) by l | sort l - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testStatsBeforeInlineStats() { @@ -1460,7 +1494,7 @@ public void testStatsBeforeInlineStats() { from test | stats min = min(salary) by languages | inline stats max(min) by languages - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testKeepBeforeInlineStats() { @@ -1469,7 +1503,7 @@ public void testKeepBeforeInlineStats() { from test | keep languages, salary | inline stats max(salary) by languages - """, Set.of("languages", "languages.*", "salary", "salary.*")); + """, Set.of("_index", "languages", "languages.*", "salary", "salary.*")); } public void testCountStar() { @@ -1492,7 +1526,7 @@ public void testDissectOverwriteName() { assertFieldNames(""" from employees | dissect first_name "%{first_name} %{more}" - | keep emp_no, first_name, more""", Set.of("emp_no", "emp_no.*", "first_name", "first_name.*")); + | keep emp_no, first_name, more""", Set.of("_index", "emp_no", "emp_no.*", "first_name", "first_name.*")); } /** @@ -1507,7 +1541,7 @@ public void testAvoidGrokAttributesRemoval() { | drop message | grok type "%{WORD:b}" | stats x = max(b) - | keep x""", Set.of("x", "b", "type", "message", "x.*", "message.*", "type.*", "b.*")); + | keep x""", Set.of("_index", "x", "b", "type", "message", "x.*", "message.*", "type.*", "b.*")); } public void testAvoidGrokAttributesRemoval2() { @@ -1518,18 +1552,21 @@ public void testAvoidGrokAttributesRemoval2() { | lookup join message_types_lookup on message | stats count = count(*) by type | keep count - | sort count""", Set.of("type", "message", "count", "message.*", "type.*", "count.*")); + | sort count""", Set.of("_index", "type", "message", "count", "message.*", "type.*", "count.*")); } public void testAvoidGrokAttributesRemoval3() { - assertFieldNames(""" - from sample_data - | grok message "%{WORD:type}" - | drop type - | lookup join message_types_lookup on message - | stats max = max(event_duration) by type - | keep max - | sort max""", Set.of("type", "event_duration", "message", "max", "event_duration.*", "message.*", "type.*", "max.*")); + assertFieldNames( + """ + from sample_data + | grok message "%{WORD:type}" + | drop type + | lookup join message_types_lookup on message + | stats max = max(event_duration) by type + | keep max + | sort max""", + Set.of("_index", "type", "event_duration", "message", "max", "event_duration.*", "message.*", "type.*", "max.*") + ); } /** @@ -1543,7 +1580,7 @@ public void testAvoidGrokAttributesRemoval4() { | drop message | grok type "%{WORD:b}" | stats x = max(b) - | keep x""", Set.of("x", "b", "type", "message", "x.*", "message.*", "type.*", "b.*")); + | keep x""", Set.of("_index", "x", "b", "type", "message", "x.*", "message.*", "type.*", "b.*")); } /** @@ -1566,6 +1603,7 @@ public void testAvoidGrokAttributesRemoval5() { | SORT message DESC | LIMIT 1""", Set.of( + "_index", "message", "type", "languages", @@ -1592,6 +1630,7 @@ public void testMetrics() { assertFieldNames( query, Set.of( + "_index", "@timestamp", "@timestamp.*", "network.total_bytes_in", @@ -1608,7 +1647,7 @@ public void testMetrics() { public void testLookupJoin() { assertFieldNames( "FROM employees | KEEP languages | RENAME languages AS language_code | LOOKUP JOIN languages_lookup ON language_code", - Set.of("languages", "languages.*", "language_code", "language_code.*"), + Set.of("_index", "languages", "languages.*", "language_code", "language_code.*"), Set.of("languages_lookup") // Since we have KEEP before the LOOKUP JOIN we need to wildcard the lookup index ); } @@ -1621,7 +1660,7 @@ public void testLookupJoinKeep() { | RENAME languages AS language_code | LOOKUP JOIN languages_lookup ON language_code | KEEP languages, language_code, language_name""", - Set.of("languages", "languages.*", "language_code", "language_code.*", "language_name", "language_name.*"), + Set.of("_index", "languages", "languages.*", "language_code", "language_code.*", "language_name", "language_name.*"), Set.of() // Since we have KEEP after the LOOKUP, we can use the global field names instead of wildcarding the lookup index ); } @@ -1634,7 +1673,7 @@ public void testLookupJoinKeepWildcard() { | RENAME languages AS language_code | LOOKUP JOIN languages_lookup ON language_code | KEEP language*""", - Set.of("language*", "languages", "languages.*", "language_code", "language_code.*"), + Set.of("_index", "language*", "languages", "languages.*", "language_code", "language_code.*"), Set.of() // Since we have KEEP after the LOOKUP, we can use the global field names instead of wildcarding the lookup index ); } @@ -1659,7 +1698,17 @@ public void testMultiLookupJoinKeepBefore() { | KEEP @timestamp, client_ip, event_duration, message | LOOKUP JOIN clientips_lookup ON client_ip | LOOKUP JOIN message_types_lookup ON message""", - Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "message", "message.*"), + Set.of( + "_index", + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*" + ), Set.of("clientips_lookup", "message_types_lookup") // Since the KEEP is before both JOINS we need to wildcard both indices ); } @@ -1673,6 +1722,7 @@ public void testMultiLookupJoinKeepBetween() { | KEEP @timestamp, client_ip, event_duration, message, env | LOOKUP JOIN message_types_lookup ON message""", Set.of( + "_index", "@timestamp", "@timestamp.*", "client_ip", @@ -1697,6 +1747,7 @@ public void testMultiLookupJoinKeepAfter() { | LOOKUP JOIN message_types_lookup ON message | KEEP @timestamp, client_ip, event_duration, message, env, type""", Set.of( + "_index", "@timestamp", "@timestamp.*", "client_ip", @@ -1722,7 +1773,7 @@ public void testMultiLookupJoinKeepAfterWildcard() { | LOOKUP JOIN clientips_lookup ON client_ip | LOOKUP JOIN message_types_lookup ON message | KEEP *env*, *type*""", - Set.of("*env*", "*type*", "client_ip", "client_ip.*", "message", "message.*"), + Set.of("_index", "*env*", "*type*", "client_ip", "client_ip.*", "message", "message.*"), Set.of() // Since the KEEP is after both JOINs, we can use the global field names ); } @@ -1749,7 +1800,17 @@ public void testMultiLookupJoinSameIndexKeepBefore() { | LOOKUP JOIN clientips_lookup ON client_ip | EVAL client_ip = message | LOOKUP JOIN clientips_lookup ON client_ip""", - Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "message", "message.*"), + Set.of( + "_index", + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*" + ), Set.of("clientips_lookup") // Since there is no KEEP after the last JOIN, we need to wildcard the index ); } @@ -1764,6 +1825,7 @@ public void testMultiLookupJoinSameIndexKeepBetween() { | EVAL client_ip = message | LOOKUP JOIN clientips_lookup ON client_ip""", Set.of( + "_index", "@timestamp", "@timestamp.*", "client_ip", @@ -1789,6 +1851,7 @@ public void testMultiLookupJoinSameIndexKeepAfter() { | LOOKUP JOIN clientips_lookup ON client_ip | KEEP @timestamp, client_ip, event_duration, message, env""", Set.of( + "_index", "@timestamp", "@timestamp.*", "client_ip", @@ -1808,7 +1871,7 @@ public void testInsist_fieldIsMappedToNonKeywordSingleIndex() { assumeTrue("UNMAPPED_FIELDS available as snapshot only", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); assertFieldNames( "FROM partial_mapping_sample_data | INSIST_🐔 client_ip | KEEP @timestamp, client_ip", - Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*"), + Set.of("_index", "@timestamp", "@timestamp.*", "client_ip", "client_ip.*"), Set.of() ); } @@ -1817,7 +1880,7 @@ public void testInsist_fieldIsMappedToKeywordSingleIndex() { assumeTrue("UNMAPPED_FIELDS available as snapshot only", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); assertFieldNames( "FROM partial_mapping_sample_data | INSIST_🐔 message | KEEP @timestamp, message", - Set.of("@timestamp", "@timestamp.*", "message", "message.*"), + Set.of("_index", "@timestamp", "@timestamp.*", "message", "message.*"), Set.of() ); } @@ -1826,7 +1889,7 @@ public void testInsist_fieldDoesNotExistSingleIndex() { assumeTrue("UNMAPPED_FIELDS available as snapshot only", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); assertFieldNames( "FROM partial_mapping_sample_data | INSIST_🐔 foo | KEEP @timestamp, foo", - Set.of("@timestamp", "@timestamp.*", "foo", "foo.*"), + Set.of("_index", "@timestamp", "@timestamp.*", "foo", "foo.*"), Set.of() ); } @@ -1835,7 +1898,7 @@ public void testInsist_fieldIsUnmappedSingleIndex() { assumeTrue("UNMAPPED_FIELDS available as snapshot only", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); assertFieldNames( "FROM partial_mapping_sample_data | INSIST_🐔 unmapped_message | KEEP @timestamp, unmapped_message", - Set.of("@timestamp", "@timestamp.*", "unmapped_message", "unmapped_message.*"), + Set.of("_index", "@timestamp", "@timestamp.*", "unmapped_message", "unmapped_message.*"), Set.of() ); } @@ -1845,6 +1908,7 @@ public void testInsist_multiFieldTestSingleIndex() { assertFieldNames( "FROM partial_mapping_sample_data | INSIST_🐔 message, unmapped_message, client_ip, foo | KEEP @timestamp, unmapped_message", Set.of( + "_index", "@timestamp", "@timestamp.*", "message", @@ -1864,7 +1928,7 @@ public void testInsist_fieldIsMappedToDifferentTypesMultiIndex() { assumeTrue("UNMAPPED_FIELDS available as snapshot only", EsqlCapabilities.Cap.UNMAPPED_FIELDS.isEnabled()); assertFieldNames( "FROM sample_data_ts_long, sample_data METADATA _index | INSIST_🐔 @timestamp | KEEP _index, @timestamp", - Set.of("@timestamp", "@timestamp.*"), + Set.of("_index", "@timestamp", "@timestamp.*"), Set.of() ); } @@ -1877,7 +1941,17 @@ public void testInsist_multiFieldMappedMultiIndex() { | INSIST_🐔 @timestamp, unmapped_message | INSIST_🐔 message, foo | KEEP _index, @timestamp, message, foo""", - Set.of("@timestamp", "@timestamp.*", "message", "message.*", "unmapped_message", "unmapped_message.*", "foo", "foo.*"), + Set.of( + "_index", + "@timestamp", + "@timestamp.*", + "message", + "message.*", + "unmapped_message", + "unmapped_message.*", + "foo", + "foo.*" + ), Set.of() ); } @@ -1892,7 +1966,17 @@ public void testJoinMaskingKeep() { | rename type as message | lookup join message_types_lookup on message | keep `language.name`""", - Set.of("language.name", "type", "language_name", "message", "language_name.*", "message.*", "type.*", "language.name.*") + Set.of( + "_index", + "language.name", + "type", + "language_name", + "message", + "language_name.*", + "message.*", + "type.*", + "language.name.*" + ) ); } @@ -1904,30 +1988,36 @@ public void testJoinMaskingKeep2() { | lookup join message_types_lookup on message | rename type as message | lookup join message_types_lookup on message - | keep `language.name`""", Set.of("language.name", "type", "message", "message.*", "type.*", "language.name.*")); + | keep `language.name`""", Set.of("_index", "language.name", "type", "message", "message.*", "type.*", "language.name.*")); } public void testEnrichMaskingEvalOn() { - assertFieldNames(""" - from employees - | eval language_name = null - | enrich languages_policy on languages - | rename language_name as languages - | eval languages = length(languages) - | enrich languages_policy on languages - | keep emp_no, language_name""", Set.of("emp_no", "language_name", "languages", "language_name.*", "languages.*", "emp_no.*")); + assertFieldNames( + """ + from employees + | eval language_name = null + | enrich languages_policy on languages + | rename language_name as languages + | eval languages = length(languages) + | enrich languages_policy on languages + | keep emp_no, language_name""", + Set.of("_index", "emp_no", "language_name", "languages", "language_name.*", "languages.*", "emp_no.*") + ); } public void testEnrichAndJoinMaskingEvalWh() { - assertFieldNames(""" - from employees - | eval language_name = null - | enrich languages_policy on languages - | rename language_name as languages - | eval languages = length(languages) - | enrich languages_policy on languages - | lookup join message_types_lookup on language_name - | keep emp_no, language_name""", Set.of("emp_no", "language_name", "languages", "language_name.*", "languages.*", "emp_no.*")); + assertFieldNames( + """ + from employees + | eval language_name = null + | enrich languages_policy on languages + | rename language_name as languages + | eval languages = length(languages) + | enrich languages_policy on languages + | lookup join message_types_lookup on language_name + | keep emp_no, language_name""", + Set.of("_index", "emp_no", "language_name", "languages", "language_name.*", "languages.*", "emp_no.*") + ); } public void testDropAgainWithWildcardAfterEval() { @@ -1937,7 +2027,7 @@ public void testDropAgainWithWildcardAfterEval() { | drop full_name | drop *name | keep emp_no - """, Set.of("emp_no", "emp_no.*", "*name", "*name.*")); + """, Set.of("_index", "emp_no", "emp_no.*", "*name", "*name.*")); } public void testDropWildcardFieldsAfterRename() { @@ -1949,7 +2039,7 @@ public void testDropWildcardFieldsAfterRename() { | drop first_names | drop *_names | keep gender""", - Set.of("first_name", "first_name.*", "last_name", "last_name.*", "*_names", "*_names.*", "gender", "gender.*") + Set.of("_index", "first_name", "first_name.*", "last_name", "last_name.*", "*_names", "*_names.*", "gender", "gender.*") ); } @@ -1984,7 +2074,7 @@ public void testDropWildcardFieldsAfterLookupJoinsAndKeep() { | KEEP @timestamp, message, *e* | SORT @timestamp | DROP *e""", - Set.of("client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), + Set.of("_index", "client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), Set.of() ); } @@ -1999,7 +2089,7 @@ public void testDropWildcardFieldsAfterLookupJoinKeepLookupJoin() { | LOOKUP JOIN message_types_lookup ON message | SORT @timestamp | DROP *e""", - Set.of("client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), + Set.of("_index", "client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), Set.of("message_types_lookup") ); } @@ -2014,7 +2104,7 @@ public void testDropWildcardFieldsAfterKeepAndLookupJoins() { | LOOKUP JOIN message_types_lookup ON message | SORT @timestamp | DROP *e""", - Set.of("client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), + Set.of("_index", "client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), Set.of("clientips_lookup", "message_types_lookup") ); } @@ -2030,7 +2120,7 @@ public void testDropWildcardFieldsAfterKeepAndLookupJoins2() { | LOOKUP JOIN message_types_lookup ON message | SORT @timestamp | DROP *e, client_ip""", - Set.of("client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), + Set.of("_index", "client_ip", "client_ip.*", "message", "message.*", "@timestamp", "@timestamp.*", "*e*", "*e", "*e.*"), Set.of("clientips_lookup", "message_types_lookup") ); } @@ -2044,7 +2134,7 @@ public void testForkFieldsWithKeepAfterFork() { (WHERE d > 1000 AND e == "aaa" | EVAL c = a + 200) | WHERE x > y | KEEP a, b, c, d, x - """, Set.of("a", "x", "y", "c", "d", "e", "e.*", "d.*", "y.*", "x.*", "a.*", "c.*")); + """, Set.of("_index", "a", "x", "y", "c", "d", "e", "e.*", "d.*", "y.*", "x.*", "a.*", "c.*")); } public void testForkFieldsWithKeepBeforeFork() { @@ -2056,7 +2146,7 @@ public void testForkFieldsWithKeepBeforeFork() { | FORK (WHERE c > 1 AND a < 10000 | EVAL d = a + 500) (WHERE d > 1000 AND e == "aaa" | EVAL c = a + 200) | WHERE x > y - """, Set.of("x", "y", "a", "d", "e", "b", "c", "e.*", "d.*", "y.*", "x.*", "a.*", "c.*", "b.*")); + """, Set.of("_index", "x", "y", "a", "d", "e", "b", "c", "e.*", "d.*", "y.*", "x.*", "a.*", "c.*", "b.*")); } public void testForkFieldsWithNoProjection() { @@ -2078,7 +2168,7 @@ public void testForkFieldsWithStatsInOneBranch() { | FORK (WHERE c > 1 AND a < 10000 | EVAL d = a + 500) (STATS x = count(*), y=min(z)) | WHERE x > y - """, Set.of("x", "y", "a", "c", "z", "y.*", "x.*", "z.*", "a.*", "c.*")); + """, Set.of("_index", "x", "y", "a", "c", "z", "y.*", "x.*", "z.*", "a.*", "c.*")); } public void testForkFieldsWithEnrichAndLookupJoins() { @@ -2094,7 +2184,27 @@ public void testForkFieldsWithEnrichAndLookupJoins() { | LOOKUP JOIN my_lookup_index ON xyz | WHERE x > y OR _fork == "fork1" """, - Set.of("x", "y", "a", "c", "abc", "b", "def", "z", "xyz", "def.*", "y.*", "x.*", "xyz.*", "z.*", "abc.*", "a.*", "c.*", "b.*"), + Set.of( + "_index", + "x", + "y", + "a", + "c", + "abc", + "b", + "def", + "z", + "xyz", + "def.*", + "y.*", + "x.*", + "xyz.*", + "z.*", + "abc.*", + "a.*", + "c.*", + "b.*" + ), Set.of("my_lookup_index") ); } @@ -2108,7 +2218,7 @@ public void testForkWithStatsInAllBranches() { (EVAL z = a * b | STATS m = max(z)) (STATS x = count(*), y=min(z)) | WHERE x > y - """, Set.of("x", "y", "c", "a", "z", "y.*", "x.*", "z.*", "a.*", "c.*")); + """, Set.of("_index", "x", "y", "c", "a", "z", "y.*", "x.*", "z.*", "a.*", "c.*")); } public void testForkWithStatsInAllBranches1() { @@ -2117,7 +2227,7 @@ public void testForkWithStatsInAllBranches1() { | FORK ( STATS x = min(last_name)) ( EVAL last_name = first_name | STATS y = max(last_name)) - """, Set.of("first_name", "last_name", "first_name.*", "last_name.*")); + """, Set.of("_index", "first_name", "last_name", "first_name.*", "last_name.*")); } public void testForkWithStatsInAllBranches2() { @@ -2126,7 +2236,7 @@ public void testForkWithStatsInAllBranches2() { | FORK ( EVAL last_name = first_name | STATS y = VALUES(last_name)) ( STATS x = VALUES(last_name)) - """, Set.of("first_name", "last_name", "first_name.*", "last_name.*")); + """, Set.of("_index", "first_name", "last_name", "first_name.*", "last_name.*")); } public void testForkWithStatsAndWhere() { @@ -2169,7 +2279,7 @@ public void testForkRefs1() { | FORK ( EVAL x = first_name) ( EVAL x = last_name) - """, Set.of("first_name", "last_name", "last_name.*", "first_name.*")); + """, Set.of("_index", "first_name", "last_name", "last_name.*", "first_name.*")); } public void testForkRefs2() { @@ -2178,7 +2288,7 @@ public void testForkRefs2() { | FORK ( KEEP first_name | EVAL x = first_name) ( KEEP last_name | EVAL x = last_name) - """, Set.of("first_name", "last_name", "last_name.*", "first_name.*")); + """, Set.of("_index", "first_name", "last_name", "last_name.*", "first_name.*")); } public void testForkRefs3() { @@ -2187,18 +2297,21 @@ public void testForkRefs3() { | FORK ( KEEP first_name | EVAL last_name = first_name) ( KEEP first_name | EVAL x = first_name) - """, Set.of("first_name", "first_name.*")); + """, Set.of("_index", "first_name", "first_name.*")); } public void testForkRef4() { - assertFieldNames(""" - from employees - | sort emp_no - | limit 1 - | FORK - (eval x = to_string(languages) | enrich languages_policy on x | keep language_name) - (eval y = to_string(emp_no) | enrich languages_policy on y | keep emp_no) - """, Set.of("emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*", "y", "y.*")); + assertFieldNames( + """ + from employees + | sort emp_no + | limit 1 + | FORK + (eval x = to_string(languages) | enrich languages_policy on x | keep language_name) + (eval y = to_string(emp_no) | enrich languages_policy on y | keep emp_no) + """, + Set.of("_index", "emp_no", "emp_no.*", "languages", "languages.*", "language_name", "language_name.*", "x", "x.*", "y", "y.*") + ); } public void testRerankerAfterFuse() { @@ -2211,7 +2324,7 @@ public void testRerankerAfterFuse() { | EVAL _score=ROUND(_score, 2) | SORT _score DESC, book_no ASC | LIMIT 2 - | KEEP book_no, title, author, _score""", Set.of("book_no", "title", "author", "title.*", "author.*", "book_no.*")); + | KEEP book_no, title, author, _score""", Set.of("_index", "book_no", "title", "author", "title.*", "author.*", "book_no.*")); } public void testSimpleFuse() { @@ -2222,7 +2335,7 @@ public void testSimpleFuse() { | FUSE | EVAL _score = round(_score, 4) | KEEP _score, _fork, emp_no - | SORT _score, _fork, emp_no""", Set.of("emp_no", "emp_no.*")); + | SORT _score, _fork, emp_no""", Set.of("_index", "emp_no", "emp_no.*")); } public void testFuseWithMatchAndScore() { @@ -2234,7 +2347,7 @@ public void testFuseWithMatchAndScore() { | SORT _score DESC, _id, _index | EVAL _fork = mv_sort(_fork) | EVAL _score = round(_score, 5) - | KEEP _score, _fork, _id""", Set.of("title", "author", "title.*", "author.*")); + | KEEP _score, _fork, _id""", Set.of("_index", "title", "author", "title.*", "author.*")); } public void testFuseWithDisjunctionAndPostFilter() { @@ -2247,7 +2360,7 @@ public void testFuseWithDisjunctionAndPostFilter() { | EVAL _fork = mv_sort(_fork) | EVAL _score = round(_score, 5) | KEEP _score, _fork, _id - | WHERE _score > 0.014""", Set.of("title", "author", "title.*", "author.*")); + | WHERE _score > 0.014""", Set.of("_index", "title", "author", "title.*", "author.*")); } public void testFuseWithStats() { @@ -2258,7 +2371,7 @@ public void testFuseWithStats() { ( WHERE author:"Ursula K. Le Guin" AND title:"short stories" | SORT _score, _id DESC | LIMIT 3) | FUSE | STATS count_fork=COUNT(*) BY _fork - | SORT _fork""", Set.of("title", "author", "title.*", "author.*")); + | SORT _fork""", Set.of("_index", "title", "author", "title.*", "author.*")); } public void testFuseWithMultipleForkBranches() { @@ -2273,7 +2386,7 @@ public void testFuseWithMultipleForkBranches() { | EVAL _fork = mv_sort(_fork) | EVAL _score = round(_score, 4) | EVAL title = trim(substring(title, 1, 20)) - | KEEP _score, author, title, _fork""", Set.of("author", "title", "year", "title.*", "author.*", "year.*")); + | KEEP _score, author, title, _fork""", Set.of("_index", "author", "title", "year", "title.*", "author.*", "year.*")); } public void testFuseWithSemanticSearch() { @@ -2285,7 +2398,7 @@ public void testFuseWithSemanticSearch() { | SORT _score DESC, _id, _index | EVAL _score = round(_score, 4) | EVAL _fork = mv_sort(_fork) - | KEEP _fork, _score, _id, semantic_text_field""", Set.of("semantic_text_field", "semantic_text_field.*")); + | KEEP _fork, _score, _id, semantic_text_field""", Set.of("_index", "semantic_text_field", "semantic_text_field.*")); } public void testSimpleFork() { @@ -2294,7 +2407,7 @@ public void testSimpleFork() { | FORK ( WHERE emp_no == 10001 ) ( WHERE emp_no == 10002 ) | KEEP emp_no, _fork - | SORT emp_no""", Set.of("emp_no", "emp_no.*")); + | SORT emp_no""", Set.of("_index", "emp_no", "emp_no.*")); } public void testSimpleForkWithStats() { @@ -2304,7 +2417,7 @@ public void testSimpleForkWithStats() { | EVAL score = round(_score, 2) | FORK (SORT score DESC, author | LIMIT 5 | KEEP author, score) (STATS total = COUNT(*)) - | SORT _fork, score DESC, author""", Set.of("score", "author", "score.*", "author.*")); + | SORT _fork, score DESC, author""", Set.of("_index", "score", "author", "score.*", "author.*")); } public void testForkWithWhereSortAndLimit() { @@ -2313,7 +2426,7 @@ public void testForkWithWhereSortAndLimit() { | FORK ( WHERE hire_date < "1985-03-01T00:00:00Z" | SORT first_name | LIMIT 5 ) ( WHERE hire_date < "1988-03-01T00:00:00Z" | SORT first_name | LIMIT 5 ) | KEEP emp_no, first_name, _fork - | SORT emp_no, _fork""", Set.of("emp_no", "first_name", "hire_date", "first_name.*", "hire_date.*", "emp_no.*")); + | SORT emp_no, _fork""", Set.of("_index", "emp_no", "first_name", "hire_date", "first_name.*", "hire_date.*", "emp_no.*")); } public void testFiveFork() { @@ -2325,16 +2438,19 @@ public void testFiveFork() { ( WHERE emp_no == 10002 ) ( WHERE emp_no == 10001 ) | KEEP _fork, emp_no - | SORT _fork""", Set.of("emp_no", "emp_no.*")); + | SORT _fork""", Set.of("_index", "emp_no", "emp_no.*")); } public void testForkWithWhereSortDescAndLimit() { - assertFieldNames(""" - FROM employees - | FORK ( WHERE hire_date < "1985-03-01T00:00:00Z" | SORT first_name DESC | LIMIT 2 ) - ( WHERE hire_date < "1988-03-01T00:00:00Z" | SORT first_name DESC NULLS LAST | LIMIT 2 ) - | KEEP _fork, emp_no, first_name - | SORT _fork, first_name DESC""", Set.of("first_name", "emp_no", "hire_date", "first_name.*", "hire_date.*", "emp_no.*")); + assertFieldNames( + """ + FROM employees + | FORK ( WHERE hire_date < "1985-03-01T00:00:00Z" | SORT first_name DESC | LIMIT 2 ) + ( WHERE hire_date < "1988-03-01T00:00:00Z" | SORT first_name DESC NULLS LAST | LIMIT 2 ) + | KEEP _fork, emp_no, first_name + | SORT _fork, first_name DESC""", + Set.of("_index", "first_name", "emp_no", "hire_date", "first_name.*", "hire_date.*", "emp_no.*") + ); } public void testForkWithCommonPrefilter() { @@ -2344,7 +2460,7 @@ public void testForkWithCommonPrefilter() { | FORK ( SORT emp_no ASC | LIMIT 2 ) ( SORT emp_no DESC NULLS LAST | LIMIT 2 ) | KEEP _fork, emp_no - | SORT _fork, emp_no""", Set.of("emp_no", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "emp_no.*")); } public void testForkWithSemanticSearchAndScore() { @@ -2354,7 +2470,7 @@ public void testForkWithSemanticSearchAndScore() { ( WHERE semantic_text_field:"something else" | SORT _score DESC | LIMIT 2) | EVAL _score = round(_score, 4) | SORT _fork, _score, _id - | KEEP _fork, _score, _id, semantic_text_field""", Set.of("semantic_text_field", "semantic_text_field.*")); + | KEEP _fork, _score, _id, semantic_text_field""", Set.of("_index", "semantic_text_field", "semantic_text_field.*")); } public void testForkWithEvals() { @@ -2363,7 +2479,7 @@ public void testForkWithEvals() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081 | EVAL x = "abc" | EVAL y = 1) (WHERE emp_no == 10081 OR emp_no == 10087 | EVAL x = "def" | EVAL z = 2) | KEEP _fork, emp_no, x, y, z - | SORT _fork, emp_no""", Set.of("emp_no", "x", "y", "z", "y.*", "x.*", "z.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "x", "y", "z", "y.*", "x.*", "z.*", "emp_no.*")); } public void testForkWithStats() { @@ -2374,7 +2490,7 @@ public void testForkWithStats() { (STATS x = COUNT(*), y = MAX(emp_no), z = MIN(emp_no)) (STATS x = COUNT(*), y = MIN(emp_no)) | KEEP _fork, emp_no, x, y, z - | SORT _fork, emp_no""", Set.of("emp_no", "x", "y", "z", "y.*", "x.*", "z.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "x", "y", "z", "y.*", "x.*", "z.*", "emp_no.*")); } public void testForkWithDissect() { @@ -2389,6 +2505,7 @@ public void testForkWithDissect() { | KEEP _fork, emp_no, x, y, z, w | SORT _fork, emp_no""", Set.of( + "_index", "emp_no", "x", "y", @@ -2421,6 +2538,7 @@ public void testForkWithMixOfCommands() { | KEEP _fork, emp_no, x, y, z, a | SORT _fork, emp_no""", Set.of( + "_index", "emp_no", "x", "y", @@ -2450,7 +2568,7 @@ public void testForkWithFiltersOnConstantValues() { (STATS x = COUNT(*), y = MIN(emp_no)) | WHERE _fork == "fork2" OR a == "y" | KEEP _fork, emp_no, x, y, z - | SORT _fork, emp_no""", Set.of("emp_no", "a", "a.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "a", "a.*", "emp_no.*")); } public void testForkWithUnsupportedAttributes() { @@ -2473,6 +2591,7 @@ public void testForkAfterLookupJoin() { | KEEP _fork, emp_no, language_code, language_name | SORT _fork, emp_no""", Set.of( + "_index", "emp_no", "language_code", "language_name", @@ -2499,6 +2618,7 @@ public void testForkBeforeLookupJoin() { | KEEP _fork, emp_no, language_code, language_name | SORT _fork, emp_no""", Set.of( + "_index", "emp_no", "language_code", "language_name", @@ -2524,6 +2644,7 @@ public void testForkBranchWithLookupJoin() { | KEEP _fork, emp_no, language_code, language_name | SORT _fork, emp_no""", Set.of( + "_index", "emp_no", "language_code", "language_name", @@ -2550,7 +2671,7 @@ public void testForkBeforeStats() { ( SORT emp_no ASC | LIMIT 2 | EVAL x = last_name ) ( EVAL x = "abc" | EVAL y = "aaa" ) | STATS c = count(*), m = max(_fork)""", - Set.of("first_name", "emp_no", "last_name", "last_name.*", "first_name.*", "emp_no.*") + Set.of("_index", "first_name", "emp_no", "last_name", "last_name.*", "first_name.*", "emp_no.*") ); } @@ -2565,7 +2686,7 @@ public void testForkBeforeStatsWithWhere() { ( SORT emp_no ASC | LIMIT 2 | EVAL x = last_name ) ( EVAL x = "abc" | EVAL y = "aaa" ) | STATS a = count(*) WHERE _fork == "fork1", - b = max(_fork)""", Set.of("first_name", "emp_no", "last_name", "last_name.*", "first_name.*", "emp_no.*")); + b = max(_fork)""", Set.of("_index", "first_name", "emp_no", "last_name", "last_name.*", "first_name.*", "emp_no.*")); } public void testForkBeforeStatsByWithWhere() { @@ -2580,7 +2701,7 @@ public void testForkBeforeStatsByWithWhere() { ( EVAL x = "abc" | EVAL y = "aaa" ) | STATS a = count(*) WHERE emp_no > 10000, b = max(x) WHERE _fork == "fork1" BY _fork - | SORT _fork""", Set.of("emp_no", "x", "first_name", "last_name", "last_name.*", "x.*", "first_name.*", "emp_no.*")); + | SORT _fork""", Set.of("_index", "emp_no", "x", "first_name", "last_name", "last_name.*", "x.*", "first_name.*", "emp_no.*")); } public void testForkAfterDrop() { @@ -2593,12 +2714,15 @@ public void testForkAfterDrop() { } public void testForkBranchWithDrop() { - assertFieldNames(""" - FROM languages - | FORK ( EVAL x = 1 | DROP language_code | WHERE language_name == "English" | DROP x ) - ( WHERE language_name != "English" ) - | SORT _fork, language_name - | KEEP language_name, language_code, _fork""", Set.of("language_name", "language_code", "language_code.*", "language_name.*")); + assertFieldNames( + """ + FROM languages + | FORK ( EVAL x = 1 | DROP language_code | WHERE language_name == "English" | DROP x ) + ( WHERE language_name != "English" ) + | SORT _fork, language_name + | KEEP language_name, language_code, _fork""", + Set.of("_index", "language_name", "language_code", "language_code.*", "language_name.*") + ); } public void testForkBeforeDrop() { @@ -2615,7 +2739,7 @@ public void testForkBranchWithKeep() { FROM languages | FORK ( WHERE language_name == "English" | KEEP language_name, language_code ) ( WHERE language_name != "English" ) - | SORT _fork, language_name""", Set.of("language_name", "language_code", "language_code.*", "language_name.*")); + | SORT _fork, language_name""", Set.of("_index", "language_name", "language_code", "language_code.*", "language_name.*")); } public void testForkBeforeRename() { @@ -2628,21 +2752,27 @@ public void testForkBeforeRename() { } public void testForkBranchWithRenameAs() { - assertFieldNames(""" - FROM languages - | FORK (RENAME language_code AS code | WHERE code == 1 OR code == 2) - (WHERE language_code == 1 | RENAME language_code AS x) - | SORT _fork, language_name - | KEEP code, language_name, x, _fork""", Set.of("language_name", "language_code", "language_code.*", "language_name.*")); + assertFieldNames( + """ + FROM languages + | FORK (RENAME language_code AS code | WHERE code == 1 OR code == 2) + (WHERE language_code == 1 | RENAME language_code AS x) + | SORT _fork, language_name + | KEEP code, language_name, x, _fork""", + Set.of("_index", "language_name", "language_code", "language_code.*", "language_name.*") + ); } public void testForkBranchWithRenameEquals() { - assertFieldNames(""" - FROM languages - | FORK (RENAME code = language_code | WHERE code == 1 OR code == 2) - (WHERE language_code == 1 | RENAME x = language_code) - | SORT _fork, language_name - | KEEP code, language_name, x, _fork""", Set.of("language_name", "language_code", "language_code.*", "language_name.*")); + assertFieldNames( + """ + FROM languages + | FORK (RENAME code = language_code | WHERE code == 1 OR code == 2) + (WHERE language_code == 1 | RENAME x = language_code) + | SORT _fork, language_name + | KEEP code, language_name, x, _fork""", + Set.of("_index", "language_name", "language_code", "language_code.*", "language_name.*") + ); } public void testForkAfterRename() { @@ -2662,7 +2792,7 @@ public void testForkBeforeDissect() { | EVAL x = concat(gender, " foobar") | DISSECT x "%{a} %{b}" | SORT _fork, emp_no - | KEEP emp_no, gender, x, a, b, _fork""", Set.of("emp_no", "gender", "gender.*", "emp_no.*")); + | KEEP emp_no, gender, x, a, b, _fork""", Set.of("_index", "emp_no", "gender", "gender.*", "emp_no.*")); } public void testForkBranchWithDissect() { @@ -2673,7 +2803,7 @@ public void testForkBranchWithDissect() { | DISSECT x "%{a} %{b}") (WHERE emp_no == 10081 OR emp_no == 10087) | SORT _fork, emp_no - | KEEP emp_no, gender, x, a, b, _fork""", Set.of("emp_no", "gender", "gender.*", "emp_no.*")); + | KEEP emp_no, gender, x, a, b, _fork""", Set.of("_index", "emp_no", "gender", "gender.*", "emp_no.*")); } public void testForkAfterDissect() { @@ -2684,7 +2814,7 @@ public void testForkAfterDissect() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) | SORT _fork, emp_no - | KEEP emp_no, gender, x, a, b, _fork""", Set.of("emp_no", "gender", "gender.*", "emp_no.*")); + | KEEP emp_no, gender, x, a, b, _fork""", Set.of("_index", "emp_no", "gender", "gender.*", "emp_no.*")); } public void testForkAfterEnrich() { @@ -2698,6 +2828,7 @@ public void testForkAfterEnrich() { (WHERE city.country.name == "Japan") | SORT _fork, city.name""", Set.of( + "_index", "city.name", "airport", "city.country.continent.planet.name", @@ -2720,6 +2851,7 @@ public void testForkBranchWithEnrich() { (ENRICH city_names ON city.name WITH city.country.continent.planet.name = airport) | SORT _fork, city.name""", Set.of( + "_index", "city.name", "airport", "city.country.continent.planet.name", @@ -2743,6 +2875,7 @@ public void testForkBeforeEnrich() { | ENRICH city_names ON city.name WITH city.country.continent.planet.name = airport | SORT _fork, city.name""", Set.of( + "_index", "city.name", "airport", "city.country.name", @@ -2762,7 +2895,7 @@ public void testForkBeforeMvExpand() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) | MV_EXPAND job_positions - | SORT _fork, emp_no, job_positions""", Set.of("emp_no", "job_positions", "job_positions.*", "emp_no.*")); + | SORT _fork, emp_no, job_positions""", Set.of("_index", "emp_no", "job_positions", "job_positions.*", "emp_no.*")); } public void testForkBranchWithMvExpand() { @@ -2771,7 +2904,7 @@ public void testForkBranchWithMvExpand() { | KEEP emp_no, job_positions | FORK (WHERE emp_no == 10048 OR emp_no == 10081 | MV_EXPAND job_positions) (WHERE emp_no == 10081 OR emp_no == 10087) - | SORT _fork, emp_no, job_positions""", Set.of("emp_no", "job_positions", "job_positions.*", "emp_no.*")); + | SORT _fork, emp_no, job_positions""", Set.of("_index", "emp_no", "job_positions", "job_positions.*", "emp_no.*")); } public void testForkAfterMvExpand() { @@ -2781,7 +2914,7 @@ public void testForkAfterMvExpand() { | MV_EXPAND job_positions | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) - | SORT _fork, emp_no, job_positions""", Set.of("emp_no", "job_positions", "job_positions.*", "emp_no.*")); + | SORT _fork, emp_no, job_positions""", Set.of("_index", "emp_no", "job_positions", "job_positions.*", "emp_no.*")); } public void testForkBeforeInlineStatsIgnore() { @@ -2793,7 +2926,7 @@ public void testForkBeforeInlineStatsIgnore() { (WHERE emp_no == 10081 OR emp_no == 10087) | INLINE STATS max_lang = MAX(languages) BY gender | SORT emp_no, gender, _fork - | LIMIT 5""", Set.of("emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); + | LIMIT 5""", Set.of("_index", "emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); } public void testForkBranchWithInlineStatsIgnore() { @@ -2806,7 +2939,7 @@ public void testForkBranchWithInlineStatsIgnore() { (WHERE emp_no == 10081 OR emp_no == 10087 | INLINE STATS x = MIN(languages)) (WHERE emp_no == 10012 OR emp_no == 10012) - | SORT emp_no, gender, _fork""", Set.of("emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); + | SORT emp_no, gender, _fork""", Set.of("_index", "emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); } public void testForkAfterInlineStatsIgnore() { @@ -2817,7 +2950,7 @@ public void testForkAfterInlineStatsIgnore() { | INLINE STATS max_lang = MAX(languages) BY gender | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) - | SORT emp_no, gender, _fork""", Set.of("emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); + | SORT emp_no, gender, _fork""", Set.of("_index", "emp_no", "gender", "languages", "gender.*", "languages.*", "emp_no.*")); } public void testForkBeforeChangePoint() { @@ -2829,7 +2962,7 @@ public void testForkBeforeChangePoint() { (WHERE emp_no <= 10100) | CHANGE_POINT salary ON emp_no | STATS COUNT() by type - | SORT type""", Set.of("type", "emp_no", "salary", "type.*", "salary.*", "emp_no.*")); + | SORT type""", Set.of("_index", "type", "emp_no", "salary", "type.*", "salary.*", "emp_no.*")); } public void testForkBranchWithChangePoint() { @@ -2841,18 +2974,21 @@ public void testForkBranchWithChangePoint() { (EVAL salary=CASE(emp_no==10087, 1000000, salary) | CHANGE_POINT salary ON emp_no) | STATS COUNT() by type, _fork - | SORT _fork, type""", Set.of("type", "emp_no", "salary", "type.*", "salary.*", "emp_no.*")); + | SORT _fork, type""", Set.of("_index", "type", "emp_no", "salary", "type.*", "salary.*", "emp_no.*")); } public void testForkAfterChangePoint() { - assertFieldNames(""" - FROM employees - | KEEP emp_no, salary - | EVAL salary = CASE(emp_no==10042, 1000000, salary) - | CHANGE_POINT salary ON emp_no - | FORK (STATS a = COUNT() by type) - (STATS b = VALUES(type)) - | SORT _fork, a, type, b""", Set.of("a", "type", "b", "emp_no", "salary", "type.*", "a.*", "salary.*", "b.*", "emp_no.*")); + assertFieldNames( + """ + FROM employees + | KEEP emp_no, salary + | EVAL salary = CASE(emp_no==10042, 1000000, salary) + | CHANGE_POINT salary ON emp_no + | FORK (STATS a = COUNT() by type) + (STATS b = VALUES(type)) + | SORT _fork, a, type, b""", + Set.of("_index", "a", "type", "b", "emp_no", "salary", "type.*", "a.*", "salary.*", "b.*", "emp_no.*") + ); } public void testForkBeforeCompletion() { @@ -2862,7 +2998,7 @@ public void testForkBeforeCompletion() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) | COMPLETION x=CONCAT(first_name, " ", last_name) WITH { "inference_id" : "test_completion" } - | SORT _fork, emp_no""", Set.of("emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); } public void testForkBranchWithCompletion() { @@ -2872,7 +3008,7 @@ public void testForkBranchWithCompletion() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081 | COMPLETION x=CONCAT(first_name, " ", last_name) WITH { "inference_id" : "test_completion" }) (WHERE emp_no == 10081 OR emp_no == 10087) - | SORT _fork, emp_no""", Set.of("emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); } public void testForkAfterCompletion() { @@ -2882,7 +3018,7 @@ public void testForkAfterCompletion() { | COMPLETION x=CONCAT(first_name, " ", last_name) WITH { "inference_id" : "test_completion" } | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) - | SORT _fork, emp_no""", Set.of("emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); + | SORT _fork, emp_no""", Set.of("_index", "emp_no", "first_name", "last_name", "last_name.*", "first_name.*", "emp_no.*")); } public void testForkAfterGrok() { @@ -2893,7 +3029,7 @@ public void testForkAfterGrok() { | FORK (WHERE emp_no == 10048 OR emp_no == 10081) (WHERE emp_no == 10081 OR emp_no == 10087) | SORT _fork, emp_no - | KEEP emp_no, gender, x, a, b, _fork""", Set.of("emp_no", "gender", "gender.*", "emp_no.*")); + | KEEP emp_no, gender, x, a, b, _fork""", Set.of("_index", "emp_no", "gender", "gender.*", "emp_no.*")); } public void testForkBranchWithGrok() { @@ -2907,7 +3043,21 @@ public void testForkBranchWithGrok() { | GROK b "%{WORD:x} %{WORD:y} %{WORD:z}" ) | KEEP _fork, emp_no, x, y, z | SORT _fork, emp_no""", - Set.of("emp_no", "x", "y", "z", "first_name", "last_name", "y.*", "last_name.*", "x.*", "z.*", "first_name.*", "emp_no.*") + Set.of( + "_index", + "emp_no", + "x", + "y", + "z", + "first_name", + "last_name", + "y.*", + "last_name.*", + "x.*", + "z.*", + "first_name.*", + "emp_no.*" + ) ); } @@ -2919,13 +3069,13 @@ public void testForkBeforeGrok() { | EVAL x = concat(gender, " foobar") | GROK x "%{WORD:a} %{WORD:b}" | SORT _fork, emp_no - | KEEP emp_no, gender, x, a, b, _fork""", Set.of("emp_no", "gender", "gender.*", "emp_no.*")); + | KEEP emp_no, gender, x, a, b, _fork""", Set.of("_index", "emp_no", "gender", "gender.*", "emp_no.*")); } public void testImplicitFieldNames() { assertFieldNames(""" FROM sample_data - | STATS x = 1 year + TBUCKET(1 day) BY b1d = TBUCKET(1 day)""", Set.of("@timestamp", "@timestamp.*")); + | STATS x = 1 year + TBUCKET(1 day) BY b1d = TBUCKET(1 day)""", Set.of("_index", "@timestamp", "@timestamp.*")); } public void testKeepTimestampBeforeStats() { @@ -2935,7 +3085,7 @@ public void testKeepTimestampBeforeStats() { | KEEP @timestamp, client_ip | STATS count = COUNT(*), avg_dur = AVG(event_duration) BY hour = TBUCKET(1h), client_ip | SORT hour ASC - """, Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*")); + """, Set.of("_index", "@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*")); } public void testKeepAtWildcardBeforeStats() { @@ -2945,17 +3095,20 @@ public void testKeepAtWildcardBeforeStats() { | KEEP @*, message | STATS errors = COUNT() BY day = TBUCKET(1d), message | SORT day ASC - """, Set.of("@timestamp", "@timestamp.*", "@*", "message", "message.*")); + """, Set.of("_index", "@timestamp", "@timestamp.*", "@*", "message", "message.*")); } public void testKeepWildcardBeforeStats() { - assertFieldNames(""" - FROM sample_data - | WHERE client_ip IS NOT NULL - | KEEP *stamp*, client_ip - | STATS p95 = PERCENTILE(event_duration, 95) BY ten_min = TBUCKET(10min), client_ip - | SORT ten_min ASC - """, Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "*stamp*")); + assertFieldNames( + """ + FROM sample_data + | WHERE client_ip IS NOT NULL + | KEEP *stamp*, client_ip + | STATS p95 = PERCENTILE(event_duration, 95) BY ten_min = TBUCKET(10min), client_ip + | SORT ten_min ASC + """, + Set.of("_index", "@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "*stamp*") + ); } public void testStatsChainingWithTimestampCarriedForward() { @@ -2966,7 +3119,7 @@ public void testStatsChainingWithTimestampCarriedForward() { | WHERE day_count > 0 | STATS hour_count = COUNT(), hour_p95 = PERCENTILE(day_p95, 95) BY hour = TBUCKET(1h), day | SORT day ASC, hour ASC - """, Set.of("@timestamp", "@timestamp.*", "event_duration", "event_duration.*")); + """, Set.of("_index", "@timestamp", "@timestamp.*", "event_duration", "event_duration.*")); } public void testStatsChainingWithTimestampEval() { @@ -2977,7 +3130,7 @@ public void testStatsChainingWithTimestampEval() { | STATS total = COUNT(*), med = MEDIAN(event_duration) BY d = TBUCKET(1d), message, t | WHERE total > 5 | STATS day_total = SUM(total), hour_med = MEDIAN(med) BY h = TBUCKET(1h), message - """, Set.of("@timestamp", "@timestamp.*", "event_duration", "event_duration.*", "message", "message.*")); + """, Set.of("_index", "@timestamp", "@timestamp.*", "event_duration", "event_duration.*", "message", "message.*")); } public void testStatsChainingWithTimestampCarriedForwardAsByKey() { @@ -2987,7 +3140,7 @@ public void testStatsChainingWithTimestampCarriedForwardAsByKey() { | STATS reqs = COUNT(), max_dur = MAX(event_duration) BY day = TBUCKET(1d), client_ip, @timestamp | WHERE max_dur > 1000 | STATS spikes = COUNT() BY hour = TBUCKET(1h), client_ip, day - """, Set.of("@timestamp", "@timestamp.*", "event_duration", "event_duration.*", "client_ip", "client_ip.*")); + """, Set.of("_index", "@timestamp", "@timestamp.*", "event_duration", "event_duration.*", "client_ip", "client_ip.*")); } private void assertFieldNames(String query, Set expected) { diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml index e38b661b2baa8..c09e851cb821d 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml @@ -661,3 +661,223 @@ version is not allowed: - length: {values: 2} - match: {values.0: ["1981-01-12T15:00:00.000Z","2000-01-01T00:00:00.000Z"]} - match: {values.1: ["1999-12-31T20:59:00.000Z", null]} + + +--- +"No columns left after drop": + - requires: + test_runner_features: [capabilities, allowed_warnings_regex] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [fix_no_columns] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test_range | keep date_1 | drop date_1 | limit 2' + + - length: {columns: 0} + - length: {values: 2} + - length: {values.0: 0 } + - length: {values.1: 0 } + + +--- +"Drop all columns after stats": + - requires: + test_runner_features: [capabilities, allowed_warnings_regex] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [fix_no_columns] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | stats by color | keep color | drop color' + + - length: {columns: 0} + - length: {values: 3} + - length: {values.0: 0 } + - length: {values.1: 0 } + - length: {values.2: 0 } + +--- +"Drop all columns after eval where false": + - requires: + test_runner_features: [capabilities, allowed_warnings_regex] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [fix_no_columns] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | eval c = 1 | drop c, color | where false' + + - length: {columns: 0} + - length: {values: 0} + +--- +"Drop all columns with inline stats ": + - requires: + test_runner_features: [capabilities, allowed_warnings_regex] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [fix_no_columns] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | drop color | eval x = 1 | inline stats a = count(), b = max(x) | limit 3' + + - length: {columns: 3} + - match: { columns.0.name: "x" } + - match: { columns.1.name: "a" } + - match: { columns.2.name: "b" } + - length: {values: 3} + - match: {values.0: [1, 40, 1]} + - match: {values.1: [1, 40, 1]} + - match: {values.2: [1, 40, 1]} + +--- +"Drop all columns with inline stats and by": + - requires: + test_runner_features: [capabilities, allowed_warnings_regex] + capabilities: + - method: POST + path: /_query + parameters: [] + capabilities: [fix_no_columns] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | drop color | eval x = 1, y = 1 | inline stats a = count(), b = max(x) by y | limit 3' + + - length: {columns: 4} + - match: { columns.0.name: "x" } + - match: { columns.1.name: "a" } + - match: { columns.2.name: "b" } + - match: { columns.3.name: "y" } + - length: {values: 3} + - match: {values.0: [1, 40, 1, 1]} + - match: {values.1: [1, 40, 1, 1]} + - match: {values.2: [1, 40, 1, 1]} + +--- +"Drop all columns with mutiple indices": + - requires: + test_runner_features: [ capabilities, allowed_warnings_regex ] + capabilities: + - method: POST + path: /_query + parameters: [ ] + capabilities: [ fix_no_columns ] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test* | keep color | drop color | eval x = 1, y = 1 | inline stats a = count(), b = max(x) by y | limit 3' + - length: { columns: 4 } + - match: { columns.0.name: "x" } + - match: { columns.1.name: "a" } + - match: { columns.2.name: "b" } + - match: { columns.3.name: "y" } + - length: { values: 3 } + - match: {values.0: [1, 43, 1, 1]} + - match: {values.1: [1, 43, 1, 1]} + - match: {values.2: [1, 43, 1, 1]} + +--- +"Drop all columns with fork": + - requires: + test_runner_features: [ capabilities, allowed_warnings_regex ] + capabilities: + - method: POST + path: /_query + parameters: [ ] + capabilities: [ fix_no_columns ] + reason: "plans witn no columns only recently supported" + + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | drop color | fork (stats c=count()) (stats c=count())' + + - length: { columns: 2 } + - match: { columns.0.name: "c" } + - match: { columns.1.name: "_fork" } + - length: { values: 2 } + - match: {values.0.0: 40} + - match: {values.1.0: 40} + + +--- +"Drop all columns with fork drop all": + - requires: + test_runner_features: [ capabilities, allowed_warnings_regex ] + capabilities: + - method: POST + path: /_query + parameters: [ ] + capabilities: [ fix_no_columns ] + reason: "plans witn no columns only recently supported" + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | drop color | fork (stats c=count()) (stats c=count()) | drop _fork*, c' + + - length: { columns: 0 } + - length: { values: 2 } + + +--- +"Drop all columns with fork drop all count": + - requires: + test_runner_features: [ capabilities, allowed_warnings_regex ] + capabilities: + - method: POST + path: /_query + parameters: [ ] + capabilities: [ fix_no_columns ] + reason: "plans witn no columns only recently supported" + - do: + allowed_warnings_regex: + - "No limit defined, adding default limit of \\[.*\\]" + esql.query: + body: + query: 'from test | keep color | drop color | fork (stats c=count()) (stats c=count()) | drop _fork*, c | stats count(*)' + + - length: { columns: 1 } + - length: { values: 1 } + - match: {values.0.0: 2}