diff --git a/.gitignore b/.gitignore index 5d0c5b2a4af57..03e2446ec4f13 100644 --- a/.gitignore +++ b/.gitignore @@ -68,3 +68,4 @@ testfixtures_shared/ # Generated checkstyle_ide.xml +x-pack/plugin/esql/gen/ diff --git a/benchmarks/build.gradle b/benchmarks/build.gradle index cd20a68ae5c1b..7cfa23e69ff96 100644 --- a/benchmarks/build.gradle +++ b/benchmarks/build.gradle @@ -35,6 +35,9 @@ dependencies { exclude group: 'net.sf.jopt-simple', module: 'jopt-simple' } api(project(':modules:aggregations')) + api(project(':x-pack:plugin:ql')) + api(project(':x-pack:plugin:esql')) + api(project(':x-pack:plugin:esql:compute')) expression(project(path: ':modules:lang-expression', configuration: 'zip')) painless(project(path: ':modules:lang-painless', configuration: 'zip')) api "org.openjdk.jmh:jmh-core:$versions.jmh" diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/AggregatorBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/AggregatorBenchmark.java new file mode 100644 index 0000000000000..aa16523e38097 --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/AggregatorBenchmark.java @@ -0,0 +1,579 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.benchmark.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.aggregation.CountAggregatorFunction; +import org.elasticsearch.compute.aggregation.CountDistinctDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MinDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MinLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AggregationOperator; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.Operator; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +@Warmup(iterations = 5) +@Measurement(iterations = 7) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Thread) +@Fork(1) +public class AggregatorBenchmark { + static final int BLOCK_LENGTH = 8 * 1024; + private static final int OP_COUNT = 1024; + private static final int GROUPS = 5; + + private static final BigArrays BIG_ARRAYS = BigArrays.NON_RECYCLING_INSTANCE; // TODO real big arrays? + + private static final String LONGS = "longs"; + private static final String INTS = "ints"; + private static final String DOUBLES = "doubles"; + private static final String BOOLEANS = "booleans"; + private static final String BYTES_REFS = "bytes_refs"; + private static final String TWO_LONGS = "two_" + LONGS; + private static final String LONGS_AND_BYTES_REFS = LONGS + "_and_" + BYTES_REFS; + private static final String TWO_LONGS_AND_BYTES_REFS = "two_" + LONGS + "_and_" + BYTES_REFS; + + private static final String VECTOR_DOUBLES = "vector_doubles"; + private static final String HALF_NULL_DOUBLES = "half_null_doubles"; + private static final String VECTOR_LONGS = "vector_" + LONGS; + private static final String HALF_NULL_LONGS = "half_null_" + LONGS; + private static final String MULTIVALUED_LONGS = "multivalued"; + + private static final String AVG = "avg"; + private static final String COUNT = "count"; + private static final String COUNT_DISTINCT = "count_distinct"; + private static final String MIN = "min"; + private static final String MAX = "max"; + private static final String SUM = "sum"; + + private static final String NONE = "none"; + + static { + // Smoke test all the expected values and force loading subclasses more like prod + try { + for (String grouping : AggregatorBenchmark.class.getField("grouping").getAnnotationsByType(Param.class)[0].value()) { + for (String op : AggregatorBenchmark.class.getField("op").getAnnotationsByType(Param.class)[0].value()) { + for (String blockType : AggregatorBenchmark.class.getField("blockType").getAnnotationsByType(Param.class)[0].value()) { + run(grouping, op, blockType, 50); + } + } + } + } catch (NoSuchFieldException e) { + throw new AssertionError(); + } + } + + @Param({ NONE, LONGS, INTS, DOUBLES, BOOLEANS, BYTES_REFS, TWO_LONGS, LONGS_AND_BYTES_REFS, TWO_LONGS_AND_BYTES_REFS }) + public String grouping; + + @Param({ COUNT, COUNT_DISTINCT, MIN, MAX, SUM }) + public String op; + + @Param({ VECTOR_LONGS, HALF_NULL_LONGS, VECTOR_DOUBLES, HALF_NULL_DOUBLES }) + public String blockType; + + private static Operator operator(String grouping, String op, String dataType) { + if (grouping.equals("none")) { + return new AggregationOperator(List.of(supplier(op, dataType, 0).aggregatorFactory(AggregatorMode.SINGLE).get())); + } + List groups = switch (grouping) { + case LONGS -> List.of(new HashAggregationOperator.GroupSpec(0, ElementType.LONG)); + case INTS -> List.of(new HashAggregationOperator.GroupSpec(0, ElementType.INT)); + case DOUBLES -> List.of(new HashAggregationOperator.GroupSpec(0, ElementType.DOUBLE)); + case BOOLEANS -> List.of(new HashAggregationOperator.GroupSpec(0, ElementType.BOOLEAN)); + case BYTES_REFS -> List.of(new HashAggregationOperator.GroupSpec(0, ElementType.BYTES_REF)); + case TWO_LONGS -> List.of( + new HashAggregationOperator.GroupSpec(0, ElementType.LONG), + new HashAggregationOperator.GroupSpec(1, ElementType.LONG) + ); + case LONGS_AND_BYTES_REFS -> List.of( + new HashAggregationOperator.GroupSpec(0, ElementType.LONG), + new HashAggregationOperator.GroupSpec(1, ElementType.BYTES_REF) + ); + case TWO_LONGS_AND_BYTES_REFS -> List.of( + new HashAggregationOperator.GroupSpec(0, ElementType.LONG), + new HashAggregationOperator.GroupSpec(1, ElementType.LONG), + new HashAggregationOperator.GroupSpec(2, ElementType.BYTES_REF) + ); + default -> throw new IllegalArgumentException("unsupported grouping [" + grouping + "]"); + }; + return new HashAggregationOperator( + List.of(supplier(op, dataType, groups.size()).groupingAggregatorFactory(AggregatorMode.SINGLE)), + () -> BlockHash.build(groups, BIG_ARRAYS, 16 * 1024), + new DriverContext() + ); + } + + private static AggregatorFunctionSupplier supplier(String op, String dataType, int dataChannel) { + return switch (op) { + case COUNT -> CountAggregatorFunction.supplier(BIG_ARRAYS, List.of(dataChannel)); + case COUNT_DISTINCT -> switch (dataType) { + case LONGS -> new CountDistinctLongAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel), 3000); + case DOUBLES -> new CountDistinctDoubleAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel), 3000); + default -> throw new IllegalArgumentException("unsupported data type [" + dataType + "]"); + }; + case MAX -> switch (dataType) { + case LONGS -> new MaxLongAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + case DOUBLES -> new MaxDoubleAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + default -> throw new IllegalArgumentException("unsupported data type [" + dataType + "]"); + }; + case MIN -> switch (dataType) { + case LONGS -> new MinLongAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + case DOUBLES -> new MinDoubleAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + default -> throw new IllegalArgumentException("unsupported data type [" + dataType + "]"); + }; + case SUM -> switch (dataType) { + case LONGS -> new SumLongAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + case DOUBLES -> new SumDoubleAggregatorFunctionSupplier(BIG_ARRAYS, List.of(dataChannel)); + default -> throw new IllegalArgumentException("unsupported data type [" + dataType + "]"); + }; + default -> throw new IllegalArgumentException("unsupported op [" + op + "]"); + }; + } + + private static void checkExpected(String grouping, String op, String blockType, String dataType, Page page, int opCount) { + String prefix = String.format("[%s][%s][%s] ", grouping, op, blockType); + if (grouping.equals("none")) { + checkUngrouped(prefix, op, dataType, page, opCount); + return; + } + checkGrouped(prefix, grouping, op, dataType, page, opCount); + } + + private static void checkGrouped(String prefix, String grouping, String op, String dataType, Page page, int opCount) { + switch (grouping) { + case TWO_LONGS -> { + checkGroupingBlock(prefix, LONGS, page.getBlock(0)); + checkGroupingBlock(prefix, LONGS, page.getBlock(1)); + } + case LONGS_AND_BYTES_REFS -> { + checkGroupingBlock(prefix, LONGS, page.getBlock(0)); + checkGroupingBlock(prefix, BYTES_REFS, page.getBlock(1)); + } + case TWO_LONGS_AND_BYTES_REFS -> { + checkGroupingBlock(prefix, LONGS, page.getBlock(0)); + checkGroupingBlock(prefix, LONGS, page.getBlock(1)); + checkGroupingBlock(prefix, BYTES_REFS, page.getBlock(2)); + } + default -> checkGroupingBlock(prefix, grouping, page.getBlock(0)); + } + Block values = page.getBlock(page.getBlockCount() - 1); + int groups = switch (grouping) { + case BOOLEANS -> 2; + default -> GROUPS; + }; + switch (op) { + case AVG -> { + DoubleBlock dValues = (DoubleBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long sum = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).sum(); + long count = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).count(); + double expected = (double) sum / count; + if (dValues.getDouble(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + dValues.getDouble(g) + "]"); + } + } + } + case COUNT -> { + LongBlock lValues = (LongBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).count() * opCount; + if (lValues.getLong(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + lValues.getLong(g) + "]"); + } + } + } + case COUNT_DISTINCT -> { + LongBlock lValues = (LongBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).distinct().count(); + long count = lValues.getLong(g); + // count should be within 10% from the expected value + if (count < expected * 0.9 || count > expected * 1.1) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + count + "]"); + } + } + } + case MIN -> { + switch (dataType) { + case LONGS -> { + LongBlock lValues = (LongBlock) values; + for (int g = 0; g < groups; g++) { + if (lValues.getLong(g) != (long) g) { + throw new AssertionError(prefix + "expected [" + g + "] but was [" + lValues.getLong(g) + "]"); + } + } + } + case DOUBLES -> { + DoubleBlock dValues = (DoubleBlock) values; + for (int g = 0; g < groups; g++) { + if (dValues.getDouble(g) != (long) g) { + throw new AssertionError(prefix + "expected [" + g + "] but was [" + dValues.getDouble(g) + "]"); + } + } + } + default -> throw new IllegalArgumentException("bad data type " + dataType); + } + } + case MAX -> { + switch (dataType) { + case LONGS -> { + LongBlock lValues = (LongBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).max().getAsLong(); + if (lValues.getLong(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + lValues.getLong(g) + "]"); + } + } + } + case DOUBLES -> { + DoubleBlock dValues = (DoubleBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).max().getAsLong(); + if (dValues.getDouble(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + dValues.getDouble(g) + "]"); + } + } + } + default -> throw new IllegalArgumentException("bad data type " + dataType); + } + } + case SUM -> { + switch (dataType) { + case LONGS -> { + LongBlock lValues = (LongBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).sum() * opCount; + if (lValues.getLong(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + lValues.getLong(g) + "]"); + } + } + } + case DOUBLES -> { + DoubleBlock dValues = (DoubleBlock) values; + for (int g = 0; g < groups; g++) { + long group = g; + long expected = LongStream.range(0, BLOCK_LENGTH).filter(l -> l % groups == group).sum() * opCount; + if (dValues.getDouble(g) != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + dValues.getDouble(g) + "]"); + } + } + } + default -> throw new IllegalArgumentException("bad data type " + dataType); + } + } + default -> throw new IllegalArgumentException("bad op " + op); + } + } + + private static void checkGroupingBlock(String prefix, String grouping, Block block) { + switch (grouping) { + case LONGS -> { + LongBlock groups = (LongBlock) block; + for (int g = 0; g < GROUPS; g++) { + if (groups.getLong(g) != (long) g) { + throw new AssertionError(prefix + "bad group expected [" + g + "] but was [" + groups.getLong(g) + "]"); + } + } + } + case INTS -> { + IntBlock groups = (IntBlock) block; + for (int g = 0; g < GROUPS; g++) { + if (groups.getInt(g) != g) { + throw new AssertionError(prefix + "bad group expected [" + g + "] but was [" + groups.getInt(g) + "]"); + } + } + } + case DOUBLES -> { + DoubleBlock groups = (DoubleBlock) block; + for (int g = 0; g < GROUPS; g++) { + if (groups.getDouble(g) != (double) g) { + throw new AssertionError(prefix + "bad group expected [" + (double) g + "] but was [" + groups.getDouble(g) + "]"); + } + } + } + case BOOLEANS -> { + BooleanBlock groups = (BooleanBlock) block; + if (groups.getBoolean(0) != false) { + throw new AssertionError(prefix + "bad group expected [false] but was [" + groups.getBoolean(0) + "]"); + } + if (groups.getBoolean(1) != true) { + throw new AssertionError(prefix + "bad group expected [true] but was [" + groups.getBoolean(1) + "]"); + } + } + case BYTES_REFS -> { + BytesRefBlock groups = (BytesRefBlock) block; + for (int g = 0; g < GROUPS; g++) { + if (false == groups.getBytesRef(g, new BytesRef()).equals(bytesGroup(g))) { + throw new AssertionError( + prefix + "bad group expected [" + bytesGroup(g) + "] but was [" + groups.getBytesRef(g, new BytesRef()) + "]" + ); + } + } + } + default -> throw new IllegalArgumentException("bad grouping [" + grouping + "]"); + } + } + + private static void checkUngrouped(String prefix, String op, String dataType, Page page, int opCount) { + Block block = page.getBlock(0); + switch (op) { + case AVG -> { + DoubleBlock dBlock = (DoubleBlock) block; + if (dBlock.getDouble(0) != (BLOCK_LENGTH - 1) / 2.0) { + throw new AssertionError( + prefix + "expected [" + ((BLOCK_LENGTH - 1) / 2.0) + "] but was [" + dBlock.getDouble(0) + "]" + ); + } + } + case COUNT -> { + LongBlock lBlock = (LongBlock) block; + if (lBlock.getLong(0) != (long) BLOCK_LENGTH * opCount) { + throw new AssertionError(prefix + "expected [" + (BLOCK_LENGTH * opCount) + "] but was [" + lBlock.getLong(0) + "]"); + } + } + case COUNT_DISTINCT -> { + LongBlock lBlock = (LongBlock) block; + long expected = BLOCK_LENGTH; + long count = lBlock.getLong(0); + // count should be within 10% from the expected value + if (count < expected * 0.9 || count > expected * 1.1) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + count + "]"); + } + } + case MIN -> { + long expected = 0L; + var val = switch (dataType) { + case LONGS -> ((LongBlock) block).getLong(0); + case DOUBLES -> ((DoubleBlock) block).getDouble(0); + default -> throw new IllegalStateException("Unexpected aggregation type: " + dataType); + }; + if (val != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + val + "]"); + } + } + case MAX -> { + long expected = BLOCK_LENGTH - 1; + var val = switch (dataType) { + case LONGS -> ((LongBlock) block).getLong(0); + case DOUBLES -> ((DoubleBlock) block).getDouble(0); + default -> throw new IllegalStateException("Unexpected aggregation type: " + dataType); + }; + if (val != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + val + "]"); + } + } + case SUM -> { + long expected = (BLOCK_LENGTH * (BLOCK_LENGTH - 1L)) * ((long) opCount) / 2; + var val = switch (dataType) { + case LONGS -> ((LongBlock) block).getLong(0); + case DOUBLES -> ((DoubleBlock) block).getDouble(0); + default -> throw new IllegalStateException("Unexpected aggregation type: " + dataType); + }; + if (val != expected) { + throw new AssertionError(prefix + "expected [" + expected + "] but was [" + val + "]"); + } + } + default -> throw new IllegalArgumentException("bad op " + op); + } + } + + private static Page page(String grouping, String blockType) { + Block dataBlock = dataBlock(blockType); + if (grouping.equals("none")) { + return new Page(dataBlock); + } + List blocks = groupingBlocks(grouping, blockType); + return new Page(Stream.concat(blocks.stream(), Stream.of(dataBlock)).toArray(Block[]::new)); + } + + private static Block dataBlock(String blockType) { + return switch (blockType) { + case VECTOR_LONGS -> new LongArrayVector(LongStream.range(0, BLOCK_LENGTH).toArray(), BLOCK_LENGTH).asBlock(); + case VECTOR_DOUBLES -> new DoubleArrayVector( + LongStream.range(0, BLOCK_LENGTH).mapToDouble(l -> Long.valueOf(l).doubleValue()).toArray(), + BLOCK_LENGTH + ).asBlock(); + case MULTIVALUED_LONGS -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + builder.beginPositionEntry(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendLong(i); + if (i % 5 == 0) { + builder.endPositionEntry(); + builder.beginPositionEntry(); + } + } + builder.endPositionEntry(); + yield builder.build(); + } + case HALF_NULL_LONGS -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendLong(i); + builder.appendNull(); + } + yield builder.build(); + } + case HALF_NULL_DOUBLES -> { + var builder = DoubleBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendDouble(i); + builder.appendNull(); + } + yield builder.build(); + } + default -> throw new IllegalArgumentException("bad blockType: " + blockType); + }; + } + + private static List groupingBlocks(String grouping, String blockType) { + return switch (grouping) { + case TWO_LONGS -> List.of(groupingBlock(LONGS, blockType), groupingBlock(LONGS, blockType)); + case LONGS_AND_BYTES_REFS -> List.of(groupingBlock(LONGS, blockType), groupingBlock(BYTES_REFS, blockType)); + case TWO_LONGS_AND_BYTES_REFS -> List.of( + groupingBlock(LONGS, blockType), + groupingBlock(LONGS, blockType), + groupingBlock(BYTES_REFS, blockType) + ); + default -> List.of(groupingBlock(grouping, blockType)); + }; + } + + private static Block groupingBlock(String grouping, String blockType) { + int valuesPerGroup = switch (blockType) { + case VECTOR_LONGS, VECTOR_DOUBLES -> 1; + case HALF_NULL_LONGS, HALF_NULL_DOUBLES -> 2; + default -> throw new UnsupportedOperationException("bad grouping [" + grouping + "]"); + }; + return switch (grouping) { + case LONGS -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + for (int v = 0; v < valuesPerGroup; v++) { + builder.appendLong(i % GROUPS); + } + } + yield builder.build(); + } + case INTS -> { + var builder = IntBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + for (int v = 0; v < valuesPerGroup; v++) { + builder.appendInt(i % GROUPS); + } + } + yield builder.build(); + } + case DOUBLES -> { + var builder = DoubleBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + for (int v = 0; v < valuesPerGroup; v++) { + builder.appendDouble(i % GROUPS); + } + } + yield builder.build(); + } + case BOOLEANS -> { + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + for (int v = 0; v < valuesPerGroup; v++) { + builder.appendBoolean(i % 2 == 1); + } + } + yield builder.build(); + } + case BYTES_REFS -> { + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + for (int v = 0; v < valuesPerGroup; v++) { + builder.appendBytesRef(bytesGroup(i % GROUPS)); + } + } + yield builder.build(); + } + default -> throw new UnsupportedOperationException("unsupported grouping [" + grouping + "]"); + }; + } + + private static BytesRef bytesGroup(int group) { + return new BytesRef(switch (group) { + case 0 -> "cat"; + case 1 -> "dog"; + case 2 -> "chicken"; + case 3 -> "pig"; + case 4 -> "cow"; + default -> throw new UnsupportedOperationException("can't handle [" + group + "]"); + }); + } + + @Benchmark + @OperationsPerInvocation(OP_COUNT * BLOCK_LENGTH) + public void run() { + run(grouping, op, blockType, OP_COUNT); + } + + private static void run(String grouping, String op, String blockType, int opCount) { + String dataType = switch (blockType) { + case VECTOR_LONGS, HALF_NULL_LONGS -> LONGS; + case VECTOR_DOUBLES, HALF_NULL_DOUBLES -> DOUBLES; + default -> throw new IllegalArgumentException(); + }; + + Operator operator = operator(grouping, op, dataType); + Page page = page(grouping, blockType); + for (int i = 0; i < opCount; i++) { + operator.addInput(page); + } + operator.finish(); + checkExpected(grouping, op, blockType, dataType, operator.getOutput(), opCount); + } +} diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/EvalBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/EvalBenchmark.java new file mode 100644 index 0000000000000..e8b79bff00975 --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/EvalBenchmark.java @@ -0,0 +1,262 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.benchmark.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Abs; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMin; +import org.elasticsearch.xpack.esql.planner.EvalMapper; +import org.elasticsearch.xpack.esql.planner.Layout; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +@Warmup(iterations = 5) +@Measurement(iterations = 7) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Thread) +@Fork(1) +public class EvalBenchmark { + private static final int BLOCK_LENGTH = 8 * 1024; + + static { + // Smoke test all the expected values and force loading subclasses more like prod + try { + for (String operation : EvalBenchmark.class.getField("operation").getAnnotationsByType(Param.class)[0].value()) { + run(operation); + } + } catch (NoSuchFieldException e) { + throw new AssertionError(); + } + } + + @Param({ "abs", "add", "date_trunc", "equal_to_const", "long_equal_to_long", "long_equal_to_int", "mv_min", "mv_min_ascending" }) + public String operation; + + private static Operator operator(String operation) { + return new EvalOperator(evaluator(operation)); + } + + private static EvalOperator.ExpressionEvaluator evaluator(String operation) { + return switch (operation) { + case "abs" -> { + FieldAttribute longField = longField(); + yield EvalMapper.toEvaluator(new Abs(Source.EMPTY, longField), layout(longField)).get(); + } + case "add" -> { + FieldAttribute longField = longField(); + yield EvalMapper.toEvaluator( + new Add(Source.EMPTY, longField, new Literal(Source.EMPTY, 1L, DataTypes.LONG)), + layout(longField) + ).get(); + } + case "date_trunc" -> { + FieldAttribute timestamp = new FieldAttribute( + Source.EMPTY, + "timestamp", + new EsField("timestamp", DataTypes.DATETIME, Map.of(), true) + ); + yield EvalMapper.toEvaluator( + new DateTrunc(Source.EMPTY, timestamp, new Literal(Source.EMPTY, Duration.ofHours(24), EsqlDataTypes.TIME_DURATION)), + layout(timestamp) + ).get(); + } + case "equal_to_const" -> { + FieldAttribute longField = longField(); + yield EvalMapper.toEvaluator( + new Equals(Source.EMPTY, longField, new Literal(Source.EMPTY, 100_000L, DataTypes.LONG)), + layout(longField) + ).get(); + } + case "long_equal_to_long" -> { + FieldAttribute lhs = longField(); + FieldAttribute rhs = longField(); + yield EvalMapper.toEvaluator(new Equals(Source.EMPTY, lhs, rhs), layout(lhs, rhs)).get(); + } + case "long_equal_to_int" -> { + FieldAttribute lhs = longField(); + FieldAttribute rhs = intField(); + yield EvalMapper.toEvaluator(new Equals(Source.EMPTY, lhs, rhs), layout(lhs, rhs)).get(); + } + case "mv_min", "mv_min_ascending" -> { + FieldAttribute longField = longField(); + yield EvalMapper.toEvaluator(new MvMin(Source.EMPTY, longField), layout(longField)).get(); + } + default -> throw new UnsupportedOperationException(); + }; + } + + private static FieldAttribute longField() { + return new FieldAttribute(Source.EMPTY, "long", new EsField("long", DataTypes.LONG, Map.of(), true)); + } + + private static FieldAttribute intField() { + return new FieldAttribute(Source.EMPTY, "int", new EsField("int", DataTypes.INTEGER, Map.of(), true)); + } + + private static Layout layout(FieldAttribute... fields) { + Layout.Builder layout = new Layout.Builder(); + for (FieldAttribute field : fields) { + layout.appendChannel(field.id()); + } + return layout.build(); + } + + private static void checkExpected(String operation, Page actual) { + switch (operation) { + case "abs" -> { + LongVector v = actual.getBlock(1).asVector(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + if (v.getLong(i) != i * 100_000) { + throw new AssertionError("[" + operation + "] expected [" + (i * 100_000) + "] but was [" + v.getLong(i) + "]"); + } + } + } + case "add" -> { + LongVector v = actual.getBlock(1).asVector(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + if (v.getLong(i) != i * 100_000 + 1) { + throw new AssertionError("[" + operation + "] expected [" + (i * 100_000 + 1) + "] but was [" + v.getLong(i) + "]"); + } + } + } + case "date_trunc" -> { + LongVector v = actual.getBlock(1).asVector(); + long oneDay = TimeValue.timeValueHours(24).millis(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + long expected = i * 100_000; + expected -= expected % oneDay; + if (v.getLong(i) != expected) { + throw new AssertionError("[" + operation + "] expected [" + expected + "] but was [" + v.getLong(i) + "]"); + } + } + } + case "equal_to_const" -> { + BooleanVector v = actual.getBlock(1).asVector(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + if (v.getBoolean(i) != (i == 1)) { + throw new AssertionError("[" + operation + "] expected [" + (i == 1) + "] but was [" + v.getBoolean(i) + "]"); + } + } + } + case "long_equal_to_long", "long_equal_to_int" -> { + BooleanVector v = actual.getBlock(2).asVector(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + if (v.getBoolean(i) != true) { + throw new AssertionError("[" + operation + "] expected [" + (i == 1) + "] but was [" + v.getBoolean(i) + "]"); + } + } + } + case "mv_min", "mv_min_ascending" -> { + LongVector v = actual.getBlock(1).asVector(); + for (int i = 0; i < BLOCK_LENGTH; i++) { + if (v.getLong(i) != i) { + throw new AssertionError("[" + operation + "] expected [" + i + "] but was [" + v.getLong(i) + "]"); + } + } + } + default -> throw new UnsupportedOperationException(); + } + } + + private static Page page(String operation) { + return switch (operation) { + case "abs", "add", "date_trunc", "equal_to_const" -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendLong(i * 100_000); + } + yield new Page(builder.build()); + } + case "long_equal_to_long" -> { + var lhs = LongBlock.newBlockBuilder(BLOCK_LENGTH); + var rhs = LongBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + lhs.appendLong(i * 100_000); + rhs.appendLong(i * 100_000); + } + yield new Page(lhs.build(), rhs.build()); + } + case "long_equal_to_int" -> { + var lhs = LongBlock.newBlockBuilder(BLOCK_LENGTH); + var rhs = IntBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + lhs.appendLong(i * 100_000); + rhs.appendInt(i * 100_000); + } + yield new Page(lhs.build(), rhs.build()); + } + case "mv_min", "mv_min_ascending" -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + if (operation.endsWith("ascending")) { + builder.mvOrdering(Block.MvOrdering.ASCENDING); + } + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.beginPositionEntry(); + builder.appendLong(i); + builder.appendLong(i + 1); + builder.appendLong(i + 2); + builder.endPositionEntry(); + } + yield new Page(builder.build()); + } + default -> throw new UnsupportedOperationException(); + }; + } + + @Benchmark + @OperationsPerInvocation(1024 * BLOCK_LENGTH) + public void run() { + run(operation); + } + + private static void run(String operation) { + try (Operator operator = operator(operation)) { + Page page = page(operation); + Page output = null; + for (int i = 0; i < 1024; i++) { + operator.addInput(page); + output = operator.getOutput(); + } + // We only check the last one + checkExpected(operation, output); + } + } +} diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java new file mode 100644 index 0000000000000..7e13d1ec0354c --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/MultivalueDedupeBenchmark.java @@ -0,0 +1,175 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.benchmark.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +@Warmup(iterations = 3) +@Measurement(iterations = 3) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Thread) +@Fork(1) +public class MultivalueDedupeBenchmark { + @Param({ "BOOLEAN", "BYTES_REF", "DOUBLE", "INT", "LONG" }) + private ElementType elementType; + + @Param({ "3", "5", "10", "50", "100", "1000" }) + private int size; + + @Param({ "0", "2", "10", "100", "1000" }) + private int repeats; + + private Block block; + + @Setup + public void setup() { + this.block = switch (elementType) { + case BOOLEAN -> { + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(AggregatorBenchmark.BLOCK_LENGTH * (size + repeats)); + for (int p = 0; p < AggregatorBenchmark.BLOCK_LENGTH; p++) { + List values = new ArrayList<>(); + for (int i = 0; i < size; i++) { + values.add(i % 2 == 0); + } + for (int r = 0; r < repeats; r++) { + values.add(r < size ? r % 2 == 0 : false); + } + Randomness.shuffle(values); + builder.beginPositionEntry(); + for (Boolean v : values) { + builder.appendBoolean(v); + } + builder.endPositionEntry(); + } + yield builder.build(); + } + case BYTES_REF -> { + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(AggregatorBenchmark.BLOCK_LENGTH * (size + repeats)); + for (int p = 0; p < AggregatorBenchmark.BLOCK_LENGTH; p++) { + List values = new ArrayList<>(); + for (int i = 0; i < size; i++) { + values.add(new BytesRef("SAFADFASDFSADFDAFS" + i)); + } + for (int r = 0; r < repeats; r++) { + values.add(new BytesRef("SAFADFASDFSADFDAFS" + ((r < size ? r : 0)))); + } + Randomness.shuffle(values); + builder.beginPositionEntry(); + for (BytesRef v : values) { + builder.appendBytesRef(v); + } + builder.endPositionEntry(); + } + yield builder.build(); + } + case DOUBLE -> { + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(AggregatorBenchmark.BLOCK_LENGTH * (size + repeats)); + for (int p = 0; p < AggregatorBenchmark.BLOCK_LENGTH; p++) { + List values = new ArrayList<>(); + for (int i = 0; i < size; i++) { + values.add((double) i); + } + for (int r = 0; r < repeats; r++) { + values.add(r < size ? (double) r : 0.0); + } + Randomness.shuffle(values); + builder.beginPositionEntry(); + for (Double v : values) { + builder.appendDouble(v); + } + builder.endPositionEntry(); + } + yield builder.build(); + } + case INT -> { + IntBlock.Builder builder = IntBlock.newBlockBuilder(AggregatorBenchmark.BLOCK_LENGTH * (size + repeats)); + for (int p = 0; p < AggregatorBenchmark.BLOCK_LENGTH; p++) { + List values = new ArrayList<>(); + for (int i = 0; i < size; i++) { + values.add(i); + } + for (int r = 0; r < repeats; r++) { + values.add(r < size ? r : 0); + } + Randomness.shuffle(values); + builder.beginPositionEntry(); + for (Integer v : values) { + builder.appendInt(v); + } + builder.endPositionEntry(); + } + yield builder.build(); + } + case LONG -> { + LongBlock.Builder builder = LongBlock.newBlockBuilder(AggregatorBenchmark.BLOCK_LENGTH * (size + repeats)); + for (int p = 0; p < AggregatorBenchmark.BLOCK_LENGTH; p++) { + List values = new ArrayList<>(); + for (long i = 0; i < size; i++) { + values.add(i); + } + for (int r = 0; r < repeats; r++) { + values.add(r < size ? r : 0L); + } + Randomness.shuffle(values); + builder.beginPositionEntry(); + for (Long v : values) { + builder.appendLong(v); + } + builder.endPositionEntry(); + } + yield builder.build(); + } + default -> throw new IllegalArgumentException(); + }; + } + + @Benchmark + @OperationsPerInvocation(AggregatorBenchmark.BLOCK_LENGTH) + public Block adaptive() { + return MultivalueDedupe.dedupeToBlockAdaptive(block); + } + + @Benchmark + @OperationsPerInvocation(AggregatorBenchmark.BLOCK_LENGTH) + public Block copyAndSort() { + return MultivalueDedupe.dedupeToBlockUsingCopyAndSort(block); + } + + @Benchmark + @OperationsPerInvocation(AggregatorBenchmark.BLOCK_LENGTH) + public Block copyMissing() { + return MultivalueDedupe.dedupeToBlockUsingCopyMissing(block); + } +} diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java new file mode 100644 index 0000000000000..c53d08b063ba9 --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java @@ -0,0 +1,163 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.benchmark.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +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.operator.Operator; +import org.elasticsearch.compute.operator.TopNOperator; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; + +@Warmup(iterations = 5) +@Measurement(iterations = 7) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Thread) +@Fork(1) +public class TopNBenchmark { + private static final int BLOCK_LENGTH = 8 * 1024; + + private static final String LONGS = "longs"; + private static final String INTS = "ints"; + private static final String DOUBLES = "doubles"; + private static final String BOOLEANS = "booleans"; + private static final String BYTES_REFS = "bytes_refs"; + private static final String TWO_LONGS = "two_" + LONGS; + private static final String LONGS_AND_BYTES_REFS = LONGS + "_and_" + BYTES_REFS; + + static { + // Smoke test all the expected values and force loading subclasses more like prod + try { + for (String data : TopNBenchmark.class.getField("data").getAnnotationsByType(Param.class)[0].value()) { + for (String topCount : TopNBenchmark.class.getField("topCount").getAnnotationsByType(Param.class)[0].value()) { + run(data, Integer.parseInt(topCount)); + } + } + } catch (NoSuchFieldException e) { + throw new AssertionError(); + } + } + + @Param({ LONGS, INTS, DOUBLES, BOOLEANS, BYTES_REFS, TWO_LONGS, LONGS_AND_BYTES_REFS }) + public String data; + + @Param({ "10", "10000" }) + public int topCount; + + private static Operator operator(String data, int topCount) { + int count = switch (data) { + case LONGS, INTS, DOUBLES, BOOLEANS, BYTES_REFS -> 1; + case TWO_LONGS, LONGS_AND_BYTES_REFS -> 2; + default -> throw new IllegalArgumentException("unsupported data type [" + data + "]"); + }; + return new TopNOperator( + topCount, + IntStream.range(0, count).mapToObj(c -> new TopNOperator.SortOrder(c, false, false)).toList(), + 16 * 1024 + ); + } + + private static void checkExpected(int topCount, List pages) { + if (topCount != pages.size()) { + throw new AssertionError("expected [" + topCount + "] but got [" + pages.size() + "]"); + } + } + + private static Page page(String data) { + return switch (data) { + case TWO_LONGS -> new Page(block(LONGS), block(LONGS)); + case LONGS_AND_BYTES_REFS -> new Page(block(LONGS), block(BYTES_REFS)); + default -> new Page(block(data)); + }; + } + + private static Block block(String data) { + return switch (data) { + case LONGS -> { + var builder = LongBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendLong(i); + } + yield builder.build(); + } + case INTS -> { + var builder = IntBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendInt(i); + } + yield builder.build(); + } + case DOUBLES -> { + var builder = DoubleBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendDouble(i); + } + yield builder.build(); + } + case BOOLEANS -> { + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendBoolean(i % 2 == 1); + } + yield builder.build(); + } + case BYTES_REFS -> { + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(BLOCK_LENGTH); + for (int i = 0; i < BLOCK_LENGTH; i++) { + builder.appendBytesRef(new BytesRef(Integer.toString(i))); + } + yield builder.build(); + } + default -> throw new UnsupportedOperationException("unsupported data [" + data + "]"); + }; + } + + @Benchmark + @OperationsPerInvocation(1024 * BLOCK_LENGTH) + public void run() { + run(data, topCount); + } + + private static void run(String data, int topCount) { + try (Operator operator = operator(data, topCount)) { + Page page = page(data); + for (int i = 0; i < 1024; i++) { + operator.addInput(page); + } + operator.finish(); + List results = new ArrayList<>(); + Page p; + while ((p = operator.getOutput()) != null) { + results.add(p); + } + checkExpected(topCount, results); + } + } +} diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java new file mode 100644 index 0000000000000..d94b10cd5c54d --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java @@ -0,0 +1,356 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.benchmark.compute.operator; + +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.store.ByteBuffersDirectory; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.NumericUtils; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.ValueSourceInfo; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.TopNOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.fielddata.FieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.fielddata.plain.SortedDoublesIndexFieldData; +import org.elasticsearch.index.fielddata.plain.SortedNumericIndexFieldData; +import org.elasticsearch.index.fielddata.plain.SortedSetOrdinalsIndexFieldData; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.script.field.KeywordDocValuesField; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.FieldContext; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.PrimitiveIterator; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; + +@Warmup(iterations = 5) +@Measurement(iterations = 7) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Thread) +@Fork(1) +public class ValuesSourceReaderBenchmark { + private static final int BLOCK_LENGTH = 16 * 1024; + private static final int INDEX_SIZE = 10 * BLOCK_LENGTH; + private static final int COMMIT_INTERVAL = 500; + + static { + // Smoke test all the expected values and force loading subclasses more like prod + try { + ValuesSourceReaderBenchmark benchmark = new ValuesSourceReaderBenchmark(); + benchmark.setupIndex(); + try { + for (String layout : ValuesSourceReaderBenchmark.class.getField("layout").getAnnotationsByType(Param.class)[0].value()) { + for (String name : ValuesSourceReaderBenchmark.class.getField("name").getAnnotationsByType(Param.class)[0].value()) { + benchmark.layout = layout; + benchmark.name = name; + benchmark.setupPages(); + benchmark.benchmark(); + } + } + } finally { + benchmark.teardownIndex(); + } + } catch (IOException | NoSuchFieldException e) { + throw new AssertionError(e); + } + } + + private static ValueSourceInfo info(IndexReader reader, String name) { + return switch (name) { + case "long" -> numericInfo(reader, name, IndexNumericFieldData.NumericType.LONG, ElementType.LONG); + case "int" -> numericInfo(reader, name, IndexNumericFieldData.NumericType.INT, ElementType.INT); + case "double" -> { + SortedDoublesIndexFieldData fd = new SortedDoublesIndexFieldData( + name, + IndexNumericFieldData.NumericType.DOUBLE, + CoreValuesSourceType.NUMERIC, + null + ); + FieldContext context = new FieldContext(name, fd, null); + yield new ValueSourceInfo( + CoreValuesSourceType.NUMERIC, + CoreValuesSourceType.NUMERIC.getField(context, null), + ElementType.DOUBLE, + reader + ); + } + case "keyword" -> { + SortedSetOrdinalsIndexFieldData fd = new SortedSetOrdinalsIndexFieldData( + new IndexFieldDataCache.None(), + "keyword", + CoreValuesSourceType.KEYWORD, + new NoneCircuitBreakerService(), + (dv, n) -> new KeywordDocValuesField(FieldData.toString(dv), n) + ); + FieldContext context = new FieldContext(name, fd, null); + yield new ValueSourceInfo( + CoreValuesSourceType.KEYWORD, + CoreValuesSourceType.KEYWORD.getField(context, null), + ElementType.BYTES_REF, + reader + ); + } + default -> throw new IllegalArgumentException("can't read [" + name + "]"); + }; + } + + private static ValueSourceInfo numericInfo( + IndexReader reader, + String name, + IndexNumericFieldData.NumericType numericType, + ElementType elementType + ) { + SortedNumericIndexFieldData fd = new SortedNumericIndexFieldData(name, numericType, CoreValuesSourceType.NUMERIC, null); + FieldContext context = new FieldContext(name, fd, null); + return new ValueSourceInfo(CoreValuesSourceType.NUMERIC, CoreValuesSourceType.NUMERIC.getField(context, null), elementType, reader); + } + + /** + * Layouts for the input blocks. + *
    + *
  • {@code in_order} is how {@link LuceneSourceOperator} produces them to read in + * the most efficient possible way. We
  • + *
  • {@code shuffled} is chunked the same size as {@link LuceneSourceOperator} but + * loads in a shuffled order, like a hypothetical {@link TopNOperator} that can + * output large blocks would output.
  • + *
  • {@code shuffled_singles} is shuffled in the same order as {@code shuffled} but + * each page has a single document rather than {@code BLOCK_SIZE} docs.
  • + *
+ */ + @Param({ "in_order", "shuffled", "shuffled_singles" }) + public String layout; + + @Param({ "long", "int", "double", "keyword" }) + public String name; + + private Directory directory; + private IndexReader reader; + private List pages; + + @Benchmark + @OperationsPerInvocation(INDEX_SIZE) + public void benchmark() { + ValuesSourceReaderOperator op = new ValuesSourceReaderOperator(List.of(info(reader, name)), 0, name); + long sum = 0; + for (Page page : pages) { + op.addInput(page); + switch (name) { + case "long" -> { + LongVector values = op.getOutput().getBlock(1).asVector(); + for (int p = 0; p < values.getPositionCount(); p++) { + sum += values.getLong(p); + } + } + case "int" -> { + IntVector values = op.getOutput().getBlock(1).asVector(); + for (int p = 0; p < values.getPositionCount(); p++) { + sum += values.getInt(p); + } + } + case "double" -> { + DoubleVector values = op.getOutput().getBlock(1).asVector(); + for (int p = 0; p < values.getPositionCount(); p++) { + sum += values.getDouble(p); + } + } + case "keyword" -> { + BytesRef scratch = new BytesRef(); + BytesRefVector values = op.getOutput().getBlock(1).asVector(); + for (int p = 0; p < values.getPositionCount(); p++) { + sum += Integer.parseInt(values.getBytesRef(p, scratch).utf8ToString()); + } + } + } + } + long expected = INDEX_SIZE; + expected = expected * (expected - 1) / 2; + if (expected != sum) { + throw new AssertionError("[" + layout + "][" + name + "] expected [" + expected + "] but was [" + sum + "]"); + } + } + + @Setup + public void setup() throws IOException { + setupIndex(); + setupPages(); + } + + private void setupIndex() throws IOException { + directory = new ByteBuffersDirectory(); + try (IndexWriter iw = new IndexWriter(directory, new IndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE))) { + for (int i = 0; i < INDEX_SIZE; i++) { + iw.addDocument( + List.of( + new NumericDocValuesField("long", i), + new NumericDocValuesField("int", i), + new NumericDocValuesField("double", NumericUtils.doubleToSortableLong(i)), + new KeywordFieldMapper.KeywordField( + "keyword", + new BytesRef(Integer.toString(i)), + KeywordFieldMapper.Defaults.FIELD_TYPE + ) + ) + ); + if (i % COMMIT_INTERVAL == 0) { + iw.commit(); + } + } + } + reader = DirectoryReader.open(directory); + } + + private void setupPages() { + pages = new ArrayList<>(); + switch (layout) { + case "in_order" -> { + IntVector.Builder docs = IntVector.newVectorBuilder(BLOCK_LENGTH); + for (LeafReaderContext ctx : reader.leaves()) { + int begin = 0; + while (begin < ctx.reader().maxDoc()) { + int end = Math.min(begin + BLOCK_LENGTH, ctx.reader().maxDoc()); + for (int doc = 0; doc < ctx.reader().maxDoc(); doc++) { + docs.appendInt(doc); + } + pages.add( + new Page( + new DocVector( + IntBlock.newConstantBlockWith(0, end - begin).asVector(), + IntBlock.newConstantBlockWith(ctx.ord, end - begin).asVector(), + docs.build(), + true + ).asBlock() + ) + ); + docs = IntVector.newVectorBuilder(BLOCK_LENGTH); + begin = end; + } + } + } + case "shuffled" -> { + record ItrAndOrd(PrimitiveIterator.OfInt itr, int ord) {} + List docItrs = new ArrayList<>(reader.leaves().size()); + for (LeafReaderContext ctx : reader.leaves()) { + docItrs.add(new ItrAndOrd(IntStream.range(0, ctx.reader().maxDoc()).iterator(), ctx.ord)); + } + IntVector.Builder docs = IntVector.newVectorBuilder(BLOCK_LENGTH); + IntVector.Builder leafs = IntVector.newVectorBuilder(BLOCK_LENGTH); + int size = 0; + while (docItrs.isEmpty() == false) { + Iterator itrItr = docItrs.iterator(); + while (itrItr.hasNext()) { + ItrAndOrd next = itrItr.next(); + if (false == next.itr.hasNext()) { + itrItr.remove(); + continue; + } + docs.appendInt(next.itr.nextInt()); + leafs.appendInt(next.ord); + size++; + if (size >= BLOCK_LENGTH) { + pages.add( + new Page( + new DocVector(IntBlock.newConstantBlockWith(0, size).asVector(), leafs.build(), docs.build(), null) + .asBlock() + ) + ); + docs = IntVector.newVectorBuilder(BLOCK_LENGTH); + leafs = IntVector.newVectorBuilder(BLOCK_LENGTH); + size = 0; + } + } + } + if (size > 0) { + pages.add( + new Page( + new DocVector( + IntBlock.newConstantBlockWith(0, size).asVector(), + leafs.build().asBlock().asVector(), + docs.build(), + null + ).asBlock() + ) + ); + } + } + case "shuffled_singles" -> { + record ItrAndOrd(PrimitiveIterator.OfInt itr, int ord) {} + List docItrs = new ArrayList<>(reader.leaves().size()); + for (LeafReaderContext ctx : reader.leaves()) { + docItrs.add(new ItrAndOrd(IntStream.range(0, ctx.reader().maxDoc()).iterator(), ctx.ord)); + } + while (docItrs.isEmpty() == false) { + Iterator itrItr = docItrs.iterator(); + while (itrItr.hasNext()) { + ItrAndOrd next = itrItr.next(); + if (false == next.itr.hasNext()) { + itrItr.remove(); + continue; + } + pages.add( + new Page( + new DocVector( + IntBlock.newConstantBlockWith(0, 1).asVector(), + IntBlock.newConstantBlockWith(next.ord, 1).asVector(), + IntBlock.newConstantBlockWith(next.itr.nextInt(), 1).asVector(), + true + ).asBlock() + ) + ); + } + } + } + default -> throw new IllegalArgumentException("unsupported layout [" + layout + "]"); + } + } + + @TearDown + public void teardownIndex() throws IOException { + IOUtils.close(reader, directory); + } +} diff --git a/build-tools-internal/src/main/groovy/elasticsearch.ide.gradle b/build-tools-internal/src/main/groovy/elasticsearch.ide.gradle index 4423af83f98b6..683a2d5604055 100644 --- a/build-tools-internal/src/main/groovy/elasticsearch.ide.gradle +++ b/build-tools-internal/src/main/groovy/elasticsearch.ide.gradle @@ -115,6 +115,8 @@ if (providers.systemProperty('idea.active').getOrNull() == 'true') { dependsOn([':client:rest-high-level:shadowJar', ':plugins:repository-hdfs:hadoop-client-api:shadowJar', ':libs:elasticsearch-x-content:generateProviderImpl', + ':x-pack:plugin:esql:compute:ann:jar', + ':x-pack:plugin:esql:compute:gen:jar', ':server:generateModulesList', ':server:generatePluginsList'].collect { elasticsearchProject.right()?.task(it) ?: it }) } diff --git a/build-tools-internal/src/main/resources/checkstyle_suppressions.xml b/build-tools-internal/src/main/resources/checkstyle_suppressions.xml index 54312eec8d61b..6a12ee5b0403b 100644 --- a/build-tools-internal/src/main/resources/checkstyle_suppressions.xml +++ b/build-tools-internal/src/main/resources/checkstyle_suppressions.xml @@ -12,6 +12,7 @@ + diff --git a/docs/changelog/98309.yaml b/docs/changelog/98309.yaml new file mode 100644 index 0000000000000..550f50b3569a1 --- /dev/null +++ b/docs/changelog/98309.yaml @@ -0,0 +1,5 @@ +pr: 98309 +summary: "Integrate Elasticsearch Query Language, ES|QL" +area: Query Languages +type: feature +issues: [] diff --git a/docs/reference/esql/aggregation-functions.asciidoc b/docs/reference/esql/aggregation-functions.asciidoc new file mode 100644 index 0000000000000..996228b8be45b --- /dev/null +++ b/docs/reference/esql/aggregation-functions.asciidoc @@ -0,0 +1,30 @@ +[[esql-agg-functions]] +== {esql} aggregation functions + +++++ +Aggregation functions +++++ + +<> support these functions: + +// tag::functions[] +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +// end::functions[] + +include::aggregation-functions/avg.asciidoc[] +include::aggregation-functions/count.asciidoc[] +include::aggregation-functions/count-distinct.asciidoc[] +include::aggregation-functions/max.asciidoc[] +include::aggregation-functions/median.asciidoc[] +include::aggregation-functions/median-absolute-deviation.asciidoc[] +include::aggregation-functions/min.asciidoc[] +include::aggregation-functions/percentile.asciidoc[] +include::aggregation-functions/sum.asciidoc[] diff --git a/docs/reference/esql/aggregation-functions/avg.asciidoc b/docs/reference/esql/aggregation-functions/avg.asciidoc new file mode 100644 index 0000000000000..93469a78cf405 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/avg.asciidoc @@ -0,0 +1,14 @@ +[[esql-agg-avg]] +=== `AVG` +The average of a numeric field. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats.csv-spec[tag=avg] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats.csv-spec[tag=avg-result] +|=== + +The result is always a `double` not matter the input type. diff --git a/docs/reference/esql/aggregation-functions/count-distinct.asciidoc b/docs/reference/esql/aggregation-functions/count-distinct.asciidoc new file mode 100644 index 0000000000000..cc1d96de688af --- /dev/null +++ b/docs/reference/esql/aggregation-functions/count-distinct.asciidoc @@ -0,0 +1,43 @@ +[[esql-agg-count-distinct]] +=== `COUNT_DISTINCT` +The approximate number of distinct values. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats_count_distinct.csv-spec[tag=count-distinct] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats_count_distinct.csv-spec[tag=count-distinct-result] +|=== + +Can take any field type as input and the result is always a `long` not matter +the input type. + +==== Counts are approximate + +Computing exact counts requires loading values into a set and returning its +size. This doesn't scale when working on high-cardinality sets and/or large +values as the required memory usage and the need to communicate those +per-shard sets between nodes would utilize too many resources of the cluster. + +This `COUNT_DISTINCT` function is based on the +https://static.googleusercontent.com/media/research.google.com/fr//pubs/archive/40671.pdf[HyperLogLog++] +algorithm, which counts based on the hashes of the values with some interesting +properties: + +include::../../aggregations/metrics/cardinality-aggregation.asciidoc[tag=explanation] + +==== Precision is configurable + +The `COUNT_DISTINCT` function takes an optional second parameter to configure the +precision discussed previously. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats_count_distinct.csv-spec[tag=count-distinct-precision] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats_count_distinct.csv-spec[tag=count-distinct-precision-result] +|=== diff --git a/docs/reference/esql/aggregation-functions/count.asciidoc b/docs/reference/esql/aggregation-functions/count.asciidoc new file mode 100644 index 0000000000000..280b612fb3bc1 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/count.asciidoc @@ -0,0 +1,18 @@ +[[esql-agg-count]] +=== `COUNT` +Counts field values. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats.csv-spec[tag=count] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats.csv-spec[tag=count-result] +|=== + +Can take any field type as input and the result is always a `long` not matter +the input type. + +NOTE: There isn't yet a `COUNT(*)`. Please count a single valued field if you + need a count of rows. diff --git a/docs/reference/esql/aggregation-functions/max.asciidoc b/docs/reference/esql/aggregation-functions/max.asciidoc new file mode 100644 index 0000000000000..924858844f264 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/max.asciidoc @@ -0,0 +1,12 @@ +[[esql-agg-max]] +=== `MAX` +The maximum value of a numeric field. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats.csv-spec[tag=max] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats.csv-spec[tag=max-result] +|=== diff --git a/docs/reference/esql/aggregation-functions/median-absolute-deviation.asciidoc b/docs/reference/esql/aggregation-functions/median-absolute-deviation.asciidoc new file mode 100644 index 0000000000000..83f391a821339 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/median-absolute-deviation.asciidoc @@ -0,0 +1,28 @@ +[[esql-agg-median-absolute-deviation]] +=== `MEDIAN_ABSOLUTE_DEVIATION` +The median absolute deviation, a measure of variability. It is a robust +statistic, meaning that it is useful for describing data that may have outliers, +or may not be normally distributed. For such data it can be more descriptive than +standard deviation. + +It is calculated as the median of each data point’s deviation from the median of +the entire sample. That is, for a random variable `X`, the median absolute deviation +is `median(|median(X) - Xi|)`. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats_percentile.csv-spec[tag=median-absolute-deviation] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats_percentile.csv-spec[tag=median-absolute-deviation-result] +|=== + +NOTE: Like <>, `MEDIAN_ABSOLUTE_DEVIATION` is + <>. + +[WARNING] +==== +`MEDIAN_ABSOLUTE_DEVIATION` is also {wikipedia}/Nondeterministic_algorithm[non-deterministic]. +This means you can get slightly different results using the same data. +==== diff --git a/docs/reference/esql/aggregation-functions/median.asciidoc b/docs/reference/esql/aggregation-functions/median.asciidoc new file mode 100644 index 0000000000000..8a482a69e06f4 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/median.asciidoc @@ -0,0 +1,21 @@ +[[esql-agg-median]] +=== `MEDIAN` +The value that is greater than half of all values and less than half of +all values, also known as the 50% <>. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats_percentile.csv-spec[tag=median] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats_percentile.csv-spec[tag=median-result] +|=== + +NOTE: Like <>, `MEDIAN` is <>. + +[WARNING] +==== +`MEDIAN` is also {wikipedia}/Nondeterministic_algorithm[non-deterministic]. +This means you can get slightly different results using the same data. +==== diff --git a/docs/reference/esql/aggregation-functions/min.asciidoc b/docs/reference/esql/aggregation-functions/min.asciidoc new file mode 100644 index 0000000000000..5d6457d36c709 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/min.asciidoc @@ -0,0 +1,12 @@ +[[esql-agg-min]] +=== `MIN` +The minimum value of a numeric field. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats.csv-spec[tag=min] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats.csv-spec[tag=min-result] +|=== diff --git a/docs/reference/esql/aggregation-functions/percentile.asciidoc b/docs/reference/esql/aggregation-functions/percentile.asciidoc new file mode 100644 index 0000000000000..e18346937cee6 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/percentile.asciidoc @@ -0,0 +1,28 @@ +[[esql-agg-percentile]] +=== `PERCENTILE` +The value at which a certain percentage of observed values occur. For example, +the 95th percentile is the value which is greater than 95% of the observed values and +the 50th percentile is the <>. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats_percentile.csv-spec[tag=percentile] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats_percentile.csv-spec[tag=percentile-result] +|=== + +[[esql-agg-percentile-approximate]] +==== `PERCENTILE` is (usually) approximate + +include::../../aggregations/metrics/percentile-aggregation.asciidoc[tag=approximate] + +[WARNING] +==== +`PERCENTILE` is also {wikipedia}/Nondeterministic_algorithm[non-deterministic]. +This means you can get slightly different results using the same data. +==== + + + diff --git a/docs/reference/esql/aggregation-functions/sum.asciidoc b/docs/reference/esql/aggregation-functions/sum.asciidoc new file mode 100644 index 0000000000000..758fdea654156 --- /dev/null +++ b/docs/reference/esql/aggregation-functions/sum.asciidoc @@ -0,0 +1,12 @@ +[[esql-agg-sum]] +=== `SUM` +The sum of a numeric field. + +[source.merge.styled,esql] +---- +include::{esql-specs}/stats.csv-spec[tag=sum] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/stats.csv-spec[tag=sum-result] +|=== diff --git a/docs/reference/esql/esql-functions.asciidoc b/docs/reference/esql/esql-functions.asciidoc new file mode 100644 index 0000000000000..2a4b64331390b --- /dev/null +++ b/docs/reference/esql/esql-functions.asciidoc @@ -0,0 +1,120 @@ +[[esql-functions]] +== {esql} functions + +++++ +Functions +++++ + +<>, <> and <> support +these functions: + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +include::functions/abs.asciidoc[] +include::functions/acos.asciidoc[] +include::functions/asin.asciidoc[] +include::functions/atan.asciidoc[] +include::functions/atan2.asciidoc[] +include::functions/auto_bucket.asciidoc[] +include::functions/case.asciidoc[] +include::functions/cidr_match.asciidoc[] +include::functions/concat.asciidoc[] +include::functions/cos.asciidoc[] +include::functions/cosh.asciidoc[] +include::functions/date_extract.asciidoc[] +include::functions/date_format.asciidoc[] +include::functions/date_parse.asciidoc[] +include::functions/date_trunc.asciidoc[] +include::functions/e.asciidoc[] +include::functions/floor.asciidoc[] +include::functions/is_finite.asciidoc[] +include::functions/is_infinite.asciidoc[] +include::functions/is_nan.asciidoc[] +include::functions/length.asciidoc[] +include::functions/log10.asciidoc[] +include::functions/mv_avg.asciidoc[] +include::functions/mv_concat.asciidoc[] +include::functions/mv_count.asciidoc[] +include::functions/mv_dedupe.asciidoc[] +include::functions/mv_max.asciidoc[] +include::functions/mv_median.asciidoc[] +include::functions/mv_min.asciidoc[] +include::functions/mv_sum.asciidoc[] +include::functions/now.asciidoc[] +include::functions/pi.asciidoc[] +include::functions/pow.asciidoc[] +include::functions/round.asciidoc[] +include::functions/sin.asciidoc[] +include::functions/sinh.asciidoc[] +include::functions/split.asciidoc[] +include::functions/sqrt.asciidoc[] +include::functions/starts_with.asciidoc[] +include::functions/substring.asciidoc[] +include::functions/tan.asciidoc[] +include::functions/tanh.asciidoc[] +include::functions/tau.asciidoc[] +include::functions/to_boolean.asciidoc[] +include::functions/to_datetime.asciidoc[] +include::functions/to_degrees.asciidoc[] +include::functions/to_double.asciidoc[] +include::functions/to_integer.asciidoc[] +include::functions/to_ip.asciidoc[] +include::functions/to_long.asciidoc[] +include::functions/to_radians.asciidoc[] +include::functions/to_string.asciidoc[] +include::functions/to_unsigned_long.asciidoc[] +include::functions/to_version.asciidoc[] +include::functions/trim.asciidoc[] diff --git a/docs/reference/esql/esql-get-started.asciidoc b/docs/reference/esql/esql-get-started.asciidoc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/docs/reference/esql/esql-processing-commands.asciidoc b/docs/reference/esql/esql-processing-commands.asciidoc new file mode 100644 index 0000000000000..d949494d1352d --- /dev/null +++ b/docs/reference/esql/esql-processing-commands.asciidoc @@ -0,0 +1,39 @@ +[[esql-processing-commands]] +== {esql} processing commands + +++++ +Processing commands +++++ + +{esql} processing commands change an input table by adding, removing, or changing +rows and columns. + +image::images/esql/processing-command.svg[A processing command changing an input table,align="center"] + +{esql} supports these processing commands: + +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +include::processing-commands/dissect.asciidoc[] +include::processing-commands/drop.asciidoc[] +include::processing-commands/enrich.asciidoc[] +include::processing-commands/eval.asciidoc[] +include::processing-commands/grok.asciidoc[] +include::processing-commands/keep.asciidoc[] +include::processing-commands/limit.asciidoc[] +include::processing-commands/mv_expand.asciidoc[] +include::processing-commands/rename.asciidoc[] +include::processing-commands/sort.asciidoc[] +include::processing-commands/stats.asciidoc[] +include::processing-commands/where.asciidoc[] diff --git a/docs/reference/esql/esql-source-commands.asciidoc b/docs/reference/esql/esql-source-commands.asciidoc new file mode 100644 index 0000000000000..5ca4d6980ddd6 --- /dev/null +++ b/docs/reference/esql/esql-source-commands.asciidoc @@ -0,0 +1,20 @@ +[[esql-source-commands]] +== {esql} source commands + +++++ +Source commands +++++ + +An {esql} source command produces a table, typically with data from {es}. + +image::images/esql/source-command.svg[A source command producing a table from {es},align="center"] + +{esql} supports these source commands: + +* <> +* <> +* <> + +include::source-commands/from.asciidoc[] +include::source-commands/row.asciidoc[] +include::source-commands/show.asciidoc[] diff --git a/docs/reference/esql/esql-syntax.asciidoc b/docs/reference/esql/esql-syntax.asciidoc new file mode 100644 index 0000000000000..53d80b9e29ab7 --- /dev/null +++ b/docs/reference/esql/esql-syntax.asciidoc @@ -0,0 +1,154 @@ +[[esql-syntax]] +== {esql} syntax reference + +++++ +Syntax reference +++++ + +[discrete] +[[esql-basic-syntax]] +=== Basic syntax + +An {esql} query is composed of a <> followed +by an optional series of <>, +separated by a pipe character: `|`. For example: + +[source,esql] +---- +source-command +| processing-command1 +| processing-command2 +---- + +The result of a query is the table produced by the final processing command. + +For readability, this documentation puts each processing command on a new line. +However, you can write an {esql} query as a single line. The following query is +identical to the previous one: + +[source,esql] +---- +source-command | processing-command1 | processing-command2 +---- + +[discrete] +[[esql-comments]] +=== Comments +{esql} uses C++ style comments: + +* double slash `//` for single line comments +* `/*` and `*/` for block comments + +[source,esql] +---- +// Query the employees index +FROM employees +| WHERE height > 2 +---- + +[source,esql] +---- +FROM /* Query the employees index */ employees +| WHERE height > 2 +---- + +[source,esql] +---- +FROM employees +/* Query the + * employees + * index */ +| WHERE height > 2 +---- + +[discrete] +[[esql-operators]] +=== Operators +These binary comparison operators are supported: + +* equality: `==` +* inequality: `!=` +* less than: `<` +* less than or equal: `<=` +* larger than: `>` +* larger than or equal: `>=` + +The `IN` operator allows testing whether a field or expression equals +an element in a list of literals, fields or expressions: + +[source,esql] +---- +include::{esql-specs}/row.csv-spec[tag=in-with-expressions] +---- + +For string comparison using wildcards or regular expressions, use `LIKE` or +`RLIKE`: + +* Use `LIKE` to match strings using wildcards. The following wildcard characters +are supported: ++ +-- +** `*` matches zero or more characters. +** `?` matches one character. + +[source,esql] +---- +FROM employees +| WHERE first_name LIKE "?b*" +| KEEP first_name, last_name +---- +-- + +* Use `RLIKE` to match strings using <>: ++ +[source,esql] +---- +FROM employees +| WHERE first_name RLIKE ".leja.*" +| KEEP first_name, last_name +---- + +The following boolean operators are supported: + +* `AND` +* `OR` +* `NOT` + +[discrete] +[[esql-predicates]] +=== Predicates + +For NULL comparison use the `IS NULL` and `IS NOT NULL` predicates: + +[source,esql] +---- +include::{esql-specs}/conditional.csv-spec[tag=is-null] +---- + +[source,esql] +---- +include::{esql-specs}/conditional.csv-spec[tag=is-not-null] +---- + +[discrete] +[[esql-timespan-literals]] +=== Timespan literals + +Datetime intervals and timespans can be expressed using timespan literals. +Timespan literals are a combination of a number and a qualifier. These +qualifiers are supported: + +* `millisecond`/`milliseconds` +* `second`/`seconds` +* `minute`/`minutes` +* `hour`/`hours` +* `day`/`days` +* `week`/`weeks` +* `month`/`months` +* `year`/`years` + +Timespan literals are not whitespace sensitive. These expressions are all valid: + +* `1day` +* `1 day` +* `1 day` diff --git a/docs/reference/esql/functions/abs.asciidoc b/docs/reference/esql/functions/abs.asciidoc new file mode 100644 index 0000000000000..b944359480db0 --- /dev/null +++ b/docs/reference/esql/functions/abs.asciidoc @@ -0,0 +1,10 @@ +[[esql-abs]] +=== `ABS` +Returns the absolute value. + +[source,esql] +---- +FROM employees +| KEEP first_name, last_name, height +| EVAL abs_height = ABS(0.0 - height) +---- diff --git a/docs/reference/esql/functions/acos.asciidoc b/docs/reference/esql/functions/acos.asciidoc new file mode 100644 index 0000000000000..383e4224a0e1b --- /dev/null +++ b/docs/reference/esql/functions/acos.asciidoc @@ -0,0 +1,12 @@ +[[esql-acos]] +=== `ACOS` +Inverse https://en.wikipedia.org/wiki/Inverse_trigonometric_functions[cosine] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=acos] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=acos-result] +|=== diff --git a/docs/reference/esql/functions/asin.asciidoc b/docs/reference/esql/functions/asin.asciidoc new file mode 100644 index 0000000000000..a7ddfde444edd --- /dev/null +++ b/docs/reference/esql/functions/asin.asciidoc @@ -0,0 +1,12 @@ +[[esql-asin]] +=== `ASIN` +Inverse https://en.wikipedia.org/wiki/Inverse_trigonometric_functions[sine] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=asin] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=asin-result] +|=== diff --git a/docs/reference/esql/functions/atan.asciidoc b/docs/reference/esql/functions/atan.asciidoc new file mode 100644 index 0000000000000..cda085ec8eb68 --- /dev/null +++ b/docs/reference/esql/functions/atan.asciidoc @@ -0,0 +1,12 @@ +[[esql-atan]] +=== `ATAN` +Inverse https://en.wikipedia.org/wiki/Inverse_trigonometric_functions[tangent] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=atan] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=atan-result] +|=== diff --git a/docs/reference/esql/functions/atan2.asciidoc b/docs/reference/esql/functions/atan2.asciidoc new file mode 100644 index 0000000000000..47dee88ddc740 --- /dev/null +++ b/docs/reference/esql/functions/atan2.asciidoc @@ -0,0 +1,14 @@ +[[esql-atan2]] +=== `ATAN2` + +The https://en.wikipedia.org/wiki/Atan2[angle] between the positive x-axis and the +ray from the origin to the point (x , y) in the Cartesian plane. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=atan2] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=atan2-result] +|=== diff --git a/docs/reference/esql/functions/auto_bucket.asciidoc b/docs/reference/esql/functions/auto_bucket.asciidoc new file mode 100644 index 0000000000000..a61e8365716c5 --- /dev/null +++ b/docs/reference/esql/functions/auto_bucket.asciidoc @@ -0,0 +1,71 @@ +[[esql-auto_bucket]] +=== `AUTO_BUCKET` +Creates human-friendly buckets and returns a `datetime` value for each row that +corresponds to the resulting bucket the row falls into. Combine `AUTO_BUCKET` +with <> to create a date histogram. + +You provide a target number of buckets, a start date, and an end date, and it +picks an appropriate bucket size to generate the target number of buckets or +fewer. For example, this asks for at most 20 buckets over a whole year, which +picks monthly buckets: + +[source.merge.styled,esql] +---- +include::{esql-specs}/date.csv-spec[tag=auto_bucket_month] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/date.csv-spec[tag=auto_bucket_month-result] +|=== + +The goal isn't to provide *exactly* the target number of buckets, it's to pick a +range that people are comfortable with that provides at most the target number of +buckets. + +If you ask for more buckets then `AUTO_BUCKET` can pick a smaller range. For example, +asking for at most 100 buckets in a year will get you week long buckets: + +[source.merge.styled,esql] +---- +include::{esql-specs}/date.csv-spec[tag=auto_bucket_week] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/date.csv-spec[tag=auto_bucket_week-result] +|=== + +`AUTO_BUCKET` does not filter any rows. It only uses the provided time range to +pick a good bucket size. For rows with a date outside of the range, it returns a +`datetime` that corresponds to a bucket outside the range. Combine `AUTO_BUCKET` +with <> to filter rows. + +A more complete example might look like: + +[source.merge.styled,esql] +---- +include::{esql-specs}/date.csv-spec[tag=auto_bucket_in_agg] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/date.csv-spec[tag=auto_bucket_in_agg-result] +|=== + +NOTE: `AUTO_BUCKET` does not create buckets that don't match any documents. That's +why the example above is missing `1985-02-01` and other dates. + +==== Numeric fields + +`auto_bucket` can also operate on numeric fields like this: +[source.merge.styled,esql] +---- +include::{esql-specs}/ints.csv-spec[tag=auto_bucket] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/ints.csv-spec[tag=auto_bucket-result] +|=== + +Unlike the example above where you are intentionally filtering on a date range, +you rarely want to filter on a numeric range. So you have find the `min` and `max` +separately. We don't yet have an easy way to do that automatically. Improvements +coming! diff --git a/docs/reference/esql/functions/case.asciidoc b/docs/reference/esql/functions/case.asciidoc new file mode 100644 index 0000000000000..c1d72f661241c --- /dev/null +++ b/docs/reference/esql/functions/case.asciidoc @@ -0,0 +1,17 @@ +[[esql-case]] +=== `CASE` + +Accepts pairs of conditions and values. The function returns the value that +belongs to the first condition that evaluates to `true`. If the number of +arguments is odd, the last argument is the default value which is returned when +no condition matches. + +[source,esql] +---- +FROM employees +| EVAL type = CASE( + languages <= 1, "monolingual", + languages <= 2, "bilingual", + "polyglot") +| KEEP first_name, last_name, type +---- diff --git a/docs/reference/esql/functions/cidr_match.asciidoc b/docs/reference/esql/functions/cidr_match.asciidoc new file mode 100644 index 0000000000000..e42b2e99b7c76 --- /dev/null +++ b/docs/reference/esql/functions/cidr_match.asciidoc @@ -0,0 +1,15 @@ +[[esql-cidr_match]] +=== `CIDR_MATCH` + +Returns `true` if the provided IP is contained in one of the provided CIDR +blocks. + +`CIDR_MATCH` accepts two or more arguments. The first argument is the IP +address of type `ip` (both IPv4 and IPv6 are supported). Subsequent arguments +are the CIDR blocks to test the IP against. + +[source,esql] +---- +FROM hosts +| WHERE CIDR_MATCH(ip, "127.0.0.2/32", "127.0.0.3/32") +---- diff --git a/docs/reference/esql/functions/concat.asciidoc b/docs/reference/esql/functions/concat.asciidoc new file mode 100644 index 0000000000000..9ebf814fdc3d9 --- /dev/null +++ b/docs/reference/esql/functions/concat.asciidoc @@ -0,0 +1,10 @@ +[[esql-concat]] +=== `CONCAT` +Concatenates two or more strings. + +[source,esql] +---- +FROM employees +| KEEP first_name, last_name, height +| EVAL fullname = CONCAT(first_name, " ", last_name) +---- diff --git a/docs/reference/esql/functions/cos.asciidoc b/docs/reference/esql/functions/cos.asciidoc new file mode 100644 index 0000000000000..39d2564dd7d73 --- /dev/null +++ b/docs/reference/esql/functions/cos.asciidoc @@ -0,0 +1,12 @@ +[[esql-cos]] +=== `COS` +https://en.wikipedia.org/wiki/Sine_and_cosine[Cosine] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=cos] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=cos-result] +|=== diff --git a/docs/reference/esql/functions/cosh.asciidoc b/docs/reference/esql/functions/cosh.asciidoc new file mode 100644 index 0000000000000..99eb19948e8e4 --- /dev/null +++ b/docs/reference/esql/functions/cosh.asciidoc @@ -0,0 +1,12 @@ +[[esql-cosh]] +=== `COSH` +https://en.wikipedia.org/wiki/Hyperbolic_functions[Cosine] hyperbolic function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=cosh] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=cosh-result] +|=== diff --git a/docs/reference/esql/functions/date_extract.asciidoc b/docs/reference/esql/functions/date_extract.asciidoc new file mode 100644 index 0000000000000..efa1a98ac0a5e --- /dev/null +++ b/docs/reference/esql/functions/date_extract.asciidoc @@ -0,0 +1,14 @@ +[[esql-date_extract]] +=== `DATE_EXTRACT` +Extracts parts of a date, like year, month, day, hour. +The supported field types are those provided by https://docs.oracle.com/javase/8/docs/api/java/time/temporal/ChronoField.html[java.time.temporal.ChronoField] + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=dateExtract] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=dateExtract-result] +|=== + diff --git a/docs/reference/esql/functions/date_format.asciidoc b/docs/reference/esql/functions/date_format.asciidoc new file mode 100644 index 0000000000000..3f61e07221111 --- /dev/null +++ b/docs/reference/esql/functions/date_format.asciidoc @@ -0,0 +1,11 @@ +[[esql-date_format]] +=== `DATE_FORMAT` +Returns a string representation of a date in the provided format. If no format +is specified, the `yyyy-MM-dd'T'HH:mm:ss.SSSZ` format is used. + +[source,esql] +---- +FROM employees +| KEEP first_name, last_name, hire_date +| EVAL hired = DATE_FORMAT(hire_date, "YYYY-MM-dd") +---- diff --git a/docs/reference/esql/functions/date_parse.asciidoc b/docs/reference/esql/functions/date_parse.asciidoc new file mode 100644 index 0000000000000..962d9bc782e3d --- /dev/null +++ b/docs/reference/esql/functions/date_parse.asciidoc @@ -0,0 +1,9 @@ +[[esql-date_parse]] +=== `DATE_PARSE` +Converts a string to a date, in the provided format. If no format +is specified, the `yyyy-MM-dd'T'HH:mm:ss.SSSZ` format is used. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=dateParse] +---- diff --git a/docs/reference/esql/functions/date_trunc.asciidoc b/docs/reference/esql/functions/date_trunc.asciidoc new file mode 100644 index 0000000000000..b3dab7255f676 --- /dev/null +++ b/docs/reference/esql/functions/date_trunc.asciidoc @@ -0,0 +1,12 @@ +[[esql-date_trunc]] +=== `DATE_TRUNC` +Rounds down a date to the closest interval. Intervals can be expressed using the +<>. + +[source,esql] +---- +FROM employees +| EVAL year_hired = DATE_TRUNC(hire_date, 1 year) +| STATS count(emp_no) BY year_hired +| SORT year_hired +---- diff --git a/docs/reference/esql/functions/e.asciidoc b/docs/reference/esql/functions/e.asciidoc new file mode 100644 index 0000000000000..a9d2f3fa340e0 --- /dev/null +++ b/docs/reference/esql/functions/e.asciidoc @@ -0,0 +1,12 @@ +[[esql-e]] +=== `E` +{wikipedia}/E_(mathematical_constant)[Euler's number]. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=e] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=e-result] +|=== diff --git a/docs/reference/esql/functions/floor.asciidoc b/docs/reference/esql/functions/floor.asciidoc new file mode 100644 index 0000000000000..595e60e98a6d2 --- /dev/null +++ b/docs/reference/esql/functions/floor.asciidoc @@ -0,0 +1,16 @@ +[[esql-floor]] +=== `FLOOR` +Round a number down to the nearest integer. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=floor] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=floor-result] +|=== + +NOTE: This is a noop for `long` and `integer`. For `double` this picks the + the closest `double` value to the integer ala + {javadoc}/java.base/java/lang/Math.html#floor(double)[Math.floor]. diff --git a/docs/reference/esql/functions/is_finite.asciidoc b/docs/reference/esql/functions/is_finite.asciidoc new file mode 100644 index 0000000000000..ff0f0170fcac7 --- /dev/null +++ b/docs/reference/esql/functions/is_finite.asciidoc @@ -0,0 +1,9 @@ +[[esql-is_finite]] +=== `IS_FINITE` +Returns a boolean that indicates whether its input is a finite number. + +[source,esql] +---- +ROW d = 1.0 +| EVAL s = IS_FINITE(d/0) +---- diff --git a/docs/reference/esql/functions/is_infinite.asciidoc b/docs/reference/esql/functions/is_infinite.asciidoc new file mode 100644 index 0000000000000..31b685a37c976 --- /dev/null +++ b/docs/reference/esql/functions/is_infinite.asciidoc @@ -0,0 +1,9 @@ +[[esql-is_infinite]] +=== `IS_INFINITE` +Returns a boolean that indicates whether its input is infinite. + +[source,esql] +---- +ROW d = 1.0 +| EVAL s = IS_INFINITE(d/0) +---- diff --git a/docs/reference/esql/functions/is_nan.asciidoc b/docs/reference/esql/functions/is_nan.asciidoc new file mode 100644 index 0000000000000..66bb79054792e --- /dev/null +++ b/docs/reference/esql/functions/is_nan.asciidoc @@ -0,0 +1,9 @@ +[[esql-is_nan]] +=== `IS_NAN` +Returns a boolean that indicates whether its input is not a number. + +[source,esql] +---- +ROW d = 1.0 +| EVAL s = IS_NAN(d) +---- diff --git a/docs/reference/esql/functions/length.asciidoc b/docs/reference/esql/functions/length.asciidoc new file mode 100644 index 0000000000000..955e09872c213 --- /dev/null +++ b/docs/reference/esql/functions/length.asciidoc @@ -0,0 +1,10 @@ +[[esql-length]] +=== `LENGTH` +Returns the character length of a string. + +[source,esql] +---- +FROM employees +| KEEP first_name, last_name, height +| EVAL fn_length = LENGTH(first_name) +---- diff --git a/docs/reference/esql/functions/log10.asciidoc b/docs/reference/esql/functions/log10.asciidoc new file mode 100644 index 0000000000000..ee19d5a61d1fa --- /dev/null +++ b/docs/reference/esql/functions/log10.asciidoc @@ -0,0 +1,15 @@ +[[esql-log10]] +=== `LOG10` +Returns the log base 10. The input can be any numeric value, the return value +is always a double. + +Logs of negative numbers are NaN. Logs of infinites are infinite, as is the log of 0. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=log10] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=log10-result] +|=== diff --git a/docs/reference/esql/functions/mv_avg.asciidoc b/docs/reference/esql/functions/mv_avg.asciidoc new file mode 100644 index 0000000000000..68f859b84d588 --- /dev/null +++ b/docs/reference/esql/functions/mv_avg.asciidoc @@ -0,0 +1,16 @@ +[[esql-mv_avg]] +=== `MV_AVG` +Converts a multivalued field into a single valued field containing the average +of all of the values. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_avg] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_avg-result] +|=== + + +NOTE: The output type is always a `double` and the input type can be any number. diff --git a/docs/reference/esql/functions/mv_concat.asciidoc b/docs/reference/esql/functions/mv_concat.asciidoc new file mode 100644 index 0000000000000..60c90ed96b8ee --- /dev/null +++ b/docs/reference/esql/functions/mv_concat.asciidoc @@ -0,0 +1,25 @@ +[[esql-mv_concat]] +=== `MV_CONCAT` +Converts a multivalued string field into a single valued field containing the +concatenation of all values separated by a delimiter: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_concat] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_concat-result] +|=== + +If you want to concat non-string fields call <> on them first: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_concat-to_string] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_concat-to_string-result] +|=== + diff --git a/docs/reference/esql/functions/mv_count.asciidoc b/docs/reference/esql/functions/mv_count.asciidoc new file mode 100644 index 0000000000000..e4fd7f0299e66 --- /dev/null +++ b/docs/reference/esql/functions/mv_count.asciidoc @@ -0,0 +1,15 @@ +[[esql-mv_count]] +=== `MV_COUNT` +Converts a multivalued field into a single valued field containing a count of the number +of values: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_count] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_count-result] +|=== + +NOTE: This function accepts all types and always returns an `integer`. diff --git a/docs/reference/esql/functions/mv_dedupe.asciidoc b/docs/reference/esql/functions/mv_dedupe.asciidoc new file mode 100644 index 0000000000000..8037dba7cf19a --- /dev/null +++ b/docs/reference/esql/functions/mv_dedupe.asciidoc @@ -0,0 +1,14 @@ +[[esql-mv_dedupe]] +=== `MV_DEDUPE` +Removes duplicates from a multivalued field. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_dedupe] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_dedupe-result] +|=== + +NOTE: `MV_DEDUPE` may, but won't always, sort the values in the field. diff --git a/docs/reference/esql/functions/mv_max.asciidoc b/docs/reference/esql/functions/mv_max.asciidoc new file mode 100644 index 0000000000000..64e8b79401bca --- /dev/null +++ b/docs/reference/esql/functions/mv_max.asciidoc @@ -0,0 +1,24 @@ +[[esql-mv_max]] +=== `MV_MAX` +Converts a multivalued field into a single valued field containing the maximum value. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_max] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_max-result] +|=== + +It can be used by any field type, including `keyword` fields. In that case picks the +last string, comparing their utf-8 representation byte by byte: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_max] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_max-result] +|=== diff --git a/docs/reference/esql/functions/mv_median.asciidoc b/docs/reference/esql/functions/mv_median.asciidoc new file mode 100644 index 0000000000000..decfb4aad3884 --- /dev/null +++ b/docs/reference/esql/functions/mv_median.asciidoc @@ -0,0 +1,26 @@ +[[esql-mv_median]] +=== `MV_MEDIAN` +Converts a multivalued field into a single valued field containing the median value. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_median] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_median-result] +|=== + +It can be used by any numeric field type and returns a value of the same type. If the +row has an even number of values for a column the result will be the average of the +middle two entries. If the field is not floating point then the average rounds *down*: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_median_round_down] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_median_round_down-result] +|=== + diff --git a/docs/reference/esql/functions/mv_min.asciidoc b/docs/reference/esql/functions/mv_min.asciidoc new file mode 100644 index 0000000000000..8d565a72b7735 --- /dev/null +++ b/docs/reference/esql/functions/mv_min.asciidoc @@ -0,0 +1,24 @@ +[[esql-mv_min]] +=== `MV_MIN` +Converts a multivalued field into a single valued field containing the minimum value. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_min] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_min-result] +|=== + +It can be used by any field type, including `keyword` fields. In that case picks the +first string, comparing their utf-8 representation byte by byte: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=mv_min] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=mv_min-result] +|=== diff --git a/docs/reference/esql/functions/mv_sum.asciidoc b/docs/reference/esql/functions/mv_sum.asciidoc new file mode 100644 index 0000000000000..ae105104c97b9 --- /dev/null +++ b/docs/reference/esql/functions/mv_sum.asciidoc @@ -0,0 +1,15 @@ +[[esql-mv_sum]] +=== `MV_SUM` +Converts a multivalued field into a single valued field containing the sum +of all of the values. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=mv_sum] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=mv_sum-result] +|=== + +NOTE: The input type can be any number and the output type is the same as the input type. diff --git a/docs/reference/esql/functions/now.asciidoc b/docs/reference/esql/functions/now.asciidoc new file mode 100644 index 0000000000000..1213ef6bfa627 --- /dev/null +++ b/docs/reference/esql/functions/now.asciidoc @@ -0,0 +1,8 @@ +[[esql-now]] +=== `NOW` +Returns current date and time. + +[source,esql] +---- +ROW current_date = NOW() +---- diff --git a/docs/reference/esql/functions/pi.asciidoc b/docs/reference/esql/functions/pi.asciidoc new file mode 100644 index 0000000000000..631018fed0055 --- /dev/null +++ b/docs/reference/esql/functions/pi.asciidoc @@ -0,0 +1,12 @@ +[[esql-pi]] +=== `PI` +The {wikipedia}/Pi[ratio] of a circle's circumference to its diameter. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=pi] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=pi-result] +|=== diff --git a/docs/reference/esql/functions/pow.asciidoc b/docs/reference/esql/functions/pow.asciidoc new file mode 100644 index 0000000000000..c80b64912e976 --- /dev/null +++ b/docs/reference/esql/functions/pow.asciidoc @@ -0,0 +1,115 @@ +[[esql-pow]] +=== `POW` +Returns the value of a base (first argument) raised to the power of an exponent (second argument). +Both arguments must be numeric. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=powDI] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=powDI-result] +|=== + +==== Type rules + +The type of the returned value is determined by the types of the base and exponent. +The following rules are applied to determine the result type: + +* If either of the base or exponent are of a floating point type, the result will be a double +* Otherwise, if either the base of the exponent are 64-bit (long or unsigned long), the result will be a long +* Otherwise, the result will be a 32-bit integer (all other numeric types, including int, short and byte) + +For example, using simple integers as arguments will lead to an integer result: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=powII] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=powII-result] +|=== + +Note: The actual power function is performed using double precision values for all cases. +This means that for very large non-floating point values the operation can lead to very slightly different answers. +However, a more likely outcome of very large non-floating point values is numerical overflow. + +==== Arithmetic errors + +Arithmetic errors and numeric overflow do not result in an error, instead the result will be `null` +and a warning for the `ArithmeticException` added. +For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=powULOverrun] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=powULOverrun-warning] +|=== +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=powULOverrun-result] +|=== + +If it is desired to protect against numerical overruns, use `to_double` on any one of the arguments: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=pow2d] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=pow2d-result] +|=== + +==== Fractional exponents + +The exponent can be a fraction, which is similar to performing a root. +For example, the exponent of `0.5` will give the square root of the base: + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=powID-sqrt] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=powID-sqrt-result] +|=== + +==== Table of supported input and output types + +For clarity, the following table describes the output result type for all combinations of numeric input types: + +[cols="1,1,1"] +|=== +|Base | Exponent | Result + +|double/float/half_float +|*footnote:all[All numeric types] +|double + +|*footnote:all[] +|double/float/half_float +|double + +|long/unsigned long +|*footnote:all_but_float[All except double/float/half_float] +|long + +|*footnote:all_but_float[] +|long/unsigned long +|long + +|*footnote:all_but_float_and_64[All except floating point and 64-bit types] +|*footnote:all_but_float_and_64[] +|int + +|*footnote:all_but_float_and_64[] +|*footnote:all_but_float_and_64[] +|int + +|=== diff --git a/docs/reference/esql/functions/round.asciidoc b/docs/reference/esql/functions/round.asciidoc new file mode 100644 index 0000000000000..88dbf23440a71 --- /dev/null +++ b/docs/reference/esql/functions/round.asciidoc @@ -0,0 +1,14 @@ +[[esql-round]] +=== `ROUND` +Rounds a number to the closest number with the specified number of digits. +Defaults to 0 digits if no number of digits is provided. If the specified number +of digits is negative, rounds to the number of digits left of the decimal point. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=round] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=round-result] +|=== diff --git a/docs/reference/esql/functions/sin.asciidoc b/docs/reference/esql/functions/sin.asciidoc new file mode 100644 index 0000000000000..7c02ded0a2f72 --- /dev/null +++ b/docs/reference/esql/functions/sin.asciidoc @@ -0,0 +1,12 @@ +[[esql-sin]] +=== `SIN` +https://en.wikipedia.org/wiki/Sine_and_cosine[Sine] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=sin] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=sin-result] +|=== diff --git a/docs/reference/esql/functions/sinh.asciidoc b/docs/reference/esql/functions/sinh.asciidoc new file mode 100644 index 0000000000000..241b4f978349d --- /dev/null +++ b/docs/reference/esql/functions/sinh.asciidoc @@ -0,0 +1,12 @@ +[[esql-sinh]] +=== `SINH` +https://en.wikipedia.org/wiki/Hyperbolic_functions[Sine] hyperbolic function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=sinh] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=sinh-result] +|=== diff --git a/docs/reference/esql/functions/split.asciidoc b/docs/reference/esql/functions/split.asciidoc new file mode 100644 index 0000000000000..396e8b2beaf3a --- /dev/null +++ b/docs/reference/esql/functions/split.asciidoc @@ -0,0 +1,17 @@ +[[esql-split]] +=== `SPLIT` +Split a single valued string into multiple strings. For example: + +[source,esql] +---- +include::{esql-specs}/string.csv-spec[tag=split] +---- + +Which splits `"foo;bar;baz;qux;quux;corge"` on `;` and returns an array: + +[%header,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=split-result] +|=== + +WARNING: Only single byte delimiters are currently supported. diff --git a/docs/reference/esql/functions/sqrt.asciidoc b/docs/reference/esql/functions/sqrt.asciidoc new file mode 100644 index 0000000000000..189deefa2cf90 --- /dev/null +++ b/docs/reference/esql/functions/sqrt.asciidoc @@ -0,0 +1,15 @@ +[[esql-sqrt]] +=== `SQRT` +Returns the square root of a number. The input can be any numeric value, the return value +is always a double. + +Square roots of negative numbers are NaN. Square roots of infinites are infinite. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=sqrt] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=sqrt-result] +|=== diff --git a/docs/reference/esql/functions/starts_with.asciidoc b/docs/reference/esql/functions/starts_with.asciidoc new file mode 100644 index 0000000000000..b54caf362ac93 --- /dev/null +++ b/docs/reference/esql/functions/starts_with.asciidoc @@ -0,0 +1,13 @@ +[[esql-starts_with]] +=== `STARTS_WITH` +Returns a boolean that indicates whether a keyword string starts with another +string: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=startsWith] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=startsWith-result] +|=== diff --git a/docs/reference/esql/functions/substring.asciidoc b/docs/reference/esql/functions/substring.asciidoc new file mode 100644 index 0000000000000..b1a24617af188 --- /dev/null +++ b/docs/reference/esql/functions/substring.asciidoc @@ -0,0 +1,37 @@ +[[esql-substring]] +=== `SUBSTRING` +Returns a substring of a string, specified by a start position and an optional +length. This example returns the first three characters of every last name: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=substring] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=substring-result] +|=== + +A negative start position is interpreted as being relative to the end of the +string. This example returns the last three characters of of every last name: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=substringEnd] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=substringEnd-result] +|=== + +If length is omitted, substring returns the remainder of the string. This +example returns all characters except for the first: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=substringRemainder] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=substringRemainder-result] +|=== diff --git a/docs/reference/esql/functions/tan.asciidoc b/docs/reference/esql/functions/tan.asciidoc new file mode 100644 index 0000000000000..fc64317135a44 --- /dev/null +++ b/docs/reference/esql/functions/tan.asciidoc @@ -0,0 +1,12 @@ +[[esql-tan]] +=== `TAN` +https://en.wikipedia.org/wiki/Sine_and_cosine[Tangent] trigonometric function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=tan] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=tan-result] +|=== diff --git a/docs/reference/esql/functions/tanh.asciidoc b/docs/reference/esql/functions/tanh.asciidoc new file mode 100644 index 0000000000000..f9fcec10394d6 --- /dev/null +++ b/docs/reference/esql/functions/tanh.asciidoc @@ -0,0 +1,12 @@ +[[esql-tanh]] +=== `TANH` +https://en.wikipedia.org/wiki/Hyperbolic_functions[Tangent] hyperbolic function. + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=tanh] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=tanh-result] +|=== diff --git a/docs/reference/esql/functions/tau.asciidoc b/docs/reference/esql/functions/tau.asciidoc new file mode 100644 index 0000000000000..f2891baf73db6 --- /dev/null +++ b/docs/reference/esql/functions/tau.asciidoc @@ -0,0 +1,12 @@ +[[esql-tau]] +=== `TAU` +The https://tauday.com/tau-manifesto[ratio] of a circle's circumference to its radius. + +[source.merge.styled,esql] +---- +include::{esql-specs}/math.csv-spec[tag=tau] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/math.csv-spec[tag=tau-result] +|=== diff --git a/docs/reference/esql/functions/to_boolean.asciidoc b/docs/reference/esql/functions/to_boolean.asciidoc new file mode 100644 index 0000000000000..4839678cebbde --- /dev/null +++ b/docs/reference/esql/functions/to_boolean.asciidoc @@ -0,0 +1,24 @@ +[[esql-to_boolean]] +=== `TO_BOOLEAN` +Converts an input value to a boolean value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a string or numeric type. + +A string value of *"true"* will be case-insensitive converted to the Boolean +*true*. For anything else, including the empty string, the function will +return *false*. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/boolean.csv-spec[tag=to_boolean] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/boolean.csv-spec[tag=to_boolean-result] +|=== + +The numerical value of *0* will be converted to *false*, anything else will be +converted to *true*. + +Alias: TO_BOOL diff --git a/docs/reference/esql/functions/to_datetime.asciidoc b/docs/reference/esql/functions/to_datetime.asciidoc new file mode 100644 index 0000000000000..2429600b75a96 --- /dev/null +++ b/docs/reference/esql/functions/to_datetime.asciidoc @@ -0,0 +1,46 @@ +[[esql-to_datetime]] +=== `TO_DATETIME` +Converts an input value to a date value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a string or numeric type. + +A string will only be successfully converted if it's respecting the format +`yyyy-MM-dd'T'HH:mm:ss.SSS'Z'` (to convert dates in other formats, use <>). For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/date.csv-spec[tag=to_datetime-str] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/date.csv-spec[tag=to_datetime-str-result] +|=== + +Note that in this example, the last value in the source multi-valued +field has not been converted. The reason being that if the date format is not +respected, the conversion will result in a *null* value. When this happens a +_Warning_ header is added to the response. The header will provide information +on the source of the failure: + +`"Line 1:112: evaluation of [TO_DATETIME(string)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"java.lang.IllegalArgumentException: failed to parse date field [1964-06-02 00:00:00] with format [yyyy-MM-dd'T'HH:mm:ss.SSS'Z']"` + + +If the input parameter is of a numeric type, its value will be interpreted as +milliseconds since the https://en.wikipedia.org/wiki/Unix_time[Unix epoch]. +For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/date.csv-spec[tag=to_datetime-int] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/date.csv-spec[tag=to_datetime-int-result] +|=== + +Alias: TO_DT diff --git a/docs/reference/esql/functions/to_degrees.asciidoc b/docs/reference/esql/functions/to_degrees.asciidoc new file mode 100644 index 0000000000000..6df4b9cee32cd --- /dev/null +++ b/docs/reference/esql/functions/to_degrees.asciidoc @@ -0,0 +1,18 @@ +[[esql-to_degrees]] +=== `TO_DEGREES` +Converts a number in https://en.wikipedia.org/wiki/Radian[radians] +to https://en.wikipedia.org/wiki/Degree_(angle)[degrees]. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a numeric type and result is always `double`. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=to_degrees] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=to_degrees-result] +|=== diff --git a/docs/reference/esql/functions/to_double.asciidoc b/docs/reference/esql/functions/to_double.asciidoc new file mode 100644 index 0000000000000..abc4217d36b5a --- /dev/null +++ b/docs/reference/esql/functions/to_double.asciidoc @@ -0,0 +1,37 @@ +[[esql-to_double]] +=== `TO_DOUBLE` +Converts an input value to a double value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a boolean, date, string or numeric type. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=to_double-str] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=to_double-str-result] +|=== + +Note that in this example, the last conversion of the string isn't +possible. When this happens, the result is a *null* value. In this case a +_Warning_ header is added to the response. The header will provide information +on the source of the failure: + +`"Line 1:115: evaluation of [TO_DOUBLE(str2)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"java.lang.NumberFormatException: For input string: \"foo\""` + + +If the input parameter is of a date type, its value will be interpreted as +milliseconds since the https://en.wikipedia.org/wiki/Unix_time[Unix epoch], +converted to double. + +Boolean *true* will be converted to double *1.0*, *false* to *0.0*. + +Alias: TO_DBL diff --git a/docs/reference/esql/functions/to_integer.asciidoc b/docs/reference/esql/functions/to_integer.asciidoc new file mode 100644 index 0000000000000..d90649d1aceee --- /dev/null +++ b/docs/reference/esql/functions/to_integer.asciidoc @@ -0,0 +1,37 @@ +[[esql-to_integer]] +=== `TO_INTEGER` +Converts an input value to an integer value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a boolean, date, string or numeric type. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/ints.csv-spec[tag=to_int-long] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/ints.csv-spec[tag=to_int-long-result] +|=== + +Note that in this example, the last value of the multi-valued field cannot +be converted as an integer. When this happens, the result is a *null* value. +In this case a _Warning_ header is added to the response. The header will +provide information on the source of the failure: + +`"Line 1:61: evaluation of [TO_INTEGER(long)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"org.elasticsearch.xpack.ql.QlIllegalArgumentException: [501379200000] out of [integer] range"` + + +If the input parameter is of a date type, its value will be interpreted as +milliseconds since the https://en.wikipedia.org/wiki/Unix_time[Unix epoch], +converted to integer. + +Boolean *true* will be converted to integer *1*, *false* to *0*. + +Alias: TO_INT diff --git a/docs/reference/esql/functions/to_ip.asciidoc b/docs/reference/esql/functions/to_ip.asciidoc new file mode 100644 index 0000000000000..8deeb70f611a8 --- /dev/null +++ b/docs/reference/esql/functions/to_ip.asciidoc @@ -0,0 +1,27 @@ +[[esql-to_ip]] +=== `TO_IP` +Converts an input string to an IP value. + +The input can be a single- or multi-valued field or an expression. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/ip.csv-spec[tag=to_ip] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/ip.csv-spec[tag=to_ip-result] +|=== + +Note that in the example above the last conversion of the string isn't +possible. When this happens, the result is a *null* value. In this case a +_Warning_ header is added to the response. The header will provide information +on the source of the failure: + +`"Line 1:68: evaluation of [TO_IP(str2)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"java.lang.IllegalArgumentException: 'foo' is not an IP string literal."` diff --git a/docs/reference/esql/functions/to_long.asciidoc b/docs/reference/esql/functions/to_long.asciidoc new file mode 100644 index 0000000000000..6d5c7be279ec3 --- /dev/null +++ b/docs/reference/esql/functions/to_long.asciidoc @@ -0,0 +1,35 @@ +[[esql-to_long]] +=== `TO_LONG` +Converts an input value to a long value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a boolean, date, string or numeric type. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/ints.csv-spec[tag=to_long-str] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/ints.csv-spec[tag=to_long-str-result] +|=== + +Note that in this example, the last conversion of the string isn't +possible. When this happens, the result is a *null* value. In this case a +_Warning_ header is added to the response. The header will provide information +on the source of the failure: + +`"Line 1:113: evaluation of [TO_LONG(str3)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"java.lang.NumberFormatException: For input string: \"foo\""` + + +If the input parameter is of a date type, its value will be interpreted as +milliseconds since the https://en.wikipedia.org/wiki/Unix_time[Unix epoch], +converted to long. + +Boolean *true* will be converted to long *1*, *false* to *0*. diff --git a/docs/reference/esql/functions/to_radians.asciidoc b/docs/reference/esql/functions/to_radians.asciidoc new file mode 100644 index 0000000000000..9ad5964e880d5 --- /dev/null +++ b/docs/reference/esql/functions/to_radians.asciidoc @@ -0,0 +1,18 @@ +[[esql-to_radians]] +=== `TO_RADIANS` +Converts a number in https://en.wikipedia.org/wiki/Degree_(angle)[degrees] to +https://en.wikipedia.org/wiki/Radian[radians]. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a numeric type and result is always `double`. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/floats.csv-spec[tag=to_radians] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/floats.csv-spec[tag=to_radians-result] +|=== diff --git a/docs/reference/esql/functions/to_string.asciidoc b/docs/reference/esql/functions/to_string.asciidoc new file mode 100644 index 0000000000000..d5a08a6398189 --- /dev/null +++ b/docs/reference/esql/functions/to_string.asciidoc @@ -0,0 +1,25 @@ +[[esql-to_string]] +=== `TO_STRING` +Converts a field into a string. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=to_string] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=to_string-result] +|=== + +It also works fine on multivalued fields: + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=to_string_multivalue] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=to_string_multivalue-result] +|=== + +Alias: TO_STR diff --git a/docs/reference/esql/functions/to_unsigned_long.asciidoc b/docs/reference/esql/functions/to_unsigned_long.asciidoc new file mode 100644 index 0000000000000..ebbf8079a5875 --- /dev/null +++ b/docs/reference/esql/functions/to_unsigned_long.asciidoc @@ -0,0 +1,37 @@ +[[esql-to_unsigned_long]] +=== `TO_UNSIGNED_LONG` +Converts an input value to an unsigned long value. + +The input can be a single- or multi-valued field or an expression. The input +type must be of a boolean, date, string or numeric type. + +Example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/ints.csv-spec[tag=to_unsigned_long-str] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/ints.csv-spec[tag=to_unsigned_long-str-result] +|=== + +Note that in this example, the last conversion of the string isn't +possible. When this happens, the result is a *null* value. In this case a +_Warning_ header is added to the response. The header will provide information +on the source of the failure: + +`"Line 1:133: evaluation of [TO_UL(str3)] failed, treating result as null. Only first 20 failures recorded."` + +A following header will contain the failure reason and the offending value: + +`"java.lang.NumberFormatException: Character f is neither a decimal digit number, decimal point, nor \"e\" notation exponential mark."` + + +If the input parameter is of a date type, its value will be interpreted as +milliseconds since the https://en.wikipedia.org/wiki/Unix_time[Unix epoch], +converted to unsigned long. + +Boolean *true* will be converted to unsigned long *1*, *false* to *0*. + +Alias: TO_ULONG, TO_UL diff --git a/docs/reference/esql/functions/to_version.asciidoc b/docs/reference/esql/functions/to_version.asciidoc new file mode 100644 index 0000000000000..7ca42b89aa6e6 --- /dev/null +++ b/docs/reference/esql/functions/to_version.asciidoc @@ -0,0 +1,16 @@ +[[esql-to_version]] +=== `TO_VERSION` +Converts an input string to a version value. For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/version.csv-spec[tag=to_version] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/version.csv-spec[tag=to_version-result] +|=== + +The input can be a single- or multi-valued field or an expression. + +Alias: TO_VER diff --git a/docs/reference/esql/functions/trim.asciidoc b/docs/reference/esql/functions/trim.asciidoc new file mode 100644 index 0000000000000..0b6cf96ed1c5a --- /dev/null +++ b/docs/reference/esql/functions/trim.asciidoc @@ -0,0 +1,12 @@ +[[esql-trim]] +=== `TRIM` +Removes leading and trailing whitespaces from strings. + +[source.merge.styled,esql] +---- +include::{esql-specs}/string.csv-spec[tag=trim] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/string.csv-spec[tag=trim-result] +|=== diff --git a/docs/reference/esql/index.asciidoc b/docs/reference/esql/index.asciidoc new file mode 100644 index 0000000000000..45b4f664db47f --- /dev/null +++ b/docs/reference/esql/index.asciidoc @@ -0,0 +1,142 @@ +[[esql]] += {esql} + +:esql-tests: {xes-repo-dir}/../../plugin/esql/qa +:esql-specs: {esql-tests}/testFixtures/src/main/resources + +[partintro] +-- + +preview::[] + +The {es} Query Language ({esql}) is a query language that enables the iterative +exploration of data. + +An {esql} query consists of a series of commands, separated by pipes. Each query +starts with a <>. A source command produces +a table, typically with data from {es}. + +image::images/esql/source-command.svg[A source command producing a table from {es},align="center"] + +A source command can be followed by one or more +<>. Processing commands change an +input table by adding, removing, or changing rows and columns. + +image::images/esql/processing-command.svg[A processing command changing an input table,align="center"] + +You can chain processing commands, separated by a pipe character: `|`. Each +processing command works on the output table of the previous command. + +image::images/esql/chaining-processing-commands.svg[Processing commands can be chained,align="center"] + +The result of a query is the table produced by the final processing command. + +[discrete] +[[esql-console]] +=== Run an {esql} query + +[discrete] +==== The {esql} API + +Use the `_query` endpoint to run an {esql} query: + +[source,console] +---- +POST /_query +{ + "query": """ + FROM library + | EVAL year = DATE_TRUNC(release_date, 1 YEARS) + | STATS MAX(page_count) BY year + | SORT year + | LIMIT 5 + """ +} +---- +// TEST[setup:library] + +The results come back in rows: + +[source,console-result] +---- +{ + "columns": [ + { "name": "MAX(page_count)", "type": "integer"}, + { "name": "year" , "type": "date"} + ], + "values": [ + [268, "1932-01-01T00:00:00.000Z"], + [224, "1951-01-01T00:00:00.000Z"], + [227, "1953-01-01T00:00:00.000Z"], + [335, "1959-01-01T00:00:00.000Z"], + [604, "1965-01-01T00:00:00.000Z"] + ] +} +---- + +By default, results are returned as JSON. To return results formatted as text, +CSV, or TSV, use the `format` parameter: + +[source,console] +---- +POST /_query?format=txt +{ + "query": """ + FROM library + | EVAL year = DATE_TRUNC(release_date, 1 YEARS) + | STATS MAX(page_count) BY year + | SORT year + | LIMIT 5 + """ +} +---- +// TEST[setup:library] + +[discrete] +==== {kib} + +{esql} can be used in Discover to explore a data set, and in Lens to visualize it. +First, enable the `enableTextBased` setting in *Advanced Settings*. Next, in +Discover or Lens, from the data view dropdown, select *{esql}*. + +NOTE: {esql} queries in Discover and Lens are subject to the time range selected +with the time filter. + +[discrete] +[[esql-limitations]] +=== Limitations + +{esql} currently supports the following <>: + +- `alias` +- `boolean` +- `date` +- `double` (`float`, `half_float`, `scaled_float` are represented as `double`) +- `ip` +- `keyword` family including `keyword`, `constant_keyword`, and `wildcard` +- `int` (`short` and `byte` are represented as `int`) +- `long` +- `null` +- `text` +- `unsigned_long` +- `version` +-- + +include::esql-get-started.asciidoc[] + +include::esql-syntax.asciidoc[] + +include::esql-source-commands.asciidoc[] + +include::esql-processing-commands.asciidoc[] + +include::esql-functions.asciidoc[] + +include::aggregation-functions.asciidoc[] + +include::multivalued-fields.asciidoc[] + +include::task-management.asciidoc[] + +:esql-tests!: +:esql-specs!: diff --git a/docs/reference/esql/multivalued-fields.asciidoc b/docs/reference/esql/multivalued-fields.asciidoc new file mode 100644 index 0000000000000..e304c25bae0a9 --- /dev/null +++ b/docs/reference/esql/multivalued-fields.asciidoc @@ -0,0 +1,240 @@ +[[esql-multivalued-fields]] +== {esql} multivalued fields + +++++ +Multivalued fields +++++ + +{esql} is fine reading from multivalued fields: + +[source,console,id=esql-multivalued-fields-reorders] +---- +POST /mv/_bulk?refresh +{ "index" : {} } +{ "a": 1, "b": [2, 1] } +{ "index" : {} } +{ "a": 2, "b": 3 } + +POST /_query +{ + "query": "FROM mv" +} +---- + +Multivalued fields come back as a JSON array: + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "long"} + ], + "values": [ + [1, [1, 2]], + [2, 3] + ] +} +---- + +The relative order of values in a multivalued field is undefined. They'll frequently be in +ascending order but don't rely on that. + +[discrete] +[[esql-multivalued-fields-dups]] +==== Duplicate values + +Some field types, like <> remove duplicate values on write: + +[source,console,id=esql-multivalued-fields-kwdups] +---- +PUT /mv +{ + "mappings": { + "properties": { + "b": {"type": "keyword"} + } + } +} + +POST /mv/_bulk?refresh +{ "index" : {} } +{ "a": 1, "b": ["foo", "foo", "bar"] } +{ "index" : {} } +{ "a": 2, "b": ["bar", "bar"] } + +POST /_query +{ + "query": "FROM mv" +} +---- + +And {esql} sees that removal: + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "keyword"} + ], + "values": [ + [1, ["bar", "foo"]], + [2, "bar"] + ] +} +---- + +But other types, like `long` don't remove duplicates. + +[source,console,id=esql-multivalued-fields-longdups] +---- +PUT /mv +{ + "mappings": { + "properties": { + "b": {"type": "long"} + } + } +} + +POST /mv/_bulk?refresh +{ "index" : {} } +{ "a": 1, "b": [2, 2, 1] } +{ "index" : {} } +{ "a": 2, "b": [1, 1] } + +POST /_query +{ + "query": "FROM mv" +} +---- + +And {esql} also sees that: + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "long"} + ], + "values": [ + [1, [1, 2, 2]], + [2, [1, 1]] + ] +} +---- + +This is all at the storage layer. If you store duplicate `long`s and then +convert them to strings the duplicates will stay: + +[source,console,id=esql-multivalued-fields-longdups-tostring] +---- +PUT /mv +{ + "mappings": { + "properties": { + "b": {"type": "long"} + } + } +} + +POST /mv/_bulk?refresh +{ "index" : {} } +{ "a": 1, "b": [2, 2, 1] } +{ "index" : {} } +{ "a": 2, "b": [1, 1] } + +POST /_query +{ + "query": "FROM mv | EVAL b=TO_STRING(b)" +} +---- + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "keyword"} + ], + "values": [ + [1, ["1", "2", "2"]], + [2, ["1", "1"]] + ] +} +---- + +[discrete] +[[esql-multivalued-fields-functions]] +==== Functions + +Unless otherwise documented functions will return `null` when applied to a multivalued +field. This behavior may change in a later version. + +[source,console,id=esql-multivalued-fields-mv-into-null] +---- +POST /mv/_bulk?refresh +{ "index" : {} } +{ "a": 1, "b": [2, 1] } +{ "index" : {} } +{ "a": 2, "b": 3 } + +POST /_query +{ + "query": "FROM mv | EVAL b + 2, a + b" +} +---- + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "long"}, + { "name": "b+2", "type": "long"}, + { "name": "a+b", "type": "long"} + ], + "values": [ + [1, [1, 2], null, null], + [2, 3, 5, 5] + ] +} +---- + +Work around this limitation by converting the field to single value with one of: + +* <> +* <> +* <> +* <> +* <> +* <> +* <> + +[source,console,esql-multivalued-fields-mv-into-null] +---- +POST /_query +{ + "query": "FROM mv | EVAL b=MV_MIN(b) | EVAL b + 2, a + b" +} +---- +// TEST[continued] + +[source,console-result] +---- +{ + "columns": [ + { "name": "a", "type": "long"}, + { "name": "b", "type": "long"}, + { "name": "b+2", "type": "long"}, + { "name": "a+b", "type": "long"} + ], + "values": [ + [1, 1, 3, 2], + [2, 3, 5, 5] + ] +} +---- + diff --git a/docs/reference/esql/processing-commands/dissect.asciidoc b/docs/reference/esql/processing-commands/dissect.asciidoc new file mode 100644 index 0000000000000..93db2ac9f3460 --- /dev/null +++ b/docs/reference/esql/processing-commands/dissect.asciidoc @@ -0,0 +1,18 @@ +[[esql-dissect]] +=== `DISSECT` + +`DISSECT` enables you to extract structured data out of a string. `DISSECT` +matches the string against a delimiter-based pattern, and extracts the specified +keys as columns. + +Refer to the <> for the +syntax of dissect patterns. + +[source.merge.styled,esql] +---- +include::{esql-specs}/dissect.csv-spec[tag=dissect] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/dissect.csv-spec[tag=dissect-result] +|=== diff --git a/docs/reference/esql/processing-commands/drop.asciidoc b/docs/reference/esql/processing-commands/drop.asciidoc new file mode 100644 index 0000000000000..28573d968103b --- /dev/null +++ b/docs/reference/esql/processing-commands/drop.asciidoc @@ -0,0 +1,17 @@ +[[esql-drop]] +=== `DROP` + +Use `DROP` to remove columns: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=dropheight] +---- + +Rather than specify each column by name, you can use wildcards to drop all +columns with a name that matches a pattern: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=dropheightwithwildcard] +---- diff --git a/docs/reference/esql/processing-commands/enrich.asciidoc b/docs/reference/esql/processing-commands/enrich.asciidoc new file mode 100644 index 0000000000000..520d4dc8765c9 --- /dev/null +++ b/docs/reference/esql/processing-commands/enrich.asciidoc @@ -0,0 +1,60 @@ +[[esql-enrich]] +=== `ENRICH` +You can use `ENRICH` to add data from your existing indices to incoming records. +It's similar to <>, but it works at query time. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich-result] +|=== + +`ENRICH` requires an <> to be executed. +The enrich policy defines a match field (a key field) and a set of enrich fields. + +`ENRICH` will look for records in the <> based on the match field value. +The matching key in the input dataset can be defined using `ON `; if it's not specified, +the match will be performed on a field with the same name as the match field defined in the <>. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_on] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_on-result] +|=== + + +You can specify which attributes (between those defined as enrich fields in the policy) have to be added to the result, +using `WITH , ...` syntax. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_with] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_with-result] +|=== + + +Attributes can also be renamed using `WITH new_name=` + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_rename] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs-ignoreCsvTests.csv-spec[tag=enrich_rename-result] +|=== + + +By default (if no `WITH` is defined), `ENRICH` will add all the enrich fields defined in the <> +to the result. + +In case of name collisions, the newly created fields will override the existing fields. diff --git a/docs/reference/esql/processing-commands/eval.asciidoc b/docs/reference/esql/processing-commands/eval.asciidoc new file mode 100644 index 0000000000000..c53a1b76a0656 --- /dev/null +++ b/docs/reference/esql/processing-commands/eval.asciidoc @@ -0,0 +1,29 @@ +[[esql-eval]] +=== `EVAL` +`EVAL` enables you to append new columns: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=eval] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=eval-result] +|=== + +If the specified column already exists, the existing column will be dropped, and +the new column will be appended to the table: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=evalReplace] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=evalReplace-result] +|=== + +[discrete] +==== Functions +`EVAL` supports various functions for calculating values. Refer to +<> for more information. diff --git a/docs/reference/esql/processing-commands/grok.asciidoc b/docs/reference/esql/processing-commands/grok.asciidoc new file mode 100644 index 0000000000000..bda726202d9cd --- /dev/null +++ b/docs/reference/esql/processing-commands/grok.asciidoc @@ -0,0 +1,20 @@ +[[esql-grok]] +=== `GROK` + +`GROK` enables you to extract structured data out of a string. `GROK` matches +the string against patterns, based on regular expressions, and extracts the +specified patterns as columns. + +Refer to the <> for the syntax for +of grok patterns. + +For example: + +[source.merge.styled,esql] +---- +include::{esql-specs}/grok.csv-spec[tag=grok] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/grok.csv-spec[tag=grok-result] +|=== diff --git a/docs/reference/esql/processing-commands/keep.asciidoc b/docs/reference/esql/processing-commands/keep.asciidoc new file mode 100644 index 0000000000000..dfa7ca701e67a --- /dev/null +++ b/docs/reference/esql/processing-commands/keep.asciidoc @@ -0,0 +1,34 @@ +[[esql-keep]] +=== `KEEP` + +The `KEEP` command enables you to specify what columns are returned and the +order in which they are returned. + +To limit the columns that are returned, use a comma-separated list of column +names. The columns are returned in the specified order: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=keep] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=keep-result] +|=== + +Rather than specify each column by name, you can use wildcards to return all +columns with a name that matches a pattern: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=keepWildcard] +---- + +The asterisk wildcard (`*`) by itself translates to all columns that do not +match the other arguments. This query will first return all columns with a name +that starts with an h, followed by all other columns: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=keepDoubleWildcard] +---- diff --git a/docs/reference/esql/processing-commands/limit.asciidoc b/docs/reference/esql/processing-commands/limit.asciidoc new file mode 100644 index 0000000000000..963ea2eea37ce --- /dev/null +++ b/docs/reference/esql/processing-commands/limit.asciidoc @@ -0,0 +1,9 @@ +[[esql-limit]] +=== `LIMIT` + +The `LIMIT` processing command enables you to limit the number of rows: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=limit] +---- diff --git a/docs/reference/esql/processing-commands/mv_expand.asciidoc b/docs/reference/esql/processing-commands/mv_expand.asciidoc new file mode 100644 index 0000000000000..f8656888403a5 --- /dev/null +++ b/docs/reference/esql/processing-commands/mv_expand.asciidoc @@ -0,0 +1,13 @@ +[[esql-mv_expand]] +=== `MV_EXPAND` + +The `MV_EXPAND` processing command expands multivalued fields into one row per value, duplicating other fields: + +[source.merge.styled,esql] +---- +include::{esql-specs}/mv_expand.csv-spec[tag=simple] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/mv_expand.csv-spec[tag=simple-result] +|=== diff --git a/docs/reference/esql/processing-commands/rename.asciidoc b/docs/reference/esql/processing-commands/rename.asciidoc new file mode 100644 index 0000000000000..646036ccc3d12 --- /dev/null +++ b/docs/reference/esql/processing-commands/rename.asciidoc @@ -0,0 +1,26 @@ +[[esql-rename]] +=== `RENAME` + +Use `RENAME` to rename a column using the following syntax: + +[source,esql] +---- +RENAME AS +---- + +For example: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=rename] +---- + +If a column with the new name already exists, it will be replaced by the new +column. + +Multiple columns can be renamed with a single `RENAME` command: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=renameMultipleColumns] +---- diff --git a/docs/reference/esql/processing-commands/sort.asciidoc b/docs/reference/esql/processing-commands/sort.asciidoc new file mode 100644 index 0000000000000..6da782a4acd07 --- /dev/null +++ b/docs/reference/esql/processing-commands/sort.asciidoc @@ -0,0 +1,36 @@ +[[esql-sort]] +=== `SORT` +Use the `SORT` command to sort rows on one or more fields: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=sort] +---- + +The default sort order is ascending. Set an explicit sort order using `ASC` or +`DESC`: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=sortDesc] +---- + +Two rows with the same sort key are considered equal. You can provide additional +sort expressions to act as tie breakers: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=sortTie] +---- + +[discrete] +==== `null` values +By default, `null` values are treated as being larger than any other value. With +an ascending sort order, `null` values are sorted last, and with a descending +sort order, `null` values are sorted first. You can change that by providing +`NULLS FIRST` or `NULLS LAST`: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=sortNullsFirst] +---- diff --git a/docs/reference/esql/processing-commands/stats.asciidoc b/docs/reference/esql/processing-commands/stats.asciidoc new file mode 100644 index 0000000000000..84529db17e098 --- /dev/null +++ b/docs/reference/esql/processing-commands/stats.asciidoc @@ -0,0 +1,44 @@ +[[esql-stats-by]] +=== `STATS ... BY` +Use `STATS ... BY` to group rows according to a common value and calculate one +or more aggregated values over the grouped rows. + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=stats] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=stats-result] +|=== + +If `BY` is omitted, the output table contains exactly one row with the +aggregations applied over the entire dataset: + +[source.merge.styled,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=statsWithoutBy] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/docs.csv-spec[tag=statsWithoutBy-result] +|=== + +It's possible to calculate multiple values: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=statsCalcMultipleValues] +---- + +It's also possible to group by multiple values (only supported for long and +keyword family fields): + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=statsGroupByMultipleValues] +---- + +The following aggregation functions are supported: + +include::../aggregation-functions.asciidoc[tag=functions] diff --git a/docs/reference/esql/processing-commands/where.asciidoc b/docs/reference/esql/processing-commands/where.asciidoc new file mode 100644 index 0000000000000..e6e3e1059cb34 --- /dev/null +++ b/docs/reference/esql/processing-commands/where.asciidoc @@ -0,0 +1,32 @@ +[[esql-where]] +=== `WHERE` + +Use `WHERE` to produce a table that contains all the rows from the input table +for which the provided condition evaluates to `true`: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=where] +---- + +Which, if `still_hired` is a boolean field, can be simplified to: + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=whereBoolean] +---- + +[discrete] +==== Operators + +Refer to <> for an overview of the supported operators. + +[discrete] +==== Functions +`WHERE` supports various functions for calculating values. Refer to +<> for more information. + +[source,esql] +---- +include::{esql-specs}/docs.csv-spec[tag=whereFunction] +---- diff --git a/docs/reference/esql/source-commands/from.asciidoc b/docs/reference/esql/source-commands/from.asciidoc new file mode 100644 index 0000000000000..64bd6f8c8dd88 --- /dev/null +++ b/docs/reference/esql/source-commands/from.asciidoc @@ -0,0 +1,29 @@ +[[esql-from]] +=== `FROM` + +The `FROM` source command returns a table with up to 10,000 documents from a +data stream, index, or alias. Each row in the resulting table represents a +document. Each column corresponds to a field, and can be accessed by the name +of that field. + +[source,esql] +---- +FROM employees +---- + +You can use <> to refer to indices, aliases +and data streams. This can be useful for time series data, for example to access +today's index: + +[source,esql] +---- +FROM +---- + +Use comma-separated lists or wildcards to query multiple data streams, indices, +or aliases: + +[source,esql] +---- +FROM employees-00001,employees-* +---- diff --git a/docs/reference/esql/source-commands/row.asciidoc b/docs/reference/esql/source-commands/row.asciidoc new file mode 100644 index 0000000000000..44ab65024b910 --- /dev/null +++ b/docs/reference/esql/source-commands/row.asciidoc @@ -0,0 +1,28 @@ +[[esql-row]] +=== `ROW` + +The `ROW` source command produces a row with one or more columns with values +that you specify. This can be useful for testing. + +[source.merge.styled,esql] +---- +include::{esql-specs}/row.csv-spec[tag=example] +---- +[%header.monospaced.styled,format=dsv,separator=|] +|=== +include::{esql-specs}/row.csv-spec[tag=example-result] +|=== + +Use square brackets to create multi-value columns: + +[source,esql] +---- +include::{esql-specs}/row.csv-spec[tag=multivalue] +---- + +`ROW` supports the use of <>: + +[source,esql] +---- +include::{esql-specs}/row.csv-spec[tag=function] +---- diff --git a/docs/reference/esql/source-commands/show.asciidoc b/docs/reference/esql/source-commands/show.asciidoc new file mode 100644 index 0000000000000..84614cfe9396d --- /dev/null +++ b/docs/reference/esql/source-commands/show.asciidoc @@ -0,0 +1,9 @@ +[[esql-show]] +=== `SHOW ` + +The `SHOW ` source command returns information about the deployment and +its capabilities: + +* Use `SHOW INFO` to return the deployment's version, build date and hash. +* Use `SHOW FUNCTIONS` to return a list of all supported functions and a +synopsis of each function. diff --git a/docs/reference/esql/task-management.asciidoc b/docs/reference/esql/task-management.asciidoc new file mode 100644 index 0000000000000..bc06e70f24bd7 --- /dev/null +++ b/docs/reference/esql/task-management.asciidoc @@ -0,0 +1,35 @@ +[[esql-task-management]] +== {esql} task management + +++++ +Task management +++++ + +You can get running {esql} queries with the <>: + +[source,console,id=esql-task-management-get-all] +---- +GET /_tasks?pretty&detailed&group_by=parents&human&actions=*data/read/esql +---- + +Which returns a list of statuses like this: + +[source,js] +---- +include::{esql-specs}/query_task.json[] +---- +// NOTCONSOLE +// Tested in a unit test + +<1> The user submitted query. +<2> Time the query has been running. + +You can use this to find long running queries and, if you need to, cancel them +with the <>: + +[source,console,id=esql-task-management-cancelEsqlQueryRequestTests] +---- +POST _tasks/2j8UKw1bRO283PMwDugNNg:5326/_cancel +---- + +It may take a few seconds for the query to be stopped. diff --git a/docs/reference/images/esql/chaining-processing-commands.svg b/docs/reference/images/esql/chaining-processing-commands.svg new file mode 100644 index 0000000000000..d163eaad5fb1e --- /dev/null +++ b/docs/reference/images/esql/chaining-processing-commands.svg @@ -0,0 +1,332 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/reference/images/esql/processing-command.svg b/docs/reference/images/esql/processing-command.svg new file mode 100644 index 0000000000000..aa161b850148d --- /dev/null +++ b/docs/reference/images/esql/processing-command.svg @@ -0,0 +1,207 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/reference/images/esql/source-command.svg b/docs/reference/images/esql/source-command.svg new file mode 100644 index 0000000000000..ebdb6af6785d8 --- /dev/null +++ b/docs/reference/images/esql/source-command.svg @@ -0,0 +1,109 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/reference/index.asciidoc b/docs/reference/index.asciidoc index e107bfe0366f7..e3d84c91c4d28 100644 --- a/docs/reference/index.asciidoc +++ b/docs/reference/index.asciidoc @@ -45,6 +45,8 @@ include::geospatial-analysis.asciidoc[] include::eql/eql.asciidoc[] +include::esql/index.asciidoc[] + include::sql/index.asciidoc[] include::scripting.asciidoc[] diff --git a/docs/reference/rest-api/info.asciidoc b/docs/reference/rest-api/info.asciidoc index 27ba1bbea6501..31daa86d2c423 100644 --- a/docs/reference/rest-api/info.asciidoc +++ b/docs/reference/rest-api/info.asciidoc @@ -109,6 +109,10 @@ Example response: "available" : true, "enabled" : true }, + "esql" : { + "available" : true, + "enabled" : true + }, "monitoring" : { "available" : true, "enabled" : true diff --git a/docs/reference/rest-api/usage.asciidoc b/docs/reference/rest-api/usage.asciidoc index 2cbc2ce3fc747..14a92eca519f4 100644 --- a/docs/reference/rest-api/usage.asciidoc +++ b/docs/reference/rest-api/usage.asciidoc @@ -203,6 +203,33 @@ GET /_xpack/usage "available" : true, "enabled" : true }, + "esql" : { + "available" : true, + "enabled" : true, + "features" : { + "eval" : 0, + "stats" : 0, + "dissect": 0, + "grok" : 0, + "limit" : 0, + "where" : 0, + "sort" : 0 + }, + "queries" : { + "rest" : { + "total" : 0, + "failed" : 0 + }, + "kibana" : { + "total" : 0, + "failed" : 0 + }, + "_all" : { + "total" : 0, + "failed" : 0 + } + } + }, "sql" : { "available" : true, "enabled" : true, diff --git a/gradle/verification-metadata.xml b/gradle/verification-metadata.xml index ce9661d635ccf..dad9bfcd1efee 100644 --- a/gradle/verification-metadata.xml +++ b/gradle/verification-metadata.xml @@ -821,6 +821,11 @@ + + + + + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/esql.query.json b/rest-api-spec/src/main/resources/rest-api-spec/api/esql.query.json new file mode 100644 index 0000000000000..ffcd30fa6c717 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/esql.query.json @@ -0,0 +1,39 @@ +{ + "esql.query":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/esql-query-api.html", + "description":"Executes an ESQL request" + }, + "stability":"experimental", + "visibility":"private", + "headers":{ + "accept": [ "application/json"], + "content_type": ["application/json"] + }, + "url":{ + "paths":[ + { + "path":"/_query", + "methods":[ + "POST" + ] + } + ] + }, + "params":{ + "format":{ + "type":"string", + "description":"a short version of the Accept header, e.g. json, yaml" + }, + "delimiter":{ + "type":"string", + "description":"The character to use between values within a CSV row. Only valid for the csv format.", + "default":false + } + }, + "body":{ + "description":"Use the `query` element to start a query. Use `time_zone` to specify an execution time zone and 'columnar' to format the answer.", + "required":true + } + } +} diff --git a/server/src/main/java/org/elasticsearch/TransportVersion.java b/server/src/main/java/org/elasticsearch/TransportVersion.java index 8f0f95f9e6f0d..4897e009db8d0 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersion.java +++ b/server/src/main/java/org/elasticsearch/TransportVersion.java @@ -175,6 +175,7 @@ private static TransportVersion registerTransportVersion(int id, String uniqueId public static final TransportVersion V_8_500_060 = registerTransportVersion(8_500_060, "ec065a44-b468-4f8a-aded-7b90ca8d792b"); // 8.10.0 release version is: public static final TransportVersion V_8_500_061 = registerTransportVersion(8_500_061, "4e07f830-8be4-448c-851e-62b3d2f0bf0a"); + public static final TransportVersion V_8_500_062 = registerTransportVersion(8_500_062, "09CD9C9B-3207-4B40-8756-B7A12001A885"); /* * STOP! READ THIS FIRST! No, really, * ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _ @@ -197,7 +198,7 @@ private static TransportVersion registerTransportVersion(int id, String uniqueId */ private static class CurrentHolder { - private static final TransportVersion CURRENT = findCurrent(V_8_500_061); + private static final TransportVersion CURRENT = findCurrent(V_8_500_062); // finds the pluggable current version, or uses the given fallback private static TransportVersion findCurrent(TransportVersion fallback) { diff --git a/x-pack/plugin/core/src/main/java/module-info.java b/x-pack/plugin/core/src/main/java/module-info.java index bb397ffe1699f..3a869aca80c4f 100644 --- a/x-pack/plugin/core/src/main/java/module-info.java +++ b/x-pack/plugin/core/src/main/java/module-info.java @@ -62,6 +62,7 @@ exports org.elasticsearch.xpack.core.enrich.action; exports org.elasticsearch.xpack.core.enrich; exports org.elasticsearch.xpack.core.eql; + exports org.elasticsearch.xpack.core.esql; exports org.elasticsearch.xpack.core.frozen.action; exports org.elasticsearch.xpack.core.frozen; exports org.elasticsearch.xpack.core.graph.action; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index dd3fb982b6478..7e2c9dd494086 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -45,6 +45,7 @@ import org.elasticsearch.xpack.core.enrich.EnrichFeatureSetUsage; import org.elasticsearch.xpack.core.enrich.action.ExecuteEnrichPolicyStatus; import org.elasticsearch.xpack.core.eql.EqlFeatureSetUsage; +import org.elasticsearch.xpack.core.esql.EsqlFeatureSetUsage; import org.elasticsearch.xpack.core.frozen.FrozenIndicesFeatureSetUsage; import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction; import org.elasticsearch.xpack.core.graph.GraphFeatureSetUsage; @@ -448,6 +449,8 @@ public List getNamedWriteables() { new NamedWriteableRegistry.Entry(RoleMapperExpression.class, ExceptExpression.NAME, ExceptExpression::new), // eql new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.EQL, EqlFeatureSetUsage::new), + // esql + new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.ESQL, EsqlFeatureSetUsage::new), // sql new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.SQL, SqlFeatureSetUsage::new), // watcher diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java index 5f74699aa061a..f78f755517d99 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackField.java @@ -30,6 +30,8 @@ public final class XPackField { public static final String SETTINGS_NAME = "xpack"; /** Name constant for the eql feature. */ public static final String EQL = "eql"; + /** Name constant for the esql feature. */ + public static final String ESQL = "esql"; /** Name constant for the sql feature. */ public static final String SQL = "sql"; /** Name constant for the rollup feature. */ diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureAction.java index 311502c61ece3..1f943a5c68646 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackInfoFeatureAction.java @@ -32,6 +32,7 @@ public class XPackInfoFeatureAction extends ActionType public static final XPackInfoFeatureAction MACHINE_LEARNING = new XPackInfoFeatureAction(XPackField.MACHINE_LEARNING); public static final XPackInfoFeatureAction LOGSTASH = new XPackInfoFeatureAction(XPackField.LOGSTASH); public static final XPackInfoFeatureAction EQL = new XPackInfoFeatureAction(XPackField.EQL); + public static final XPackInfoFeatureAction ESQL = new XPackInfoFeatureAction(XPackField.ESQL); public static final XPackInfoFeatureAction SQL = new XPackInfoFeatureAction(XPackField.SQL); public static final XPackInfoFeatureAction ROLLUP = new XPackInfoFeatureAction(XPackField.ROLLUP); public static final XPackInfoFeatureAction INDEX_LIFECYCLE = new XPackInfoFeatureAction(XPackField.INDEX_LIFECYCLE); @@ -62,6 +63,7 @@ public class XPackInfoFeatureAction extends ActionType MACHINE_LEARNING, LOGSTASH, EQL, + ESQL, SQL, ROLLUP, INDEX_LIFECYCLE, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java index 3ac4cdd1e90f4..a63d535e2a06c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/XPackUsageFeatureAction.java @@ -29,6 +29,7 @@ public class XPackUsageFeatureAction extends ActionType stats; + + public EsqlFeatureSetUsage(StreamInput in) throws IOException { + super(in); + stats = in.readMap(); + } + + public EsqlFeatureSetUsage(Map stats) { + this(true, true, stats); + } + + private EsqlFeatureSetUsage(boolean available, boolean enabled, Map stats) { + super(XPackField.ESQL, available, enabled); + this.stats = stats; + } + + /** Returns a feature set usage where the feature is not available or enabled, and has an empty stats. */ + public static EsqlFeatureSetUsage unavailable() { + return new EsqlFeatureSetUsage(false, false, Map.of()); + } + + public Map stats() { + return stats; + } + + @Override + protected void innerXContent(XContentBuilder builder, Params params) throws IOException { + super.innerXContent(builder, params); + if (enabled) { + for (Map.Entry entry : stats.entrySet()) { + builder.field(entry.getKey(), entry.getValue()); + } + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeGenericMap(stats); + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return TransportVersion.V_8_500_062; + } + +} diff --git a/x-pack/plugin/esql/build.gradle b/x-pack/plugin/esql/build.gradle new file mode 100644 index 0000000000000..38ce55335802e --- /dev/null +++ b/x-pack/plugin/esql/build.gradle @@ -0,0 +1,160 @@ +import org.elasticsearch.gradle.internal.info.BuildParams + +apply plugin: 'elasticsearch.internal-es-plugin' +apply plugin: 'elasticsearch.internal-cluster-test' +esplugin { + name 'x-pack-esql' + description 'The plugin that powers ESQL for Elasticsearch' + classname 'org.elasticsearch.xpack.esql.plugin.EsqlPlugin' + extendedPlugins = ['x-pack-ql', 'lang-painless'] +} + +base { + archivesName = 'x-pack-esql' +} + +dependencies { + compileOnly project(path: xpackModule('core')) + compileOnly project(':modules:lang-painless:spi') + compileOnly project(xpackModule('ql')) + implementation project('compute') + implementation project('compute:ann') + implementation project(':libs:elasticsearch-dissect') + implementation project(':libs:elasticsearch-grok') + annotationProcessor project('compute:gen') + + testImplementation project('qa:testFixtures') + testImplementation project(':test:framework') + testImplementation(testArtifact(project(xpackModule('core')))) + testImplementation(testArtifact(project(xpackModule('security')))) + testImplementation project(path: ':modules:reindex') + testImplementation project(path: ':modules:parent-join') + testImplementation project(path: ':modules:analysis-common') + + internalClusterTestImplementation project(":client:rest-high-level") +} + +/* + * IntelliJ will always put the java files generated by the annotation processor + * into src/main/java/generated so we make gradle play along. This block makes + * it put the generated files into the same spot and the next block stops it from + * trying to compile the generated files in the regular compile - it'll regenerate + * them and *then* compile them. + */ +tasks.named("compileJava").configure { + options.compilerArgs.addAll(["-s", "${projectDir}/src/main/java/generated"]) +} + +sourceSets.main.java { + exclude 'generated/**' +} + +/**************************************************************** + * Enable QA/rest integration tests for snapshot builds only * + * TODO: Enable for all builds upon this feature release * + ****************************************************************/ +if (BuildParams.isSnapshotBuild()) { + addQaCheckDependencies(project) +} + +/********************************************** + * ESQL Parser regeneration * + **********************************************/ + +configurations { + regenerate +} + +dependencies { + regenerate "org.antlr:antlr4:${versions.antlr4}" +} + +String grammarPath = 'src/main/antlr' +String outputPath = 'src/main/java/org/elasticsearch/xpack/esql/parser' + +pluginManager.withPlugin('com.diffplug.spotless') { + spotless { + java { + // for some reason "${outputPath}/EsqlBaseParser*.java" does not match the same files... + targetExclude "src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer*.java", + "src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser*.java", + "src/main/java/generated/**/*.java" + } + } +} + +tasks.register("cleanGenerated", Delete) { + delete fileTree(grammarPath) { + include '*.tokens' + } + delete fileTree(outputPath) { + include 'EsqlBase*.java' + } +} + +tasks.register("regenLexer", JavaExec) { + dependsOn "cleanGenerated" + mainClass = 'org.antlr.v4.Tool' + classpath = configurations.regenerate + systemProperty 'file.encoding', 'UTF-8' + systemProperty 'user.language', 'en' + systemProperty 'user.country', 'US' + systemProperty 'user.variant', '' + args '-Werror', + '-package', 'org.elasticsearch.xpack.esql.parser', + '-listener', + '-visitor', + '-o', outputPath, + "${file(grammarPath)}/EsqlBaseLexer.g4" +} + +tasks.register("regenParser", JavaExec) { + dependsOn "cleanGenerated" + dependsOn "regenLexer" + mainClass = 'org.antlr.v4.Tool' + classpath = configurations.regenerate + systemProperty 'file.encoding', 'UTF-8' + systemProperty 'user.language', 'en' + systemProperty 'user.country', 'US' + systemProperty 'user.variant', '' + args '-Werror', + '-package', 'org.elasticsearch.xpack.esql.parser', + '-listener', + '-visitor', + '-o', outputPath, + '-lib', outputPath, + "${file(grammarPath)}/EsqlBaseParser.g4" +} + +tasks.register("regen") { + dependsOn "regenParser" + doLast { + // moves token files to grammar directory for use with IDE's + ant.move(file: "${outputPath}/EsqlBaseLexer.tokens", toDir: grammarPath) + ant.move(file: "${outputPath}/EsqlBaseParser.tokens", toDir: grammarPath) + // make the generated classes package private + ant.replaceregexp( + match: 'public ((interface|class) \\QEsqlBase(Parser|Lexer)\\E\\w+)', + replace: '\\1', + encoding: 'UTF-8' + ) { + fileset(dir: outputPath, includes: 'EsqlBase*.java') + } + // nuke timestamps/filenames in generated files + ant.replaceregexp( + match: '\\Q// Generated from \\E.*', + replace: '\\/\\/ ANTLR GENERATED CODE: DO NOT EDIT', + encoding: 'UTF-8' + ) { + fileset(dir: outputPath, includes: 'EsqlBase*.java') + } + // remove tabs in antlr generated files + ant.replaceregexp(match: '\t', flags: 'g', replace: ' ', encoding: 'UTF-8') { + fileset(dir: outputPath, includes: 'EsqlBase*.java') + } + // fix line endings + ant.fixcrlf(srcdir: outputPath, eol: 'lf') { + patternset(includes: 'EsqlBase*.java') + } + } +} diff --git a/x-pack/plugin/esql/compute/ann/build.gradle b/x-pack/plugin/esql/compute/ann/build.gradle new file mode 100644 index 0000000000000..ee8d8c62dff39 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/build.gradle @@ -0,0 +1,6 @@ +apply plugin: 'elasticsearch.build' + +tasks.named('forbiddenApisMain').configure { + // doesn't depend on anything + replaceSignatureFiles 'jdk-signatures' +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/module-info.java b/x-pack/plugin/esql/compute/ann/src/main/java/module-info.java new file mode 100644 index 0000000000000..2326c37990530 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/module-info.java @@ -0,0 +1,10 @@ +/* + * 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. + */ + +module org.elasticsearch.compute.ann { + exports org.elasticsearch.compute.ann; +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Aggregator.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Aggregator.java new file mode 100644 index 0000000000000..b57008fd6544d --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Aggregator.java @@ -0,0 +1,60 @@ +/* + * 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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Trigger for generating {@code AggregatorFunction} implementations. + *

+ * The first thing the aggregator generation code does is find a static + * method names {@code init} or {@code initSingle}. That's the method that + * initializes an empty aggregation state. The method can either return + * a subclass of {@code AggregatorState} or it can return an {@code int}, + * {@code long}, or {@code double} which will automatically be adapted into + * a small {@code AggregatorState} implementation that wraps a mutable reference + * to the primitive. + *

+ *

+ * Next the generation code finds a static method named {@code combine} which + * "combines" the state with a new value. The first parameter of this method + * must the state type detected in the previous section or be a primitive that + * lines up with one of the primitive state types from the previous section. + * This is called once per value to "combine" the value into the state. + *

+ *

+ * If the state type has a method called {@code seen} then the generated + * aggregation will call it at least once if it'll ever call {@code combine}. + * Think of this as a lower overhead way of detecting the cases where no values + * are ever collected. + *

+ *

+ * The generation code will also look for a method called {@code combineValueCount} + * which is called once per received block with a count of values. NOTE: We may + * not need this after we convert AVG into a composite operation. + *

+ *

+ * The generation code also looks for the optional methods {@code combineStates} + * and {@code evaluateFinal} which are used to combine intermediate states and + * produce the final output. If the first is missing then the generated code will + * call the {@code combine} method to combine intermediate states. If the second + * is missing the generated code will make a block containing the primitive from + * the state. If either of those don't have sensible interpretations then the code + * generation code will throw an error, aborting the compilation. + *

+ */ +@Target(ElementType.TYPE) +@Retention(RetentionPolicy.SOURCE) +public @interface Aggregator { + + IntermediateState[] value() default {}; + +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/ConvertEvaluator.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/ConvertEvaluator.java new file mode 100644 index 0000000000000..69a015b8d5ae9 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/ConvertEvaluator.java @@ -0,0 +1,28 @@ +/* + * 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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Implement an evaluator for a function applying a static {@code process} + * method to each value of a multivalued field. + */ +@Target(ElementType.METHOD) +@Retention(RetentionPolicy.SOURCE) +public @interface ConvertEvaluator { + /** + * Extra part of the name of the evaluator. Use for disambiguating + * when there are multiple ways to evaluate a function. + */ + String extraName() default ""; + +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Evaluator.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Evaluator.java new file mode 100644 index 0000000000000..ab77a8f5b1973 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Evaluator.java @@ -0,0 +1,45 @@ +/* + * 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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Implement an evaluator from a static {@code process} method. The generated + * evaluator provides each argument in one of three ways: + *
    + *
  1. If the argument isn't annotated or an array then it is considered + * to be a sub-evaluator and the generated Evaluator will take an + * Evaluator for this on construction and call it for each position.
  2. + *
  3. If the argument isn't annotated but is an array then it is considered + * to be an array of evaluators and the generated Evaluator will take + * an array of Evaluators on construction and evaluate each of them for + * each position.
  4. + *
  5. If parameter has the {@link Fixed} annotation then it must be + * provided at construction time and is passed unchanged to the process + * method.
  6. + *
+ */ +@Target(ElementType.METHOD) +@Retention(RetentionPolicy.SOURCE) +public @interface Evaluator { + /** + * Extra part of the name of the evaluator. Use for disambiguating + * when there are multiple ways to evaluate a function. + */ + String extraName() default ""; + + /** + * Exceptions thrown by the process method to catch and convert + * into a warning and turn into a null value. + */ + Class[] warnExceptions() default {}; +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java new file mode 100644 index 0000000000000..286c36ab2314d --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java @@ -0,0 +1,24 @@ +/* + * 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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Used on parameters on methods annotated with {@link Evaluator} to indicate + * parameters that are provided to the generated evaluator's constructor rather + * than recalculated for every row. + */ +@Target(ElementType.PARAMETER) +@Retention(RetentionPolicy.SOURCE) +public @interface Fixed { + boolean includeInToString() default true; +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/GroupingAggregator.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/GroupingAggregator.java new file mode 100644 index 0000000000000..bb7b2cc888c2c --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/GroupingAggregator.java @@ -0,0 +1,20 @@ +/* + * 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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Target(ElementType.TYPE) +@Retention(RetentionPolicy.SOURCE) +public @interface GroupingAggregator { + + IntermediateState[] value() default {}; +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/IntermediateState.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/IntermediateState.java new file mode 100644 index 0000000000000..54a5caa05d149 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/IntermediateState.java @@ -0,0 +1,15 @@ +/* + * 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.compute.ann; + +public @interface IntermediateState { + + String name(); + + String type(); +} diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/MvEvaluator.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/MvEvaluator.java new file mode 100644 index 0000000000000..97f165e67cb44 --- /dev/null +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/MvEvaluator.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.compute.ann; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Implement an evaluator for a function reducing multivalued fields into a + * single valued field from a static {@code process} method. + *

+ * Annotated methods can have two "shapes": pairwise processing and + * accumulator processing. Pairwise is generally + * simpler and looks like {@code int process(int current, int next)}. + * Use it when the result is a primitive. Accumulator processing is + * a bit more complex and looks like {@code void process(State state, int v)} + * and it useful when you need to accumulate more data than fits + * in a primitive result. Think Kahan summation. + *

+ *

+ * Both method shapes support at {@code finish = "finish_method"} parameter + * on the annotation which is used to, well, "finish" processing after + * all values have been received. Again, think reading the sum from the + * Kahan summation. Or doing the division for an "average" operation. + * This method is required for accumulator processing. + *

+ */ +@Target(ElementType.METHOD) +@Retention(RetentionPolicy.SOURCE) +public @interface MvEvaluator { + /** + * Extra part of the name of the evaluator. Use for disambiguating + * when there are multiple ways to evaluate a function. + */ + String extraName() default ""; + + /** + * Optional method called to convert state into result. + */ + String finish() default ""; + + /** + * Optional method called to process single valued fields. If this + * is missing then blocks containing only single valued fields will + * be returned exactly as is. If this is present then single valued + * fields will not call the process or finish function and instead + * just call this function. + */ + String single() default ""; + + /** + * Optional method called to process blocks whose values are sorted + * in ascending order. + */ + String ascending() default ""; + + /** + * Exceptions thrown by the process method to catch and convert + * into a warning and turn into a null value. + */ + Class[] warnExceptions() default {}; +} diff --git a/x-pack/plugin/esql/compute/build.gradle b/x-pack/plugin/esql/compute/build.gradle new file mode 100644 index 0000000000000..d6a27b4122edb --- /dev/null +++ b/x-pack/plugin/esql/compute/build.gradle @@ -0,0 +1,399 @@ +apply plugin: 'elasticsearch.build' +apply plugin: 'elasticsearch.string-templates' + +dependencies { + compileOnly project(':server') + compileOnly project('ann') + annotationProcessor project('gen') + + testImplementation project(':test:framework') + testImplementation(project(xpackModule('ql'))) +} + +tasks.named("compileJava").configure { + options.compilerArgs.addAll(["-s", "${projectDir}/src/main/generated"]) +} + +tasks.named('checkstyleMain').configure { + source = "src/main/java" + excludes = [ "**/*.java.st" ] +} + +spotlessJava.dependsOn stringTemplates + +spotless { + java { + /* + * Generated files go here. + */ + targetExclude "src/main/generated/**/*.java" + } +} + +def prop(Type, type, TYPE, BYTES) { + return [ + "Type" : Type, + "type" : type, + "TYPE" : TYPE, + "BYTES" : BYTES, + + "int" : type == "int" ? "true" : "", + "long" : type == "long" ? "true" : "", + "double" : type == "double" ? "true" : "", + "BytesRef" : type == "BytesRef" ? "true" : "", + "boolean" : type == "boolean" ? "true" : "", + ] +} + +tasks.named('stringTemplates').configure { + var intProperties = prop("Int", "int", "INT", "Integer.BYTES") + var longProperties = prop("Long", "long", "LONG", "Long.BYTES") + var doubleProperties = prop("Double", "double", "DOUBLE", "Double.BYTES") + var bytesRefProperties = prop("BytesRef", "BytesRef", "BYTES_REF", "org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF") + var booleanProperties = prop("Boolean", "boolean", "BOOLEAN", "Boolean.BYTES") + // primitive vectors + File vectorInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-Vector.java.st") + template { + it.properties = intProperties + it.inputFile = vectorInputFile + it.outputFile = "org/elasticsearch/compute/data/IntVector.java" + } + template { + it.properties = longProperties + it.inputFile = vectorInputFile + it.outputFile = "org/elasticsearch/compute/data/LongVector.java" + } + template { + it.properties = doubleProperties + it.inputFile = vectorInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleVector.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = vectorInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefVector.java" + } + template { + it.properties = booleanProperties + it.inputFile = vectorInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanVector.java" + } + // array vector implementations + File arrayVectorInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-ArrayVector.java.st") + template { + it.properties = intProperties + it.inputFile = arrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/IntArrayVector.java" + } + template { + it.properties = longProperties + it.inputFile = arrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/LongArrayVector.java" + } + template { + it.properties = doubleProperties + it.inputFile = arrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleArrayVector.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = arrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefArrayVector.java" + } + template { + it.properties = booleanProperties + it.inputFile = arrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanArrayVector.java" + } + // BigArray vector implementations + File bigArrayVectorInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-BigArrayVector.java.st") + template { + it.properties = intProperties + it.inputFile = bigArrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/IntBigArrayVector.java" + } + template { + it.properties = longProperties + it.inputFile = bigArrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/LongBigArrayVector.java" + } + template { + it.properties = doubleProperties + it.inputFile = bigArrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleBigArrayVector.java" + } + template { + it.properties = booleanProperties + it.inputFile = bigArrayVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanBigArrayVector.java" + } + // filter vectors + File filterVectorInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-FilterVector.java.st") + template { + it.properties = intProperties + it.inputFile = filterVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterIntVector.java" + } + template { + it.properties = longProperties + it.inputFile = filterVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterLongVector.java" + } + template { + it.properties = doubleProperties + it.inputFile = filterVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterDoubleVector.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = filterVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterBytesRefVector.java" + } + template { + it.properties = booleanProperties + it.inputFile = filterVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterBooleanVector.java" + } + // constant vectors + File constantVectorInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-ConstantVector.java.st") + template { + it.properties = intProperties + it.inputFile = constantVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/ConstantIntVector.java" + } + template { + it.properties = longProperties + it.inputFile = constantVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/ConstantLongVector.java" + } + template { + it.properties = doubleProperties + it.inputFile = constantVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/ConstantDoubleVector.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = constantVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/ConstantBytesRefVector.java" + } + template { + it.properties = booleanProperties + it.inputFile = constantVectorInputFile + it.outputFile = "org/elasticsearch/compute/data/ConstantBooleanVector.java" + } + // primitive blocks + File blockInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-Block.java.st") + template { + it.properties = intProperties + it.inputFile = blockInputFile + it.outputFile = "org/elasticsearch/compute/data/IntBlock.java" + } + template { + it.properties = longProperties + it.inputFile = blockInputFile + it.outputFile = "org/elasticsearch/compute/data/LongBlock.java" + } + template { + it.properties = doubleProperties + it.inputFile = blockInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleBlock.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = blockInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefBlock.java" + } + template { + it.properties = booleanProperties + it.inputFile = blockInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanBlock.java" + } + // array blocks + File arrayBlockInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st") + template { + it.properties = intProperties + it.inputFile = arrayBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/IntArrayBlock.java" + } + template { + it.properties = longProperties + it.inputFile = arrayBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/LongArrayBlock.java" + } + template { + it.properties = doubleProperties + it.inputFile = arrayBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleArrayBlock.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = arrayBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefArrayBlock.java" + } + template { + it.properties = booleanProperties + it.inputFile = arrayBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanArrayBlock.java" + } + // filter blocks + File filterBlockInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-FilterBlock.java.st") + template { + it.properties = intProperties + it.inputFile = filterBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterIntBlock.java" + } + template { + it.properties = longProperties + it.inputFile = filterBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterLongBlock.java" + } + template { + it.properties = doubleProperties + it.inputFile = filterBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterDoubleBlock.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = filterBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterBytesRefBlock.java" + } + template { + it.properties = booleanProperties + it.inputFile = filterBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/FilterBooleanBlock.java" + } + // vector blocks + File vectorBlockInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-VectorBlock.java.st") + template { + it.properties = intProperties + it.inputFile = vectorBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/IntVectorBlock.java" + } + template { + it.properties = longProperties + it.inputFile = vectorBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/LongVectorBlock.java" + } + template { + it.properties = doubleProperties + it.inputFile = vectorBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleVectorBlock.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = vectorBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefVectorBlock.java" + } + template { + it.properties = booleanProperties + it.inputFile = vectorBlockInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanVectorBlock.java" + } + // block builders + File blockBuildersInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st") + template { + it.properties = intProperties + it.inputFile = blockBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/IntBlockBuilder.java" + } + template { + it.properties = longProperties + it.inputFile = blockBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/LongBlockBuilder.java" + } + template { + it.properties = doubleProperties + it.inputFile = blockBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleBlockBuilder.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = blockBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefBlockBuilder.java" + } + template { + it.properties = booleanProperties + it.inputFile = blockBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanBlockBuilder.java" + } + // vector builders + File vectorBuildersInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/data/X-VectorBuilder.java.st") + template { + it.properties = intProperties + it.inputFile = vectorBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/IntVectorBuilder.java" + } + template { + it.properties = longProperties + it.inputFile = vectorBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/LongVectorBuilder.java" + } + template { + it.properties = doubleProperties + it.inputFile = vectorBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/DoubleVectorBuilder.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = vectorBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/BytesRefVectorBuilder.java" + } + template { + it.properties = booleanProperties + it.inputFile = vectorBuildersInputFile + it.outputFile = "org/elasticsearch/compute/data/BooleanVectorBuilder.java" + } + File stateInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/aggregation/X-State.java.st") + template { + it.properties = intProperties + it.inputFile = stateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/IntState.java" + } + template { + it.properties = longProperties + it.inputFile = stateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/LongState.java" + } + template { + it.properties = doubleProperties + it.inputFile = stateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/DoubleState.java" + } + File arrayStateInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/aggregation/X-ArrayState.java.st") + template { + it.properties = intProperties + it.inputFile = arrayStateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/IntArrayState.java" + } + template { + it.properties = longProperties + it.inputFile = arrayStateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/LongArrayState.java" + } + template { + it.properties = doubleProperties + it.inputFile = arrayStateInputFile + it.outputFile = "org/elasticsearch/compute/aggregation/DoubleArrayState.java" + } + File multivalueDedupeInputFile = new File("${projectDir}/src/main/java/org/elasticsearch/compute/operator/X-MultivalueDedupe.java.st") + template { + it.properties = intProperties + it.inputFile = multivalueDedupeInputFile + it.outputFile = "org/elasticsearch/compute/operator/MultivalueDedupeInt.java" + } + template { + it.properties = longProperties + it.inputFile = multivalueDedupeInputFile + it.outputFile = "org/elasticsearch/compute/operator/MultivalueDedupeLong.java" + } + template { + it.properties = doubleProperties + it.inputFile = multivalueDedupeInputFile + it.outputFile = "org/elasticsearch/compute/operator/MultivalueDedupeDouble.java" + } + template { + it.properties = bytesRefProperties + it.inputFile = multivalueDedupeInputFile + it.outputFile = "org/elasticsearch/compute/operator/MultivalueDedupeBytesRef.java" + } +} diff --git a/x-pack/plugin/esql/compute/gen/build.gradle b/x-pack/plugin/esql/compute/gen/build.gradle new file mode 100644 index 0000000000000..c9f2e8c3632bf --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/build.gradle @@ -0,0 +1,11 @@ +apply plugin: 'elasticsearch.build' + +dependencies { + api project(':x-pack:plugin:esql:compute:ann') + api 'com.squareup:javapoet:1.13.0' +} + +tasks.named('forbiddenApisMain').configure { + // doesn't depend on core + replaceSignatureFiles 'jdk-signatures' +} diff --git a/x-pack/plugin/esql/compute/gen/licenses/javapoet-LICENSE.txt b/x-pack/plugin/esql/compute/gen/licenses/javapoet-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/licenses/javapoet-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/x-pack/plugin/esql/compute/gen/licenses/javapoet-NOTICE.txt b/x-pack/plugin/esql/compute/gen/licenses/javapoet-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/module-info.java b/x-pack/plugin/esql/compute/gen/src/main/java/module-info.java new file mode 100644 index 0000000000000..877f2d5c04f90 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/module-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +import org.elasticsearch.compute.gen.AggregatorProcessor; +import org.elasticsearch.compute.gen.ConsumeProcessor; +import org.elasticsearch.compute.gen.EvaluatorProcessor; + +module org.elasticsearch.compute.gen { + requires com.squareup.javapoet; + requires org.elasticsearch.compute.ann; + requires java.compiler; + + exports org.elasticsearch.compute.gen; + + provides javax.annotation.processing.Processor with AggregatorProcessor, ConsumeProcessor, EvaluatorProcessor; +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorFunctionSupplierImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorFunctionSupplierImplementer.java new file mode 100644 index 0000000000000..e51fa1a199afb --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorFunctionSupplierImplementer.java @@ -0,0 +1,157 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeSpec; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.util.Elements; + +import static org.elasticsearch.compute.gen.Types.AGGREGATOR_FUNCTION_SUPPLIER; +import static org.elasticsearch.compute.gen.Types.BIG_ARRAYS; +import static org.elasticsearch.compute.gen.Types.LIST_INTEGER; + +/** + * Implements "AggregationFunctionSupplier" from a class annotated with both + * {@link Aggregator} and {@link GroupingAggregator}. + */ +public class AggregatorFunctionSupplierImplementer { + private final TypeElement declarationType; + private final AggregatorImplementer aggregatorImplementer; + private final GroupingAggregatorImplementer groupingAggregatorImplementer; + private final List createParameters; + private final ClassName implementation; + + public AggregatorFunctionSupplierImplementer( + Elements elements, + TypeElement declarationType, + AggregatorImplementer aggregatorImplementer, + GroupingAggregatorImplementer groupingAggregatorImplementer + ) { + this.declarationType = declarationType; + this.aggregatorImplementer = aggregatorImplementer; + this.groupingAggregatorImplementer = groupingAggregatorImplementer; + + Set createParameters = new LinkedHashSet<>(); + createParameters.addAll(aggregatorImplementer.createParameters()); + createParameters.addAll(groupingAggregatorImplementer.createParameters()); + List sortedParameters = new ArrayList<>(createParameters); + for (Parameter p : sortedParameters) { + if (p.type().equals(BIG_ARRAYS) && false == p.name().equals("bigArrays")) { + throw new IllegalArgumentException("BigArrays should always be named bigArrays but was " + p); + } + } + + /* + * We like putting BigArrays first and then channels second + * regardless of the order that the aggs actually want them. + * Just a little bit of standardization here. + */ + Parameter bigArraysParam = new Parameter(BIG_ARRAYS, "bigArrays"); + sortedParameters.remove(bigArraysParam); + sortedParameters.add(0, bigArraysParam); + sortedParameters.add(1, new Parameter(LIST_INTEGER, "channels")); + + this.createParameters = sortedParameters; + + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + (declarationType.getSimpleName() + "AggregatorFunctionSupplier").replace("AggregatorAggregator", "Aggregator") + ); + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", AGGREGATOR_FUNCTION_SUPPLIER, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + builder.addSuperinterface(AGGREGATOR_FUNCTION_SUPPLIER); + + createParameters.stream().forEach(p -> p.declareField(builder)); + builder.addMethod(ctor()); + builder.addMethod(aggregator()); + builder.addMethod(groupingAggregator()); + builder.addMethod(describe()); + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + createParameters.stream().forEach(p -> p.buildCtor(builder)); + return builder.build(); + } + + private MethodSpec aggregator() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("aggregator").returns(aggregatorImplementer.implementation()); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addStatement( + "return $T.create($L)", + aggregatorImplementer.implementation(), + Stream.concat(Stream.of("channels"), aggregatorImplementer.createParameters().stream().map(Parameter::name)) + .collect(Collectors.joining(", ")) + ); + + return builder.build(); + } + + private MethodSpec groupingAggregator() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("groupingAggregator").returns(groupingAggregatorImplementer.implementation()); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addStatement( + "return $T.create($L)", + groupingAggregatorImplementer.implementation(), + Stream.concat(Stream.of("channels"), groupingAggregatorImplementer.createParameters().stream().map(Parameter::name)) + .collect(Collectors.joining(", ")) + ); + return builder.build(); + } + + private MethodSpec describe() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("describe").returns(String.class); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + + String name = declarationType.getSimpleName().toString(); + name = name.replace("BytesRef", "Byte"); // The hack expects one word types so let's make BytesRef into Byte + String[] parts = name.split("(?=\\p{Upper})"); + if (false == parts[parts.length - 1].equals("Aggregator") || parts.length < 3) { + throw new IllegalArgumentException("Can't generate description for " + declarationType.getSimpleName()); + } + + String operation = Arrays.stream(parts, 0, parts.length - 2).map(s -> s.toLowerCase(Locale.ROOT)).collect(Collectors.joining("_")); + String type = parts[parts.length - 2]; + + builder.addStatement("return $S", operation + " of " + type.toLowerCase(Locale.ROOT) + "s"); + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java new file mode 100644 index 0000000000000..785e69d1c3364 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorImplementer.java @@ -0,0 +1,538 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.CodeBlock; +import com.squareup.javapoet.FieldSpec; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +import java.util.Arrays; +import java.util.List; +import java.util.Locale; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.util.Elements; + +import static java.util.stream.Collectors.joining; +import static org.elasticsearch.compute.gen.Methods.findMethod; +import static org.elasticsearch.compute.gen.Methods.findRequiredMethod; +import static org.elasticsearch.compute.gen.Methods.vectorAccessorName; +import static org.elasticsearch.compute.gen.Types.AGGREGATOR_FUNCTION; +import static org.elasticsearch.compute.gen.Types.BLOCK; +import static org.elasticsearch.compute.gen.Types.BLOCK_ARRAY; +import static org.elasticsearch.compute.gen.Types.BOOLEAN_BLOCK; +import static org.elasticsearch.compute.gen.Types.BOOLEAN_VECTOR; +import static org.elasticsearch.compute.gen.Types.BYTES_REF; +import static org.elasticsearch.compute.gen.Types.BYTES_REF_BLOCK; +import static org.elasticsearch.compute.gen.Types.BYTES_REF_VECTOR; +import static org.elasticsearch.compute.gen.Types.DOUBLE_BLOCK; +import static org.elasticsearch.compute.gen.Types.DOUBLE_VECTOR; +import static org.elasticsearch.compute.gen.Types.ELEMENT_TYPE; +import static org.elasticsearch.compute.gen.Types.INTERMEDIATE_STATE_DESC; +import static org.elasticsearch.compute.gen.Types.INT_BLOCK; +import static org.elasticsearch.compute.gen.Types.INT_VECTOR; +import static org.elasticsearch.compute.gen.Types.LIST_AGG_FUNC_DESC; +import static org.elasticsearch.compute.gen.Types.LIST_INTEGER; +import static org.elasticsearch.compute.gen.Types.LONG_BLOCK; +import static org.elasticsearch.compute.gen.Types.LONG_VECTOR; +import static org.elasticsearch.compute.gen.Types.PAGE; +import static org.elasticsearch.compute.gen.Types.blockType; +import static org.elasticsearch.compute.gen.Types.vectorType; + +/** + * Implements "AggregationFunction" from a class containing static methods + * annotated with {@link Aggregator}. + *

The goal here is the implement an AggregationFunction who's inner loops + * don't contain any {@code invokevirtual}s. Instead, we generate a class + * that calls static methods in the inner loops. + *

A secondary goal is to make the generated code as readable, debuggable, + * and break-point-able as possible. + */ +public class AggregatorImplementer { + private final TypeElement declarationType; + private final ExecutableElement init; + private final ExecutableElement combine; + private final ExecutableElement combineValueCount; + private final ExecutableElement combineStates; + private final ExecutableElement combineIntermediate; + private final ExecutableElement evaluateFinal; + private final ClassName implementation; + private final TypeName stateType; + private final boolean stateTypeHasSeen; + private final boolean valuesIsBytesRef; + private final List intermediateState; + + public AggregatorImplementer(Elements elements, TypeElement declarationType, IntermediateState[] interStateAnno) { + this.declarationType = declarationType; + + this.init = findRequiredMethod(declarationType, new String[] { "init", "initSingle" }, e -> true); + this.stateType = choseStateType(); + stateTypeHasSeen = elements.getAllMembers(elements.getTypeElement(stateType.toString())) + .stream() + .anyMatch(e -> e.toString().equals("seen()")); + + this.combine = findRequiredMethod(declarationType, new String[] { "combine" }, e -> { + if (e.getParameters().size() == 0) { + return false; + } + TypeName firstParamType = TypeName.get(e.getParameters().get(0).asType()); + return firstParamType.isPrimitive() || firstParamType.toString().equals(stateType.toString()); + }); + this.combineValueCount = findMethod(declarationType, "combineValueCount"); + this.combineStates = findMethod(declarationType, "combineStates"); + this.combineIntermediate = findMethod(declarationType, "combineIntermediate"); + this.evaluateFinal = findMethod(declarationType, "evaluateFinal"); + + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + (declarationType.getSimpleName() + "AggregatorFunction").replace("AggregatorAggregator", "Aggregator") + ); + this.valuesIsBytesRef = BYTES_REF.equals(TypeName.get(combine.getParameters().get(combine.getParameters().size() - 1).asType())); + intermediateState = Arrays.stream(interStateAnno).map(state -> new IntermediateStateDesc(state.name(), state.type())).toList(); + } + + record IntermediateStateDesc(String name, String elementType) {} + + ClassName implementation() { + return implementation; + } + + List createParameters() { + return init.getParameters().stream().map(Parameter::from).toList(); + } + + private TypeName choseStateType() { + TypeName initReturn = TypeName.get(init.getReturnType()); + if (false == initReturn.isPrimitive()) { + return initReturn; + } + return ClassName.get("org.elasticsearch.compute.aggregation", firstUpper(initReturn.toString()) + "State"); + } + + static String valueType(ExecutableElement init, ExecutableElement combine) { + if (combine != null) { + // If there's an explicit combine function it's final parameter is the type of the value. + return combine.getParameters().get(combine.getParameters().size() - 1).asType().toString(); + } + String initReturn = init.getReturnType().toString(); + switch (initReturn) { + case "double": + return "double"; + case "long": + return "long"; + case "int": + return "int"; + case "boolean": + return "boolean"; + default: + throw new IllegalArgumentException("unknown primitive type for " + initReturn); + } + } + + static ClassName valueBlockType(ExecutableElement init, ExecutableElement combine) { + return switch (valueType(init, combine)) { + case "boolean" -> BOOLEAN_BLOCK; + case "double" -> DOUBLE_BLOCK; + case "long" -> LONG_BLOCK; + case "int" -> INT_BLOCK; + case "org.apache.lucene.util.BytesRef" -> BYTES_REF_BLOCK; + default -> throw new IllegalArgumentException("unknown block type for " + valueType(init, combine)); + }; + } + + static ClassName valueVectorType(ExecutableElement init, ExecutableElement combine) { + return switch (valueType(init, combine)) { + case "boolean" -> BOOLEAN_VECTOR; + case "double" -> DOUBLE_VECTOR; + case "long" -> LONG_VECTOR; + case "int" -> INT_VECTOR; + case "org.apache.lucene.util.BytesRef" -> BYTES_REF_VECTOR; + default -> throw new IllegalArgumentException("unknown vector type for " + valueType(init, combine)); + }; + } + + public static String firstUpper(String s) { + String head = s.toString().substring(0, 1).toUpperCase(Locale.ROOT); + String tail = s.toString().substring(1); + return head + tail; + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", AGGREGATOR_FUNCTION, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + builder.addSuperinterface(AGGREGATOR_FUNCTION); + builder.addField( + FieldSpec.builder(LIST_AGG_FUNC_DESC, "INTERMEDIATE_STATE_DESC", Modifier.PRIVATE, Modifier.STATIC, Modifier.FINAL) + .initializer(initInterState()) + .build() + ); + builder.addField(stateType, "state", Modifier.PRIVATE, Modifier.FINAL); + builder.addField(LIST_INTEGER, "channels", Modifier.PRIVATE, Modifier.FINAL); + + for (VariableElement p : init.getParameters()) { + builder.addField(TypeName.get(p.asType()), p.getSimpleName().toString(), Modifier.PRIVATE, Modifier.FINAL); + } + + builder.addMethod(create()); + builder.addMethod(ctor()); + builder.addMethod(intermediateStateDesc()); + builder.addMethod(intermediateBlockCount()); + builder.addMethod(addRawInput()); + builder.addMethod(addRawVector()); + builder.addMethod(addRawBlock()); + builder.addMethod(addIntermediateInput()); + builder.addMethod(evaluateIntermediate()); + builder.addMethod(evaluateFinal()); + builder.addMethod(toStringMethod()); + builder.addMethod(close()); + return builder.build(); + } + + private MethodSpec create() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("create"); + builder.addModifiers(Modifier.PUBLIC, Modifier.STATIC).returns(implementation); + builder.addParameter(LIST_INTEGER, "channels"); + for (VariableElement p : init.getParameters()) { + builder.addParameter(TypeName.get(p.asType()), p.getSimpleName().toString()); + } + if (init.getParameters().isEmpty()) { + builder.addStatement("return new $T(channels, $L)", implementation, callInit()); + } else { + builder.addStatement("return new $T(channels, $L, $L)", implementation, callInit(), initParameters()); + } + return builder.build(); + } + + private String initParameters() { + return init.getParameters().stream().map(p -> p.getSimpleName().toString()).collect(joining(", ")); + } + + private CodeBlock callInit() { + CodeBlock.Builder builder = CodeBlock.builder(); + if (init.getReturnType().toString().equals(stateType.toString())) { + builder.add("$T.$L($L)", declarationType, init.getSimpleName(), initParameters()); + } else { + builder.add("new $T($T.$L($L))", stateType, declarationType, init.getSimpleName(), initParameters()); + } + return builder.build(); + } + + private CodeBlock initInterState() { + CodeBlock.Builder builder = CodeBlock.builder(); + builder.add("List.of("); + boolean addComma = false; + for (var interState : intermediateState) { + if (addComma) builder.add(","); + builder.add("$Wnew $T($S, $T." + interState.elementType() + ")", INTERMEDIATE_STATE_DESC, interState.name(), ELEMENT_TYPE); + addComma = true; + } + builder.add("$W$W)"); + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + builder.addParameter(LIST_INTEGER, "channels"); + builder.addParameter(stateType, "state"); + builder.addStatement("this.channels = channels"); + builder.addStatement("this.state = state"); + + for (VariableElement p : init.getParameters()) { + builder.addParameter(TypeName.get(p.asType()), p.getSimpleName().toString()); + builder.addStatement("this.$N = $N", p.getSimpleName(), p.getSimpleName()); + } + return builder.build(); + } + + private MethodSpec intermediateStateDesc() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("intermediateStateDesc"); + builder.addModifiers(Modifier.PUBLIC, Modifier.STATIC).returns(LIST_AGG_FUNC_DESC); + builder.addStatement("return INTERMEDIATE_STATE_DESC"); + return builder.build(); + } + + private MethodSpec intermediateBlockCount() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("intermediateBlockCount"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).returns(TypeName.INT); + builder.addStatement("return INTERMEDIATE_STATE_DESC.size()"); + return builder.build(); + } + + private MethodSpec addRawInput() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addRawInput"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).addParameter(PAGE, "page"); + builder.addStatement("$T uncastBlock = page.getBlock(channels.get(0))", BLOCK); + builder.beginControlFlow("if (uncastBlock.areAllValuesNull())").addStatement("return").endControlFlow(); + builder.addStatement("$T block = ($T) uncastBlock", valueBlockType(init, combine), valueBlockType(init, combine)); + builder.addStatement("$T vector = block.asVector()", valueVectorType(init, combine)); + builder.beginControlFlow("if (vector != null)").addStatement("addRawVector(vector)"); + builder.nextControlFlow("else").addStatement("addRawBlock(block)").endControlFlow(); + return builder.build(); + } + + private MethodSpec addRawVector() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addRawVector"); + builder.addModifiers(Modifier.PRIVATE).addParameter(valueVectorType(init, combine), "vector"); + + if (stateTypeHasSeen) { + builder.addStatement("state.seen(true)"); + } + if (valuesIsBytesRef) { + // Add bytes_ref scratch var that will be used for bytes_ref blocks/vectors + builder.addStatement("$T scratch = new $T()", BYTES_REF, BYTES_REF); + } + + builder.beginControlFlow("for (int i = 0; i < vector.getPositionCount(); i++)"); + { + combineRawInput(builder, "vector"); + } + builder.endControlFlow(); + if (combineValueCount != null) { + builder.addStatement("$T.combineValueCount(state, vector.getPositionCount())", declarationType); + } + return builder.build(); + } + + private MethodSpec addRawBlock() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addRawBlock"); + builder.addModifiers(Modifier.PRIVATE).addParameter(valueBlockType(init, combine), "block"); + + if (valuesIsBytesRef) { + // Add bytes_ref scratch var that will only be used for bytes_ref blocks/vectors + builder.addStatement("$T scratch = new $T()", BYTES_REF, BYTES_REF); + } + builder.beginControlFlow("for (int p = 0; p < block.getPositionCount(); p++)"); + { + builder.beginControlFlow("if (block.isNull(p))"); + builder.addStatement("continue"); + builder.endControlFlow(); + if (stateTypeHasSeen) { + builder.addStatement("state.seen(true)"); + } + builder.addStatement("int start = block.getFirstValueIndex(p)"); + builder.addStatement("int end = start + block.getValueCount(p)"); + builder.beginControlFlow("for (int i = start; i < end; i++)"); + combineRawInput(builder, "block"); + builder.endControlFlow(); + } + builder.endControlFlow(); + if (combineValueCount != null) { + builder.addStatement("$T.combineValueCount(state, block.getTotalValueCount())", declarationType); + } + return builder.build(); + } + + private void combineRawInput(MethodSpec.Builder builder, String blockVariable) { + if (valuesIsBytesRef) { + combineRawInputForBytesRef(builder, blockVariable); + return; + } + TypeName returnType = TypeName.get(combine.getReturnType()); + if (returnType.isPrimitive()) { + combineRawInputForPrimitive(returnType, builder, blockVariable); + return; + } + if (returnType == TypeName.VOID) { + combineRawInputForVoid(builder, blockVariable); + return; + } + throw new IllegalArgumentException("combine must return void or a primitive"); + } + + private void combineRawInputForPrimitive(TypeName returnType, MethodSpec.Builder builder, String blockVariable) { + builder.addStatement( + "state.$TValue($T.combine(state.$TValue(), $L.get$L(i)))", + returnType, + declarationType, + returnType, + blockVariable, + firstUpper(combine.getParameters().get(1).asType().toString()) + ); + } + + private void combineRawInputForVoid(MethodSpec.Builder builder, String blockVariable) { + builder.addStatement( + "$T.combine(state, $L.get$L(i))", + declarationType, + blockVariable, + firstUpper(combine.getParameters().get(1).asType().toString()) + ); + } + + private void combineRawInputForBytesRef(MethodSpec.Builder builder, String blockVariable) { + // scratch is a BytesRef var that must have been defined before the iteration starts + builder.addStatement("$T.combine(state, $L.getBytesRef(i, scratch))", declarationType, blockVariable); + } + + private MethodSpec addIntermediateInput() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addIntermediateInput"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).addParameter(PAGE, "page"); + builder.addStatement("assert channels.size() == intermediateBlockCount()"); + builder.addStatement("assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size()"); + int count = 0; + for (var interState : intermediateState) { + builder.addStatement( + "$T " + interState.name() + " = page.<$T>getBlock(channels.get(" + count + ")).asVector()", + vectorType(interState.elementType()), + blockType(interState.elementType()) + ); + count++; + } + final String first = intermediateState.get(0).name(); + builder.addStatement("assert " + first + ".getPositionCount() == 1"); + if (intermediateState.size() > 1) { + builder.addStatement( + "assert " + + intermediateState.stream() + .map(IntermediateStateDesc::name) + .skip(1) + .map(s -> first + ".getPositionCount() == " + s + ".getPositionCount()") + .collect(joining(" && ")) + ); + } + if (hasPrimitiveState()) { + assert intermediateState.size() == 2; + assert intermediateState.get(1).name().equals("seen"); + builder.beginControlFlow("if (seen.getBoolean(0))"); + { + var state = intermediateState.get(0); + var s = "state.$L($T.combine(state.$L(), " + state.name() + "." + vectorAccessorName(state.elementType()) + "(0)))"; + builder.addStatement(s, primitiveStateMethod(), declarationType, primitiveStateMethod()); + builder.addStatement("state.seen(true)"); + builder.endControlFlow(); + } + } else { + if (intermediateState.stream().map(IntermediateStateDesc::elementType).anyMatch(n -> n.equals("BYTES_REF"))) { + builder.addStatement("$T scratch = new $T()", BYTES_REF, BYTES_REF); + } + builder.addStatement("$T.combineIntermediate(state, " + intermediateStateRowAccess() + ")", declarationType); + } + return builder.build(); + } + + String intermediateStateRowAccess() { + return intermediateState.stream().map(AggregatorImplementer::vectorAccess).collect(joining(", ")); + } + + static String vectorAccess(IntermediateStateDesc isd) { + String s = isd.name() + "." + vectorAccessorName(isd.elementType()) + "(0"; + if (isd.elementType().equals("BYTES_REF")) { + s += ", scratch"; + } + return s + ")"; + } + + private String primitiveStateMethod() { + switch (stateType.toString()) { + case "org.elasticsearch.compute.aggregation.IntState": + return "intValue"; + case "org.elasticsearch.compute.aggregation.LongState": + return "longValue"; + case "org.elasticsearch.compute.aggregation.DoubleState": + return "doubleValue"; + default: + throw new IllegalArgumentException( + "don't know how to fetch primitive values from " + stateType + ". define combineStates." + ); + } + } + + private MethodSpec evaluateIntermediate() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evaluateIntermediate"); + builder.addAnnotation(Override.class) + .addModifiers(Modifier.PUBLIC) + .addParameter(BLOCK_ARRAY, "blocks") + .addParameter(TypeName.INT, "offset"); + builder.addStatement("state.toIntermediate(blocks, offset)"); + return builder.build(); + } + + private MethodSpec evaluateFinal() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evaluateFinal"); + builder.addAnnotation(Override.class) + .addModifiers(Modifier.PUBLIC) + .addParameter(BLOCK_ARRAY, "blocks") + .addParameter(TypeName.INT, "offset"); + if (stateTypeHasSeen) { + builder.beginControlFlow("if (state.seen() == false)"); + builder.addStatement("blocks[offset] = $T.constantNullBlock(1)", BLOCK); + builder.addStatement("return"); + builder.endControlFlow(); + } + if (evaluateFinal == null) { + primitiveStateToResult(builder); + } else { + builder.addStatement("blocks[offset] = $T.evaluateFinal(state)", declarationType); + } + return builder.build(); + } + + private void primitiveStateToResult(MethodSpec.Builder builder) { + switch (stateType.toString()) { + case "org.elasticsearch.compute.aggregation.IntState": + builder.addStatement("blocks[offset] = $T.newConstantBlockWith(state.intValue(), 1)", INT_BLOCK); + return; + case "org.elasticsearch.compute.aggregation.LongState": + builder.addStatement("blocks[offset] = $T.newConstantBlockWith(state.longValue(), 1)", LONG_BLOCK); + return; + case "org.elasticsearch.compute.aggregation.DoubleState": + builder.addStatement("blocks[offset] = $T.newConstantBlockWith(state.doubleValue(), 1)", DOUBLE_BLOCK); + return; + default: + throw new IllegalArgumentException("don't know how to convert state to result: " + stateType); + } + } + + private MethodSpec toStringMethod() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("toString"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).returns(String.class); + builder.addStatement("$T sb = new $T()", StringBuilder.class, StringBuilder.class); + builder.addStatement("sb.append(getClass().getSimpleName()).append($S)", "["); + builder.addStatement("sb.append($S).append(channels)", "channels="); + builder.addStatement("sb.append($S)", "]"); + builder.addStatement("return sb.toString()"); + return builder.build(); + } + + private MethodSpec close() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("close"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addStatement("state.close()"); + return builder.build(); + } + + private boolean hasPrimitiveState() { + return switch (stateType.toString()) { + case "org.elasticsearch.compute.aggregation.IntState", "org.elasticsearch.compute.aggregation.LongState", + "org.elasticsearch.compute.aggregation.DoubleState" -> true; + default -> false; + }; + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorProcessor.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorProcessor.java new file mode 100644 index 0000000000000..b724ee9152ca8 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/AggregatorProcessor.java @@ -0,0 +1,132 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.JavaFile; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +import java.io.IOException; +import java.io.Writer; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Set; + +import javax.annotation.processing.Completion; +import javax.annotation.processing.Filer; +import javax.annotation.processing.ProcessingEnvironment; +import javax.annotation.processing.Processor; +import javax.annotation.processing.RoundEnvironment; +import javax.lang.model.SourceVersion; +import javax.lang.model.element.AnnotationMirror; +import javax.lang.model.element.Element; +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.TypeElement; +import javax.tools.Diagnostic; +import javax.tools.JavaFileObject; + +/** + * Glues the {@link AggregatorImplementer} into the jdk's annotation + * processing framework. + */ +public class AggregatorProcessor implements Processor { + private ProcessingEnvironment env; + + @Override + public Set getSupportedOptions() { + return Set.of(); + } + + @Override + public Set getSupportedAnnotationTypes() { + return Set.of(Aggregator.class.getName(), GroupingAggregator.class.getName()); + } + + @Override + public SourceVersion getSupportedSourceVersion() { + return SourceVersion.RELEASE_17; + } + + @Override + public void init(ProcessingEnvironment processingEnvironment) { + this.env = processingEnvironment; + } + + @Override + public Iterable getCompletions( + Element element, + AnnotationMirror annotationMirror, + ExecutableElement executableElement, + String s + ) { + return List.of(); + } + + @Override + public boolean process(Set set, RoundEnvironment roundEnvironment) { + Set annotatedClasses = Collections.newSetFromMap(new IdentityHashMap<>()); + for (TypeElement ann : set) { + for (Element aggClass : roundEnvironment.getElementsAnnotatedWith(ann)) { + annotatedClasses.add((TypeElement) aggClass); + } + } + for (TypeElement aggClass : annotatedClasses) { + AggregatorImplementer implementer = null; + if (aggClass.getAnnotation(Aggregator.class) != null) { + IntermediateState[] intermediateState = aggClass.getAnnotation(Aggregator.class).value(); + implementer = new AggregatorImplementer(env.getElementUtils(), aggClass, intermediateState); + write(aggClass, "aggregator", implementer.sourceFile(), env); + } + GroupingAggregatorImplementer groupingAggregatorImplementer = null; + if (aggClass.getAnnotation(Aggregator.class) != null) { + assert aggClass.getAnnotation(GroupingAggregator.class) != null; + IntermediateState[] intermediateState = aggClass.getAnnotation(GroupingAggregator.class).value(); + if (intermediateState.length == 0) { + intermediateState = aggClass.getAnnotation(Aggregator.class).value(); + } + + groupingAggregatorImplementer = new GroupingAggregatorImplementer(env.getElementUtils(), aggClass, intermediateState); + write(aggClass, "grouping aggregator", groupingAggregatorImplementer.sourceFile(), env); + } + if (implementer != null && groupingAggregatorImplementer != null) { + write( + aggClass, + "aggregator function supplier", + new AggregatorFunctionSupplierImplementer(env.getElementUtils(), aggClass, implementer, groupingAggregatorImplementer) + .sourceFile(), + env + ); + } + } + return true; + } + + /** + * Just like {@link JavaFile#writeTo(Filer)} but on windows it replaces {@code \n} with {@code \r\n}. + */ + public static void write(Object origination, String what, JavaFile file, ProcessingEnvironment env) { + try { + String fileName = file.packageName + "." + file.typeSpec.name; + JavaFileObject filerSourceFile = env.getFiler() + .createSourceFile(fileName, file.typeSpec.originatingElements.toArray(Element[]::new)); + try (Writer w = filerSourceFile.openWriter()) { + if (System.getProperty("line.separator").equals("\n")) { + file.writeTo(w); + } else { + w.write(file.toString().replace("\n", System.getProperty("line.separator"))); + } + } + } catch (IOException e) { + env.getMessager().printMessage(Diagnostic.Kind.ERROR, "failed generating " + what + " for " + origination); + throw new RuntimeException(e); + } + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConsumeProcessor.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConsumeProcessor.java new file mode 100644 index 0000000000000..d33f5e3924ea9 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConsumeProcessor.java @@ -0,0 +1,62 @@ +/* + * 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.compute.gen; + +import org.elasticsearch.compute.ann.Fixed; + +import java.util.List; +import java.util.Set; + +import javax.annotation.processing.Completion; +import javax.annotation.processing.ProcessingEnvironment; +import javax.annotation.processing.Processor; +import javax.annotation.processing.RoundEnvironment; +import javax.lang.model.SourceVersion; +import javax.lang.model.element.AnnotationMirror; +import javax.lang.model.element.Element; +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.TypeElement; + +/** + * Consumes the "Nullable" and "Inject" annotations and does nothing with them + * to prevent warnings when running annotation processors. + */ +public class ConsumeProcessor implements Processor { + @Override + public Set getSupportedOptions() { + return Set.of(); + } + + @Override + public Set getSupportedAnnotationTypes() { + return Set.of("org.elasticsearch.core.Nullable", "org.elasticsearch.common.inject.Inject", Fixed.class.getName()); + } + + @Override + public SourceVersion getSupportedSourceVersion() { + return SourceVersion.latest(); + } + + @Override + public void init(ProcessingEnvironment processingEnvironment) {} + + @Override + public boolean process(Set set, RoundEnvironment roundEnvironment) { + return true; + } + + @Override + public Iterable getCompletions( + Element element, + AnnotationMirror annotationMirror, + ExecutableElement executableElement, + String s + ) { + return List.of(); + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConvertEvaluatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConvertEvaluatorImplementer.java new file mode 100644 index 0000000000000..a7dce57348fb6 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/ConvertEvaluatorImplementer.java @@ -0,0 +1,282 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import java.util.BitSet; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.util.Elements; + +import static org.elasticsearch.compute.gen.Methods.appendMethod; +import static org.elasticsearch.compute.gen.Methods.getMethod; +import static org.elasticsearch.compute.gen.Types.ABSTRACT_CONVERT_FUNCTION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.BIG_ARRAYS; +import static org.elasticsearch.compute.gen.Types.BLOCK; +import static org.elasticsearch.compute.gen.Types.BYTES_REF; +import static org.elasticsearch.compute.gen.Types.BYTES_REF_ARRAY; +import static org.elasticsearch.compute.gen.Types.EXPRESSION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.SOURCE; +import static org.elasticsearch.compute.gen.Types.VECTOR; +import static org.elasticsearch.compute.gen.Types.arrayBlockType; +import static org.elasticsearch.compute.gen.Types.arrayVectorType; +import static org.elasticsearch.compute.gen.Types.blockType; +import static org.elasticsearch.compute.gen.Types.constantVectorType; +import static org.elasticsearch.compute.gen.Types.vectorType; + +public class ConvertEvaluatorImplementer { + + private final TypeElement declarationType; + private final ExecutableElement processFunction; + private final ClassName implementation; + private final TypeName argumentType; + private final TypeName resultType; + + public ConvertEvaluatorImplementer(Elements elements, ExecutableElement processFunction, String extraName) { + this.declarationType = (TypeElement) processFunction.getEnclosingElement(); + this.processFunction = processFunction; + if (processFunction.getParameters().size() != 1) { + throw new IllegalArgumentException("processing function should have exactly one parameter"); + } + this.argumentType = TypeName.get(processFunction.getParameters().get(0).asType()); + this.resultType = TypeName.get(processFunction.getReturnType()); + + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + declarationType.getSimpleName() + extraName + "Evaluator" + ); + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", EXPRESSION_EVALUATOR, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + builder.superclass(ABSTRACT_CONVERT_FUNCTION_EVALUATOR); + + builder.addMethod(ctor()); + builder.addMethod(name()); + builder.addMethod(evalVector()); + builder.addMethod(evalValue(true)); + builder.addMethod(evalBlock()); + builder.addMethod(evalValue(false)); + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + builder.addParameter(EXPRESSION_EVALUATOR, "field"); + builder.addParameter(SOURCE, "source"); + builder.addStatement("super($N, $N)", "field", "source"); + return builder.build(); + } + + private MethodSpec name() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("name").addModifiers(Modifier.PUBLIC); + builder.addAnnotation(Override.class).returns(String.class); + builder.addStatement("return $S", declarationType.getSimpleName()); + return builder.build(); + } + + private MethodSpec evalVector() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evalVector").addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addParameter(VECTOR, "v").returns(BLOCK); + + TypeName vectorType = vectorType(argumentType); + builder.addStatement("$T vector = ($T) v", vectorType, vectorType); + builder.addStatement("int positionCount = v.getPositionCount()"); + + String scratchPadName = null; + if (argumentType.equals(BYTES_REF)) { + scratchPadName = "scratchPad"; + builder.addStatement("BytesRef $N = new BytesRef()", scratchPadName); + } + + builder.beginControlFlow("if (vector.isConstant())"); + { + builder.beginControlFlow("try"); + { + var constVectType = constantVectorType(resultType); + builder.addStatement( + "return new $T($N, positionCount).asBlock()", + constVectType, + evalValueCall("vector", "0", scratchPadName) + ); + } + builder.nextControlFlow("catch (Exception e)"); + { + builder.addStatement("registerException(e)"); + builder.addStatement("return Block.constantNullBlock(positionCount)"); + } + builder.endControlFlow(); + } + builder.endControlFlow(); + + builder.addStatement("$T nullsMask = null", BitSet.class); + if (resultType.equals(BYTES_REF)) { + builder.addStatement( + "$T values = new $T(positionCount, $T.NON_RECYCLING_INSTANCE)", // TODO: see note in MvEvaluatorImplementer + BYTES_REF_ARRAY, + BYTES_REF_ARRAY, + BIG_ARRAYS + ); + } else { + builder.addStatement("$T[] values = new $T[positionCount]", resultType, resultType); + } + builder.beginControlFlow("for (int p = 0; p < positionCount; p++)"); + { + builder.beginControlFlow("try"); + { + if (resultType.equals(BYTES_REF)) { + builder.addStatement("values.append($N)", evalValueCall("vector", "p", scratchPadName)); + } else { + builder.addStatement("values[p] = $N", evalValueCall("vector", "p", scratchPadName)); + } + } + builder.nextControlFlow("catch (Exception e)"); + { + builder.addStatement("registerException(e)"); + builder.beginControlFlow("if (nullsMask == null)"); + { + builder.addStatement("nullsMask = new BitSet(positionCount)"); + } + builder.endControlFlow(); + builder.addStatement("nullsMask.set(p)"); + } + builder.endControlFlow(); + } + builder.endControlFlow(); + + builder.addStatement( + """ + return nullsMask == null + ? new $T(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new $T(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED)""", + arrayVectorType(resultType), + arrayBlockType(resultType) + ); + + return builder.build(); + } + + private MethodSpec evalBlock() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evalBlock").addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addParameter(BLOCK, "b").returns(BLOCK); + + TypeName blockType = blockType(argumentType); + builder.addStatement("$T block = ($T) b", blockType, blockType); + builder.addStatement("int positionCount = block.getPositionCount()"); + TypeName resultBlockType = blockType(resultType); + builder.addStatement("$T.Builder builder = $T.newBlockBuilder(positionCount)", resultBlockType, resultBlockType); + String scratchPadName = null; + if (argumentType.equals(BYTES_REF)) { + scratchPadName = "scratchPad"; + builder.addStatement("BytesRef $N = new BytesRef()", scratchPadName); + } + + String appendMethod = appendMethod(resultType); + builder.beginControlFlow("for (int p = 0; p < positionCount; p++)"); + { + builder.addStatement("int valueCount = block.getValueCount(p)"); + builder.addStatement("int start = block.getFirstValueIndex(p)"); + builder.addStatement("int end = start + valueCount"); + builder.addStatement("boolean positionOpened = false"); + builder.addStatement("boolean valuesAppended = false"); + // builder.addStatement("builder.beginPositionEntry()"); + builder.beginControlFlow("for (int i = start; i < end; i++)"); + { + builder.beginControlFlow("try"); + { + builder.addStatement("$T value = $N", resultType, evalValueCall("block", "i", scratchPadName)); + builder.beginControlFlow("if (positionOpened == false && valueCount > 1)"); + { + builder.addStatement("builder.beginPositionEntry()"); + builder.addStatement("positionOpened = true"); + } + builder.endControlFlow(); + builder.addStatement("builder.$N(value)", appendMethod); + builder.addStatement("valuesAppended = true"); + } + builder.nextControlFlow("catch (Exception e)"); + { + builder.addStatement("registerException(e)"); + } + builder.endControlFlow(); + } + builder.endControlFlow(); + builder.beginControlFlow("if (valuesAppended == false)"); + { + builder.addStatement("builder.appendNull()"); + } + builder.nextControlFlow("else if (positionOpened)"); + { + builder.addStatement("builder.endPositionEntry()"); + } + builder.endControlFlow(); + } + builder.endControlFlow(); + + builder.addStatement("return builder.build()"); + + return builder.build(); + } + + private String evalValueCall(String container, String index, String scratchPad) { + StringBuilder builder = new StringBuilder("evalValue("); + builder.append(container); + builder.append(", "); + builder.append(index); + if (scratchPad != null) { + builder.append(", "); + builder.append(scratchPad); + } + builder.append(")"); + return builder.toString(); + } + + private MethodSpec evalValue(boolean forVector) { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evalValue") + .addModifiers(Modifier.PRIVATE, Modifier.STATIC) + .returns(resultType); + + if (forVector) { + builder.addParameter(vectorType(argumentType), "container"); + } else { + builder.addParameter(blockType(argumentType), "container"); + } + builder.addParameter(TypeName.INT, "index"); + if (argumentType.equals(BYTES_REF)) { + builder.addParameter(BYTES_REF, "scratchPad"); + builder.addStatement("$T value = container.$N(index, scratchPad)", argumentType, getMethod(argumentType)); + } else { + builder.addStatement("$T value = container.$N(index)", argumentType, getMethod(argumentType)); + } + + builder.addStatement("return $T.$N(value)", declarationType, processFunction.getSimpleName()); + + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java new file mode 100644 index 0000000000000..4ea351084bcc4 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java @@ -0,0 +1,642 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ArrayTypeName; +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import org.elasticsearch.compute.ann.Fixed; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.type.ArrayType; +import javax.lang.model.type.TypeKind; +import javax.lang.model.type.TypeMirror; +import javax.lang.model.util.Elements; + +import static org.elasticsearch.compute.gen.Methods.appendMethod; +import static org.elasticsearch.compute.gen.Methods.getMethod; +import static org.elasticsearch.compute.gen.Types.BLOCK; +import static org.elasticsearch.compute.gen.Types.BYTES_REF; +import static org.elasticsearch.compute.gen.Types.EXPRESSION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.PAGE; +import static org.elasticsearch.compute.gen.Types.SOURCE; +import static org.elasticsearch.compute.gen.Types.WARNINGS; +import static org.elasticsearch.compute.gen.Types.blockType; +import static org.elasticsearch.compute.gen.Types.vectorType; + +public class EvaluatorImplementer { + private final TypeElement declarationType; + private final ProcessFunction processFunction; + private final ClassName implementation; + + public EvaluatorImplementer(Elements elements, ExecutableElement processFunction, String extraName, List warnExceptions) { + this.declarationType = (TypeElement) processFunction.getEnclosingElement(); + this.processFunction = new ProcessFunction(processFunction, warnExceptions); + + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + declarationType.getSimpleName() + extraName + "Evaluator" + ); + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", EXPRESSION_EVALUATOR, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + builder.addSuperinterface(EXPRESSION_EVALUATOR); + + if (processFunction.warnExceptions.isEmpty() == false) { + builder.addField(WARNINGS, "warnings", Modifier.PRIVATE, Modifier.FINAL); + } + processFunction.args.stream().forEach(a -> a.declareField(builder)); + + builder.addMethod(ctor()); + builder.addMethod(eval()); + if (processFunction.args.stream().anyMatch(x -> x instanceof FixedProcessFunctionArg == false)) { + builder.addMethod(realEval(true)); + } + builder.addMethod(realEval(false)); + builder.addMethod(toStringMethod()); + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + if (processFunction.warnExceptions.isEmpty() == false) { + builder.addParameter(SOURCE, "source"); + builder.addStatement("this.warnings = new Warnings(source)"); + } + processFunction.args.stream().forEach(a -> a.implementCtor(builder)); + return builder.build(); + } + + private MethodSpec eval() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("eval").addAnnotation(Override.class); + builder.addModifiers(Modifier.PUBLIC).returns(BLOCK).addParameter(PAGE, "page"); + + processFunction.args.stream().forEach(a -> a.evalToBlock(builder)); + String invokeBlockEval = invokeRealEval(true); + processFunction.args.stream().forEach(a -> a.resolveVectors(builder, invokeBlockEval)); + builder.addStatement(invokeRealEval(false)); + return builder.build(); + } + + private String invokeRealEval(boolean blockStyle) { + StringBuilder builder = new StringBuilder("return eval(page.getPositionCount()"); + String params = processFunction.args.stream() + .map(a -> a.paramName(blockStyle)) + .filter(a -> a != null) + .collect(Collectors.joining(", ")); + if (params.length() > 0) { + builder.append(", "); + builder.append(params); + } + builder.append(")"); + if (processFunction.resultDataType(blockStyle).simpleName().endsWith("Vector")) { + builder.append(".asBlock()"); + } + return builder.toString(); + } + + private MethodSpec realEval(boolean blockStyle) { + ClassName resultDataType = processFunction.resultDataType(blockStyle); + MethodSpec.Builder builder = MethodSpec.methodBuilder("eval"); + builder.addModifiers(Modifier.PUBLIC).returns(resultDataType); + builder.addParameter(TypeName.INT, "positionCount"); + + processFunction.args.stream().forEach(a -> { + if (a.paramName(blockStyle) != null) { + builder.addParameter(a.dataType(blockStyle), a.paramName(blockStyle)); + } + }); + builder.addStatement( + "$T.Builder result = $T.$L(positionCount)", + resultDataType, + resultDataType, + resultDataType.simpleName().endsWith("Vector") ? "newVectorBuilder" : "newBlockBuilder" + ); + processFunction.args.stream().forEach(a -> a.createScratch(builder)); + + builder.beginControlFlow("position: for (int p = 0; p < positionCount; p++)"); + { + if (blockStyle) { + processFunction.args.stream().forEach(a -> a.skipNull(builder)); + } + processFunction.args.stream().forEach(a -> a.unpackValues(builder, blockStyle)); + + StringBuilder pattern = new StringBuilder(); + List args = new ArrayList<>(); + pattern.append("$T.$N("); + args.add(declarationType); + args.add(processFunction.function.getSimpleName()); + processFunction.args.stream().forEach(a -> { + if (args.size() > 2) { + pattern.append(", "); + } + a.buildInvocation(pattern, args, blockStyle); + }); + pattern.append(")"); + String builtPattern; + if (processFunction.builderArg == null) { + builtPattern = "result.$L(" + pattern + ")"; + args.add(0, appendMethod(resultDataType)); + } else { + builtPattern = pattern.toString(); + } + + if (processFunction.warnExceptions.isEmpty() == false) { + builder.beginControlFlow("try"); + } + builder.addStatement(builtPattern, args.toArray()); + if (processFunction.warnExceptions.isEmpty() == false) { + String catchPattern = "catch (" + + processFunction.warnExceptions.stream().map(m -> "$T").collect(Collectors.joining(" | ")) + + " e)"; + builder.nextControlFlow(catchPattern, processFunction.warnExceptions.stream().map(m -> TypeName.get(m)).toArray()); + builder.addStatement("warnings.registerException(e)"); + builder.addStatement("result.appendNull()"); + builder.endControlFlow(); + } + } + builder.endControlFlow(); + builder.addStatement("return result.build()"); + return builder.build(); + } + + private static void skipNull(MethodSpec.Builder builder, String value) { + builder.beginControlFlow("if ($N.isNull(p) || $N.getValueCount(p) != 1)", value, value); + { + builder.addStatement("result.appendNull()"); + builder.addStatement("continue position"); + } + builder.endControlFlow(); + } + + private MethodSpec toStringMethod() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("toString").addAnnotation(Override.class); + builder.addModifiers(Modifier.PUBLIC).returns(String.class); + + StringBuilder pattern = new StringBuilder(); + List args = new ArrayList<>(); + pattern.append("return $S"); + args.add(implementation.simpleName() + "["); + processFunction.args.stream().forEach(a -> a.buildToStringInvocation(pattern, args, args.size() > 2 ? ", " : "")); + pattern.append(" + $S"); + args.add("]"); + builder.addStatement(pattern.toString(), args.toArray()); + return builder.build(); + } + + private interface ProcessFunctionArg { + /** + * Type containing the actual data for a page of values for this field. Usually a + * Block or Vector, but for fixed fields will be the original fixed type. + */ + TypeName dataType(boolean blockStyle); + + /** + * The parameter passed to the real evaluation function + */ + String paramName(boolean blockStyle); + + /** + * Declare any required fields on the type for this parameter. + */ + void declareField(TypeSpec.Builder builder); + + /** + * Implement the ctor for this parameter. Will declare parameters + * and assign values to declared fields. + */ + void implementCtor(MethodSpec.Builder builder); + + /** + * Emits code to evaluate this parameter to a Block or array of Blocks. + * Noop if the parameter is {@link Fixed}. + */ + void evalToBlock(MethodSpec.Builder builder); + + /** + * Emits code to check if this parameter is a vector or a block, and to + * call the block flavored evaluator if this is a block. Noop if the + * parameter is {@link Fixed}. + */ + void resolveVectors(MethodSpec.Builder builder, String invokeBlockEval); + + /** + * Create any scratch structures needed by {@link EvaluatorImplementer#realEval}. + */ + void createScratch(MethodSpec.Builder builder); + + /** + * Skip any null values in blocks containing this field. + */ + void skipNull(MethodSpec.Builder builder); + + /** + * Unpacks values from blocks and repacks them into an appropriate local. Noop + * except for arrays. + */ + void unpackValues(MethodSpec.Builder builder, boolean blockStyle); + + /** + * Build the invocation of the process method for this parameter. + */ + void buildInvocation(StringBuilder pattern, List args, boolean blockStyle); + + void buildToStringInvocation(StringBuilder pattern, List args, String prefix); + } + + private record StandardProcessFunctionArg(TypeName type, String name) implements ProcessFunctionArg { + @Override + public TypeName dataType(boolean blockStyle) { + if (blockStyle) { + return blockType(type); + } + return vectorType(type); + } + + @Override + public String paramName(boolean blockStyle) { + return name + (blockStyle ? "Block" : "Vector"); + } + + @Override + public void declareField(TypeSpec.Builder builder) { + builder.addField(EXPRESSION_EVALUATOR, name, Modifier.PRIVATE, Modifier.FINAL); + } + + @Override + public void implementCtor(MethodSpec.Builder builder) { + builder.addParameter(EXPRESSION_EVALUATOR, name); + builder.addStatement("this.$L = $L", name, name); + } + + @Override + public void evalToBlock(MethodSpec.Builder builder) { + TypeName blockType = blockType(type); + builder.addStatement("Block $LUncastBlock = $L.eval(page)", name, name); + builder.beginControlFlow("if ($LUncastBlock.areAllValuesNull())", name); + builder.addStatement("return Block.constantNullBlock(page.getPositionCount())"); + builder.endControlFlow(); + builder.addStatement("$T $LBlock = ($T) $LUncastBlock", blockType, name, blockType, name); + } + + @Override + public void resolveVectors(MethodSpec.Builder builder, String invokeBlockEval) { + builder.addStatement("$T $LVector = $LBlock.asVector()", vectorType(type), name, name); + builder.beginControlFlow("if ($LVector == null)", name).addStatement(invokeBlockEval).endControlFlow(); + } + + @Override + public void createScratch(MethodSpec.Builder builder) { + if (type.equals(BYTES_REF)) { + builder.addStatement("$T $LScratch = new $T()", BYTES_REF, name, BYTES_REF); + } + } + + @Override + public void skipNull(MethodSpec.Builder builder) { + EvaluatorImplementer.skipNull(builder, paramName(true)); + } + + @Override + public void unpackValues(MethodSpec.Builder builder, boolean blockStyle) { + // nothing to do + } + + @Override + public void buildInvocation(StringBuilder pattern, List args, boolean blockStyle) { + if (type.equals(BYTES_REF)) { + if (blockStyle) { + pattern.append("$L.getBytesRef($L.getFirstValueIndex(p), $LScratch)"); + args.add(paramName(true)); + } else { + pattern.append("$L.getBytesRef(p, $LScratch)"); + } + args.add(paramName(blockStyle)); + args.add(name); + return; + } + if (blockStyle) { + pattern.append("$L.$L($L.getFirstValueIndex(p))"); + } else { + pattern.append("$L.$L(p)"); + } + args.add(paramName(blockStyle)); + args.add(getMethod(type)); + if (blockStyle) { + args.add(paramName(true)); + } + } + + @Override + public void buildToStringInvocation(StringBuilder pattern, List args, String prefix) { + pattern.append(" + $S + $L"); + args.add(prefix + name + "="); + args.add(name); + } + } + + private record ArrayProcessFunctionArg(TypeName componentType, String name) implements ProcessFunctionArg { + @Override + public TypeName dataType(boolean blockStyle) { + if (blockStyle) { + return ArrayTypeName.of(blockType(componentType)); + } + return ArrayTypeName.of(vectorType(componentType)); + } + + @Override + public String paramName(boolean blockStyle) { + return name + (blockStyle ? "Block" : "Vector") + "s"; + } + + @Override + public void declareField(TypeSpec.Builder builder) { + builder.addField(ArrayTypeName.of(EXPRESSION_EVALUATOR), name, Modifier.PRIVATE, Modifier.FINAL); + } + + @Override + public void implementCtor(MethodSpec.Builder builder) { + builder.addParameter(ArrayTypeName.of(EXPRESSION_EVALUATOR), name); + builder.addStatement("this.$L = $L", name, name); + } + + @Override + public void evalToBlock(MethodSpec.Builder builder) { + TypeName blockType = blockType(componentType); + builder.addStatement("$T[] $LBlocks = new $T[$L.length]", blockType, name, blockType, name); + builder.beginControlFlow("for (int i = 0; i < $LBlocks.length; i++)", name); + { + builder.addStatement("Block block = $L[i].eval(page)", name); + builder.beginControlFlow("if (block.areAllValuesNull())"); + builder.addStatement("return Block.constantNullBlock(page.getPositionCount())"); + builder.endControlFlow(); + builder.addStatement("$LBlocks[i] = ($T) block", name, blockType); + } + builder.endControlFlow(); + } + + @Override + public void resolveVectors(MethodSpec.Builder builder, String invokeBlockEval) { + TypeName vectorType = vectorType(componentType); + builder.addStatement("$T[] $LVectors = new $T[$L.length]", vectorType, name, vectorType, name); + builder.beginControlFlow("for (int i = 0; i < $LBlocks.length; i++)", name); + builder.addStatement("$LVectors[i] = $LBlocks[i].asVector()", name, name); + builder.beginControlFlow("if ($LVectors[i] == null)", name).addStatement(invokeBlockEval).endControlFlow(); + builder.endControlFlow(); + } + + @Override + public void createScratch(MethodSpec.Builder builder) { + builder.addStatement("$T[] $LValues = new $T[$L.length]", componentType, name, componentType, name); + if (componentType.equals(BYTES_REF)) { + builder.addStatement("$T[] $LScratch = new $T[$L.length]", componentType, name, componentType, name); + builder.beginControlFlow("for (int i = 0; i < $L.length; i++)", name); + builder.addStatement("$LScratch[i] = new $T()", name, BYTES_REF); + builder.endControlFlow(); + } + } + + @Override + public void skipNull(MethodSpec.Builder builder) { + builder.beginControlFlow("for (int i = 0; i < $L.length; i++)", paramName(true)); + EvaluatorImplementer.skipNull(builder, paramName(true) + "[i]"); + builder.endControlFlow(); + } + + @Override + public void unpackValues(MethodSpec.Builder builder, boolean blockStyle) { + builder.addComment("unpack $L into $LValues", paramName(blockStyle), name); + builder.beginControlFlow("for (int i = 0; i < $L.length; i++)", paramName(blockStyle)); + String lookupVar; + if (blockStyle) { + lookupVar = "o"; + builder.addStatement("int o = $LBlocks[i].getFirstValueIndex(p)", name); + } else { + lookupVar = "p"; + } + if (componentType.equals(BYTES_REF)) { + builder.addStatement("$LValues[i] = $L[i].getBytesRef($L, $LScratch[i])", name, paramName(blockStyle), lookupVar, name); + } else { + builder.addStatement("$LValues[i] = $L[i].$L($L)", name, paramName(blockStyle), getMethod(componentType), lookupVar); + } + builder.endControlFlow(); + } + + @Override + public void buildInvocation(StringBuilder pattern, List args, boolean blockStyle) { + pattern.append("$LValues"); + args.add(name); + } + + @Override + public void buildToStringInvocation(StringBuilder pattern, List args, String prefix) { + pattern.append(" + $S + $T.toString($L)"); + args.add(prefix + name + "="); + args.add(Arrays.class); + args.add(name); + } + } + + private record FixedProcessFunctionArg(TypeName type, String name, boolean includeInToString) implements ProcessFunctionArg { + @Override + public TypeName dataType(boolean blockStyle) { + return type; + } + + @Override + public String paramName(boolean blockStyle) { + // No need to pass it + return null; + } + + @Override + public void declareField(TypeSpec.Builder builder) { + builder.addField(type, name, Modifier.PRIVATE, Modifier.FINAL); + } + + @Override + public void implementCtor(MethodSpec.Builder builder) { + builder.addParameter(type, name); + builder.addStatement("this.$L = $L", name, name); + } + + @Override + public void evalToBlock(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void resolveVectors(MethodSpec.Builder builder, String invokeBlockEval) { + // nothing to do + } + + @Override + public void createScratch(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void skipNull(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void unpackValues(MethodSpec.Builder builder, boolean blockStyle) { + // nothing to do + } + + @Override + public void buildInvocation(StringBuilder pattern, List args, boolean blockStyle) { + pattern.append("$L"); + args.add(name); + } + + @Override + public void buildToStringInvocation(StringBuilder pattern, List args, String prefix) { + if (includeInToString) { + pattern.append(" + $S + $L"); + args.add(prefix + name + "="); + args.add(name); + } + } + } + + private record BuilderProcessFunctionArg(ClassName type, String name) implements ProcessFunctionArg { + @Override + public TypeName dataType(boolean blockStyle) { + return type; + } + + @Override + public String paramName(boolean blockStyle) { + // never passed as a parameter + return null; + } + + @Override + public void declareField(TypeSpec.Builder builder) { + // Nothing to declare + } + + @Override + public void implementCtor(MethodSpec.Builder builder) { + // Nothing to do + } + + @Override + public void evalToBlock(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void resolveVectors(MethodSpec.Builder builder, String invokeBlockEval) { + // nothing to do + } + + @Override + public void createScratch(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void skipNull(MethodSpec.Builder builder) { + // nothing to do + } + + @Override + public void unpackValues(MethodSpec.Builder builder, boolean blockStyle) { + // nothing to do + } + + @Override + public void buildInvocation(StringBuilder pattern, List args, boolean blockStyle) { + pattern.append("$L"); + args.add("result"); + } + + @Override + public void buildToStringInvocation(StringBuilder pattern, List args, String prefix) { + // Don't want to include + } + } + + private static class ProcessFunction { + private final ExecutableElement function; + private final List args; + private final BuilderProcessFunctionArg builderArg; + private final List warnExceptions; + + private ProcessFunction(ExecutableElement function, List warnExceptions) { + this.function = function; + args = new ArrayList<>(); + BuilderProcessFunctionArg builderArg = null; + for (VariableElement v : function.getParameters()) { + TypeName type = TypeName.get(v.asType()); + String name = v.getSimpleName().toString(); + Fixed fixed = v.getAnnotation(Fixed.class); + if (fixed != null) { + args.add(new FixedProcessFunctionArg(type, name, fixed.includeInToString())); + continue; + } + if (type instanceof ClassName c + && c.simpleName().equals("Builder") + && c.enclosingClassName() != null + && c.enclosingClassName().simpleName().endsWith("Block")) { + if (builderArg != null) { + throw new IllegalArgumentException("only one builder allowed"); + } + builderArg = new BuilderProcessFunctionArg(c, name); + args.add(builderArg); + continue; + } + if (v.asType().getKind() == TypeKind.ARRAY) { + TypeMirror componentType = ((ArrayType) v.asType()).getComponentType(); + args.add(new ArrayProcessFunctionArg(TypeName.get(componentType), name)); + continue; + } + args.add(new StandardProcessFunctionArg(type, name)); + } + this.builderArg = builderArg; + this.warnExceptions = warnExceptions; + } + + private ClassName resultDataType(boolean blockStyle) { + if (builderArg != null) { + return builderArg.type.enclosingClassName(); + } + boolean useBlockStyle = blockStyle || warnExceptions.isEmpty() == false; + return useBlockStyle ? blockType(TypeName.get(function.getReturnType())) : vectorType(TypeName.get(function.getReturnType())); + } + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorProcessor.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorProcessor.java new file mode 100644 index 0000000000000..d640e2b8633f2 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorProcessor.java @@ -0,0 +1,137 @@ +/* + * 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.compute.gen; + +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.MvEvaluator; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import javax.annotation.processing.Completion; +import javax.annotation.processing.ProcessingEnvironment; +import javax.annotation.processing.Processor; +import javax.annotation.processing.RoundEnvironment; +import javax.lang.model.SourceVersion; +import javax.lang.model.element.AnnotationMirror; +import javax.lang.model.element.AnnotationValue; +import javax.lang.model.element.Element; +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.TypeElement; +import javax.lang.model.type.TypeMirror; + +/** + * Glues the {@link EvaluatorImplementer} into the jdk's annotation + * processing framework. + */ +public class EvaluatorProcessor implements Processor { + private ProcessingEnvironment env; + + @Override + public Set getSupportedOptions() { + return Set.of(); + } + + @Override + public Set getSupportedAnnotationTypes() { + return Set.of(Evaluator.class.getName(), MvEvaluator.class.getName(), ConvertEvaluator.class.getName()); + } + + @Override + public SourceVersion getSupportedSourceVersion() { + return SourceVersion.RELEASE_17; + } + + @Override + public void init(ProcessingEnvironment processingEnvironment) { + this.env = processingEnvironment; + } + + @Override + public Iterable getCompletions( + Element element, + AnnotationMirror annotationMirror, + ExecutableElement executableElement, + String s + ) { + return List.of(); + } + + @Override + public boolean process(Set set, RoundEnvironment roundEnvironment) { + for (TypeElement ann : set) { + for (Element evaluatorMethod : roundEnvironment.getElementsAnnotatedWith(ann)) { + Evaluator evaluatorAnn = evaluatorMethod.getAnnotation(Evaluator.class); + if (evaluatorAnn != null) { + AggregatorProcessor.write( + evaluatorMethod, + "evaluator", + new EvaluatorImplementer( + env.getElementUtils(), + (ExecutableElement) evaluatorMethod, + evaluatorAnn.extraName(), + warnExceptions(evaluatorMethod) + ).sourceFile(), + env + ); + } + MvEvaluator mvEvaluatorAnn = evaluatorMethod.getAnnotation(MvEvaluator.class); + if (mvEvaluatorAnn != null) { + AggregatorProcessor.write( + evaluatorMethod, + "evaluator", + new MvEvaluatorImplementer( + env.getElementUtils(), + (ExecutableElement) evaluatorMethod, + mvEvaluatorAnn.extraName(), + mvEvaluatorAnn.finish(), + mvEvaluatorAnn.single(), + mvEvaluatorAnn.ascending(), + warnExceptions(evaluatorMethod) + ).sourceFile(), + env + ); + } + ConvertEvaluator convertEvaluatorAnn = evaluatorMethod.getAnnotation(ConvertEvaluator.class); + if (convertEvaluatorAnn != null) { + AggregatorProcessor.write( + evaluatorMethod, + "evaluator", + new ConvertEvaluatorImplementer( + env.getElementUtils(), + (ExecutableElement) evaluatorMethod, + convertEvaluatorAnn.extraName() + ).sourceFile(), + env + ); + } + } + } + return true; + } + + private static List warnExceptions(Element evaluatorMethod) { + List result = new ArrayList<>(); + for (var mirror : evaluatorMethod.getAnnotationMirrors()) { + String annotationType = mirror.getAnnotationType().toString(); + if (annotationType.equals(Evaluator.class.getName()) || annotationType.equals(MvEvaluator.class.getName())) { + for (var e : mirror.getElementValues().entrySet()) { + if (false == e.getKey().getSimpleName().toString().equals("warnExceptions")) { + continue; + } + for (var v : (List) e.getValue().getValue()) { + result.add((TypeMirror) ((AnnotationValue) v).getValue()); + } + } + } + } + return result; + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/GroupingAggregatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/GroupingAggregatorImplementer.java new file mode 100644 index 0000000000000..cda4ac3ddff80 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/GroupingAggregatorImplementer.java @@ -0,0 +1,554 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.CodeBlock; +import com.squareup.javapoet.FieldSpec; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.util.Elements; + +import static java.util.stream.Collectors.joining; +import static org.elasticsearch.compute.gen.AggregatorImplementer.valueBlockType; +import static org.elasticsearch.compute.gen.AggregatorImplementer.valueVectorType; +import static org.elasticsearch.compute.gen.Methods.findMethod; +import static org.elasticsearch.compute.gen.Methods.findRequiredMethod; +import static org.elasticsearch.compute.gen.Methods.vectorAccessorName; +import static org.elasticsearch.compute.gen.Types.BIG_ARRAYS; +import static org.elasticsearch.compute.gen.Types.BLOCK; +import static org.elasticsearch.compute.gen.Types.BLOCK_ARRAY; +import static org.elasticsearch.compute.gen.Types.BYTES_REF; +import static org.elasticsearch.compute.gen.Types.ELEMENT_TYPE; +import static org.elasticsearch.compute.gen.Types.GROUPING_AGGREGATOR_FUNCTION; +import static org.elasticsearch.compute.gen.Types.GROUPING_AGGREGATOR_FUNCTION_ADD_INPUT; +import static org.elasticsearch.compute.gen.Types.INTERMEDIATE_STATE_DESC; +import static org.elasticsearch.compute.gen.Types.INT_VECTOR; +import static org.elasticsearch.compute.gen.Types.LIST_AGG_FUNC_DESC; +import static org.elasticsearch.compute.gen.Types.LIST_INTEGER; +import static org.elasticsearch.compute.gen.Types.LONG_BLOCK; +import static org.elasticsearch.compute.gen.Types.LONG_VECTOR; +import static org.elasticsearch.compute.gen.Types.PAGE; +import static org.elasticsearch.compute.gen.Types.SEEN_GROUP_IDS; +import static org.elasticsearch.compute.gen.Types.blockType; +import static org.elasticsearch.compute.gen.Types.vectorType; + +/** + * Implements "GroupingAggregationFunction" from a class containing static methods + * annotated with {@link Aggregator}. + *

The goal here is the implement an GroupingAggregationFunction who's inner loops + * don't contain any {@code invokevirtual}s. Instead, we generate a class + * that calls static methods in the inner loops. + *

A secondary goal is to make the generated code as readable, debuggable, + * and break-point-able as possible. + */ +public class GroupingAggregatorImplementer { + private final TypeElement declarationType; + private final ExecutableElement init; + private final ExecutableElement combine; + private final ExecutableElement combineStates; + private final ExecutableElement evaluateFinal; + private final ExecutableElement combineIntermediate; + private final TypeName stateType; + private final boolean valuesIsBytesRef; + private final List createParameters; + private final ClassName implementation; + private final List intermediateState; + + public GroupingAggregatorImplementer(Elements elements, TypeElement declarationType, IntermediateState[] interStateAnno) { + this.declarationType = declarationType; + + this.init = findRequiredMethod(declarationType, new String[] { "init", "initGrouping" }, e -> true); + this.stateType = choseStateType(); + + this.combine = findRequiredMethod(declarationType, new String[] { "combine" }, e -> { + if (e.getParameters().size() == 0) { + return false; + } + TypeName firstParamType = TypeName.get(e.getParameters().get(0).asType()); + return firstParamType.isPrimitive() || firstParamType.toString().equals(stateType.toString()); + }); + this.combineStates = findMethod(declarationType, "combineStates"); + this.combineIntermediate = findMethod(declarationType, "combineIntermediate"); + this.evaluateFinal = findMethod(declarationType, "evaluateFinal"); + this.valuesIsBytesRef = BYTES_REF.equals(TypeName.get(combine.getParameters().get(combine.getParameters().size() - 1).asType())); + this.createParameters = init.getParameters().stream().map(Parameter::from).collect(Collectors.toList()); + if (false == createParameters.stream().anyMatch(p -> p.type().equals(BIG_ARRAYS))) { + createParameters.add(0, new Parameter(BIG_ARRAYS, "bigArrays")); + } + + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + (declarationType.getSimpleName() + "GroupingAggregatorFunction").replace("AggregatorGroupingAggregator", "GroupingAggregator") + ); + + intermediateState = Arrays.stream(interStateAnno).map(state -> new IntermediateStateDesc(state.name(), state.type())).toList(); + } + + record IntermediateStateDesc(String name, String elementType) {} + + public ClassName implementation() { + return implementation; + } + + List createParameters() { + return createParameters; + } + + private TypeName choseStateType() { + TypeName initReturn = TypeName.get(init.getReturnType()); + if (false == initReturn.isPrimitive()) { + return initReturn; + } + String head = initReturn.toString().substring(0, 1).toUpperCase(Locale.ROOT); + String tail = initReturn.toString().substring(1); + return ClassName.get("org.elasticsearch.compute.aggregation", head + tail + "ArrayState"); + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", GROUPING_AGGREGATOR_FUNCTION, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + builder.addSuperinterface(GROUPING_AGGREGATOR_FUNCTION); + builder.addField( + FieldSpec.builder(LIST_AGG_FUNC_DESC, "INTERMEDIATE_STATE_DESC", Modifier.PRIVATE, Modifier.STATIC, Modifier.FINAL) + .initializer(initInterState()) + .build() + ); + builder.addField(stateType, "state", Modifier.PRIVATE, Modifier.FINAL); + builder.addField(LIST_INTEGER, "channels", Modifier.PRIVATE, Modifier.FINAL); + + for (VariableElement p : init.getParameters()) { + builder.addField(TypeName.get(p.asType()), p.getSimpleName().toString(), Modifier.PRIVATE, Modifier.FINAL); + } + + builder.addMethod(create()); + builder.addMethod(ctor()); + builder.addMethod(intermediateStateDesc()); + builder.addMethod(intermediateBlockCount()); + builder.addMethod(prepareProcessPage()); + builder.addMethod(addRawInputLoop(LONG_VECTOR, valueBlockType(init, combine))); + builder.addMethod(addRawInputLoop(LONG_VECTOR, valueVectorType(init, combine))); + builder.addMethod(addRawInputLoop(LONG_BLOCK, valueBlockType(init, combine))); + builder.addMethod(addRawInputLoop(LONG_BLOCK, valueVectorType(init, combine))); + builder.addMethod(addIntermediateInput()); + builder.addMethod(addIntermediateRowInput()); + builder.addMethod(evaluateIntermediate()); + builder.addMethod(evaluateFinal()); + builder.addMethod(toStringMethod()); + builder.addMethod(close()); + return builder.build(); + } + + private MethodSpec create() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("create"); + builder.addModifiers(Modifier.PUBLIC, Modifier.STATIC).returns(implementation); + builder.addParameter(LIST_INTEGER, "channels"); + for (Parameter p : createParameters) { + builder.addParameter(p.type(), p.name()); + } + if (init.getParameters().isEmpty()) { + builder.addStatement("return new $T(channels, $L)", implementation, callInit()); + } else { + builder.addStatement("return new $T(channels, $L, $L)", implementation, callInit(), initParameters()); + } + return builder.build(); + } + + private String initParameters() { + return init.getParameters().stream().map(p -> p.getSimpleName().toString()).collect(Collectors.joining(", ")); + } + + private CodeBlock callInit() { + CodeBlock.Builder builder = CodeBlock.builder(); + if (init.getReturnType().toString().equals(stateType.toString())) { + builder.add("$T.$L($L)", declarationType, init.getSimpleName(), initParameters()); + } else { + builder.add("new $T(bigArrays, $T.$L($L))", stateType, declarationType, init.getSimpleName(), initParameters()); + } + return builder.build(); + } + + private CodeBlock initInterState() { + CodeBlock.Builder builder = CodeBlock.builder(); + builder.add("List.of("); + boolean addComma = false; + for (var interState : intermediateState) { + if (addComma) builder.add(","); + builder.add("$Wnew $T($S, $T." + interState.elementType() + ")", INTERMEDIATE_STATE_DESC, interState.name(), ELEMENT_TYPE); + addComma = true; + } + builder.add("$W$W)"); + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + builder.addParameter(LIST_INTEGER, "channels"); + builder.addParameter(stateType, "state"); + builder.addStatement("this.channels = channels"); + builder.addStatement("this.state = state"); + + for (VariableElement p : init.getParameters()) { + builder.addParameter(TypeName.get(p.asType()), p.getSimpleName().toString()); + builder.addStatement("this.$N = $N", p.getSimpleName(), p.getSimpleName()); + } + return builder.build(); + } + + private MethodSpec intermediateStateDesc() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("intermediateStateDesc"); + builder.addModifiers(Modifier.PUBLIC, Modifier.STATIC).returns(LIST_AGG_FUNC_DESC); + builder.addStatement("return INTERMEDIATE_STATE_DESC"); + return builder.build(); + } + + private MethodSpec intermediateBlockCount() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("intermediateBlockCount"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).returns(TypeName.INT); + builder.addStatement("return INTERMEDIATE_STATE_DESC.size()"); + return builder.build(); + } + + /** + * Prepare to process a single page of results. + */ + private MethodSpec prepareProcessPage() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("prepareProcessPage"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).returns(GROUPING_AGGREGATOR_FUNCTION_ADD_INPUT); + builder.addParameter(SEEN_GROUP_IDS, "seenGroupIds").addParameter(PAGE, "page"); + + builder.addStatement("$T uncastValuesBlock = page.getBlock(channels.get(0))", BLOCK); + + builder.beginControlFlow("if (uncastValuesBlock.areAllValuesNull())"); + { + builder.addStatement("state.enableGroupIdTracking(seenGroupIds)"); + builder.addStatement("return $L", addInput(b -> {})); + } + builder.endControlFlow(); + + builder.addStatement("$T valuesBlock = ($T) uncastValuesBlock", valueBlockType(init, combine), valueBlockType(init, combine)); + builder.addStatement("$T valuesVector = valuesBlock.asVector()", valueVectorType(init, combine)); + builder.beginControlFlow("if (valuesVector == null)"); + { + builder.beginControlFlow("if (valuesBlock.mayHaveNulls())"); + builder.addStatement("state.enableGroupIdTracking(seenGroupIds)"); + builder.endControlFlow(); + builder.addStatement("return $L", addInput(b -> b.addStatement("addRawInput(positionOffset, groupIds, valuesBlock)"))); + } + builder.endControlFlow(); + builder.addStatement("return $L", addInput(b -> b.addStatement("addRawInput(positionOffset, groupIds, valuesVector)"))); + return builder.build(); + } + + /** + * Generate an {@code AddInput} implementation. That's a collection path optimized for the input data. + */ + private TypeSpec addInput(Consumer addBlock) { + TypeSpec.Builder builder = TypeSpec.anonymousClassBuilder(""); + builder.addSuperinterface(GROUPING_AGGREGATOR_FUNCTION_ADD_INPUT); + + MethodSpec.Builder block = MethodSpec.methodBuilder("add").addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + block.addParameter(TypeName.INT, "positionOffset").addParameter(LONG_BLOCK, "groupIds"); + addBlock.accept(block); + builder.addMethod(block.build()); + + MethodSpec.Builder vector = MethodSpec.methodBuilder("add").addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + vector.addParameter(TypeName.INT, "positionOffset").addParameter(LONG_VECTOR, "groupIds"); + addBlock.accept(vector); + builder.addMethod(vector.build()); + + return builder.build(); + } + + /** + * Generate an {@code addRawInput} method to perform the actual aggregation. + * @param groupsType The type of the group key, always {@code LongBlock} or {@code LongVector} + * @param valuesType The type of the values to consume, always a subclass of {@code Block} or a subclass of {@code Vector} + */ + private MethodSpec addRawInputLoop(TypeName groupsType, TypeName valuesType) { + boolean groupsIsBlock = groupsType.toString().endsWith("Block"); + boolean valuesIsBlock = valuesType.toString().endsWith("Block"); + String methodName = "addRawInput"; + MethodSpec.Builder builder = MethodSpec.methodBuilder(methodName); + builder.addModifiers(Modifier.PRIVATE); + builder.addParameter(TypeName.INT, "positionOffset").addParameter(groupsType, "groups").addParameter(valuesType, "values"); + if (valuesIsBytesRef) { + // Add bytes_ref scratch var that will be used for bytes_ref blocks/vectors + builder.addStatement("$T scratch = new $T()", BYTES_REF, BYTES_REF); + } + + builder.beginControlFlow("for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++)"); + { + if (groupsIsBlock) { + // TODO we can drop this once we stop sending null group keys + builder.beginControlFlow("if (groups.isNull(groupPosition))"); + builder.addStatement("continue"); + builder.endControlFlow(); + builder.addStatement("int groupStart = groups.getFirstValueIndex(groupPosition)"); + builder.addStatement("int groupEnd = groupStart + groups.getValueCount(groupPosition)"); + builder.beginControlFlow("for (int g = groupStart; g < groupEnd; g++)"); + builder.addStatement("int groupId = Math.toIntExact(groups.getLong(g))"); + } else { + builder.addStatement("int groupId = Math.toIntExact(groups.getLong(groupPosition))"); + } + + if (valuesIsBlock) { + builder.beginControlFlow("if (values.isNull(groupPosition + positionOffset))"); + builder.addStatement("continue"); + builder.endControlFlow(); + builder.addStatement("int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset)"); + builder.addStatement("int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset)"); + builder.beginControlFlow("for (int v = valuesStart; v < valuesEnd; v++)"); + combineRawInput(builder, "values", "v"); + builder.endControlFlow(); + } else { + combineRawInput(builder, "values", "groupPosition + positionOffset"); + } + + if (groupsIsBlock) { + builder.endControlFlow(); + } + } + builder.endControlFlow(); + return builder.build(); + } + + private void combineRawInput(MethodSpec.Builder builder, String blockVariable, String offsetVariable) { + if (valuesIsBytesRef) { + combineRawInputForBytesRef(builder, blockVariable, offsetVariable); + return; + } + TypeName valueType = TypeName.get(combine.getParameters().get(combine.getParameters().size() - 1).asType()); + if (valueType.isPrimitive() == false) { + throw new IllegalArgumentException("second parameter to combine must be a primitive"); + } + String secondParameterGetter = "get" + + valueType.toString().substring(0, 1).toUpperCase(Locale.ROOT) + + valueType.toString().substring(1); + TypeName returnType = TypeName.get(combine.getReturnType()); + if (returnType.isPrimitive()) { + combineRawInputForPrimitive(builder, secondParameterGetter, blockVariable, offsetVariable); + return; + } + if (returnType == TypeName.VOID) { + combineRawInputForVoid(builder, secondParameterGetter, blockVariable, offsetVariable); + return; + } + throw new IllegalArgumentException("combine must return void or a primitive"); + } + + private void combineRawInputForPrimitive( + MethodSpec.Builder builder, + String secondParameterGetter, + String blockVariable, + String offsetVariable + ) { + builder.addStatement( + "state.set(groupId, $T.combine(state.getOrDefault(groupId), $L.$L($L)))", + declarationType, + blockVariable, + secondParameterGetter, + offsetVariable + ); + } + + private void combineRawInputForVoid( + MethodSpec.Builder builder, + String secondParameterGetter, + String blockVariable, + String offsetVariable + ) { + builder.addStatement( + "$T.combine(state, groupId, $L.$L($L))", + declarationType, + blockVariable, + secondParameterGetter, + offsetVariable + ); + } + + private void combineRawInputForBytesRef(MethodSpec.Builder builder, String blockVariable, String offsetVariable) { + // scratch is a BytesRef var that must have been defined before the iteration starts + builder.addStatement("$T.combine(state, groupId, $L.getBytesRef($L, scratch))", declarationType, blockVariable, offsetVariable); + } + + private MethodSpec addIntermediateInput() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addIntermediateInput"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addParameter(TypeName.INT, "positionOffset"); + builder.addParameter(LONG_VECTOR, "groups"); + builder.addParameter(PAGE, "page"); + + builder.addStatement("state.enableGroupIdTracking(new $T.Empty())", SEEN_GROUP_IDS); + builder.addStatement("assert channels.size() == intermediateBlockCount()"); + int count = 0; + for (var interState : intermediateState) { + builder.addStatement( + "$T " + interState.name() + " = page.<$T>getBlock(channels.get(" + count + ")).asVector()", + vectorType(interState.elementType()), + blockType(interState.elementType()) + ); + count++; + } + final String first = intermediateState.get(0).name(); + if (intermediateState.size() > 1) { + builder.addStatement( + "assert " + + intermediateState.stream() + .map(IntermediateStateDesc::name) + .skip(1) + .map(s -> first + ".getPositionCount() == " + s + ".getPositionCount()") + .collect(joining(" && ")) + ); + } + if (intermediateState.stream().map(IntermediateStateDesc::elementType).anyMatch(n -> n.equals("BYTES_REF"))) { + builder.addStatement("$T scratch = new $T()", BYTES_REF, BYTES_REF); + } + builder.beginControlFlow("for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++)"); + { + builder.addStatement("int groupId = Math.toIntExact(groups.getLong(groupPosition))"); + if (hasPrimitiveState()) { + assert intermediateState.size() == 2; + assert intermediateState.get(1).name().equals("seen"); + builder.beginControlFlow("if (seen.getBoolean(groupPosition + positionOffset))"); + { + var name = intermediateState.get(0).name(); + var m = vectorAccessorName(intermediateState.get(0).elementType()); + builder.addStatement( + "state.set(groupId, $T.combine(state.getOrDefault(groupId), $L.$L(groupPosition + positionOffset)))", + declarationType, + name, + m + ); + builder.endControlFlow(); + } + } else { + builder.addStatement("$T.combineIntermediate(state, groupId, " + intermediateStateRowAccess() + ")", declarationType); + } + builder.endControlFlow(); + } + return builder.build(); + } + + String intermediateStateRowAccess() { + return intermediateState.stream().map(GroupingAggregatorImplementer::vectorAccess).collect(joining(", ")); + } + + static String vectorAccess(IntermediateStateDesc isd) { + String s = isd.name() + "." + vectorAccessorName(isd.elementType()) + "(groupPosition + positionOffset"; + if (isd.elementType().equals("BYTES_REF")) { + s += ", scratch"; + } + return s + ")"; + } + + private void combineStates(MethodSpec.Builder builder) { + if (combineStates == null) { + builder.beginControlFlow("if (inState.hasValue(position))"); + builder.addStatement("state.set(groupId, $T.combine(state.getOrDefault(groupId), inState.get(position)))", declarationType); + builder.endControlFlow(); + return; + } + builder.addStatement("$T.combineStates(state, groupId, inState, position)", declarationType); + } + + private MethodSpec addIntermediateRowInput() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("addIntermediateRowInput"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addParameter(int.class, "groupId").addParameter(GROUPING_AGGREGATOR_FUNCTION, "input").addParameter(int.class, "position"); + builder.beginControlFlow("if (input.getClass() != getClass())"); + { + builder.addStatement("throw new IllegalArgumentException($S + getClass() + $S + input.getClass())", "expected ", "; got "); + } + builder.endControlFlow(); + builder.addStatement("$T inState = (($T) input).state", stateType, implementation); + builder.addStatement("state.enableGroupIdTracking(new $T.Empty())", SEEN_GROUP_IDS); + combineStates(builder); + return builder.build(); + } + + private MethodSpec evaluateIntermediate() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evaluateIntermediate"); + builder.addAnnotation(Override.class) + .addModifiers(Modifier.PUBLIC) + .addParameter(BLOCK_ARRAY, "blocks") + .addParameter(TypeName.INT, "offset") + .addParameter(INT_VECTOR, "selected"); + builder.addStatement("state.toIntermediate(blocks, offset, selected)"); + return builder.build(); + } + + private MethodSpec evaluateFinal() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("evaluateFinal"); + builder.addAnnotation(Override.class) + .addModifiers(Modifier.PUBLIC) + .addParameter(BLOCK_ARRAY, "blocks") + .addParameter(TypeName.INT, "offset") + .addParameter(INT_VECTOR, "selected"); + if (evaluateFinal == null) { + builder.addStatement("blocks[offset] = state.toValuesBlock(selected)"); + } else { + builder.addStatement("blocks[offset] = $T.evaluateFinal(state, selected)", declarationType); + } + return builder.build(); + } + + private MethodSpec toStringMethod() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("toString"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC).returns(String.class); + builder.addStatement("$T sb = new $T()", StringBuilder.class, StringBuilder.class); + builder.addStatement("sb.append(getClass().getSimpleName()).append($S)", "["); + builder.addStatement("sb.append($S).append(channels)", "channels="); + builder.addStatement("sb.append($S)", "]"); + builder.addStatement("return sb.toString()"); + return builder.build(); + } + + private MethodSpec close() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("close"); + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + builder.addStatement("state.close()"); + return builder.build(); + } + + private boolean hasPrimitiveState() { + return switch (stateType.toString()) { + case "org.elasticsearch.compute.aggregation.IntArrayState", "org.elasticsearch.compute.aggregation.LongArrayState", + "org.elasticsearch.compute.aggregation.DoubleArrayState" -> true; + default -> false; + }; + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Methods.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Methods.java new file mode 100644 index 0000000000000..746b8be169031 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Methods.java @@ -0,0 +1,137 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.TypeName; + +import java.util.Arrays; +import java.util.function.Predicate; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.util.ElementFilter; + +import static org.elasticsearch.compute.gen.Types.BOOLEAN_BLOCK; +import static org.elasticsearch.compute.gen.Types.BOOLEAN_VECTOR; +import static org.elasticsearch.compute.gen.Types.BYTES_REF_BLOCK; +import static org.elasticsearch.compute.gen.Types.DOUBLE_BLOCK; +import static org.elasticsearch.compute.gen.Types.DOUBLE_VECTOR; +import static org.elasticsearch.compute.gen.Types.INT_BLOCK; +import static org.elasticsearch.compute.gen.Types.INT_VECTOR; +import static org.elasticsearch.compute.gen.Types.LONG_BLOCK; +import static org.elasticsearch.compute.gen.Types.LONG_VECTOR; + +/** + * Finds declared methods for the code generator. + */ +public class Methods { + static ExecutableElement findRequiredMethod(TypeElement declarationType, String[] names, Predicate filter) { + ExecutableElement result = findMethod(declarationType, names, filter); + if (result == null) { + if (names.length == 1) { + throw new IllegalArgumentException(declarationType + "#" + names[0] + " is required"); + } + throw new IllegalArgumentException("one of " + declarationType + "#" + Arrays.toString(names) + " is required"); + } + return result; + } + + static ExecutableElement findMethod(TypeElement declarationType, String name) { + return findMethod(declarationType, new String[] { name }, e -> true); + } + + static ExecutableElement findMethod(TypeElement declarationType, String[] names, Predicate filter) { + for (ExecutableElement e : ElementFilter.methodsIn(declarationType.getEnclosedElements())) { + if (e.getModifiers().contains(Modifier.STATIC) == false) { + continue; + } + String name = e.getSimpleName().toString(); + for (String n : names) { + if (n.equals(name) && filter.test(e)) { + return e; + } + } + } + return null; + } + + /** + * Returns the arguments of a method after applying a filter. + */ + static VariableElement[] findMethodArguments(ExecutableElement method, Predicate filter) { + if (method.getParameters().isEmpty()) { + return new VariableElement[0]; + } + return method.getParameters().stream().filter(e -> filter.test(e)).toArray(VariableElement[]::new); + } + + /** + * Returns the name of the method used to add {@code valueType} instances + * to vector or block builders. + */ + static String appendMethod(TypeName t) { + if (t.equals(TypeName.BOOLEAN) || t.equals(BOOLEAN_BLOCK) || t.equals(BOOLEAN_VECTOR)) { + return "appendBoolean"; + } + if (t.equals(Types.BYTES_REF) || t.equals(BYTES_REF_BLOCK) || t.equals(Types.BYTES_REF_VECTOR)) { + return "appendBytesRef"; + } + if (t.equals(TypeName.INT) || t.equals(INT_BLOCK) || t.equals(INT_VECTOR)) { + return "appendInt"; + } + if (t.equals(TypeName.LONG) || t.equals(LONG_BLOCK) || t.equals(LONG_VECTOR)) { + return "appendLong"; + } + if (t.equals(TypeName.DOUBLE) || t.equals(DOUBLE_BLOCK) || t.equals(DOUBLE_VECTOR)) { + return "appendDouble"; + } + throw new IllegalArgumentException("unknown append method for [" + t + "]"); + } + + /** + * Returns the name of the method used to get {@code valueType} instances + * from vectors or blocks. + */ + static String getMethod(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return "getBoolean"; + } + if (elementType.equals(Types.BYTES_REF)) { + return "getBytesRef"; + } + if (elementType.equals(TypeName.INT)) { + return "getInt"; + } + if (elementType.equals(TypeName.LONG)) { + return "getLong"; + } + if (elementType.equals(TypeName.DOUBLE)) { + return "getDouble"; + } + throw new IllegalArgumentException("unknown get method for [" + elementType + "]"); + } + + /** + * Returns the name of the method used to get {@code valueType} instances + * from vectors or blocks. + */ + static String vectorAccessorName(String elementTypeName) { + return switch (elementTypeName) { + case "BOOLEAN" -> "getBoolean"; + case "INT" -> "getInt"; + case "LONG" -> "getLong"; + case "DOUBLE" -> "getDouble"; + case "BYTES_REF" -> "getBytesRef"; + default -> throw new IllegalArgumentException( + "don't know how to fetch primitive values from " + elementTypeName + ". define combineStates." + ); + }; + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java new file mode 100644 index 0000000000000..124179b7447e8 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/MvEvaluatorImplementer.java @@ -0,0 +1,492 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.JavaFile; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import javax.lang.model.element.ExecutableElement; +import javax.lang.model.element.Modifier; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.type.TypeKind; +import javax.lang.model.type.TypeMirror; +import javax.lang.model.util.Elements; + +import static org.elasticsearch.compute.gen.Methods.appendMethod; +import static org.elasticsearch.compute.gen.Methods.findMethod; +import static org.elasticsearch.compute.gen.Methods.getMethod; +import static org.elasticsearch.compute.gen.Types.ABSTRACT_MULTIVALUE_FUNCTION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.ABSTRACT_NULLABLE_MULTIVALUE_FUNCTION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.BIG_ARRAYS; +import static org.elasticsearch.compute.gen.Types.BLOCK; +import static org.elasticsearch.compute.gen.Types.BYTES_REF; +import static org.elasticsearch.compute.gen.Types.BYTES_REF_ARRAY; +import static org.elasticsearch.compute.gen.Types.EXPRESSION_EVALUATOR; +import static org.elasticsearch.compute.gen.Types.SOURCE; +import static org.elasticsearch.compute.gen.Types.VECTOR; +import static org.elasticsearch.compute.gen.Types.WARNINGS; +import static org.elasticsearch.compute.gen.Types.arrayVectorType; +import static org.elasticsearch.compute.gen.Types.blockType; + +public class MvEvaluatorImplementer { + private final TypeElement declarationType; + + /** + * Function specifying how each value in a multivalued field is processed. + */ + private final ExecutableElement processFunction; + + /** + * Optional function "finishing" the processing of a multivalued field. It + * converts {@link #workType} into {@link #resultType}. If {@code null} then + * {@link #workType} is {@link #resultType} and the work + * is returned unchanged. + */ + private final FinishFunction finishFunction; + + /** + * Optional function to process single valued fields. This is often used + * when the {@link #fieldType} isn't the same as the {@link #resultType} + * and will implement the conversion. If this is unspecified then single + * value fields are returned as is. + */ + private final SingleValueFunction singleValueFunction; + + /** + * Optional function to process {@code Block}s where all multivalued fields + * are ascending, which is how Lucene loads them so it's quite common. If + * specified then the implementation will use this method to process the + * multivalued field instead of {@link #processFunction}. + */ + private final AscendingFunction ascendingFunction; + + private final List warnExceptions; + private final ClassName implementation; + private final TypeName workType; + private final TypeName fieldType; + private final TypeName resultType; + + public MvEvaluatorImplementer( + Elements elements, + ExecutableElement processFunction, + String extraName, + String finishMethodName, + String singleValueMethodName, + String ascendingMethodName, + List warnExceptions + ) { + this.declarationType = (TypeElement) processFunction.getEnclosingElement(); + this.processFunction = processFunction; + if (processFunction.getParameters().size() != 2) { + throw new IllegalArgumentException("process should have exactly two parameters"); + } + this.workType = TypeName.get(processFunction.getParameters().get(0).asType()); + this.fieldType = TypeName.get(processFunction.getParameters().get(1).asType()); + this.finishFunction = FinishFunction.from(declarationType, finishMethodName, workType, fieldType); + this.resultType = this.finishFunction == null ? this.workType : this.finishFunction.resultType; + this.singleValueFunction = SingleValueFunction.from(declarationType, singleValueMethodName, resultType, fieldType); + this.ascendingFunction = AscendingFunction.from(this, declarationType, ascendingMethodName); + this.warnExceptions = warnExceptions; + this.implementation = ClassName.get( + elements.getPackageOf(declarationType).toString(), + declarationType.getSimpleName() + extraName + "Evaluator" + ); + } + + public JavaFile sourceFile() { + JavaFile.Builder builder = JavaFile.builder(implementation.packageName(), type()); + builder.addFileComment(""" + 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."""); + return builder.build(); + } + + private TypeSpec type() { + TypeSpec.Builder builder = TypeSpec.classBuilder(implementation); + builder.addJavadoc("{@link $T} implementation for {@link $T}.\n", EXPRESSION_EVALUATOR, declarationType); + builder.addJavadoc("This class is generated. Do not edit it."); + builder.addModifiers(Modifier.PUBLIC, Modifier.FINAL); + if (warnExceptions.isEmpty()) { + builder.superclass(ABSTRACT_MULTIVALUE_FUNCTION_EVALUATOR); + } else { + builder.superclass(ABSTRACT_NULLABLE_MULTIVALUE_FUNCTION_EVALUATOR); + + builder.addField(WARNINGS, "warnings", Modifier.PRIVATE, Modifier.FINAL); + } + + builder.addMethod(ctor()); + builder.addMethod(name()); + builder.addMethod(eval("evalNullable", true)); + if (warnExceptions.isEmpty()) { + builder.addMethod(eval("evalNotNullable", false)); + } + if (singleValueFunction != null) { + builder.addMethod(evalSingleValued("evalSingleValuedNullable", true)); + if (warnExceptions.isEmpty()) { + builder.addMethod(evalSingleValued("evalSingleValuedNotNullable", false)); + } + } + if (ascendingFunction != null) { + builder.addMethod(evalAscending("evalAscendingNullable", true)); + builder.addMethod(evalAscending("evalAscendingNotNullable", false)); + } + return builder.build(); + } + + private MethodSpec ctor() { + MethodSpec.Builder builder = MethodSpec.constructorBuilder().addModifiers(Modifier.PUBLIC); + if (warnExceptions.isEmpty() == false) { + builder.addParameter(SOURCE, "source"); + } + builder.addParameter(EXPRESSION_EVALUATOR, "field"); + builder.addStatement("super($L)", "field"); + if (warnExceptions.isEmpty() == false) { + builder.addStatement("this.warnings = new Warnings(source)"); + } + return builder.build(); + } + + private MethodSpec name() { + MethodSpec.Builder builder = MethodSpec.methodBuilder("name").addModifiers(Modifier.PUBLIC); + builder.addAnnotation(Override.class).returns(String.class); + builder.addStatement("return $S", declarationType.getSimpleName()); + return builder.build(); + } + + private MethodSpec evalShell( + String name, + boolean override, + boolean nullable, + String javadoc, + Consumer preflight, + Consumer body + ) { + MethodSpec.Builder builder = MethodSpec.methodBuilder(name); + builder.returns(nullable ? BLOCK : VECTOR).addParameter(BLOCK, "fieldVal"); + if (override) { + builder.addAnnotation(Override.class).addModifiers(Modifier.PUBLIC); + } else { + builder.addModifiers(Modifier.PRIVATE); + } + builder.addJavadoc(javadoc); + TypeName blockType = blockType(fieldType); + + preflight.accept(builder); + + builder.addStatement("$T v = ($T) fieldVal", blockType, blockType); + builder.addStatement("int positionCount = v.getPositionCount()"); + if (nullable) { + TypeName resultBlockType = blockType(resultType); + builder.addStatement("$T.Builder builder = $T.newBlockBuilder(positionCount)", resultBlockType, resultBlockType); + } else if (resultType.equals(BYTES_REF)) { + builder.addStatement( + "$T values = new $T(positionCount, $T.NON_RECYCLING_INSTANCE)", // TODO blocks should use recycling array + BYTES_REF_ARRAY, + BYTES_REF_ARRAY, + BIG_ARRAYS + ); + } else { + builder.addStatement("$T[] values = new $T[positionCount]", resultType, resultType); + } + + if (false == workType.equals(fieldType) && workType.isPrimitive() == false) { + builder.addStatement("$T work = new $T()", workType, workType); + } + if (fieldType.equals(BYTES_REF)) { + if (workType.equals(fieldType)) { + builder.addStatement("$T firstScratch = new $T()", BYTES_REF, BYTES_REF); + builder.addStatement("$T nextScratch = new $T()", BYTES_REF, BYTES_REF); + } else { + builder.addStatement("$T valueScratch = new $T()", BYTES_REF, BYTES_REF); + } + } + + builder.beginControlFlow("for (int p = 0; p < positionCount; p++)"); + { + builder.addStatement("int valueCount = v.getValueCount(p)"); + if (nullable) { + builder.beginControlFlow("if (valueCount == 0)"); + builder.addStatement("builder.appendNull()"); + builder.addStatement("continue"); + builder.endControlFlow(); + } + if (warnExceptions.isEmpty() == false) { + builder.beginControlFlow("try"); + body.accept(builder); + String catchPattern = "catch (" + warnExceptions.stream().map(m -> "$T").collect(Collectors.joining(" | ")) + " e)"; + builder.nextControlFlow(catchPattern, warnExceptions.stream().map(TypeName::get).toArray()); + builder.addStatement("warnings.registerException(e)"); + builder.addStatement("builder.appendNull()"); + builder.endControlFlow(); + } else { + body.accept(builder); + } + } + builder.endControlFlow(); + + if (nullable) { + builder.addStatement("return builder.build()"); + } else { + builder.addStatement("return new $T(values, positionCount)", arrayVectorType(resultType)); + } + return builder.build(); + } + + private MethodSpec eval(String name, boolean nullable) { + String javadoc = "Evaluate blocks containing at least one multivalued field."; + return evalShell(name, true, nullable, javadoc, builder -> { + if (ascendingFunction == null) { + return; + } + builder.beginControlFlow("if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING)"); + builder.addStatement("return $L(fieldVal)", name.replace("eval", "evalAscending")); + builder.endControlFlow(); + }, builder -> { + builder.addStatement("int first = v.getFirstValueIndex(p)"); + + if (singleValueFunction != null) { + builder.beginControlFlow("if (valueCount == 1)"); + fetch(builder, "value", fieldType, "first", workType.equals(fieldType) ? "firstScratch" : "valueScratch"); + singleValueFunction.call(builder); + writeResult(builder, nullable); + builder.addStatement("continue"); + builder.endControlFlow(); + } + + builder.addStatement("int end = first + valueCount"); + if (workType.equals(fieldType) || workType.isPrimitive()) { + // process function evaluates pairwise + fetch(builder, "value", workType, "first", "firstScratch"); + builder.beginControlFlow("for (int i = first + 1; i < end; i++)"); + { + if (fieldType.equals(BYTES_REF)) { + fetch(builder, "next", workType, "i", "nextScratch"); + builder.addStatement("$T.$L(value, next)", declarationType, processFunction.getSimpleName()); + } else { + fetch(builder, "next", fieldType, "i", "nextScratch"); + builder.addStatement("value = $T.$L(value, next)", declarationType, processFunction.getSimpleName()); + } + } + builder.endControlFlow(); + if (finishFunction == null) { + builder.addStatement("$T result = value", resultType); + } else { + finishFunction.call(builder, "value"); + } + } else { + builder.beginControlFlow("for (int i = first; i < end; i++)"); + { + fetch(builder, "value", fieldType, "i", "valueScratch"); + builder.addStatement("$T.$L(work, value)", declarationType, processFunction.getSimpleName()); + } + builder.endControlFlow(); + finishFunction.call(builder, "work"); + } + writeResult(builder, nullable); + }); + } + + private MethodSpec evalSingleValued(String name, boolean nullable) { + String javadoc = "Evaluate blocks containing only single valued fields."; + return evalShell(name, true, nullable, javadoc, builder -> {}, builder -> { + builder.addStatement("assert valueCount == 1"); + builder.addStatement("int first = v.getFirstValueIndex(p)"); + fetch(builder, "value", fieldType, "first", workType.equals(fieldType) ? "firstScratch" : "valueScratch"); + singleValueFunction.call(builder); + writeResult(builder, nullable); + }); + } + + private void fetch(MethodSpec.Builder builder, String into, TypeName intoType, String index, String scratchName) { + if (intoType.equals(BYTES_REF)) { + builder.addStatement("$T $L = v.getBytesRef($L, $L)", intoType, into, index, scratchName); + } else if (intoType.equals(fieldType) == false && intoType.isPrimitive()) { + builder.addStatement("$T $L = ($T) v.$L($L)", intoType, into, intoType, getMethod(fieldType), index); + } else { + builder.addStatement("$T $L = v.$L($L)", intoType, into, getMethod(fieldType), index); + } + } + + private MethodSpec evalAscending(String name, boolean nullable) { + String javadoc = "Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order."; + return evalShell(name, false, nullable, javadoc, builder -> {}, builder -> { + builder.addStatement("int first = v.getFirstValueIndex(p)"); + ascendingFunction.call(builder); + writeResult(builder, nullable); + }); + } + + private void writeResult(MethodSpec.Builder builder, boolean nullable) { + if (nullable) { + builder.addStatement("builder.$L(result)", appendMethod(resultType)); + } else if (fieldType.equals(BYTES_REF)) { + builder.addStatement("values.append(result)"); + } else { + builder.addStatement("values[p] = result"); + } + } + + /** + * Function "finishing" the computation on a multivalued field. It converts {@link #workType} into {@link #resultType}. + */ + private static class FinishFunction { + static FinishFunction from(TypeElement declarationType, String name, TypeName workType, TypeName fieldType) { + if (name.equals("")) { + if (false == workType.equals(fieldType)) { + throw new IllegalArgumentException( + "the [finish] enum value is required because the first and second arguments differ in type" + ); + } + return null; + } + ExecutableElement fn = findMethod( + declarationType, + new String[] { name }, + m -> TypeName.get(m.getParameters().get(0).asType()).equals(workType) + ); + if (fn == null) { + throw new IllegalArgumentException("Couldn't find " + declarationType + "#" + name + "(" + workType + "...)"); + } + TypeName resultType = TypeName.get(fn.getReturnType()); + return new FinishFunction(declarationType, fn, resultType, workType); + } + + private final TypeName resultType; + private final String invocationPattern; + private final List invocationArgs = new ArrayList<>(); + + private FinishFunction(TypeElement declarationType, ExecutableElement fn, TypeName resultType, TypeName workType) { + this.resultType = resultType; + StringBuilder pattern = new StringBuilder().append("$T result = $T.$L($work$"); + invocationArgs.add(resultType); + invocationArgs.add(declarationType); + invocationArgs.add(fn.getSimpleName()); + + for (int p = 1; p < fn.getParameters().size(); p++) { + VariableElement param = fn.getParameters().get(p); + if (p == 0) { + if (false == TypeName.get(param.asType()).equals(workType)) { + throw new IllegalArgumentException( + "First argument of " + declarationType + "#" + fn.getSimpleName() + " must have type " + workType + ); + } + continue; + } + if (param.getSimpleName().toString().equals("valueCount")) { + if (param.asType().getKind() != TypeKind.INT) { + throw new IllegalArgumentException("count argument must have type [int]"); + } + pattern.append(", valueCount"); + continue; + } + throw new IllegalArgumentException("unsupported parameter " + param); + } + invocationPattern = pattern.append(")").toString(); + } + + private void call(MethodSpec.Builder builder, String workName) { + builder.addStatement(invocationPattern.replace("$work$", workName), invocationArgs.toArray()); + } + } + + /** + * Function handling single valued fields. + */ + private static class SingleValueFunction { + static SingleValueFunction from(TypeElement declarationType, String name, TypeName resultType, TypeName fieldType) { + if (name.equals("")) { + return null; + } + ExecutableElement fn = findMethod( + declarationType, + new String[] { name }, + m -> m.getParameters().size() == 1 && TypeName.get(m.getParameters().get(0).asType()).equals(fieldType) + ); + if (fn == null) { + throw new IllegalArgumentException("Couldn't find " + declarationType + "#" + name + "(" + fieldType + ")"); + } + return new SingleValueFunction(declarationType, resultType, fn); + } + + private final List invocationArgs = new ArrayList<>(); + + private SingleValueFunction(TypeElement declarationType, TypeName resultType, ExecutableElement fn) { + invocationArgs.add(resultType); + invocationArgs.add(declarationType); + invocationArgs.add(fn.getSimpleName()); + } + + private void call(MethodSpec.Builder builder) { + builder.addStatement("$T result = $T.$L(value)", invocationArgs.toArray()); + } + } + + /** + * Function handling blocks of ascending values. + */ + private class AscendingFunction { + static AscendingFunction from(MvEvaluatorImplementer impl, TypeElement declarationType, String name) { + if (name.equals("")) { + return null; + } + + // check for index lookup + ExecutableElement fn = findMethod( + declarationType, + new String[] { name }, + m -> m.getParameters().size() == 1 && m.getParameters().get(0).asType().getKind() == TypeKind.INT + ); + if (fn != null) { + return impl.new AscendingFunction(fn, false); + } + fn = findMethod( + declarationType, + new String[] { name }, + m -> m.getParameters().size() == 3 + && m.getParameters().get(1).asType().getKind() == TypeKind.INT + && m.getParameters().get(2).asType().getKind() == TypeKind.INT + ); + if (fn == null) { + throw new IllegalArgumentException("Couldn't find " + declarationType + "#" + name + "(block, int, int)"); + } + return impl.new AscendingFunction(fn, true); + } + + private final List invocationArgs = new ArrayList<>(); + private final boolean blockMode; + + private AscendingFunction(ExecutableElement fn, boolean blockMode) { + this.blockMode = blockMode; + if (blockMode) { + invocationArgs.add(resultType); + } + invocationArgs.add(declarationType); + invocationArgs.add(fn.getSimpleName()); + } + + private void call(MethodSpec.Builder builder) { + if (blockMode) { + builder.addStatement("$T result = $T.$L(v, first, valueCount)", invocationArgs.toArray()); + } else { + builder.addStatement("int idx = $T.$L(valueCount)", invocationArgs.toArray()); + fetch(builder, "result", resultType, "first + idx", workType.equals(fieldType) ? "firstScratch" : "valueScratch"); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Parameter.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Parameter.java new file mode 100644 index 0000000000000..437b9d8f76494 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Parameter.java @@ -0,0 +1,31 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.MethodSpec; +import com.squareup.javapoet.TypeName; +import com.squareup.javapoet.TypeSpec; + +import javax.lang.model.element.Modifier; +import javax.lang.model.element.VariableElement; + +record Parameter(TypeName type, String name) { + static Parameter from(VariableElement e) { + return new Parameter(ClassName.get(e.asType()), e.getSimpleName().toString()); + } + + void declareField(TypeSpec.Builder builder) { + builder.addField(type(), name(), Modifier.PRIVATE, Modifier.FINAL); + } + + void buildCtor(MethodSpec.Builder builder) { + builder.addParameter(type(), name()); + builder.addStatement("this.$N = $N", name(), name()); + } +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Types.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Types.java new file mode 100644 index 0000000000000..2f76d1a73e480 --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Types.java @@ -0,0 +1,266 @@ +/* + * 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.compute.gen; + +import com.squareup.javapoet.ArrayTypeName; +import com.squareup.javapoet.ClassName; +import com.squareup.javapoet.ParameterizedTypeName; +import com.squareup.javapoet.TypeName; + +import java.util.List; + +/** + * Types used by the code generator. + */ +public class Types { + private static final String PACKAGE = "org.elasticsearch.compute"; + private static final String AGGREGATION_PACKAGE = PACKAGE + ".aggregation"; + private static final String OPERATOR_PACKAGE = PACKAGE + ".operator"; + private static final String DATA_PACKAGE = PACKAGE + ".data"; + + static final TypeName LIST_INTEGER = ParameterizedTypeName.get(ClassName.get(List.class), TypeName.INT.box()); + + static final ClassName PAGE = ClassName.get(DATA_PACKAGE, "Page"); + static final ClassName BLOCK = ClassName.get(DATA_PACKAGE, "Block"); + static final TypeName BLOCK_ARRAY = ArrayTypeName.of(BLOCK); + static final ClassName VECTOR = ClassName.get(DATA_PACKAGE, "Vector"); + + static final ClassName BIG_ARRAYS = ClassName.get("org.elasticsearch.common.util", "BigArrays"); + static final ClassName BYTES_REF_ARRAY = ClassName.get("org.elasticsearch.common.util", "BytesRefArray"); + + static final ClassName BOOLEAN_BLOCK = ClassName.get(DATA_PACKAGE, "BooleanBlock"); + static final ClassName BYTES_REF_BLOCK = ClassName.get(DATA_PACKAGE, "BytesRefBlock"); + static final ClassName INT_BLOCK = ClassName.get(DATA_PACKAGE, "IntBlock"); + static final ClassName LONG_BLOCK = ClassName.get(DATA_PACKAGE, "LongBlock"); + static final ClassName DOUBLE_BLOCK = ClassName.get(DATA_PACKAGE, "DoubleBlock"); + + static final ClassName BOOLEAN_BLOCK_BUILDER = BOOLEAN_BLOCK.nestedClass("Builder"); + static final ClassName BYTES_REF_BLOCK_BUILDER = BYTES_REF_BLOCK.nestedClass("Builder"); + static final ClassName INT_BLOCK_BUILDER = INT_BLOCK.nestedClass("Builder"); + static final ClassName LONG_BLOCK_BUILDER = LONG_BLOCK.nestedClass("Builder"); + static final ClassName DOUBLE_BLOCK_BUILDER = DOUBLE_BLOCK.nestedClass("Builder"); + + static final ClassName ELEMENT_TYPE = ClassName.get(DATA_PACKAGE, "ElementType"); + + static final ClassName BOOLEAN_VECTOR = ClassName.get(DATA_PACKAGE, "BooleanVector"); + static final ClassName BYTES_REF_VECTOR = ClassName.get(DATA_PACKAGE, "BytesRefVector"); + static final ClassName INT_VECTOR = ClassName.get(DATA_PACKAGE, "IntVector"); + static final ClassName LONG_VECTOR = ClassName.get(DATA_PACKAGE, "LongVector"); + static final ClassName DOUBLE_VECTOR = ClassName.get(DATA_PACKAGE, "DoubleVector"); + + static final ClassName BOOLEAN_ARRAY_VECTOR = ClassName.get(DATA_PACKAGE, "BooleanArrayVector"); + static final ClassName BYTES_REF_ARRAY_VECTOR = ClassName.get(DATA_PACKAGE, "BytesRefArrayVector"); + static final ClassName INT_ARRAY_VECTOR = ClassName.get(DATA_PACKAGE, "IntArrayVector"); + static final ClassName LONG_ARRAY_VECTOR = ClassName.get(DATA_PACKAGE, "LongArrayVector"); + static final ClassName DOUBLE_ARRAY_VECTOR = ClassName.get(DATA_PACKAGE, "DoubleArrayVector"); + + static final ClassName BOOLEAN_ARRAY_BLOCK = ClassName.get(DATA_PACKAGE, "BooleanArrayBlock"); + static final ClassName BYTES_REF_ARRAY_BLOCK = ClassName.get(DATA_PACKAGE, "BytesRefArrayBlock"); + static final ClassName INT_ARRAY_BLOCK = ClassName.get(DATA_PACKAGE, "IntArrayBlock"); + static final ClassName LONG_ARRAY_BLOCK = ClassName.get(DATA_PACKAGE, "LongArrayBlock"); + static final ClassName DOUBLE_ARRAY_BLOCK = ClassName.get(DATA_PACKAGE, "DoubleArrayBlock"); + + static final ClassName BOOLEAN_CONSTANT_VECTOR = ClassName.get(DATA_PACKAGE, "ConstantBooleanVector"); + static final ClassName BYTES_REF_CONSTANT_VECTOR = ClassName.get(DATA_PACKAGE, "ConstantBytesRefVector"); + static final ClassName INT_CONSTANT_VECTOR = ClassName.get(DATA_PACKAGE, "ConstantIntVector"); + static final ClassName LONG_CONSTANT_VECTOR = ClassName.get(DATA_PACKAGE, "ConstantLongVector"); + static final ClassName DOUBLE_CONSTANT_VECTOR = ClassName.get(DATA_PACKAGE, "ConstantDoubleVector"); + + static final ClassName INT_ARRAY_STATE = ClassName.get(AGGREGATION_PACKAGE, "IntArrayState"); + static final ClassName LONG_ARRAY_STATE = ClassName.get(AGGREGATION_PACKAGE, "LongArrayState"); + static final ClassName DOUBLE_ARRAY_STATE = ClassName.get(AGGREGATION_PACKAGE, "DoubleArrayState"); + + static final ClassName AGGREGATOR_FUNCTION = ClassName.get(AGGREGATION_PACKAGE, "AggregatorFunction"); + static final ClassName AGGREGATOR_FUNCTION_SUPPLIER = ClassName.get(AGGREGATION_PACKAGE, "AggregatorFunctionSupplier"); + static final ClassName GROUPING_AGGREGATOR_FUNCTION = ClassName.get(AGGREGATION_PACKAGE, "GroupingAggregatorFunction"); + static final ClassName GROUPING_AGGREGATOR_FUNCTION_ADD_INPUT = ClassName.get( + AGGREGATION_PACKAGE, + "GroupingAggregatorFunction", + "AddInput" + ); + static final ClassName SEEN_GROUP_IDS = ClassName.get(AGGREGATION_PACKAGE, "SeenGroupIds"); + + static final ClassName INTERMEDIATE_STATE_DESC = ClassName.get(AGGREGATION_PACKAGE, "IntermediateStateDesc"); + static final TypeName LIST_AGG_FUNC_DESC = ParameterizedTypeName.get(ClassName.get(List.class), INTERMEDIATE_STATE_DESC); + + static final ClassName EXPRESSION_EVALUATOR = ClassName.get(OPERATOR_PACKAGE, "EvalOperator", "ExpressionEvaluator"); + static final ClassName ABSTRACT_MULTIVALUE_FUNCTION_EVALUATOR = ClassName.get( + "org.elasticsearch.xpack.esql.expression.function.scalar.multivalue", + "AbstractMultivalueFunction", + "AbstractEvaluator" + ); + static final ClassName ABSTRACT_NULLABLE_MULTIVALUE_FUNCTION_EVALUATOR = ClassName.get( + "org.elasticsearch.xpack.esql.expression.function.scalar.multivalue", + "AbstractMultivalueFunction", + "AbstractNullableEvaluator" + ); + static final ClassName ABSTRACT_CONVERT_FUNCTION_EVALUATOR = ClassName.get( + "org.elasticsearch.xpack.esql.expression.function.scalar.convert", + "AbstractConvertFunction", + "AbstractEvaluator" + ); + + static final ClassName WARNINGS = ClassName.get("org.elasticsearch.xpack.esql.expression.function", "Warnings"); + + static final ClassName SOURCE = ClassName.get("org.elasticsearch.xpack.ql.tree", "Source"); + + static final ClassName BYTES_REF = ClassName.get("org.apache.lucene.util", "BytesRef"); + + static ClassName blockType(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return BOOLEAN_BLOCK; + } + if (elementType.equals(BYTES_REF)) { + return BYTES_REF_BLOCK; + } + if (elementType.equals(TypeName.INT)) { + return INT_BLOCK; + } + if (elementType.equals(TypeName.LONG)) { + return LONG_BLOCK; + } + if (elementType.equals(TypeName.DOUBLE)) { + return DOUBLE_BLOCK; + } + throw new IllegalArgumentException("unknown block type for [" + elementType + "]"); + } + + static ClassName blockType(String elementType) { + if (elementType.equalsIgnoreCase(TypeName.BOOLEAN.toString())) { + return BOOLEAN_BLOCK; + } + if (elementType.equalsIgnoreCase("BYTES_REF")) { + return BYTES_REF_BLOCK; + } + if (elementType.equalsIgnoreCase(TypeName.INT.toString())) { + return INT_BLOCK; + } + if (elementType.equalsIgnoreCase(TypeName.LONG.toString())) { + return LONG_BLOCK; + } + if (elementType.equalsIgnoreCase(TypeName.DOUBLE.toString())) { + return DOUBLE_BLOCK; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static ClassName vectorType(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return BOOLEAN_VECTOR; + } + if (elementType.equals(BYTES_REF)) { + return BYTES_REF_VECTOR; + } + if (elementType.equals(TypeName.INT)) { + return INT_VECTOR; + } + if (elementType.equals(TypeName.LONG)) { + return LONG_VECTOR; + } + if (elementType.equals(TypeName.DOUBLE)) { + return DOUBLE_VECTOR; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static ClassName vectorType(String elementType) { + if (elementType.equalsIgnoreCase(TypeName.BOOLEAN.toString())) { + return BOOLEAN_VECTOR; + } + if (elementType.equalsIgnoreCase("BYTES_REF")) { + return BYTES_REF_VECTOR; + } + if (elementType.equalsIgnoreCase(TypeName.INT.toString())) { + return INT_VECTOR; + } + if (elementType.equalsIgnoreCase(TypeName.LONG.toString())) { + return LONG_VECTOR; + } + if (elementType.equalsIgnoreCase(TypeName.DOUBLE.toString())) { + return DOUBLE_VECTOR; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static ClassName arrayVectorType(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return BOOLEAN_ARRAY_VECTOR; + } + if (elementType.equals(BYTES_REF)) { + return BYTES_REF_ARRAY_VECTOR; + } + if (elementType.equals(TypeName.INT)) { + return INT_ARRAY_VECTOR; + } + if (elementType.equals(TypeName.LONG)) { + return LONG_ARRAY_VECTOR; + } + if (elementType.equals(TypeName.DOUBLE)) { + return DOUBLE_ARRAY_VECTOR; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static ClassName arrayBlockType(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return BOOLEAN_ARRAY_BLOCK; + } + if (elementType.equals(BYTES_REF)) { + return BYTES_REF_ARRAY_BLOCK; + } + if (elementType.equals(TypeName.INT)) { + return INT_ARRAY_BLOCK; + } + if (elementType.equals(TypeName.LONG)) { + return LONG_ARRAY_BLOCK; + } + if (elementType.equals(TypeName.DOUBLE)) { + return DOUBLE_ARRAY_BLOCK; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static ClassName constantVectorType(TypeName elementType) { + if (elementType.equals(TypeName.BOOLEAN)) { + return BOOLEAN_CONSTANT_VECTOR; + } + if (elementType.equals(BYTES_REF)) { + return BYTES_REF_CONSTANT_VECTOR; + } + if (elementType.equals(TypeName.INT)) { + return INT_CONSTANT_VECTOR; + } + if (elementType.equals(TypeName.LONG)) { + return LONG_CONSTANT_VECTOR; + } + if (elementType.equals(TypeName.DOUBLE)) { + return DOUBLE_CONSTANT_VECTOR; + } + throw new IllegalArgumentException("unknown vector type for [" + elementType + "]"); + } + + static TypeName elementType(TypeName t) { + if (t.equals(BOOLEAN_BLOCK) || t.equals(BOOLEAN_VECTOR) || t.equals(BOOLEAN_BLOCK_BUILDER)) { + return TypeName.BOOLEAN; + } + if (t.equals(BYTES_REF_BLOCK) || t.equals(BYTES_REF_VECTOR) || t.equals(BYTES_REF_BLOCK_BUILDER)) { + return BYTES_REF; + } + if (t.equals(INT_BLOCK) || t.equals(INT_VECTOR) || t.equals(INT_BLOCK_BUILDER)) { + return TypeName.INT; + } + if (t.equals(LONG_BLOCK) || t.equals(LONG_VECTOR) || t.equals(LONG_BLOCK_BUILDER)) { + return TypeName.LONG; + } + if (t.equals(DOUBLE_BLOCK) || t.equals(DOUBLE_VECTOR) || t.equals(DOUBLE_BLOCK_BUILDER)) { + return TypeName.DOUBLE; + } + throw new IllegalArgumentException("unknown element type for [" + t + "]"); + } + +} diff --git a/x-pack/plugin/esql/compute/gen/src/main/resources/META-INF/services/javax.annotation.processing.Processor b/x-pack/plugin/esql/compute/gen/src/main/resources/META-INF/services/javax.annotation.processing.Processor new file mode 100644 index 0000000000000..51700a418a02b --- /dev/null +++ b/x-pack/plugin/esql/compute/gen/src/main/resources/META-INF/services/javax.annotation.processing.Processor @@ -0,0 +1,3 @@ +org.elasticsearch.compute.gen.AggregatorProcessor +org.elasticsearch.compute.gen.ConsumeProcessor +org.elasticsearch.compute.gen.EvaluatorProcessor diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleArrayState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleArrayState.java new file mode 100644 index 0000000000000..f08475f2339d0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleArrayState.java @@ -0,0 +1,111 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.core.Releasables; + +/** + * Aggregator state for an array of doubles. It is created in a mode where it + * won't track the {@code groupId}s that are sent to it and it is the + * responsibility of the caller to only fetch values for {@code groupId}s + * that it has sent using the {@code selected} parameter when building the + * results. This is fine when there are no {@code null} values in the input + * data. But once there are null values in the input data it is + * much more convenient to only send non-null values and + * the tracking built into the grouping code can't track that. In that case + * call {@link #enableGroupIdTracking} to transition the state into a mode + * where it'll track which {@code groupIds} have been written. + *

+ * This class is generated. Do not edit it. + *

+ */ +final class DoubleArrayState extends AbstractArrayState implements GroupingAggregatorState { + private final double init; + + private DoubleArray values; + + DoubleArrayState(BigArrays bigArrays, double init) { + super(bigArrays); + this.values = bigArrays.newDoubleArray(1, false); + this.values.set(0, init); + this.init = init; + } + + double get(int groupId) { + return values.get(groupId); + } + + double getOrDefault(int groupId) { + return groupId < values.size() ? values.get(groupId) : init; + } + + void set(int groupId, double value) { + ensureCapacity(groupId); + values.set(groupId, value); + trackGroupId(groupId); + } + + Block toValuesBlock(org.elasticsearch.compute.data.IntVector selected) { + if (false == trackingGroupIds()) { + DoubleVector.Builder builder = DoubleVector.newVectorBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + builder.appendDouble(values.get(selected.getInt(i))); + } + return builder.build().asBlock(); + } + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (hasValue(group)) { + builder.appendDouble(values.get(group)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + private void ensureCapacity(int groupId) { + if (groupId >= values.size()) { + long prevSize = values.size(); + values = bigArrays.grow(values, groupId + 1); + values.fill(prevSize, values.size(), init); + } + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 2; + var valuesBuilder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + var hasValueBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (group < values.size()) { + valuesBuilder.appendDouble(values.get(group)); + } else { + valuesBuilder.appendDouble(0); // TODO can we just use null? + } + hasValueBuilder.appendBoolean(hasValue(group)); + } + blocks[offset + 0] = valuesBuilder.build(); + blocks[offset + 1] = hasValueBuilder.build(); + } + + @Override + public void close() { + Releasables.close(values, super::close); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleState.java new file mode 100644 index 0000000000000..3536976d47373 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/DoubleState.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.ConstantDoubleVector; + +/** + * Aggregator state for a single double. + * This class is generated. Do not edit it. + */ +final class DoubleState implements AggregatorState { + private double value; + private boolean seen; + + DoubleState() { + this(0); + } + + DoubleState(double init) { + this.value = init; + } + + double doubleValue() { + return value; + } + + void doubleValue(double value) { + this.value = value; + } + + boolean seen() { + return seen; + } + + void seen(boolean seen) { + this.seen = seen; + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 2; + blocks[offset + 0] = new ConstantDoubleVector(value, 1).asBlock(); + blocks[offset + 1] = new ConstantBooleanVector(seen, 1).asBlock(); + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntArrayState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntArrayState.java new file mode 100644 index 0000000000000..c9f8aee229376 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntArrayState.java @@ -0,0 +1,110 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.IntArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.core.Releasables; + +/** + * Aggregator state for an array of ints. It is created in a mode where it + * won't track the {@code groupId}s that are sent to it and it is the + * responsibility of the caller to only fetch values for {@code groupId}s + * that it has sent using the {@code selected} parameter when building the + * results. This is fine when there are no {@code null} values in the input + * data. But once there are null values in the input data it is + * much more convenient to only send non-null values and + * the tracking built into the grouping code can't track that. In that case + * call {@link #enableGroupIdTracking} to transition the state into a mode + * where it'll track which {@code groupIds} have been written. + *

+ * This class is generated. Do not edit it. + *

+ */ +final class IntArrayState extends AbstractArrayState implements GroupingAggregatorState { + private final int init; + + private IntArray values; + + IntArrayState(BigArrays bigArrays, int init) { + super(bigArrays); + this.values = bigArrays.newIntArray(1, false); + this.values.set(0, init); + this.init = init; + } + + int get(int groupId) { + return values.get(groupId); + } + + int getOrDefault(int groupId) { + return groupId < values.size() ? values.get(groupId) : init; + } + + void set(int groupId, int value) { + ensureCapacity(groupId); + values.set(groupId, value); + trackGroupId(groupId); + } + + Block toValuesBlock(org.elasticsearch.compute.data.IntVector selected) { + if (false == trackingGroupIds()) { + IntVector.Builder builder = IntVector.newVectorBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + builder.appendInt(values.get(selected.getInt(i))); + } + return builder.build().asBlock(); + } + IntBlock.Builder builder = IntBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (hasValue(group)) { + builder.appendInt(values.get(group)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + private void ensureCapacity(int groupId) { + if (groupId >= values.size()) { + long prevSize = values.size(); + values = bigArrays.grow(values, groupId + 1); + values.fill(prevSize, values.size(), init); + } + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 2; + var valuesBuilder = IntBlock.newBlockBuilder(selected.getPositionCount()); + var hasValueBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (group < values.size()) { + valuesBuilder.appendInt(values.get(group)); + } else { + valuesBuilder.appendInt(0); // TODO can we just use null? + } + hasValueBuilder.appendBoolean(hasValue(group)); + } + blocks[offset + 0] = valuesBuilder.build(); + blocks[offset + 1] = hasValueBuilder.build(); + } + + @Override + public void close() { + Releasables.close(values, super::close); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntState.java new file mode 100644 index 0000000000000..8492f29f71a68 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/IntState.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.ConstantIntVector; + +/** + * Aggregator state for a single int. + * This class is generated. Do not edit it. + */ +final class IntState implements AggregatorState { + private int value; + private boolean seen; + + IntState() { + this(0); + } + + IntState(int init) { + this.value = init; + } + + int intValue() { + return value; + } + + void intValue(int value) { + this.value = value; + } + + boolean seen() { + return seen; + } + + void seen(boolean seen) { + this.seen = seen; + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 2; + blocks[offset + 0] = new ConstantIntVector(value, 1).asBlock(); + blocks[offset + 1] = new ConstantBooleanVector(seen, 1).asBlock(); + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongArrayState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongArrayState.java new file mode 100644 index 0000000000000..08da95417770f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongArrayState.java @@ -0,0 +1,117 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.core.Releasables; + +/** + * Aggregator state for an array of longs. It is created in a mode where it + * won't track the {@code groupId}s that are sent to it and it is the + * responsibility of the caller to only fetch values for {@code groupId}s + * that it has sent using the {@code selected} parameter when building the + * results. This is fine when there are no {@code null} values in the input + * data. But once there are null values in the input data it is + * much more convenient to only send non-null values and + * the tracking built into the grouping code can't track that. In that case + * call {@link #enableGroupIdTracking} to transition the state into a mode + * where it'll track which {@code groupIds} have been written. + *

+ * This class is generated. Do not edit it. + *

+ */ +final class LongArrayState extends AbstractArrayState implements GroupingAggregatorState { + private final long init; + + private LongArray values; + + LongArrayState(BigArrays bigArrays, long init) { + super(bigArrays); + this.values = bigArrays.newLongArray(1, false); + this.values.set(0, init); + this.init = init; + } + + long get(int groupId) { + return values.get(groupId); + } + + long getOrDefault(int groupId) { + return groupId < values.size() ? values.get(groupId) : init; + } + + void set(int groupId, long value) { + ensureCapacity(groupId); + values.set(groupId, value); + trackGroupId(groupId); + } + + void increment(int groupId, long value) { + ensureCapacity(groupId); + values.increment(groupId, value); + trackGroupId(groupId); + } + + Block toValuesBlock(org.elasticsearch.compute.data.IntVector selected) { + if (false == trackingGroupIds()) { + LongVector.Builder builder = LongVector.newVectorBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + builder.appendLong(values.get(selected.getInt(i))); + } + return builder.build().asBlock(); + } + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (hasValue(group)) { + builder.appendLong(values.get(group)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + private void ensureCapacity(int groupId) { + if (groupId >= values.size()) { + long prevSize = values.size(); + values = bigArrays.grow(values, groupId + 1); + values.fill(prevSize, values.size(), init); + } + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 2; + var valuesBuilder = LongBlock.newBlockBuilder(selected.getPositionCount()); + var hasValueBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (group < values.size()) { + valuesBuilder.appendLong(values.get(group)); + } else { + valuesBuilder.appendLong(0); // TODO can we just use null? + } + hasValueBuilder.appendBoolean(hasValue(group)); + } + blocks[offset + 0] = valuesBuilder.build(); + blocks[offset + 1] = hasValueBuilder.build(); + } + + @Override + public void close() { + Releasables.close(values, super::close); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongState.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongState.java new file mode 100644 index 0000000000000..bd4e8d0637077 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/aggregation/LongState.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.ConstantLongVector; + +/** + * Aggregator state for a single long. + * This class is generated. Do not edit it. + */ +final class LongState implements AggregatorState { + private long value; + private boolean seen; + + LongState() { + this(0); + } + + LongState(long init) { + this.value = init; + } + + long longValue() { + return value; + } + + void longValue(long value) { + this.value = value; + } + + boolean seen() { + return seen; + } + + void seen(boolean seen) { + this.seen = seen; + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 2; + blocks[offset + 0] = new ConstantLongVector(value, 1).asBlock(); + blocks[offset + 1] = new ConstantBooleanVector(seen, 1).asBlock(); + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java new file mode 100644 index 0000000000000..b0c2843554a69 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayBlock.java @@ -0,0 +1,84 @@ +/* + * 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.compute.data; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of boolean. + * This class is generated. Do not edit it. + */ +public final class BooleanArrayBlock extends AbstractArrayBlock implements BooleanBlock { + + private final boolean[] values; + + public BooleanArrayBlock(boolean[] values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public BooleanVector asVector() { + return null; + } + + @Override + public boolean getBoolean(int valueIndex) { + return values[valueIndex]; + } + + @Override + public BooleanBlock filter(int... positions) { + return new FilterBooleanBlock(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public BooleanBlock expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new BooleanArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new BooleanArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanBlock that) { + return BooleanBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() + + ", values=" + + Arrays.toString(values) + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayVector.java new file mode 100644 index 0000000000000..1b4374061b4e1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanArrayVector.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +import java.util.Arrays; + +/** + * Vector implementation that stores an array of boolean values. + * This class is generated. Do not edit it. + */ +public final class BooleanArrayVector extends AbstractVector implements BooleanVector { + + private final boolean[] values; + + public BooleanArrayVector(boolean[] values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public BooleanBlock asBlock() { + return new BooleanVectorBlock(this); + } + + @Override + public boolean getBoolean(int position) { + return values[position]; + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public BooleanVector filter(int... positions) { + return new FilterBooleanVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanVector that) { + return BooleanVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + Arrays.toString(values) + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayVector.java new file mode 100644 index 0000000000000..d1f43310f00d1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBigArrayVector.java @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.core.Releasable; + +/** + * Vector implementation that defers to an enclosed BooleanArray. + * This class is generated. Do not edit it. + */ +public final class BooleanBigArrayVector extends AbstractVector implements BooleanVector, Releasable { + + private final BitArray values; + + public BooleanBigArrayVector(BitArray values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public BooleanBlock asBlock() { + return new BooleanVectorBlock(this); + } + + @Override + public boolean getBoolean(int position) { + return values.get(position); + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public BooleanVector filter(int... positions) { + return new FilterBooleanVector(this, positions); + } + + @Override + public void close() { + values.close(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanVector that) { + return BooleanVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + values + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java new file mode 100644 index 0000000000000..fbab7132ba525 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java @@ -0,0 +1,203 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores boolean values. + * This class is generated. Do not edit it. + */ +public sealed interface BooleanBlock extends Block permits FilterBooleanBlock, BooleanArrayBlock, BooleanVectorBlock { + + /** + * Retrieves the boolean value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index + * @return the data value (as a boolean) + */ + boolean getBoolean(int valueIndex); + + @Override + BooleanVector asVector(); + + @Override + BooleanBlock filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "BooleanBlock", BooleanBlock::of); + + @Override + default String getWriteableName() { + return "BooleanBlock"; + } + + static BooleanBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.appendBoolean(in.readBoolean()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + out.writeBoolean(getBoolean(getFirstValueIndex(pos) + valueIndex)); + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a BooleanBlock, and both blocks are {@link #equals(BooleanBlock, BooleanBlock) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash(BooleanBlock)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the BooleanBlock interface. + */ + static boolean equals(BooleanBlock block1, BooleanBlock block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + if (block1.getBoolean(b1ValueIdx + valueIndex) != block2.getBoolean(b2ValueIdx + valueIndex)) { + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(BooleanBlock block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + result = 31 * result + Boolean.hashCode(block.getBoolean(firstValueIdx + valueIndex)); + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new BooleanBlockBuilder(estimatedSize); + } + + static BooleanBlock newConstantBlockWith(boolean value, int positions) { + return new ConstantBooleanVector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits BooleanBlockBuilder { + + /** + * Appends a boolean to the current entry. + */ + Builder appendBoolean(boolean value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(BooleanBlock block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(BooleanBlock block); + + @Override + BooleanBlock build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlockBuilder.java new file mode 100644 index 0000000000000..101998edbd3bd --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlockBuilder.java @@ -0,0 +1,187 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of BooleanBlocks. + * This class is generated. Do not edit it. + */ +final class BooleanBlockBuilder extends AbstractBlockBuilder implements BooleanBlock.Builder { + + private boolean[] values; + + BooleanBlockBuilder(int estimatedSize) { + values = new boolean[Math.max(estimatedSize, 2)]; + } + + @Override + public BooleanBlockBuilder appendBoolean(boolean value) { + ensureCapacity(); + values[valueCount] = value; + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public BooleanBlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public BooleanBlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public BooleanBlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public BooleanBlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition((BooleanBlock) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public BooleanBlockBuilder appendAllValuesToCurrentPosition(BooleanBlock block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } + final BooleanVector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { + appendBoolean(vector.getBoolean(p)); + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendBoolean(block.getBoolean(i++)); + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public BooleanBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom((BooleanBlock) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public BooleanBlockBuilder copyFrom(BooleanBlock block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + BooleanVector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock(BooleanBlock block, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendBoolean(block.getBoolean(i++)); + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector(BooleanVector vector, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendBoolean(vector.getBoolean(p)); + } + } + + @Override + public BooleanBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public BooleanBlock build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { + return new ConstantBooleanVector(values[0], 1).asBlock(); + } else { + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + if (isDense() && singleValued()) { + return new BooleanArrayVector(values, positionCount).asBlock(); + } else { + return new BooleanArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVector.java new file mode 100644 index 0000000000000..c020d41d22cab --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVector.java @@ -0,0 +1,82 @@ +/* + * 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.compute.data; + +/** + * Vector that stores boolean values. + * This class is generated. Do not edit it. + */ +public sealed interface BooleanVector extends Vector permits ConstantBooleanVector, FilterBooleanVector, BooleanArrayVector, + BooleanBigArrayVector { + boolean getBoolean(int position); + + @Override + BooleanBlock asBlock(); + + @Override + BooleanVector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a BooleanVector, and both vectors are {@link #equals(BooleanVector, BooleanVector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash(BooleanVector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the BooleanVector interface. + */ + static boolean equals(BooleanVector vector1, BooleanVector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (vector1.getBoolean(pos) != vector2.getBoolean(pos)) { + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(BooleanVector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { + result = 31 * result + Boolean.hashCode(vector.getBoolean(pos)); + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new BooleanVectorBuilder(estimatedSize); + } + + sealed interface Builder extends Vector.Builder permits BooleanVectorBuilder { + /** + * Appends a boolean to the current entry. + */ + Builder appendBoolean(boolean value); + + @Override + BooleanVector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBlock.java new file mode 100644 index 0000000000000..23e069989f4f1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBlock.java @@ -0,0 +1,111 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a BooleanVector. + * This class is generated. Do not edit it. + */ +public final class BooleanVectorBlock extends AbstractVectorBlock implements BooleanBlock { + + private final BooleanVector vector; + + BooleanVectorBlock(BooleanVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public BooleanVector asVector() { + return vector; + } + + @Override + public boolean getBoolean(int valueIndex) { + return vector.getBoolean(valueIndex); + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public BooleanBlock filter(int... positions) { + return new FilterBooleanVector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "BooleanVectorBlock", + BooleanVectorBlock::of + ); + + @Override + public String getWriteableName() { + return "BooleanVectorBlock"; + } + + static BooleanVectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new BooleanVectorBlock(new ConstantBooleanVector(in.readBoolean(), positions)); + } else { + var builder = BooleanVector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.appendBoolean(in.readBoolean()); + } + return new BooleanVectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final BooleanVector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { + out.writeBoolean(getBoolean(0)); + } else { + for (int i = 0; i < positions; i++) { + out.writeBoolean(getBoolean(i)); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanBlock that) { + return BooleanBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBuilder.java new file mode 100644 index 0000000000000..cd1f84e03dd14 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanVectorBuilder.java @@ -0,0 +1,52 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of BooleanBlocks. + * This class is generated. Do not edit it. + */ +final class BooleanVectorBuilder extends AbstractVectorBuilder implements BooleanVector.Builder { + + private boolean[] values; + + BooleanVectorBuilder(int estimatedSize) { + values = new boolean[Math.max(estimatedSize, 2)]; + } + + @Override + public BooleanVectorBuilder appendBoolean(boolean value) { + ensureCapacity(); + values[valueCount] = value; + valueCount++; + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public BooleanVector build() { + if (valueCount == 1) { + return new ConstantBooleanVector(values[0], 1); + } + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + return new BooleanArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java new file mode 100644 index 0000000000000..263378e5cf846 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayBlock.java @@ -0,0 +1,86 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefArray; + +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of BytesRef. + * This class is generated. Do not edit it. + */ +public final class BytesRefArrayBlock extends AbstractArrayBlock implements BytesRefBlock { + + private final BytesRefArray values; + + public BytesRefArrayBlock(BytesRefArray values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public BytesRefVector asVector() { + return null; + } + + @Override + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return values.get(valueIndex, dest); + } + + @Override + public BytesRefBlock filter(int... positions) { + return new FilterBytesRefBlock(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.BYTES_REF; + } + + @Override + public BytesRefBlock expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new BytesRefArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new BytesRefArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefBlock that) { + return BytesRefBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() + + ", values=" + + values.size() + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayVector.java new file mode 100644 index 0000000000000..6f1970fe66c38 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefArrayVector.java @@ -0,0 +1,68 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefArray; + +/** + * Vector implementation that stores an array of BytesRef values. + * This class is generated. Do not edit it. + */ +public final class BytesRefArrayVector extends AbstractVector implements BytesRefVector { + + private final BytesRefArray values; + + public BytesRefArrayVector(BytesRefArray values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public BytesRefBlock asBlock() { + return new BytesRefVectorBlock(this); + } + + @Override + public BytesRef getBytesRef(int position, BytesRef dest) { + return values.get(position, dest); + } + + @Override + public ElementType elementType() { + return ElementType.BYTES_REF; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public BytesRefVector filter(int... positions) { + return new FilterBytesRefVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefVector that) { + return BytesRefVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java new file mode 100644 index 0000000000000..4ee11728b72d4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java @@ -0,0 +1,206 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores BytesRef values. + * This class is generated. Do not edit it. + */ +public sealed interface BytesRefBlock extends Block permits FilterBytesRefBlock, BytesRefArrayBlock, BytesRefVectorBlock { + + /** + * Retrieves the BytesRef value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index + * @param dest the destination + * @return the data value (as a BytesRef) + */ + BytesRef getBytesRef(int valueIndex, BytesRef dest); + + @Override + BytesRefVector asVector(); + + @Override + BytesRefBlock filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "BytesRefBlock", BytesRefBlock::of); + + @Override + default String getWriteableName() { + return "BytesRefBlock"; + } + + static BytesRefBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.appendBytesRef(in.readBytesRef()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + out.writeBytesRef(getBytesRef(getFirstValueIndex(pos) + valueIndex, new BytesRef())); + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a BytesRefBlock, and both blocks are {@link #equals(BytesRefBlock, BytesRefBlock) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash(BytesRefBlock)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the BytesRefBlock interface. + */ + static boolean equals(BytesRefBlock block1, BytesRefBlock block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + if (block1.getBytesRef(b1ValueIdx + valueIndex, new BytesRef()) + .equals(block2.getBytesRef(b2ValueIdx + valueIndex, new BytesRef())) == false) { + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(BytesRefBlock block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + result = 31 * result + block.getBytesRef(firstValueIdx + valueIndex, new BytesRef()).hashCode(); + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new BytesRefBlockBuilder(estimatedSize); + } + + static BytesRefBlock newConstantBlockWith(BytesRef value, int positions) { + return new ConstantBytesRefVector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits BytesRefBlockBuilder { + + /** + * Appends a BytesRef to the current entry. + */ + Builder appendBytesRef(BytesRef value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(BytesRefBlock block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(BytesRefBlock block); + + @Override + BytesRefBlock build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlockBuilder.java new file mode 100644 index 0000000000000..aaed753cc20e1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlockBuilder.java @@ -0,0 +1,200 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; + +/** + * Block build of BytesRefBlocks. + * This class is generated. Do not edit it. + */ +final class BytesRefBlockBuilder extends AbstractBlockBuilder implements BytesRefBlock.Builder { + + private static final BytesRef NULL_VALUE = new BytesRef(); + + private BytesRefArray values; + + BytesRefBlockBuilder(int estimatedSize) { + this(estimatedSize, BigArrays.NON_RECYCLING_INSTANCE); + } + + BytesRefBlockBuilder(int estimatedSize, BigArrays bigArrays) { + values = new BytesRefArray(Math.max(estimatedSize, 2), bigArrays); + } + + @Override + public BytesRefBlockBuilder appendBytesRef(BytesRef value) { + ensureCapacity(); + values.append(value); + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { + return Integer.MAX_VALUE; // allow the BytesRefArray through its own append + } + + @Override + protected void growValuesArray(int newSize) { + throw new AssertionError("should not reach here"); + } + + @Override + public BytesRefBlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public BytesRefBlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public BytesRefBlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + + @Override + protected void writeNullValue() { + values.append(NULL_VALUE); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public BytesRefBlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition((BytesRefBlock) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public BytesRefBlockBuilder appendAllValuesToCurrentPosition(BytesRefBlock block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } + BytesRef scratch = new BytesRef(); + final BytesRefVector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { + appendBytesRef(vector.getBytesRef(p, scratch)); + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendBytesRef(block.getBytesRef(i++, scratch)); + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public BytesRefBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom((BytesRefBlock) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public BytesRefBlockBuilder copyFrom(BytesRefBlock block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + BytesRefVector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock(BytesRefBlock block, int beginInclusive, int endExclusive) { + BytesRef scratch = new BytesRef(); + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendBytesRef(block.getBytesRef(i++, scratch)); + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector(BytesRefVector vector, int beginInclusive, int endExclusive) { + BytesRef scratch = new BytesRef(); + for (int p = beginInclusive; p < endExclusive; p++) { + appendBytesRef(vector.getBytesRef(p, scratch)); + } + } + + @Override + public BytesRefBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public BytesRefBlock build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { + return new ConstantBytesRefVector(values.get(0, new BytesRef()), 1).asBlock(); + } else { + if (isDense() && singleValued()) { + return new BytesRefArrayVector(values, positionCount).asBlock(); + } else { + return new BytesRefArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVector.java new file mode 100644 index 0000000000000..27b02d5fc651b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVector.java @@ -0,0 +1,83 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +/** + * Vector that stores BytesRef values. + * This class is generated. Do not edit it. + */ +public sealed interface BytesRefVector extends Vector permits ConstantBytesRefVector, FilterBytesRefVector, BytesRefArrayVector { + BytesRef getBytesRef(int position, BytesRef dest); + + @Override + BytesRefBlock asBlock(); + + @Override + BytesRefVector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a BytesRefVector, and both vectors are {@link #equals(BytesRefVector, BytesRefVector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash(BytesRefVector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the BytesRefVector interface. + */ + static boolean equals(BytesRefVector vector1, BytesRefVector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (vector1.getBytesRef(pos, new BytesRef()).equals(vector2.getBytesRef(pos, new BytesRef())) == false) { + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(BytesRefVector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { + result = 31 * result + vector.getBytesRef(pos, new BytesRef()).hashCode(); + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new BytesRefVectorBuilder(estimatedSize); + } + + sealed interface Builder extends Vector.Builder permits BytesRefVectorBuilder { + /** + * Appends a BytesRef to the current entry. + */ + Builder appendBytesRef(BytesRef value); + + @Override + BytesRefVector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBlock.java new file mode 100644 index 0000000000000..d52ac7e66d04d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBlock.java @@ -0,0 +1,112 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a BytesRefVector. + * This class is generated. Do not edit it. + */ +public final class BytesRefVectorBlock extends AbstractVectorBlock implements BytesRefBlock { + + private final BytesRefVector vector; + + BytesRefVectorBlock(BytesRefVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public BytesRefVector asVector() { + return vector; + } + + @Override + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return vector.getBytesRef(valueIndex, dest); + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public BytesRefBlock filter(int... positions) { + return new FilterBytesRefVector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "BytesRefVectorBlock", + BytesRefVectorBlock::of + ); + + @Override + public String getWriteableName() { + return "BytesRefVectorBlock"; + } + + static BytesRefVectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new BytesRefVectorBlock(new ConstantBytesRefVector(in.readBytesRef(), positions)); + } else { + var builder = BytesRefVector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.appendBytesRef(in.readBytesRef()); + } + return new BytesRefVectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final BytesRefVector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { + out.writeBytesRef(getBytesRef(0, new BytesRef())); + } else { + for (int i = 0; i < positions; i++) { + out.writeBytesRef(getBytesRef(i, new BytesRef())); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefBlock that) { + return BytesRefBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBuilder.java new file mode 100644 index 0000000000000..6035cdf5774e4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefVectorBuilder.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; + +/** + * Block build of BytesRefBlocks. + * This class is generated. Do not edit it. + */ +final class BytesRefVectorBuilder extends AbstractVectorBuilder implements BytesRefVector.Builder { + + private BytesRefArray values; + + BytesRefVectorBuilder(int estimatedSize) { + this(estimatedSize, BigArrays.NON_RECYCLING_INSTANCE); + } + + BytesRefVectorBuilder(int estimatedSize, BigArrays bigArrays) { + values = new BytesRefArray(Math.max(estimatedSize, 2), bigArrays); + } + + @Override + public BytesRefVectorBuilder appendBytesRef(BytesRef value) { + ensureCapacity(); + values.append(value); + valueCount++; + return this; + } + + @Override + protected int valuesLength() { + return Integer.MAX_VALUE; // allow the BytesRefArray through its own append + } + + @Override + protected void growValuesArray(int newSize) { + throw new AssertionError("should not reach here"); + } + + @Override + public BytesRefVector build() { + if (valueCount == 1) { + return new ConstantBytesRefVector(values.get(0, new BytesRef()), 1); + } + return new BytesRefArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBooleanVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBooleanVector.java new file mode 100644 index 0000000000000..e802548350d39 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBooleanVector.java @@ -0,0 +1,64 @@ +/* + * 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.compute.data; + +/** + * Vector implementation that stores a constant boolean value. + * This class is generated. Do not edit it. + */ +public final class ConstantBooleanVector extends AbstractVector implements BooleanVector { + + private final boolean value; + + public ConstantBooleanVector(boolean value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override + public boolean getBoolean(int position) { + return value; + } + + @Override + public BooleanBlock asBlock() { + return new BooleanVectorBlock(this); + } + + @Override + public BooleanVector filter(int... positions) { + return new ConstantBooleanVector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanVector that) { + return BooleanVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanVector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBytesRefVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBytesRefVector.java new file mode 100644 index 0000000000000..25f07d72c1d65 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantBytesRefVector.java @@ -0,0 +1,66 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +/** + * Vector implementation that stores a constant BytesRef value. + * This class is generated. Do not edit it. + */ +public final class ConstantBytesRefVector extends AbstractVector implements BytesRefVector { + + private final BytesRef value; + + public ConstantBytesRefVector(BytesRef value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override + public BytesRef getBytesRef(int position, BytesRef ignore) { + return value; + } + + @Override + public BytesRefBlock asBlock() { + return new BytesRefVectorBlock(this); + } + + @Override + public BytesRefVector filter(int... positions) { + return new ConstantBytesRefVector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.BYTES_REF; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefVector that) { + return BytesRefVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefVector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantDoubleVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantDoubleVector.java new file mode 100644 index 0000000000000..8d196aa33f974 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantDoubleVector.java @@ -0,0 +1,64 @@ +/* + * 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.compute.data; + +/** + * Vector implementation that stores a constant double value. + * This class is generated. Do not edit it. + */ +public final class ConstantDoubleVector extends AbstractVector implements DoubleVector { + + private final double value; + + public ConstantDoubleVector(double value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override + public double getDouble(int position) { + return value; + } + + @Override + public DoubleBlock asBlock() { + return new DoubleVectorBlock(this); + } + + @Override + public DoubleVector filter(int... positions) { + return new ConstantDoubleVector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleVector that) { + return DoubleVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleVector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantIntVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantIntVector.java new file mode 100644 index 0000000000000..ad942bb79e779 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantIntVector.java @@ -0,0 +1,64 @@ +/* + * 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.compute.data; + +/** + * Vector implementation that stores a constant int value. + * This class is generated. Do not edit it. + */ +public final class ConstantIntVector extends AbstractVector implements IntVector { + + private final int value; + + public ConstantIntVector(int value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override + public int getInt(int position) { + return value; + } + + @Override + public IntBlock asBlock() { + return new IntVectorBlock(this); + } + + @Override + public IntVector filter(int... positions) { + return new ConstantIntVector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntVector that) { + return IntVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntVector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantLongVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantLongVector.java new file mode 100644 index 0000000000000..79d9ba76db48c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/ConstantLongVector.java @@ -0,0 +1,64 @@ +/* + * 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.compute.data; + +/** + * Vector implementation that stores a constant long value. + * This class is generated. Do not edit it. + */ +public final class ConstantLongVector extends AbstractVector implements LongVector { + + private final long value; + + public ConstantLongVector(long value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override + public long getLong(int position) { + return value; + } + + @Override + public LongBlock asBlock() { + return new LongVectorBlock(this); + } + + @Override + public LongVector filter(int... positions) { + return new ConstantLongVector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongVector that) { + return LongVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongVector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java new file mode 100644 index 0000000000000..c74de042da52b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayBlock.java @@ -0,0 +1,84 @@ +/* + * 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.compute.data; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of double. + * This class is generated. Do not edit it. + */ +public final class DoubleArrayBlock extends AbstractArrayBlock implements DoubleBlock { + + private final double[] values; + + public DoubleArrayBlock(double[] values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public DoubleVector asVector() { + return null; + } + + @Override + public double getDouble(int valueIndex) { + return values[valueIndex]; + } + + @Override + public DoubleBlock filter(int... positions) { + return new FilterDoubleBlock(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public DoubleBlock expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new DoubleArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new DoubleArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleBlock that) { + return DoubleBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() + + ", values=" + + Arrays.toString(values) + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayVector.java new file mode 100644 index 0000000000000..340d434907643 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleArrayVector.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +import java.util.Arrays; + +/** + * Vector implementation that stores an array of double values. + * This class is generated. Do not edit it. + */ +public final class DoubleArrayVector extends AbstractVector implements DoubleVector { + + private final double[] values; + + public DoubleArrayVector(double[] values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public DoubleBlock asBlock() { + return new DoubleVectorBlock(this); + } + + @Override + public double getDouble(int position) { + return values[position]; + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public DoubleVector filter(int... positions) { + return new FilterDoubleVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleVector that) { + return DoubleVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + Arrays.toString(values) + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayVector.java new file mode 100644 index 0000000000000..138fecbf0725b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBigArrayVector.java @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.core.Releasable; + +/** + * Vector implementation that defers to an enclosed DoubleArray. + * This class is generated. Do not edit it. + */ +public final class DoubleBigArrayVector extends AbstractVector implements DoubleVector, Releasable { + + private final DoubleArray values; + + public DoubleBigArrayVector(DoubleArray values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public DoubleBlock asBlock() { + return new DoubleVectorBlock(this); + } + + @Override + public double getDouble(int position) { + return values.get(position); + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public DoubleVector filter(int... positions) { + return new FilterDoubleVector(this, positions); + } + + @Override + public void close() { + values.close(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleVector that) { + return DoubleVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + values + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java new file mode 100644 index 0000000000000..3f5fe1c180686 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java @@ -0,0 +1,204 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores double values. + * This class is generated. Do not edit it. + */ +public sealed interface DoubleBlock extends Block permits FilterDoubleBlock, DoubleArrayBlock, DoubleVectorBlock { + + /** + * Retrieves the double value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index + * @return the data value (as a double) + */ + double getDouble(int valueIndex); + + @Override + DoubleVector asVector(); + + @Override + DoubleBlock filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "DoubleBlock", DoubleBlock::of); + + @Override + default String getWriteableName() { + return "DoubleBlock"; + } + + static DoubleBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.appendDouble(in.readDouble()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + out.writeDouble(getDouble(getFirstValueIndex(pos) + valueIndex)); + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a DoubleBlock, and both blocks are {@link #equals(DoubleBlock, DoubleBlock) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash(DoubleBlock)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the DoubleBlock interface. + */ + static boolean equals(DoubleBlock block1, DoubleBlock block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + if (block1.getDouble(b1ValueIdx + valueIndex) != block2.getDouble(b2ValueIdx + valueIndex)) { + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(DoubleBlock block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + long element = Double.doubleToLongBits(block.getDouble(firstValueIdx + valueIndex)); + result = 31 * result + (int) (element ^ (element >>> 32)); + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new DoubleBlockBuilder(estimatedSize); + } + + static DoubleBlock newConstantBlockWith(double value, int positions) { + return new ConstantDoubleVector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits DoubleBlockBuilder { + + /** + * Appends a double to the current entry. + */ + Builder appendDouble(double value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(DoubleBlock block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(DoubleBlock block); + + @Override + DoubleBlock build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlockBuilder.java new file mode 100644 index 0000000000000..ff5b1ddf6e1d2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlockBuilder.java @@ -0,0 +1,187 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of DoubleBlocks. + * This class is generated. Do not edit it. + */ +final class DoubleBlockBuilder extends AbstractBlockBuilder implements DoubleBlock.Builder { + + private double[] values; + + DoubleBlockBuilder(int estimatedSize) { + values = new double[Math.max(estimatedSize, 2)]; + } + + @Override + public DoubleBlockBuilder appendDouble(double value) { + ensureCapacity(); + values[valueCount] = value; + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public DoubleBlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public DoubleBlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public DoubleBlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public DoubleBlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition((DoubleBlock) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public DoubleBlockBuilder appendAllValuesToCurrentPosition(DoubleBlock block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } + final DoubleVector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { + appendDouble(vector.getDouble(p)); + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendDouble(block.getDouble(i++)); + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public DoubleBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom((DoubleBlock) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public DoubleBlockBuilder copyFrom(DoubleBlock block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + DoubleVector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock(DoubleBlock block, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendDouble(block.getDouble(i++)); + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector(DoubleVector vector, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendDouble(vector.getDouble(p)); + } + } + + @Override + public DoubleBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public DoubleBlock build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { + return new ConstantDoubleVector(values[0], 1).asBlock(); + } else { + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + if (isDense() && singleValued()) { + return new DoubleArrayVector(values, positionCount).asBlock(); + } else { + return new DoubleArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVector.java new file mode 100644 index 0000000000000..d6886bef60a05 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVector.java @@ -0,0 +1,83 @@ +/* + * 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.compute.data; + +/** + * Vector that stores double values. + * This class is generated. Do not edit it. + */ +public sealed interface DoubleVector extends Vector permits ConstantDoubleVector, FilterDoubleVector, DoubleArrayVector, + DoubleBigArrayVector { + double getDouble(int position); + + @Override + DoubleBlock asBlock(); + + @Override + DoubleVector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a DoubleVector, and both vectors are {@link #equals(DoubleVector, DoubleVector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash(DoubleVector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the DoubleVector interface. + */ + static boolean equals(DoubleVector vector1, DoubleVector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (vector1.getDouble(pos) != vector2.getDouble(pos)) { + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(DoubleVector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { + long element = Double.doubleToLongBits(vector.getDouble(pos)); + result = 31 * result + (int) (element ^ (element >>> 32)); + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new DoubleVectorBuilder(estimatedSize); + } + + sealed interface Builder extends Vector.Builder permits DoubleVectorBuilder { + /** + * Appends a double to the current entry. + */ + Builder appendDouble(double value); + + @Override + DoubleVector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBlock.java new file mode 100644 index 0000000000000..adc4d9d2eee01 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBlock.java @@ -0,0 +1,111 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a DoubleVector. + * This class is generated. Do not edit it. + */ +public final class DoubleVectorBlock extends AbstractVectorBlock implements DoubleBlock { + + private final DoubleVector vector; + + DoubleVectorBlock(DoubleVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public DoubleVector asVector() { + return vector; + } + + @Override + public double getDouble(int valueIndex) { + return vector.getDouble(valueIndex); + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public DoubleBlock filter(int... positions) { + return new FilterDoubleVector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "DoubleVectorBlock", + DoubleVectorBlock::of + ); + + @Override + public String getWriteableName() { + return "DoubleVectorBlock"; + } + + static DoubleVectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new DoubleVectorBlock(new ConstantDoubleVector(in.readDouble(), positions)); + } else { + var builder = DoubleVector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.appendDouble(in.readDouble()); + } + return new DoubleVectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final DoubleVector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { + out.writeDouble(getDouble(0)); + } else { + for (int i = 0; i < positions; i++) { + out.writeDouble(getDouble(i)); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleBlock that) { + return DoubleBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBuilder.java new file mode 100644 index 0000000000000..782b43c1bd9e2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleVectorBuilder.java @@ -0,0 +1,52 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of DoubleBlocks. + * This class is generated. Do not edit it. + */ +final class DoubleVectorBuilder extends AbstractVectorBuilder implements DoubleVector.Builder { + + private double[] values; + + DoubleVectorBuilder(int estimatedSize) { + values = new double[Math.max(estimatedSize, 2)]; + } + + @Override + public DoubleVectorBuilder appendDouble(double value) { + ensureCapacity(); + values[valueCount] = value; + valueCount++; + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public DoubleVector build() { + if (valueCount == 1) { + return new ConstantDoubleVector(values[0], 1); + } + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + return new DoubleArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanBlock.java new file mode 100644 index 0000000000000..a3e3793498463 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanBlock.java @@ -0,0 +1,114 @@ +/* + * 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.compute.data; + +/** + * Filter block for BooleanBlocks. + * This class is generated. Do not edit it. + */ +final class FilterBooleanBlock extends AbstractFilterBlock implements BooleanBlock { + + private final BooleanBlock block; + + FilterBooleanBlock(BooleanBlock block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public BooleanVector asVector() { + return null; + } + + @Override + public boolean getBoolean(int valueIndex) { + return block.getBoolean(valueIndex); + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public BooleanBlock filter(int... positions) { + return new FilterBooleanBlock(this, positions); + } + + @Override + public BooleanBlock expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positions.length); + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + builder.appendBoolean(block.getBoolean(i)); + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanBlock that) { + return BooleanBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanBlock.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { + sb.append(getBoolean(start)); + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } + sb.append(getBoolean(i)); + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanVector.java new file mode 100644 index 0000000000000..5f6ad76e35a09 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBooleanVector.java @@ -0,0 +1,80 @@ +/* + * 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.compute.data; + +/** + * Filter vector for BooleanVectors. + * This class is generated. Do not edit it. + */ +public final class FilterBooleanVector extends AbstractFilterVector implements BooleanVector { + + private final BooleanVector vector; + + FilterBooleanVector(BooleanVector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override + public boolean getBoolean(int position) { + return vector.getBoolean(mapPosition(position)); + } + + @Override + public BooleanBlock asBlock() { + return new BooleanVectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.BOOLEAN; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public BooleanVector filter(int... positions) { + return new FilterBooleanVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BooleanVector that) { + return BooleanVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BooleanVector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(getBoolean(i)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefBlock.java new file mode 100644 index 0000000000000..3bdd60dbedb2c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefBlock.java @@ -0,0 +1,118 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +/** + * Filter block for BytesRefBlocks. + * This class is generated. Do not edit it. + */ +final class FilterBytesRefBlock extends AbstractFilterBlock implements BytesRefBlock { + + private final BytesRefBlock block; + + FilterBytesRefBlock(BytesRefBlock block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public BytesRefVector asVector() { + return null; + } + + @Override + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return block.getBytesRef(valueIndex, dest); + } + + @Override + public ElementType elementType() { + return ElementType.BYTES_REF; + } + + @Override + public BytesRefBlock filter(int... positions) { + return new FilterBytesRefBlock(this, positions); + } + + @Override + public BytesRefBlock expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positions.length); + BytesRef scratch = new BytesRef(); + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + BytesRef v = block.getBytesRef(i, scratch); + builder.appendBytesRef(v); + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefBlock that) { + return BytesRefBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefBlock.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { + sb.append(getBytesRef(start, new BytesRef())); + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } + sb.append(getBytesRef(i, new BytesRef())); + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefVector.java new file mode 100644 index 0000000000000..63ef354fd6d36 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterBytesRefVector.java @@ -0,0 +1,82 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +/** + * Filter vector for BytesRefVectors. + * This class is generated. Do not edit it. + */ +public final class FilterBytesRefVector extends AbstractFilterVector implements BytesRefVector { + + private final BytesRefVector vector; + + FilterBytesRefVector(BytesRefVector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override + public BytesRef getBytesRef(int position, BytesRef dest) { + return vector.getBytesRef(mapPosition(position), dest); + } + + @Override + public BytesRefBlock asBlock() { + return new BytesRefVectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.BYTES_REF; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public BytesRefVector filter(int... positions) { + return new FilterBytesRefVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BytesRefVector that) { + return BytesRefVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return BytesRefVector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(getBytesRef(i, new BytesRef())); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleBlock.java new file mode 100644 index 0000000000000..2f8f24b6b134f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleBlock.java @@ -0,0 +1,114 @@ +/* + * 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.compute.data; + +/** + * Filter block for DoubleBlocks. + * This class is generated. Do not edit it. + */ +final class FilterDoubleBlock extends AbstractFilterBlock implements DoubleBlock { + + private final DoubleBlock block; + + FilterDoubleBlock(DoubleBlock block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public DoubleVector asVector() { + return null; + } + + @Override + public double getDouble(int valueIndex) { + return block.getDouble(valueIndex); + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public DoubleBlock filter(int... positions) { + return new FilterDoubleBlock(this, positions); + } + + @Override + public DoubleBlock expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positions.length); + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + builder.appendDouble(block.getDouble(i)); + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleBlock that) { + return DoubleBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleBlock.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { + sb.append(getDouble(start)); + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } + sb.append(getDouble(i)); + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleVector.java new file mode 100644 index 0000000000000..6e841ec13b4e5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterDoubleVector.java @@ -0,0 +1,80 @@ +/* + * 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.compute.data; + +/** + * Filter vector for DoubleVectors. + * This class is generated. Do not edit it. + */ +public final class FilterDoubleVector extends AbstractFilterVector implements DoubleVector { + + private final DoubleVector vector; + + FilterDoubleVector(DoubleVector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override + public double getDouble(int position) { + return vector.getDouble(mapPosition(position)); + } + + @Override + public DoubleBlock asBlock() { + return new DoubleVectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.DOUBLE; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public DoubleVector filter(int... positions) { + return new FilterDoubleVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DoubleVector that) { + return DoubleVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DoubleVector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(getDouble(i)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntBlock.java new file mode 100644 index 0000000000000..21c3bb3ebdfbd --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntBlock.java @@ -0,0 +1,114 @@ +/* + * 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.compute.data; + +/** + * Filter block for IntBlocks. + * This class is generated. Do not edit it. + */ +final class FilterIntBlock extends AbstractFilterBlock implements IntBlock { + + private final IntBlock block; + + FilterIntBlock(IntBlock block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public IntVector asVector() { + return null; + } + + @Override + public int getInt(int valueIndex) { + return block.getInt(valueIndex); + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public IntBlock filter(int... positions) { + return new FilterIntBlock(this, positions); + } + + @Override + public IntBlock expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + IntBlock.Builder builder = IntBlock.newBlockBuilder(positions.length); + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + builder.appendInt(block.getInt(i)); + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntBlock that) { + return IntBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntBlock.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { + sb.append(getInt(start)); + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } + sb.append(getInt(i)); + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntVector.java new file mode 100644 index 0000000000000..7caf0ee9ee45b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterIntVector.java @@ -0,0 +1,80 @@ +/* + * 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.compute.data; + +/** + * Filter vector for IntVectors. + * This class is generated. Do not edit it. + */ +public final class FilterIntVector extends AbstractFilterVector implements IntVector { + + private final IntVector vector; + + FilterIntVector(IntVector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override + public int getInt(int position) { + return vector.getInt(mapPosition(position)); + } + + @Override + public IntBlock asBlock() { + return new IntVectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public IntVector filter(int... positions) { + return new FilterIntVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntVector that) { + return IntVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntVector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(getInt(i)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongBlock.java new file mode 100644 index 0000000000000..d67d3e388b6ca --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongBlock.java @@ -0,0 +1,114 @@ +/* + * 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.compute.data; + +/** + * Filter block for LongBlocks. + * This class is generated. Do not edit it. + */ +final class FilterLongBlock extends AbstractFilterBlock implements LongBlock { + + private final LongBlock block; + + FilterLongBlock(LongBlock block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public LongVector asVector() { + return null; + } + + @Override + public long getLong(int valueIndex) { + return block.getLong(valueIndex); + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public LongBlock filter(int... positions) { + return new FilterLongBlock(this, positions); + } + + @Override + public LongBlock expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + LongBlock.Builder builder = LongBlock.newBlockBuilder(positions.length); + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + builder.appendLong(block.getLong(i)); + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongBlock that) { + return LongBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongBlock.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { + sb.append(getLong(start)); + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } + sb.append(getLong(i)); + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongVector.java new file mode 100644 index 0000000000000..96d72f7959474 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/FilterLongVector.java @@ -0,0 +1,80 @@ +/* + * 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.compute.data; + +/** + * Filter vector for LongVectors. + * This class is generated. Do not edit it. + */ +public final class FilterLongVector extends AbstractFilterVector implements LongVector { + + private final LongVector vector; + + FilterLongVector(LongVector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override + public long getLong(int position) { + return vector.getLong(mapPosition(position)); + } + + @Override + public LongBlock asBlock() { + return new LongVectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public LongVector filter(int... positions) { + return new FilterLongVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongVector that) { + return LongVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongVector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(getLong(i)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java new file mode 100644 index 0000000000000..2a52516148ab1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayBlock.java @@ -0,0 +1,84 @@ +/* + * 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.compute.data; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of int. + * This class is generated. Do not edit it. + */ +public final class IntArrayBlock extends AbstractArrayBlock implements IntBlock { + + private final int[] values; + + public IntArrayBlock(int[] values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public IntVector asVector() { + return null; + } + + @Override + public int getInt(int valueIndex) { + return values[valueIndex]; + } + + @Override + public IntBlock filter(int... positions) { + return new FilterIntBlock(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public IntBlock expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new IntArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new IntArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntBlock that) { + return IntBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() + + ", values=" + + Arrays.toString(values) + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayVector.java new file mode 100644 index 0000000000000..c3a55e9e63075 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntArrayVector.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +import java.util.Arrays; + +/** + * Vector implementation that stores an array of int values. + * This class is generated. Do not edit it. + */ +public final class IntArrayVector extends AbstractVector implements IntVector { + + private final int[] values; + + public IntArrayVector(int[] values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public IntBlock asBlock() { + return new IntVectorBlock(this); + } + + @Override + public int getInt(int position) { + return values[position]; + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public IntVector filter(int... positions) { + return new FilterIntVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntVector that) { + return IntVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + Arrays.toString(values) + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayVector.java new file mode 100644 index 0000000000000..a172ea8b9cdc7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBigArrayVector.java @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.IntArray; +import org.elasticsearch.core.Releasable; + +/** + * Vector implementation that defers to an enclosed IntArray. + * This class is generated. Do not edit it. + */ +public final class IntBigArrayVector extends AbstractVector implements IntVector, Releasable { + + private final IntArray values; + + public IntBigArrayVector(IntArray values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public IntBlock asBlock() { + return new IntVectorBlock(this); + } + + @Override + public int getInt(int position) { + return values.get(position); + } + + @Override + public ElementType elementType() { + return ElementType.INT; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public IntVector filter(int... positions) { + return new FilterIntVector(this, positions); + } + + @Override + public void close() { + values.close(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntVector that) { + return IntVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + values + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java new file mode 100644 index 0000000000000..0653824b5b8f4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java @@ -0,0 +1,203 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores int values. + * This class is generated. Do not edit it. + */ +public sealed interface IntBlock extends Block permits FilterIntBlock, IntArrayBlock, IntVectorBlock { + + /** + * Retrieves the int value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index + * @return the data value (as a int) + */ + int getInt(int valueIndex); + + @Override + IntVector asVector(); + + @Override + IntBlock filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "IntBlock", IntBlock::of); + + @Override + default String getWriteableName() { + return "IntBlock"; + } + + static IntBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.appendInt(in.readInt()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + out.writeInt(getInt(getFirstValueIndex(pos) + valueIndex)); + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a IntBlock, and both blocks are {@link #equals(IntBlock, IntBlock) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash(IntBlock)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the IntBlock interface. + */ + static boolean equals(IntBlock block1, IntBlock block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + if (block1.getInt(b1ValueIdx + valueIndex) != block2.getInt(b2ValueIdx + valueIndex)) { + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(IntBlock block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + result = 31 * result + block.getInt(firstValueIdx + valueIndex); + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new IntBlockBuilder(estimatedSize); + } + + static IntBlock newConstantBlockWith(int value, int positions) { + return new ConstantIntVector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits IntBlockBuilder { + + /** + * Appends a int to the current entry. + */ + Builder appendInt(int value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(IntBlock block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(IntBlock block); + + @Override + IntBlock build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java new file mode 100644 index 0000000000000..ba45611a7bdc7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlockBuilder.java @@ -0,0 +1,187 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of IntBlocks. + * This class is generated. Do not edit it. + */ +final class IntBlockBuilder extends AbstractBlockBuilder implements IntBlock.Builder { + + private int[] values; + + IntBlockBuilder(int estimatedSize) { + values = new int[Math.max(estimatedSize, 2)]; + } + + @Override + public IntBlockBuilder appendInt(int value) { + ensureCapacity(); + values[valueCount] = value; + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public IntBlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public IntBlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public IntBlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public IntBlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition((IntBlock) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public IntBlockBuilder appendAllValuesToCurrentPosition(IntBlock block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } + final IntVector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { + appendInt(vector.getInt(p)); + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendInt(block.getInt(i++)); + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public IntBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom((IntBlock) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public IntBlockBuilder copyFrom(IntBlock block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + IntVector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock(IntBlock block, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendInt(block.getInt(i++)); + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector(IntVector vector, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendInt(vector.getInt(p)); + } + } + + @Override + public IntBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public IntBlock build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { + return new ConstantIntVector(values[0], 1).asBlock(); + } else { + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + if (isDense() && singleValued()) { + return new IntArrayVector(values, positionCount).asBlock(); + } else { + return new IntArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVector.java new file mode 100644 index 0000000000000..2b9a1b8b8ccd7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVector.java @@ -0,0 +1,91 @@ +/* + * 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.compute.data; + +/** + * Vector that stores int values. + * This class is generated. Do not edit it. + */ +public sealed interface IntVector extends Vector permits ConstantIntVector, FilterIntVector, IntArrayVector, IntBigArrayVector { + + int getInt(int position); + + @Override + IntBlock asBlock(); + + @Override + IntVector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a IntVector, and both vectors are {@link #equals(IntVector, IntVector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash(IntVector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the IntVector interface. + */ + static boolean equals(IntVector vector1, IntVector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (vector1.getInt(pos) != vector2.getInt(pos)) { + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(IntVector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { + result = 31 * result + vector.getInt(pos); + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new IntVectorBuilder(estimatedSize); + } + + /** Create a vector for a range of ints. */ + static IntVector range(int startInclusive, int endExclusive) { + int[] values = new int[endExclusive - startInclusive]; + for (int i = 0; i < values.length; i++) { + values[i] = startInclusive + i; + } + return new IntArrayVector(values, values.length); + } + + sealed interface Builder extends Vector.Builder permits IntVectorBuilder { + /** + * Appends a int to the current entry. + */ + Builder appendInt(int value); + + @Override + IntVector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBlock.java new file mode 100644 index 0000000000000..4856c81966271 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBlock.java @@ -0,0 +1,111 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a IntVector. + * This class is generated. Do not edit it. + */ +public final class IntVectorBlock extends AbstractVectorBlock implements IntBlock { + + private final IntVector vector; + + IntVectorBlock(IntVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public IntVector asVector() { + return vector; + } + + @Override + public int getInt(int valueIndex) { + return vector.getInt(valueIndex); + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public IntBlock filter(int... positions) { + return new FilterIntVector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "IntVectorBlock", + IntVectorBlock::of + ); + + @Override + public String getWriteableName() { + return "IntVectorBlock"; + } + + static IntVectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new IntVectorBlock(new ConstantIntVector(in.readInt(), positions)); + } else { + var builder = IntVector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.appendInt(in.readInt()); + } + return new IntVectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final IntVector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { + out.writeInt(getInt(0)); + } else { + for (int i = 0; i < positions; i++) { + out.writeInt(getInt(i)); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IntBlock that) { + return IntBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return IntBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBuilder.java new file mode 100644 index 0000000000000..9ae625152ce8e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntVectorBuilder.java @@ -0,0 +1,52 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of IntBlocks. + * This class is generated. Do not edit it. + */ +final class IntVectorBuilder extends AbstractVectorBuilder implements IntVector.Builder { + + private int[] values; + + IntVectorBuilder(int estimatedSize) { + values = new int[Math.max(estimatedSize, 2)]; + } + + @Override + public IntVectorBuilder appendInt(int value) { + ensureCapacity(); + values[valueCount] = value; + valueCount++; + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public IntVector build() { + if (valueCount == 1) { + return new ConstantIntVector(values[0], 1); + } + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + return new IntArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java new file mode 100644 index 0000000000000..ec81eb4d59563 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayBlock.java @@ -0,0 +1,84 @@ +/* + * 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.compute.data; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of long. + * This class is generated. Do not edit it. + */ +public final class LongArrayBlock extends AbstractArrayBlock implements LongBlock { + + private final long[] values; + + public LongArrayBlock(long[] values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public LongVector asVector() { + return null; + } + + @Override + public long getLong(int valueIndex) { + return values[valueIndex]; + } + + @Override + public LongBlock filter(int... positions) { + return new FilterLongBlock(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public LongBlock expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new LongArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new LongArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongBlock that) { + return LongBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() + + ", values=" + + Arrays.toString(values) + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayVector.java new file mode 100644 index 0000000000000..997412473af1b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongArrayVector.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +import java.util.Arrays; + +/** + * Vector implementation that stores an array of long values. + * This class is generated. Do not edit it. + */ +public final class LongArrayVector extends AbstractVector implements LongVector { + + private final long[] values; + + public LongArrayVector(long[] values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public LongBlock asBlock() { + return new LongVectorBlock(this); + } + + @Override + public long getLong(int position) { + return values[position]; + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public LongVector filter(int... positions) { + return new FilterLongVector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongVector that) { + return LongVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + Arrays.toString(values) + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayVector.java new file mode 100644 index 0000000000000..30c69a5792cb7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBigArrayVector.java @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.core.Releasable; + +/** + * Vector implementation that defers to an enclosed LongArray. + * This class is generated. Do not edit it. + */ +public final class LongBigArrayVector extends AbstractVector implements LongVector, Releasable { + + private final LongArray values; + + public LongBigArrayVector(LongArray values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public LongBlock asBlock() { + return new LongVectorBlock(this); + } + + @Override + public long getLong(int position) { + return values.get(position); + } + + @Override + public ElementType elementType() { + return ElementType.LONG; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public LongVector filter(int... positions) { + return new FilterLongVector(this, positions); + } + + @Override + public void close() { + values.close(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongVector that) { + return LongVector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongVector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + values + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java new file mode 100644 index 0000000000000..df80bc8e29aa3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java @@ -0,0 +1,204 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores long values. + * This class is generated. Do not edit it. + */ +public sealed interface LongBlock extends Block permits FilterLongBlock, LongArrayBlock, LongVectorBlock { + + /** + * Retrieves the long value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index + * @return the data value (as a long) + */ + long getLong(int valueIndex); + + @Override + LongVector asVector(); + + @Override + LongBlock filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "LongBlock", LongBlock::of); + + @Override + default String getWriteableName() { + return "LongBlock"; + } + + static LongBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.appendLong(in.readLong()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + out.writeLong(getLong(getFirstValueIndex(pos) + valueIndex)); + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a LongBlock, and both blocks are {@link #equals(LongBlock, LongBlock) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash(LongBlock)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the LongBlock interface. + */ + static boolean equals(LongBlock block1, LongBlock block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + if (block1.getLong(b1ValueIdx + valueIndex) != block2.getLong(b2ValueIdx + valueIndex)) { + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(LongBlock block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + long element = block.getLong(firstValueIdx + valueIndex); + result = 31 * result + (int) (element ^ (element >>> 32)); + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new LongBlockBuilder(estimatedSize); + } + + static LongBlock newConstantBlockWith(long value, int positions) { + return new ConstantLongVector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits LongBlockBuilder { + + /** + * Appends a long to the current entry. + */ + Builder appendLong(long value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(LongBlock block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(LongBlock block); + + @Override + LongBlock build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlockBuilder.java new file mode 100644 index 0000000000000..9834de886d904 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlockBuilder.java @@ -0,0 +1,187 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of LongBlocks. + * This class is generated. Do not edit it. + */ +final class LongBlockBuilder extends AbstractBlockBuilder implements LongBlock.Builder { + + private long[] values; + + LongBlockBuilder(int estimatedSize) { + values = new long[Math.max(estimatedSize, 2)]; + } + + @Override + public LongBlockBuilder appendLong(long value) { + ensureCapacity(); + values[valueCount] = value; + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public LongBlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public LongBlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public LongBlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public LongBlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition((LongBlock) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public LongBlockBuilder appendAllValuesToCurrentPosition(LongBlock block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } + final LongVector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { + appendLong(vector.getLong(p)); + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendLong(block.getLong(i++)); + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public LongBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom((LongBlock) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public LongBlockBuilder copyFrom(LongBlock block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + LongVector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock(LongBlock block, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + appendLong(block.getLong(i++)); + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector(LongVector vector, int beginInclusive, int endExclusive) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendLong(vector.getLong(p)); + } + } + + @Override + public LongBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public LongBlock build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { + return new ConstantLongVector(values[0], 1).asBlock(); + } else { + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + if (isDense() && singleValued()) { + return new LongArrayVector(values, positionCount).asBlock(); + } else { + return new LongArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVector.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVector.java new file mode 100644 index 0000000000000..0e0b02987dd26 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVector.java @@ -0,0 +1,83 @@ +/* + * 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.compute.data; + +/** + * Vector that stores long values. + * This class is generated. Do not edit it. + */ +public sealed interface LongVector extends Vector permits ConstantLongVector, FilterLongVector, LongArrayVector, LongBigArrayVector { + + long getLong(int position); + + @Override + LongBlock asBlock(); + + @Override + LongVector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a LongVector, and both vectors are {@link #equals(LongVector, LongVector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash(LongVector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the LongVector interface. + */ + static boolean equals(LongVector vector1, LongVector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (vector1.getLong(pos) != vector2.getLong(pos)) { + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash(LongVector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { + long element = vector.getLong(pos); + result = 31 * result + (int) (element ^ (element >>> 32)); + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new LongVectorBuilder(estimatedSize); + } + + sealed interface Builder extends Vector.Builder permits LongVectorBuilder { + /** + * Appends a long to the current entry. + */ + Builder appendLong(long value); + + @Override + LongVector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBlock.java new file mode 100644 index 0000000000000..9f7c026e8687c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBlock.java @@ -0,0 +1,111 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a LongVector. + * This class is generated. Do not edit it. + */ +public final class LongVectorBlock extends AbstractVectorBlock implements LongBlock { + + private final LongVector vector; + + LongVectorBlock(LongVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public LongVector asVector() { + return vector; + } + + @Override + public long getLong(int valueIndex) { + return vector.getLong(valueIndex); + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public LongBlock filter(int... positions) { + return new FilterLongVector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "LongVectorBlock", + LongVectorBlock::of + ); + + @Override + public String getWriteableName() { + return "LongVectorBlock"; + } + + static LongVectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new LongVectorBlock(new ConstantLongVector(in.readLong(), positions)); + } else { + var builder = LongVector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.appendLong(in.readLong()); + } + return new LongVectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final LongVector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { + out.writeLong(getLong(0)); + } else { + for (int i = 0; i < positions; i++) { + out.writeLong(getLong(i)); + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LongBlock that) { + return LongBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return LongBlock.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBuilder.java new file mode 100644 index 0000000000000..ba4864bdde812 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongVectorBuilder.java @@ -0,0 +1,52 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +/** + * Block build of LongBlocks. + * This class is generated. Do not edit it. + */ +final class LongVectorBuilder extends AbstractVectorBuilder implements LongVector.Builder { + + private long[] values; + + LongVectorBuilder(int estimatedSize) { + values = new long[Math.max(estimatedSize, 2)]; + } + + @Override + public LongVectorBuilder appendLong(long value) { + ensureCapacity(); + values[valueCount] = value; + valueCount++; + return this; + } + + @Override + protected int valuesLength() { + return values.length; + } + + @Override + protected void growValuesArray(int newSize) { + values = Arrays.copyOf(values, newSize); + } + + @Override + public LongVector build() { + if (valueCount == 1) { + return new ConstantLongVector(values[0], 1); + } + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } + return new LongArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeBytesRef.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeBytesRef.java new file mode 100644 index 0000000000000..13a0849504a4a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeBytesRef.java @@ -0,0 +1,389 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.LongBlock; + +import java.util.Arrays; + +/** + * Removes duplicate values from multivalued positions. + * This class is generated. Edit {@code X-MultivalueDedupe.java.st} instead. + */ +public class MultivalueDedupeBytesRef { + /** + * The number of entries before we switch from and {@code n^2} strategy + * with low overhead to an {@code n*log(n)} strategy with higher overhead. + * The choice of number has been experimentally derived. + */ + private static final int ALWAYS_COPY_MISSING = 20; // TODO BytesRef should try adding to the hash *first* and then comparing. + private final BytesRefBlock block; + private BytesRef[] work = new BytesRef[ArrayUtil.oversize(2, org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; + private int w; + + public MultivalueDedupeBytesRef(BytesRefBlock block) { + this.block = block; + // TODO very large numbers might want a hash based implementation - and for BytesRef that might not be that big + fillWork(0, work.length); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public BytesRefBlock dedupeToBlockAdaptive() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); + default -> { + /* + * It's better to copyMissing when there are few unique values + * and better to copy and sort when there are many unique values. + * The more duplicate values there are the more comparatively worse + * copyAndSort is. But we don't know how many unique values there + * because our job is to find them. So we use the count of values + * as a proxy that is fast to test. It's not always going to be + * optimal but it has the nice property of being quite quick on + * short lists and not n^2 levels of terrible on long ones. + * + * It'd also be possible to make a truly hybrid mechanism that + * switches from copyMissing to copyUnique once it collects enough + * unique values. The trouble is that the switch is expensive and + * makes kind of a "hole" in the performance of that mechanism where + * you may as well have just gone with either of the two other + * strategies. So we just don't try it for now. + */ + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + writeUniquedWork(builder); + } else { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public BytesRefBlock dedupeToBlockUsingCopyAndSort() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); + default -> { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public BytesRefBlock dedupeToBlockUsingCopyMissing() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); + default -> { + copyMissing(first, count); + writeUniquedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + */ + public MultivalueDedupe.HashResult hash(BytesRefHash hash) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + boolean sawNull = false; + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + sawNull = true; + builder.appendLong(0); + } + case 1 -> { + BytesRef v = block.getBytesRef(first, work[0]); + hash(builder, hash, v); + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + hashUniquedWork(hash, builder); + } else { + copyAndSort(first, count); + hashSortedWork(hash, builder); + } + } + } + } + return new MultivalueDedupe.HashResult(builder.build(), sawNull); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.BytesRefs(batchSize) { + @Override + protected void readNextBatch() { + int position = firstPosition(); + if (w > 0) { + // The last block didn't fit so we have to *make* it fit + ensureCapacity(workSize(), w); + startPosition(); + encodeUniquedWork(this); + endPosition(); + position++; + } + for (; position < block.getPositionCount(); position++) { + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { + BytesRef v = block.getBytesRef(first, work[0]); + if (hasCapacity(v.length, 1)) { + startPosition(); + encode(v); + endPosition(); + } else { + work[0] = v; + w = 1; + return; + } + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + } else { + copyAndSort(first, count); + convertSortedWorkToUnique(); + } + if (hasCapacity(workSize(), w)) { + startPosition(); + encodeUniquedWork(this); + endPosition(); + } else { + return; + } + } + } + } + } + + private int workSize() { + int size = 0; + for (int i = 0; i < w; i++) { + size += work[i].length; + } + return size; + } + }; + } + + /** + * Copy all value from the position into {@link #work} and then + * sorts it {@code n * log(n)}. + */ + private void copyAndSort(int first, int count) { + grow(count); + int end = first + count; + + w = 0; + for (int i = first; i < end; i++) { + work[w] = block.getBytesRef(i, work[w]); + w++; + } + + Arrays.sort(work, 0, w); + } + + /** + * Fill {@link #work} with the unique values in the position by scanning + * all fields already copied {@code n^2}. + */ + private void copyMissing(int first, int count) { + grow(count); + int end = first + count; + + work[0] = block.getBytesRef(first, work[0]); + w = 1; + i: for (int i = first + 1; i < end; i++) { + BytesRef v = block.getBytesRef(i, work[w]); + for (int j = 0; j < w; j++) { + if (v.equals(work[j])) { + continue i; + } + } + work[w++] = v; + } + } + + /** + * Writes an already deduplicated {@link #work} to a {@link BytesRefBlock.Builder}. + */ + private void writeUniquedWork(BytesRefBlock.Builder builder) { + if (w == 1) { + builder.appendBytesRef(work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + builder.appendBytesRef(work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a {@link BytesRefBlock.Builder}, skipping duplicates. + */ + private void writeSortedWork(BytesRefBlock.Builder builder) { + if (w == 1) { + builder.appendBytesRef(work[0]); + return; + } + builder.beginPositionEntry(); + BytesRef prev = work[0]; + builder.appendBytesRef(prev); + for (int i = 1; i < w; i++) { + if (false == prev.equals(work[i])) { + prev = work[i]; + builder.appendBytesRef(prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes an already deduplicated {@link #work} to a hash. + */ + private void hashUniquedWork(BytesRefHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + hash(builder, hash, work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a hash, skipping duplicates. + */ + private void hashSortedWork(BytesRefHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + BytesRef prev = work[0]; + hash(builder, hash, prev); + for (int i = 1; i < w; i++) { + if (false == prev.equals(work[i])) { + prev = work[i]; + hash(builder, hash, prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes a deduplicated {@link #work} to a {@link BatchEncoder.BytesRefs}. + */ + private void encodeUniquedWork(BatchEncoder.BytesRefs encoder) { + for (int i = 0; i < w; i++) { + encoder.encode(work[i]); + } + } + + /** + * Converts {@link #work} from sorted array to a deduplicated array. + */ + private void convertSortedWorkToUnique() { + BytesRef prev = work[0]; + int end = w; + w = 1; + for (int i = 1; i < end; i++) { + if (false == prev.equals(work[i])) { + prev = work[i]; + work[w].bytes = prev.bytes; + work[w].offset = prev.offset; + work[w].length = prev.length; + w++; + } + } + } + + private void grow(int size) { + int prev = work.length; + work = ArrayUtil.grow(work, size); + fillWork(prev, work.length); + } + + private void fillWork(int from, int to) { + for (int i = from; i < to; i++) { + work[i] = new BytesRef(); + } + } + + private void hash(LongBlock.Builder builder, BytesRefHash hash, BytesRef v) { + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(v))); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeDouble.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeDouble.java new file mode 100644 index 0000000000000..1f451c2cdac11 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeDouble.java @@ -0,0 +1,367 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; + +import java.util.Arrays; + +/** + * Removes duplicate values from multivalued positions. + * This class is generated. Edit {@code X-MultivalueDedupe.java.st} instead. + */ +public class MultivalueDedupeDouble { + /** + * The number of entries before we switch from and {@code n^2} strategy + * with low overhead to an {@code n*log(n)} strategy with higher overhead. + * The choice of number has been experimentally derived. + */ + private static final int ALWAYS_COPY_MISSING = 110; + private final DoubleBlock block; + private double[] work = new double[ArrayUtil.oversize(2, Double.BYTES)]; + private int w; + + public MultivalueDedupeDouble(DoubleBlock block) { + this.block = block; + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public DoubleBlock dedupeToBlockAdaptive() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendDouble(block.getDouble(first)); + default -> { + /* + * It's better to copyMissing when there are few unique values + * and better to copy and sort when there are many unique values. + * The more duplicate values there are the more comparatively worse + * copyAndSort is. But we don't know how many unique values there + * because our job is to find them. So we use the count of values + * as a proxy that is fast to test. It's not always going to be + * optimal but it has the nice property of being quite quick on + * short lists and not n^2 levels of terrible on long ones. + * + * It'd also be possible to make a truly hybrid mechanism that + * switches from copyMissing to copyUnique once it collects enough + * unique values. The trouble is that the switch is expensive and + * makes kind of a "hole" in the performance of that mechanism where + * you may as well have just gone with either of the two other + * strategies. So we just don't try it for now. + */ + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + writeUniquedWork(builder); + } else { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public DoubleBlock dedupeToBlockUsingCopyAndSort() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendDouble(block.getDouble(first)); + default -> { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public DoubleBlock dedupeToBlockUsingCopyMissing() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendDouble(block.getDouble(first)); + default -> { + copyMissing(first, count); + writeUniquedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + */ + public MultivalueDedupe.HashResult hash(LongHash hash) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + boolean sawNull = false; + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + sawNull = true; + builder.appendLong(0); + } + case 1 -> { + double v = block.getDouble(first); + hash(builder, hash, v); + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + hashUniquedWork(hash, builder); + } else { + copyAndSort(first, count); + hashSortedWork(hash, builder); + } + } + } + } + return new MultivalueDedupe.HashResult(builder.build(), sawNull); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.Doubles(batchSize) { + @Override + protected void readNextBatch() { + int position = firstPosition(); + if (w > 0) { + // The last block didn't fit so we have to *make* it fit + ensureCapacity(w); + startPosition(); + encodeUniquedWork(this); + endPosition(); + position++; + } + for (; position < block.getPositionCount(); position++) { + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { + double v = block.getDouble(first); + if (hasCapacity(1)) { + startPosition(); + encode(v); + endPosition(); + } else { + work[0] = v; + w = 1; + return; + } + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + } else { + copyAndSort(first, count); + convertSortedWorkToUnique(); + } + if (hasCapacity(w)) { + startPosition(); + encodeUniquedWork(this); + endPosition(); + } else { + return; + } + } + } + } + } + + }; + } + + /** + * Copy all value from the position into {@link #work} and then + * sorts it {@code n * log(n)}. + */ + private void copyAndSort(int first, int count) { + grow(count); + int end = first + count; + + w = 0; + for (int i = first; i < end; i++) { + work[w++] = block.getDouble(i); + } + + Arrays.sort(work, 0, w); + } + + /** + * Fill {@link #work} with the unique values in the position by scanning + * all fields already copied {@code n^2}. + */ + private void copyMissing(int first, int count) { + grow(count); + int end = first + count; + + work[0] = block.getDouble(first); + w = 1; + i: for (int i = first + 1; i < end; i++) { + double v = block.getDouble(i); + for (int j = 0; j < w; j++) { + if (v == work[j]) { + continue i; + } + } + work[w++] = v; + } + } + + /** + * Writes an already deduplicated {@link #work} to a {@link DoubleBlock.Builder}. + */ + private void writeUniquedWork(DoubleBlock.Builder builder) { + if (w == 1) { + builder.appendDouble(work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + builder.appendDouble(work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a {@link DoubleBlock.Builder}, skipping duplicates. + */ + private void writeSortedWork(DoubleBlock.Builder builder) { + if (w == 1) { + builder.appendDouble(work[0]); + return; + } + builder.beginPositionEntry(); + double prev = work[0]; + builder.appendDouble(prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + builder.appendDouble(prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes an already deduplicated {@link #work} to a hash. + */ + private void hashUniquedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + hash(builder, hash, work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a hash, skipping duplicates. + */ + private void hashSortedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + double prev = work[0]; + hash(builder, hash, prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + hash(builder, hash, prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes a deduplicated {@link #work} to a {@link BatchEncoder.Doubles}. + */ + private void encodeUniquedWork(BatchEncoder.Doubles encoder) { + for (int i = 0; i < w; i++) { + encoder.encode(work[i]); + } + } + + /** + * Converts {@link #work} from sorted array to a deduplicated array. + */ + private void convertSortedWorkToUnique() { + double prev = work[0]; + int end = w; + w = 1; + for (int i = 1; i < end; i++) { + if (prev != work[i]) { + prev = work[i]; + work[w++] = prev; + } + } + } + + private void grow(int size) { + work = ArrayUtil.grow(work, size); + } + + private void hash(LongBlock.Builder builder, LongHash hash, double v) { + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(Double.doubleToLongBits(v)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeInt.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeInt.java new file mode 100644 index 0000000000000..e8e9f60189f15 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeInt.java @@ -0,0 +1,367 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; + +import java.util.Arrays; + +/** + * Removes duplicate values from multivalued positions. + * This class is generated. Edit {@code X-MultivalueDedupe.java.st} instead. + */ +public class MultivalueDedupeInt { + /** + * The number of entries before we switch from and {@code n^2} strategy + * with low overhead to an {@code n*log(n)} strategy with higher overhead. + * The choice of number has been experimentally derived. + */ + private static final int ALWAYS_COPY_MISSING = 300; + private final IntBlock block; + private int[] work = new int[ArrayUtil.oversize(2, Integer.BYTES)]; + private int w; + + public MultivalueDedupeInt(IntBlock block) { + this.block = block; + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public IntBlock dedupeToBlockAdaptive() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + IntBlock.Builder builder = IntBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendInt(block.getInt(first)); + default -> { + /* + * It's better to copyMissing when there are few unique values + * and better to copy and sort when there are many unique values. + * The more duplicate values there are the more comparatively worse + * copyAndSort is. But we don't know how many unique values there + * because our job is to find them. So we use the count of values + * as a proxy that is fast to test. It's not always going to be + * optimal but it has the nice property of being quite quick on + * short lists and not n^2 levels of terrible on long ones. + * + * It'd also be possible to make a truly hybrid mechanism that + * switches from copyMissing to copyUnique once it collects enough + * unique values. The trouble is that the switch is expensive and + * makes kind of a "hole" in the performance of that mechanism where + * you may as well have just gone with either of the two other + * strategies. So we just don't try it for now. + */ + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + writeUniquedWork(builder); + } else { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public IntBlock dedupeToBlockUsingCopyAndSort() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + IntBlock.Builder builder = IntBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendInt(block.getInt(first)); + default -> { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public IntBlock dedupeToBlockUsingCopyMissing() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + IntBlock.Builder builder = IntBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendInt(block.getInt(first)); + default -> { + copyMissing(first, count); + writeUniquedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + */ + public MultivalueDedupe.HashResult hash(LongHash hash) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + boolean sawNull = false; + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + sawNull = true; + builder.appendLong(0); + } + case 1 -> { + int v = block.getInt(first); + hash(builder, hash, v); + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + hashUniquedWork(hash, builder); + } else { + copyAndSort(first, count); + hashSortedWork(hash, builder); + } + } + } + } + return new MultivalueDedupe.HashResult(builder.build(), sawNull); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.Ints(batchSize) { + @Override + protected void readNextBatch() { + int position = firstPosition(); + if (w > 0) { + // The last block didn't fit so we have to *make* it fit + ensureCapacity(w); + startPosition(); + encodeUniquedWork(this); + endPosition(); + position++; + } + for (; position < block.getPositionCount(); position++) { + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { + int v = block.getInt(first); + if (hasCapacity(1)) { + startPosition(); + encode(v); + endPosition(); + } else { + work[0] = v; + w = 1; + return; + } + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + } else { + copyAndSort(first, count); + convertSortedWorkToUnique(); + } + if (hasCapacity(w)) { + startPosition(); + encodeUniquedWork(this); + endPosition(); + } else { + return; + } + } + } + } + } + + }; + } + + /** + * Copy all value from the position into {@link #work} and then + * sorts it {@code n * log(n)}. + */ + private void copyAndSort(int first, int count) { + grow(count); + int end = first + count; + + w = 0; + for (int i = first; i < end; i++) { + work[w++] = block.getInt(i); + } + + Arrays.sort(work, 0, w); + } + + /** + * Fill {@link #work} with the unique values in the position by scanning + * all fields already copied {@code n^2}. + */ + private void copyMissing(int first, int count) { + grow(count); + int end = first + count; + + work[0] = block.getInt(first); + w = 1; + i: for (int i = first + 1; i < end; i++) { + int v = block.getInt(i); + for (int j = 0; j < w; j++) { + if (v == work[j]) { + continue i; + } + } + work[w++] = v; + } + } + + /** + * Writes an already deduplicated {@link #work} to a {@link IntBlock.Builder}. + */ + private void writeUniquedWork(IntBlock.Builder builder) { + if (w == 1) { + builder.appendInt(work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + builder.appendInt(work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a {@link IntBlock.Builder}, skipping duplicates. + */ + private void writeSortedWork(IntBlock.Builder builder) { + if (w == 1) { + builder.appendInt(work[0]); + return; + } + builder.beginPositionEntry(); + int prev = work[0]; + builder.appendInt(prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + builder.appendInt(prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes an already deduplicated {@link #work} to a hash. + */ + private void hashUniquedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + hash(builder, hash, work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a hash, skipping duplicates. + */ + private void hashSortedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + int prev = work[0]; + hash(builder, hash, prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + hash(builder, hash, prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes a deduplicated {@link #work} to a {@link BatchEncoder.Ints}. + */ + private void encodeUniquedWork(BatchEncoder.Ints encoder) { + for (int i = 0; i < w; i++) { + encoder.encode(work[i]); + } + } + + /** + * Converts {@link #work} from sorted array to a deduplicated array. + */ + private void convertSortedWorkToUnique() { + int prev = work[0]; + int end = w; + w = 1; + for (int i = 1; i < end; i++) { + if (prev != work[i]) { + prev = work[i]; + work[w++] = prev; + } + } + } + + private void grow(int size) { + work = ArrayUtil.grow(work, size); + } + + private void hash(LongBlock.Builder builder, LongHash hash, int v) { + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(v))); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeLong.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeLong.java new file mode 100644 index 0000000000000..f334e1bd3f61f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/operator/MultivalueDedupeLong.java @@ -0,0 +1,367 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; + +import java.util.Arrays; + +/** + * Removes duplicate values from multivalued positions. + * This class is generated. Edit {@code X-MultivalueDedupe.java.st} instead. + */ +public class MultivalueDedupeLong { + /** + * The number of entries before we switch from and {@code n^2} strategy + * with low overhead to an {@code n*log(n)} strategy with higher overhead. + * The choice of number has been experimentally derived. + */ + private static final int ALWAYS_COPY_MISSING = 300; + + private final LongBlock block; + private long[] work = new long[ArrayUtil.oversize(2, Long.BYTES)]; + private int w; + + public MultivalueDedupeLong(LongBlock block) { + this.block = block; + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public LongBlock dedupeToBlockAdaptive() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendLong(block.getLong(first)); + default -> { + /* + * It's better to copyMissing when there are few unique values + * and better to copy and sort when there are many unique values. + * The more duplicate values there are the more comparatively worse + * copyAndSort is. But we don't know how many unique values there + * because our job is to find them. So we use the count of values + * as a proxy that is fast to test. It's not always going to be + * optimal but it has the nice property of being quite quick on + * short lists and not n^2 levels of terrible on long ones. + * + * It'd also be possible to make a truly hybrid mechanism that + * switches from copyMissing to copyUnique once it collects enough + * unique values. The trouble is that the switch is expensive and + * makes kind of a "hole" in the performance of that mechanism where + * you may as well have just gone with either of the two other + * strategies. So we just don't try it for now. + */ + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + writeUniquedWork(builder); + } else { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public LongBlock dedupeToBlockUsingCopyAndSort() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendLong(block.getLong(first)); + default -> { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public LongBlock dedupeToBlockUsingCopyMissing() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendLong(block.getLong(first)); + default -> { + copyMissing(first, count); + writeUniquedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + */ + public MultivalueDedupe.HashResult hash(LongHash hash) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + boolean sawNull = false; + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + sawNull = true; + builder.appendLong(0); + } + case 1 -> { + long v = block.getLong(first); + hash(builder, hash, v); + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + hashUniquedWork(hash, builder); + } else { + copyAndSort(first, count); + hashSortedWork(hash, builder); + } + } + } + } + return new MultivalueDedupe.HashResult(builder.build(), sawNull); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.Longs(batchSize) { + @Override + protected void readNextBatch() { + int position = firstPosition(); + if (w > 0) { + // The last block didn't fit so we have to *make* it fit + ensureCapacity(w); + startPosition(); + encodeUniquedWork(this); + endPosition(); + position++; + } + for (; position < block.getPositionCount(); position++) { + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { + long v = block.getLong(first); + if (hasCapacity(1)) { + startPosition(); + encode(v); + endPosition(); + } else { + work[0] = v; + w = 1; + return; + } + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + } else { + copyAndSort(first, count); + convertSortedWorkToUnique(); + } + if (hasCapacity(w)) { + startPosition(); + encodeUniquedWork(this); + endPosition(); + } else { + return; + } + } + } + } + } + + }; + } + + /** + * Copy all value from the position into {@link #work} and then + * sorts it {@code n * log(n)}. + */ + private void copyAndSort(int first, int count) { + grow(count); + int end = first + count; + + w = 0; + for (int i = first; i < end; i++) { + work[w++] = block.getLong(i); + } + + Arrays.sort(work, 0, w); + } + + /** + * Fill {@link #work} with the unique values in the position by scanning + * all fields already copied {@code n^2}. + */ + private void copyMissing(int first, int count) { + grow(count); + int end = first + count; + + work[0] = block.getLong(first); + w = 1; + i: for (int i = first + 1; i < end; i++) { + long v = block.getLong(i); + for (int j = 0; j < w; j++) { + if (v == work[j]) { + continue i; + } + } + work[w++] = v; + } + } + + /** + * Writes an already deduplicated {@link #work} to a {@link LongBlock.Builder}. + */ + private void writeUniquedWork(LongBlock.Builder builder) { + if (w == 1) { + builder.appendLong(work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + builder.appendLong(work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a {@link LongBlock.Builder}, skipping duplicates. + */ + private void writeSortedWork(LongBlock.Builder builder) { + if (w == 1) { + builder.appendLong(work[0]); + return; + } + builder.beginPositionEntry(); + long prev = work[0]; + builder.appendLong(prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + builder.appendLong(prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes an already deduplicated {@link #work} to a hash. + */ + private void hashUniquedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + hash(builder, hash, work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a hash, skipping duplicates. + */ + private void hashSortedWork(LongHash hash, LongBlock.Builder builder) { + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + long prev = work[0]; + hash(builder, hash, prev); + for (int i = 1; i < w; i++) { + if (prev != work[i]) { + prev = work[i]; + hash(builder, hash, prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes a deduplicated {@link #work} to a {@link BatchEncoder.Longs}. + */ + private void encodeUniquedWork(BatchEncoder.Longs encoder) { + for (int i = 0; i < w; i++) { + encoder.encode(work[i]); + } + } + + /** + * Converts {@link #work} from sorted array to a deduplicated array. + */ + private void convertSortedWorkToUnique() { + long prev = work[0]; + int end = w; + w = 1; + for (int i = 1; i < end; i++) { + if (prev != work[i]) { + prev = work[i]; + work[w++] = prev; + } + } + } + + private void grow(int size) { + work = ArrayUtil.grow(work, size); + } + + private void hash(LongBlock.Builder builder, LongHash hash, long v) { + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(v))); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunction.java new file mode 100644 index 0000000000000..b050c883883c3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunction.java @@ -0,0 +1,118 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link CountDistinctBooleanAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBooleanAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("fbit", ElementType.BOOLEAN), + new IntermediateStateDesc("tbit", ElementType.BOOLEAN) ); + + private final CountDistinctBooleanAggregator.SingleState state; + + private final List channels; + + public CountDistinctBooleanAggregatorFunction(List channels, + CountDistinctBooleanAggregator.SingleState state) { + this.channels = channels; + this.state = state; + } + + public static CountDistinctBooleanAggregatorFunction create(List channels) { + return new CountDistinctBooleanAggregatorFunction(channels, CountDistinctBooleanAggregator.initSingle()); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + BooleanBlock block = (BooleanBlock) uncastBlock; + BooleanVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(BooleanVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + CountDistinctBooleanAggregator.combine(state, vector.getBoolean(i)); + } + } + + private void addRawBlock(BooleanBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + CountDistinctBooleanAggregator.combine(state, block.getBoolean(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BooleanVector fbit = page.getBlock(channels.get(0)).asVector(); + BooleanVector tbit = page.getBlock(channels.get(1)).asVector(); + assert fbit.getPositionCount() == 1; + assert fbit.getPositionCount() == tbit.getPositionCount(); + CountDistinctBooleanAggregator.combineIntermediate(state, fbit.getBoolean(0), tbit.getBoolean(0)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = CountDistinctBooleanAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..40dcea66965da --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionSupplier.java @@ -0,0 +1,42 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link CountDistinctBooleanAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBooleanAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public CountDistinctBooleanAggregatorFunctionSupplier(BigArrays bigArrays, + List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public CountDistinctBooleanAggregatorFunction aggregator() { + return CountDistinctBooleanAggregatorFunction.create(channels); + } + + @Override + public CountDistinctBooleanGroupingAggregatorFunction groupingAggregator() { + return CountDistinctBooleanGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "count_distinct of booleans"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..ea850cac245c7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunction.java @@ -0,0 +1,207 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link CountDistinctBooleanAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBooleanGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("fbit", ElementType.BOOLEAN), + new IntermediateStateDesc("tbit", ElementType.BOOLEAN) ); + + private final CountDistinctBooleanAggregator.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + public CountDistinctBooleanGroupingAggregatorFunction(List channels, + CountDistinctBooleanAggregator.GroupingState state, BigArrays bigArrays) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + } + + public static CountDistinctBooleanGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new CountDistinctBooleanGroupingAggregatorFunction(channels, CountDistinctBooleanAggregator.initGrouping(bigArrays), bigArrays); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + BooleanBlock valuesBlock = (BooleanBlock) uncastValuesBlock; + BooleanVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, BooleanBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctBooleanAggregator.combine(state, groupId, values.getBoolean(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, BooleanVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctBooleanAggregator.combine(state, groupId, values.getBoolean(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, BooleanBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctBooleanAggregator.combine(state, groupId, values.getBoolean(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, BooleanVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + CountDistinctBooleanAggregator.combine(state, groupId, values.getBoolean(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BooleanVector fbit = page.getBlock(channels.get(0)).asVector(); + BooleanVector tbit = page.getBlock(channels.get(1)).asVector(); + assert fbit.getPositionCount() == tbit.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctBooleanAggregator.combineIntermediate(state, groupId, fbit.getBoolean(groupPosition + positionOffset), tbit.getBoolean(groupPosition + positionOffset)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + CountDistinctBooleanAggregator.GroupingState inState = ((CountDistinctBooleanGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + CountDistinctBooleanAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = CountDistinctBooleanAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunction.java new file mode 100644 index 0000000000000..c2eeb421f2cd2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunction.java @@ -0,0 +1,127 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link CountDistinctBytesRefAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBytesRefAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.SingleState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctBytesRefAggregatorFunction(List channels, + HllStates.SingleState state, BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctBytesRefAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctBytesRefAggregatorFunction(channels, CountDistinctBytesRefAggregator.initSingle(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + BytesRefBlock block = (BytesRefBlock) uncastBlock; + BytesRefVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(BytesRefVector vector) { + BytesRef scratch = new BytesRef(); + for (int i = 0; i < vector.getPositionCount(); i++) { + CountDistinctBytesRefAggregator.combine(state, vector.getBytesRef(i, scratch)); + } + } + + private void addRawBlock(BytesRefBlock block) { + BytesRef scratch = new BytesRef(); + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + CountDistinctBytesRefAggregator.combine(state, block.getBytesRef(i, scratch)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + assert hll.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + CountDistinctBytesRefAggregator.combineIntermediate(state, hll.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = CountDistinctBytesRefAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..1597abb20d9a6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link CountDistinctBytesRefAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBytesRefAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final int precision; + + public CountDistinctBytesRefAggregatorFunctionSupplier(BigArrays bigArrays, + List channels, int precision) { + this.bigArrays = bigArrays; + this.channels = channels; + this.precision = precision; + } + + @Override + public CountDistinctBytesRefAggregatorFunction aggregator() { + return CountDistinctBytesRefAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public CountDistinctBytesRefGroupingAggregatorFunction groupingAggregator() { + return CountDistinctBytesRefGroupingAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public String describe() { + return "count_distinct of bytes"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..f1de6efc814b9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunction.java @@ -0,0 +1,213 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link CountDistinctBytesRefAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctBytesRefGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctBytesRefGroupingAggregatorFunction(List channels, + HllStates.GroupingState state, BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctBytesRefGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctBytesRefGroupingAggregatorFunction(channels, CountDistinctBytesRefAggregator.initGrouping(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + BytesRefBlock valuesBlock = (BytesRefBlock) uncastValuesBlock; + BytesRefVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, BytesRefBlock values) { + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctBytesRefAggregator.combine(state, groupId, values.getBytesRef(v, scratch)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, BytesRefVector values) { + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctBytesRefAggregator.combine(state, groupId, values.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, BytesRefBlock values) { + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctBytesRefAggregator.combine(state, groupId, values.getBytesRef(v, scratch)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, BytesRefVector values) { + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + CountDistinctBytesRefAggregator.combine(state, groupId, values.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctBytesRefAggregator.combineIntermediate(state, groupId, hll.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + HllStates.GroupingState inState = ((CountDistinctBytesRefGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + CountDistinctBytesRefAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = CountDistinctBytesRefAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..cab6436ab97be --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunction.java @@ -0,0 +1,127 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link CountDistinctDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.SingleState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctDoubleAggregatorFunction(List channels, HllStates.SingleState state, + BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctDoubleAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctDoubleAggregatorFunction(channels, CountDistinctDoubleAggregator.initSingle(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + CountDistinctDoubleAggregator.combine(state, vector.getDouble(i)); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + CountDistinctDoubleAggregator.combine(state, block.getDouble(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + assert hll.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + CountDistinctDoubleAggregator.combineIntermediate(state, hll.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = CountDistinctDoubleAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..acfc6735e486f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link CountDistinctDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final int precision; + + public CountDistinctDoubleAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + int precision) { + this.bigArrays = bigArrays; + this.channels = channels; + this.precision = precision; + } + + @Override + public CountDistinctDoubleAggregatorFunction aggregator() { + return CountDistinctDoubleAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public CountDistinctDoubleGroupingAggregatorFunction groupingAggregator() { + return CountDistinctDoubleGroupingAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public String describe() { + return "count_distinct of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..6691d29ae712b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunction.java @@ -0,0 +1,211 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link CountDistinctDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctDoubleGroupingAggregatorFunction(List channels, + HllStates.GroupingState state, BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctDoubleGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctDoubleGroupingAggregatorFunction(channels, CountDistinctDoubleAggregator.initGrouping(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + CountDistinctDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctDoubleAggregator.combineIntermediate(state, groupId, hll.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + HllStates.GroupingState inState = ((CountDistinctDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + CountDistinctDoubleAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = CountDistinctDoubleAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunction.java new file mode 100644 index 0000000000000..6fd9df1d31528 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunction.java @@ -0,0 +1,127 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link CountDistinctIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.SingleState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctIntAggregatorFunction(List channels, HllStates.SingleState state, + BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctIntAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctIntAggregatorFunction(channels, CountDistinctIntAggregator.initSingle(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + CountDistinctIntAggregator.combine(state, vector.getInt(i)); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + CountDistinctIntAggregator.combine(state, block.getInt(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + assert hll.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + CountDistinctIntAggregator.combineIntermediate(state, hll.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = CountDistinctIntAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..d13f79e164f0b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link CountDistinctIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final int precision; + + public CountDistinctIntAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + int precision) { + this.bigArrays = bigArrays; + this.channels = channels; + this.precision = precision; + } + + @Override + public CountDistinctIntAggregatorFunction aggregator() { + return CountDistinctIntAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public CountDistinctIntGroupingAggregatorFunction groupingAggregator() { + return CountDistinctIntGroupingAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public String describe() { + return "count_distinct of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..b0935f78d95a2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunction.java @@ -0,0 +1,210 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link CountDistinctIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctIntGroupingAggregatorFunction(List channels, + HllStates.GroupingState state, BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctIntGroupingAggregatorFunction(channels, CountDistinctIntAggregator.initGrouping(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + CountDistinctIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctIntAggregator.combineIntermediate(state, groupId, hll.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + HllStates.GroupingState inState = ((CountDistinctIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + CountDistinctIntAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = CountDistinctIntAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunction.java new file mode 100644 index 0000000000000..747fc8a91eb4a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunction.java @@ -0,0 +1,127 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link CountDistinctLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.SingleState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctLongAggregatorFunction(List channels, HllStates.SingleState state, + BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctLongAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctLongAggregatorFunction(channels, CountDistinctLongAggregator.initSingle(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + CountDistinctLongAggregator.combine(state, vector.getLong(i)); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + CountDistinctLongAggregator.combine(state, block.getLong(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + assert hll.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + CountDistinctLongAggregator.combineIntermediate(state, hll.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = CountDistinctLongAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..44cf633ba7668 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link CountDistinctLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final int precision; + + public CountDistinctLongAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + int precision) { + this.bigArrays = bigArrays; + this.channels = channels; + this.precision = precision; + } + + @Override + public CountDistinctLongAggregatorFunction aggregator() { + return CountDistinctLongAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public CountDistinctLongGroupingAggregatorFunction groupingAggregator() { + return CountDistinctLongGroupingAggregatorFunction.create(channels, bigArrays, precision); + } + + @Override + public String describe() { + return "count_distinct of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..a9795b0388439 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunction.java @@ -0,0 +1,209 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link CountDistinctLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class CountDistinctLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("hll", ElementType.BYTES_REF) ); + + private final HllStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final int precision; + + public CountDistinctLongGroupingAggregatorFunction(List channels, + HllStates.GroupingState state, BigArrays bigArrays, int precision) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.precision = precision; + } + + public static CountDistinctLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, int precision) { + return new CountDistinctLongGroupingAggregatorFunction(channels, CountDistinctLongAggregator.initGrouping(bigArrays, precision), bigArrays, precision); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + CountDistinctLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + CountDistinctLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector hll = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + CountDistinctLongAggregator.combineIntermediate(state, groupId, hll.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + HllStates.GroupingState inState = ((CountDistinctLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + CountDistinctLongAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = CountDistinctLongAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..636d89b16da8d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MaxDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final DoubleState state; + + private final List channels; + + public MaxDoubleAggregatorFunction(List channels, DoubleState state) { + this.channels = channels; + this.state = state; + } + + public static MaxDoubleAggregatorFunction create(List channels) { + return new MaxDoubleAggregatorFunction(channels, new DoubleState(MaxDoubleAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.doubleValue(MaxDoubleAggregator.combine(state.doubleValue(), vector.getDouble(i))); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.doubleValue(MaxDoubleAggregator.combine(state.doubleValue(), block.getDouble(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + DoubleVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == 1; + assert max.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.doubleValue(MaxDoubleAggregator.combine(state.doubleValue(), max.getDouble(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = DoubleBlock.newConstantBlockWith(state.doubleValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..d419e4e1ce1c9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MaxDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MaxDoubleAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MaxDoubleAggregatorFunction aggregator() { + return MaxDoubleAggregatorFunction.create(channels); + } + + @Override + public MaxDoubleGroupingAggregatorFunction groupingAggregator() { + return MaxDoubleGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "max of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..ab22615e03b76 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunction.java @@ -0,0 +1,209 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MaxDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final DoubleArrayState state; + + private final List channels; + + public MaxDoubleGroupingAggregatorFunction(List channels, DoubleArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MaxDoubleGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MaxDoubleGroupingAggregatorFunction(channels, new DoubleArrayState(bigArrays, MaxDoubleAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + DoubleVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), max.getDouble(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + DoubleArrayState inState = ((MaxDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MaxDoubleAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunction.java new file mode 100644 index 0000000000000..aecef5cff2f77 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MaxIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.INT), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final IntState state; + + private final List channels; + + public MaxIntAggregatorFunction(List channels, IntState state) { + this.channels = channels; + this.state = state; + } + + public static MaxIntAggregatorFunction create(List channels) { + return new MaxIntAggregatorFunction(channels, new IntState(MaxIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.intValue(MaxIntAggregator.combine(state.intValue(), vector.getInt(i))); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.intValue(MaxIntAggregator.combine(state.intValue(), block.getInt(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + IntVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == 1; + assert max.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.intValue(MaxIntAggregator.combine(state.intValue(), max.getInt(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = IntBlock.newConstantBlockWith(state.intValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..169afd2c6783a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MaxIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MaxIntAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MaxIntAggregatorFunction aggregator() { + return MaxIntAggregatorFunction.create(channels); + } + + @Override + public MaxIntGroupingAggregatorFunction groupingAggregator() { + return MaxIntGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "max of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..b825912add9e0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunction.java @@ -0,0 +1,208 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MaxIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.INT), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final IntArrayState state; + + private final List channels; + + public MaxIntGroupingAggregatorFunction(List channels, IntArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MaxIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MaxIntGroupingAggregatorFunction(channels, new IntArrayState(bigArrays, MaxIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + IntVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), max.getInt(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + IntArrayState inState = ((MaxIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MaxIntAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunction.java new file mode 100644 index 0000000000000..8b951487fbffe --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MaxLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongState state; + + private final List channels; + + public MaxLongAggregatorFunction(List channels, LongState state) { + this.channels = channels; + this.state = state; + } + + public static MaxLongAggregatorFunction create(List channels) { + return new MaxLongAggregatorFunction(channels, new LongState(MaxLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.longValue(MaxLongAggregator.combine(state.longValue(), vector.getLong(i))); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.longValue(MaxLongAggregator.combine(state.longValue(), block.getLong(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + LongVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == 1; + assert max.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.longValue(MaxLongAggregator.combine(state.longValue(), max.getLong(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = LongBlock.newConstantBlockWith(state.longValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..287dd23ddf55b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MaxLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MaxLongAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MaxLongAggregatorFunction aggregator() { + return MaxLongAggregatorFunction.create(channels); + } + + @Override + public MaxLongGroupingAggregatorFunction groupingAggregator() { + return MaxLongGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "max of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..02f2352e361eb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunction.java @@ -0,0 +1,207 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MaxLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MaxLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("max", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongArrayState state; + + private final List channels; + + public MaxLongGroupingAggregatorFunction(List channels, LongArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MaxLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MaxLongGroupingAggregatorFunction(channels, new LongArrayState(bigArrays, MaxLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + LongVector max = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert max.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), max.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + LongArrayState inState = ((MaxLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MaxLongAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..d9920146d371b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunction.java @@ -0,0 +1,119 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MedianAbsoluteDeviationDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + public MedianAbsoluteDeviationDoubleAggregatorFunction(List channels, + QuantileStates.SingleState state) { + this.channels = channels; + this.state = state; + } + + public static MedianAbsoluteDeviationDoubleAggregatorFunction create(List channels) { + return new MedianAbsoluteDeviationDoubleAggregatorFunction(channels, MedianAbsoluteDeviationDoubleAggregator.initSingle()); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + MedianAbsoluteDeviationDoubleAggregator.combine(state, vector.getDouble(i)); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + MedianAbsoluteDeviationDoubleAggregator.combine(state, block.getDouble(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + MedianAbsoluteDeviationDoubleAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = MedianAbsoluteDeviationDoubleAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..141f23377a18a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,42 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MedianAbsoluteDeviationDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier(BigArrays bigArrays, + List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MedianAbsoluteDeviationDoubleAggregatorFunction aggregator() { + return MedianAbsoluteDeviationDoubleAggregatorFunction.create(channels); + } + + @Override + public MedianAbsoluteDeviationDoubleGroupingAggregatorFunction groupingAggregator() { + return MedianAbsoluteDeviationDoubleGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "median_absolute_deviation of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..96a8ccf0c86f2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunction.java @@ -0,0 +1,208 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MedianAbsoluteDeviationDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + public MedianAbsoluteDeviationDoubleGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + } + + public static MedianAbsoluteDeviationDoubleGroupingAggregatorFunction create( + List channels, BigArrays bigArrays) { + return new MedianAbsoluteDeviationDoubleGroupingAggregatorFunction(channels, MedianAbsoluteDeviationDoubleAggregator.initGrouping(bigArrays), bigArrays); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + MedianAbsoluteDeviationDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationDoubleAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((MedianAbsoluteDeviationDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + MedianAbsoluteDeviationDoubleAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = MedianAbsoluteDeviationDoubleAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunction.java new file mode 100644 index 0000000000000..e62cbd1c19c8d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunction.java @@ -0,0 +1,119 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MedianAbsoluteDeviationIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + public MedianAbsoluteDeviationIntAggregatorFunction(List channels, + QuantileStates.SingleState state) { + this.channels = channels; + this.state = state; + } + + public static MedianAbsoluteDeviationIntAggregatorFunction create(List channels) { + return new MedianAbsoluteDeviationIntAggregatorFunction(channels, MedianAbsoluteDeviationIntAggregator.initSingle()); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + MedianAbsoluteDeviationIntAggregator.combine(state, vector.getInt(i)); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + MedianAbsoluteDeviationIntAggregator.combine(state, block.getInt(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + MedianAbsoluteDeviationIntAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = MedianAbsoluteDeviationIntAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..e1cb7f645957d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionSupplier.java @@ -0,0 +1,42 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MedianAbsoluteDeviationIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MedianAbsoluteDeviationIntAggregatorFunctionSupplier(BigArrays bigArrays, + List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MedianAbsoluteDeviationIntAggregatorFunction aggregator() { + return MedianAbsoluteDeviationIntAggregatorFunction.create(channels); + } + + @Override + public MedianAbsoluteDeviationIntGroupingAggregatorFunction groupingAggregator() { + return MedianAbsoluteDeviationIntGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "median_absolute_deviation of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..5cbcb3e9898ed --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunction.java @@ -0,0 +1,207 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MedianAbsoluteDeviationIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + public MedianAbsoluteDeviationIntGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + } + + public static MedianAbsoluteDeviationIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MedianAbsoluteDeviationIntGroupingAggregatorFunction(channels, MedianAbsoluteDeviationIntAggregator.initGrouping(bigArrays), bigArrays); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + MedianAbsoluteDeviationIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationIntAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((MedianAbsoluteDeviationIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + MedianAbsoluteDeviationIntAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = MedianAbsoluteDeviationIntAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunction.java new file mode 100644 index 0000000000000..54fcd565c20b2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunction.java @@ -0,0 +1,119 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MedianAbsoluteDeviationLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + public MedianAbsoluteDeviationLongAggregatorFunction(List channels, + QuantileStates.SingleState state) { + this.channels = channels; + this.state = state; + } + + public static MedianAbsoluteDeviationLongAggregatorFunction create(List channels) { + return new MedianAbsoluteDeviationLongAggregatorFunction(channels, MedianAbsoluteDeviationLongAggregator.initSingle()); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + MedianAbsoluteDeviationLongAggregator.combine(state, vector.getLong(i)); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + MedianAbsoluteDeviationLongAggregator.combine(state, block.getLong(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + MedianAbsoluteDeviationLongAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = MedianAbsoluteDeviationLongAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..70dcbb14b51a8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionSupplier.java @@ -0,0 +1,42 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MedianAbsoluteDeviationLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MedianAbsoluteDeviationLongAggregatorFunctionSupplier(BigArrays bigArrays, + List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MedianAbsoluteDeviationLongAggregatorFunction aggregator() { + return MedianAbsoluteDeviationLongAggregatorFunction.create(channels); + } + + @Override + public MedianAbsoluteDeviationLongGroupingAggregatorFunction groupingAggregator() { + return MedianAbsoluteDeviationLongGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "median_absolute_deviation of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..34efae2c90a7f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunction.java @@ -0,0 +1,206 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MedianAbsoluteDeviationLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MedianAbsoluteDeviationLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + public MedianAbsoluteDeviationLongGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + } + + public static MedianAbsoluteDeviationLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MedianAbsoluteDeviationLongGroupingAggregatorFunction(channels, MedianAbsoluteDeviationLongAggregator.initGrouping(bigArrays), bigArrays); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + MedianAbsoluteDeviationLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + MedianAbsoluteDeviationLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + MedianAbsoluteDeviationLongAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((MedianAbsoluteDeviationLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + MedianAbsoluteDeviationLongAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = MedianAbsoluteDeviationLongAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..3538ace6d7426 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MinDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final DoubleState state; + + private final List channels; + + public MinDoubleAggregatorFunction(List channels, DoubleState state) { + this.channels = channels; + this.state = state; + } + + public static MinDoubleAggregatorFunction create(List channels) { + return new MinDoubleAggregatorFunction(channels, new DoubleState(MinDoubleAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.doubleValue(MinDoubleAggregator.combine(state.doubleValue(), vector.getDouble(i))); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.doubleValue(MinDoubleAggregator.combine(state.doubleValue(), block.getDouble(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + DoubleVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == 1; + assert min.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.doubleValue(MinDoubleAggregator.combine(state.doubleValue(), min.getDouble(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = DoubleBlock.newConstantBlockWith(state.doubleValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..c31b6446c4a66 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MinDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MinDoubleAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MinDoubleAggregatorFunction aggregator() { + return MinDoubleAggregatorFunction.create(channels); + } + + @Override + public MinDoubleGroupingAggregatorFunction groupingAggregator() { + return MinDoubleGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "min of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..d6fbf83b2c855 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunction.java @@ -0,0 +1,209 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MinDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final DoubleArrayState state; + + private final List channels; + + public MinDoubleGroupingAggregatorFunction(List channels, DoubleArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MinDoubleGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MinDoubleGroupingAggregatorFunction(channels, new DoubleArrayState(bigArrays, MinDoubleAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), values.getDouble(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + DoubleVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), min.getDouble(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + DoubleArrayState inState = ((MinDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MinDoubleAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunction.java new file mode 100644 index 0000000000000..70f336ef4e1a1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MinIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.INT), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final IntState state; + + private final List channels; + + public MinIntAggregatorFunction(List channels, IntState state) { + this.channels = channels; + this.state = state; + } + + public static MinIntAggregatorFunction create(List channels) { + return new MinIntAggregatorFunction(channels, new IntState(MinIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.intValue(MinIntAggregator.combine(state.intValue(), vector.getInt(i))); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.intValue(MinIntAggregator.combine(state.intValue(), block.getInt(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + IntVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == 1; + assert min.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.intValue(MinIntAggregator.combine(state.intValue(), min.getInt(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = IntBlock.newConstantBlockWith(state.intValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..1348527a9593d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MinIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MinIntAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MinIntAggregatorFunction aggregator() { + return MinIntAggregatorFunction.create(channels); + } + + @Override + public MinIntGroupingAggregatorFunction groupingAggregator() { + return MinIntGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "min of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..b9d8c316dc561 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunction.java @@ -0,0 +1,208 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MinIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.INT), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final IntArrayState state; + + private final List channels; + + public MinIntGroupingAggregatorFunction(List channels, IntArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MinIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MinIntGroupingAggregatorFunction(channels, new IntArrayState(bigArrays, MinIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + IntVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), min.getInt(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + IntArrayState inState = ((MinIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MinIntAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunction.java new file mode 100644 index 0000000000000..7fc6e6be615a2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link MinLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongState state; + + private final List channels; + + public MinLongAggregatorFunction(List channels, LongState state) { + this.channels = channels; + this.state = state; + } + + public static MinLongAggregatorFunction create(List channels) { + return new MinLongAggregatorFunction(channels, new LongState(MinLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.longValue(MinLongAggregator.combine(state.longValue(), vector.getLong(i))); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.longValue(MinLongAggregator.combine(state.longValue(), block.getLong(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + LongVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == 1; + assert min.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.longValue(MinLongAggregator.combine(state.longValue(), min.getLong(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = LongBlock.newConstantBlockWith(state.longValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..59b0f1f936661 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link MinLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public MinLongAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public MinLongAggregatorFunction aggregator() { + return MinLongAggregatorFunction.create(channels); + } + + @Override + public MinLongGroupingAggregatorFunction groupingAggregator() { + return MinLongGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "min of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..d5eca10b40286 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunction.java @@ -0,0 +1,207 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link MinLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class MinLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("min", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongArrayState state; + + private final List channels; + + public MinLongGroupingAggregatorFunction(List channels, LongArrayState state) { + this.channels = channels; + this.state = state; + } + + public static MinLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new MinLongGroupingAggregatorFunction(channels, new LongArrayState(bigArrays, MinLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + LongVector min = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert min.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), min.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + LongArrayState inState = ((MinLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, MinLongAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..fafa9c2f8aa6e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunction.java @@ -0,0 +1,123 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link PercentileDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + private final double percentile; + + public PercentileDoubleAggregatorFunction(List channels, + QuantileStates.SingleState state, double percentile) { + this.channels = channels; + this.state = state; + this.percentile = percentile; + } + + public static PercentileDoubleAggregatorFunction create(List channels, + double percentile) { + return new PercentileDoubleAggregatorFunction(channels, PercentileDoubleAggregator.initSingle(percentile), percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + PercentileDoubleAggregator.combine(state, vector.getDouble(i)); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + PercentileDoubleAggregator.combine(state, block.getDouble(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + PercentileDoubleAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = PercentileDoubleAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..348bd03e84b09 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link PercentileDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final double percentile; + + public PercentileDoubleAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + double percentile) { + this.bigArrays = bigArrays; + this.channels = channels; + this.percentile = percentile; + } + + @Override + public PercentileDoubleAggregatorFunction aggregator() { + return PercentileDoubleAggregatorFunction.create(channels, percentile); + } + + @Override + public PercentileDoubleGroupingAggregatorFunction groupingAggregator() { + return PercentileDoubleGroupingAggregatorFunction.create(channels, bigArrays, percentile); + } + + @Override + public String describe() { + return "percentile of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..859b9b3434d74 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunction.java @@ -0,0 +1,211 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link PercentileDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final double percentile; + + public PercentileDoubleGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays, double percentile) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.percentile = percentile; + } + + public static PercentileDoubleGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, double percentile) { + return new PercentileDoubleGroupingAggregatorFunction(channels, PercentileDoubleAggregator.initGrouping(bigArrays, percentile), bigArrays, percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + PercentileDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileDoubleAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((PercentileDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + PercentileDoubleAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = PercentileDoubleAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunction.java new file mode 100644 index 0000000000000..3fa4eb7820e4b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunction.java @@ -0,0 +1,122 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link PercentileIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + private final double percentile; + + public PercentileIntAggregatorFunction(List channels, QuantileStates.SingleState state, + double percentile) { + this.channels = channels; + this.state = state; + this.percentile = percentile; + } + + public static PercentileIntAggregatorFunction create(List channels, double percentile) { + return new PercentileIntAggregatorFunction(channels, PercentileIntAggregator.initSingle(percentile), percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + PercentileIntAggregator.combine(state, vector.getInt(i)); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + PercentileIntAggregator.combine(state, block.getInt(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + PercentileIntAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = PercentileIntAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..125529b7ec151 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link PercentileIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final double percentile; + + public PercentileIntAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + double percentile) { + this.bigArrays = bigArrays; + this.channels = channels; + this.percentile = percentile; + } + + @Override + public PercentileIntAggregatorFunction aggregator() { + return PercentileIntAggregatorFunction.create(channels, percentile); + } + + @Override + public PercentileIntGroupingAggregatorFunction groupingAggregator() { + return PercentileIntGroupingAggregatorFunction.create(channels, bigArrays, percentile); + } + + @Override + public String describe() { + return "percentile of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..2b86de9e0d12b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunction.java @@ -0,0 +1,210 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link PercentileIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final double percentile; + + public PercentileIntGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays, double percentile) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.percentile = percentile; + } + + public static PercentileIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, double percentile) { + return new PercentileIntGroupingAggregatorFunction(channels, PercentileIntAggregator.initGrouping(bigArrays, percentile), bigArrays, percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileIntAggregator.combine(state, groupId, values.getInt(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + PercentileIntAggregator.combine(state, groupId, values.getInt(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileIntAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((PercentileIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + PercentileIntAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = PercentileIntAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunction.java new file mode 100644 index 0000000000000..a778dbae9a9c7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunction.java @@ -0,0 +1,122 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link PercentileLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.SingleState state; + + private final List channels; + + private final double percentile; + + public PercentileLongAggregatorFunction(List channels, QuantileStates.SingleState state, + double percentile) { + this.channels = channels; + this.state = state; + this.percentile = percentile; + } + + public static PercentileLongAggregatorFunction create(List channels, double percentile) { + return new PercentileLongAggregatorFunction(channels, PercentileLongAggregator.initSingle(percentile), percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + for (int i = 0; i < vector.getPositionCount(); i++) { + PercentileLongAggregator.combine(state, vector.getLong(i)); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + PercentileLongAggregator.combine(state, block.getLong(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + assert quart.getPositionCount() == 1; + BytesRef scratch = new BytesRef(); + PercentileLongAggregator.combineIntermediate(state, quart.getBytesRef(0, scratch)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = PercentileLongAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..2cba0e693fe4f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionSupplier.java @@ -0,0 +1,45 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link PercentileLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + private final double percentile; + + public PercentileLongAggregatorFunctionSupplier(BigArrays bigArrays, List channels, + double percentile) { + this.bigArrays = bigArrays; + this.channels = channels; + this.percentile = percentile; + } + + @Override + public PercentileLongAggregatorFunction aggregator() { + return PercentileLongAggregatorFunction.create(channels, percentile); + } + + @Override + public PercentileLongGroupingAggregatorFunction groupingAggregator() { + return PercentileLongGroupingAggregatorFunction.create(channels, bigArrays, percentile); + } + + @Override + public String describe() { + return "percentile of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..abb832adc4964 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunction.java @@ -0,0 +1,209 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link PercentileLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class PercentileLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("quart", ElementType.BYTES_REF) ); + + private final QuantileStates.GroupingState state; + + private final List channels; + + private final BigArrays bigArrays; + + private final double percentile; + + public PercentileLongGroupingAggregatorFunction(List channels, + QuantileStates.GroupingState state, BigArrays bigArrays, double percentile) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + this.percentile = percentile; + } + + public static PercentileLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays, double percentile) { + return new PercentileLongGroupingAggregatorFunction(channels, PercentileLongAggregator.initGrouping(bigArrays, percentile), bigArrays, percentile); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + PercentileLongAggregator.combine(state, groupId, values.getLong(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + PercentileLongAggregator.combine(state, groupId, values.getLong(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + BytesRefVector quart = page.getBlock(channels.get(0)).asVector(); + BytesRef scratch = new BytesRef(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + PercentileLongAggregator.combineIntermediate(state, groupId, quart.getBytesRef(groupPosition + positionOffset, scratch)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + QuantileStates.GroupingState inState = ((PercentileLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + PercentileLongAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = PercentileLongAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunction.java new file mode 100644 index 0000000000000..6f5321c9636fc --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunction.java @@ -0,0 +1,127 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link SumDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumDoubleAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("value", ElementType.DOUBLE), + new IntermediateStateDesc("delta", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final SumDoubleAggregator.SumState state; + + private final List channels; + + public SumDoubleAggregatorFunction(List channels, SumDoubleAggregator.SumState state) { + this.channels = channels; + this.state = state; + } + + public static SumDoubleAggregatorFunction create(List channels) { + return new SumDoubleAggregatorFunction(channels, SumDoubleAggregator.initSingle()); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + DoubleBlock block = (DoubleBlock) uncastBlock; + DoubleVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(DoubleVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + SumDoubleAggregator.combine(state, vector.getDouble(i)); + } + } + + private void addRawBlock(DoubleBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + SumDoubleAggregator.combine(state, block.getDouble(i)); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + DoubleVector value = page.getBlock(channels.get(0)).asVector(); + DoubleVector delta = page.getBlock(channels.get(1)).asVector(); + BooleanVector seen = page.getBlock(channels.get(2)).asVector(); + assert value.getPositionCount() == 1; + assert value.getPositionCount() == delta.getPositionCount() && value.getPositionCount() == seen.getPositionCount(); + SumDoubleAggregator.combineIntermediate(state, value.getDouble(0), delta.getDouble(0), seen.getBoolean(0)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = SumDoubleAggregator.evaluateFinal(state); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..1f65689b6525c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link SumDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumDoubleAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public SumDoubleAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public SumDoubleAggregatorFunction aggregator() { + return SumDoubleAggregatorFunction.create(channels); + } + + @Override + public SumDoubleGroupingAggregatorFunction groupingAggregator() { + return SumDoubleGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "sum of doubles"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..4df1638ebb8bf --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunction.java @@ -0,0 +1,211 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link SumDoubleAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumDoubleGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("value", ElementType.DOUBLE), + new IntermediateStateDesc("delta", ElementType.DOUBLE), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final SumDoubleAggregator.GroupingSumState state; + + private final List channels; + + private final BigArrays bigArrays; + + public SumDoubleGroupingAggregatorFunction(List channels, + SumDoubleAggregator.GroupingSumState state, BigArrays bigArrays) { + this.channels = channels; + this.state = state; + this.bigArrays = bigArrays; + } + + public static SumDoubleGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new SumDoubleGroupingAggregatorFunction(channels, SumDoubleAggregator.initGrouping(bigArrays), bigArrays); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + DoubleBlock valuesBlock = (DoubleBlock) uncastValuesBlock; + DoubleVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + SumDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + SumDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + SumDoubleAggregator.combine(state, groupId, values.getDouble(v)); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, DoubleVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + SumDoubleAggregator.combine(state, groupId, values.getDouble(groupPosition + positionOffset)); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + DoubleVector value = page.getBlock(channels.get(0)).asVector(); + DoubleVector delta = page.getBlock(channels.get(1)).asVector(); + BooleanVector seen = page.getBlock(channels.get(2)).asVector(); + assert value.getPositionCount() == delta.getPositionCount() && value.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + SumDoubleAggregator.combineIntermediate(state, groupId, value.getDouble(groupPosition + positionOffset), delta.getDouble(groupPosition + positionOffset), seen.getBoolean(groupPosition + positionOffset)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + SumDoubleAggregator.GroupingSumState inState = ((SumDoubleGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + SumDoubleAggregator.combineStates(state, groupId, inState, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = SumDoubleAggregator.evaluateFinal(state, selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunction.java new file mode 100644 index 0000000000000..9fe30cb0e69c9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunction.java @@ -0,0 +1,130 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link SumIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumIntAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("sum", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongState state; + + private final List channels; + + public SumIntAggregatorFunction(List channels, LongState state) { + this.channels = channels; + this.state = state; + } + + public static SumIntAggregatorFunction create(List channels) { + return new SumIntAggregatorFunction(channels, new LongState(SumIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + IntBlock block = (IntBlock) uncastBlock; + IntVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(IntVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.longValue(SumIntAggregator.combine(state.longValue(), vector.getInt(i))); + } + } + + private void addRawBlock(IntBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.longValue(SumIntAggregator.combine(state.longValue(), block.getInt(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + LongVector sum = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert sum.getPositionCount() == 1; + assert sum.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.longValue(SumIntAggregator.combine(state.longValue(), sum.getLong(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = LongBlock.newConstantBlockWith(state.longValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..2b0ad0a6538fb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link SumIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumIntAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public SumIntAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public SumIntAggregatorFunction aggregator() { + return SumIntAggregatorFunction.create(channels); + } + + @Override + public SumIntGroupingAggregatorFunction groupingAggregator() { + return SumIntGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "sum of ints"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..27fc33a83abd6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunction.java @@ -0,0 +1,208 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link SumIntAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumIntGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("sum", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongArrayState state; + + private final List channels; + + public SumIntGroupingAggregatorFunction(List channels, LongArrayState state) { + this.channels = channels; + this.state = state; + } + + public static SumIntGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new SumIntGroupingAggregatorFunction(channels, new LongArrayState(bigArrays, SumIntAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + IntBlock valuesBlock = (IntBlock) uncastValuesBlock; + IntVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), values.getInt(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, IntVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), values.getInt(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + LongVector sum = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert sum.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), sum.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + LongArrayState inState = ((SumIntGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, SumIntAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunction.java new file mode 100644 index 0000000000000..e6b5f7c9b08b5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunction.java @@ -0,0 +1,128 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link AggregatorFunction} implementation for {@link SumLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumLongAggregatorFunction implements AggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("sum", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongState state; + + private final List channels; + + public SumLongAggregatorFunction(List channels, LongState state) { + this.channels = channels; + this.state = state; + } + + public static SumLongAggregatorFunction create(List channels) { + return new SumLongAggregatorFunction(channels, new LongState(SumLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public void addRawInput(Page page) { + Block uncastBlock = page.getBlock(channels.get(0)); + if (uncastBlock.areAllValuesNull()) { + return; + } + LongBlock block = (LongBlock) uncastBlock; + LongVector vector = block.asVector(); + if (vector != null) { + addRawVector(vector); + } else { + addRawBlock(block); + } + } + + private void addRawVector(LongVector vector) { + state.seen(true); + for (int i = 0; i < vector.getPositionCount(); i++) { + state.longValue(SumLongAggregator.combine(state.longValue(), vector.getLong(i))); + } + } + + private void addRawBlock(LongBlock block) { + for (int p = 0; p < block.getPositionCount(); p++) { + if (block.isNull(p)) { + continue; + } + state.seen(true); + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { + state.longValue(SumLongAggregator.combine(state.longValue(), block.getLong(i))); + } + } + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + LongVector sum = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert sum.getPositionCount() == 1; + assert sum.getPositionCount() == seen.getPositionCount(); + if (seen.getBoolean(0)) { + state.longValue(SumLongAggregator.combine(state.longValue(), sum.getLong(0))); + state.seen(true); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + if (state.seen() == false) { + blocks[offset] = Block.constantNullBlock(1); + return; + } + blocks[offset] = LongBlock.newConstantBlockWith(state.longValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..535998bfac47c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionSupplier.java @@ -0,0 +1,41 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; + +/** + * {@link AggregatorFunctionSupplier} implementation for {@link SumLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumLongAggregatorFunctionSupplier implements AggregatorFunctionSupplier { + private final BigArrays bigArrays; + + private final List channels; + + public SumLongAggregatorFunctionSupplier(BigArrays bigArrays, List channels) { + this.bigArrays = bigArrays; + this.channels = channels; + } + + @Override + public SumLongAggregatorFunction aggregator() { + return SumLongAggregatorFunction.create(channels); + } + + @Override + public SumLongGroupingAggregatorFunction groupingAggregator() { + return SumLongGroupingAggregatorFunction.create(channels, bigArrays); + } + + @Override + public String describe() { + return "sum of longs"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..2ae2d3c2b6338 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/generated/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunction.java @@ -0,0 +1,207 @@ +// 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.compute.aggregation; + +import java.lang.Integer; +import java.lang.Override; +import java.lang.String; +import java.lang.StringBuilder; +import java.util.List; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +/** + * {@link GroupingAggregatorFunction} implementation for {@link SumLongAggregator}. + * This class is generated. Do not edit it. + */ +public final class SumLongGroupingAggregatorFunction implements GroupingAggregatorFunction { + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("sum", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) ); + + private final LongArrayState state; + + private final List channels; + + public SumLongGroupingAggregatorFunction(List channels, LongArrayState state) { + this.channels = channels; + this.state = state; + } + + public static SumLongGroupingAggregatorFunction create(List channels, + BigArrays bigArrays) { + return new SumLongGroupingAggregatorFunction(channels, new LongArrayState(bigArrays, SumLongAggregator.init())); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + @Override + public int intermediateBlockCount() { + return INTERMEDIATE_STATE_DESC.size(); + } + + @Override + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, + Page page) { + Block uncastValuesBlock = page.getBlock(channels.get(0)); + if (uncastValuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + } + }; + } + LongBlock valuesBlock = (LongBlock) uncastValuesBlock; + LongVector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesVector); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + + private void addRawInput(int positionOffset, LongVector groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongBlock values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(groupPosition + positionOffset)) { + continue; + } + int valuesStart = values.getFirstValueIndex(groupPosition + positionOffset); + int valuesEnd = valuesStart + values.getValueCount(groupPosition + positionOffset); + for (int v = valuesStart; v < valuesEnd; v++) { + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), values.getLong(v))); + } + } + } + } + + private void addRawInput(int positionOffset, LongBlock groups, LongVector values) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), values.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + assert channels.size() == intermediateBlockCount(); + LongVector sum = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert sum.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (seen.getBoolean(groupPosition + positionOffset)) { + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), sum.getLong(groupPosition + positionOffset))); + } + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + LongArrayState inState = ((SumLongGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.set(groupId, SumLongAggregator.combine(state.getOrDefault(groupId), inState.get(position))); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + blocks[offset] = state.toValuesBlock(selected); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/module-info.java b/x-pack/plugin/esql/compute/src/main/java/module-info.java new file mode 100644 index 0000000000000..91e45baa7bda9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/module-info.java @@ -0,0 +1,24 @@ +/* + * 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. + */ + +module org.elasticsearch.compute { + requires org.apache.lucene.core; + requires org.elasticsearch.base; + requires org.elasticsearch.server; + requires org.elasticsearch.compute.ann; + requires org.elasticsearch.xcontent; + requires org.apache.logging.log4j; + requires org.elasticsearch.tdigest; + + exports org.elasticsearch.compute; + exports org.elasticsearch.compute.aggregation; + exports org.elasticsearch.compute.data; + exports org.elasticsearch.compute.lucene; + exports org.elasticsearch.compute.operator; + exports org.elasticsearch.compute.operator.exchange; + exports org.elasticsearch.compute.aggregation.blockhash; +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/Describable.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/Describable.java new file mode 100644 index 0000000000000..5e49a6b49e1e6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/Describable.java @@ -0,0 +1,21 @@ +/* + * 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.compute; + +/** + * A component capable of describing itself. + */ +public interface Describable { + + /** + * Returns a description of the component. This description can be more specific than Object::toString. + * + * @return the description + */ + String describe(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AbstractArrayState.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AbstractArrayState.java new file mode 100644 index 0000000000000..0dc008cb22396 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AbstractArrayState.java @@ -0,0 +1,53 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; + +public class AbstractArrayState implements Releasable { + protected final BigArrays bigArrays; + + private BitArray seen; + + public AbstractArrayState(BigArrays bigArrays) { + this.bigArrays = bigArrays; + } + + final boolean hasValue(int groupId) { + return seen == null || seen.get(groupId); + } + + /** + * Switches this array state into tracking which group ids are set. This is + * idempotent and fast if already tracking so it's safe to, say, call it once + * for every block of values that arrives containing {@code null}. + */ + final void enableGroupIdTracking(SeenGroupIds seenGroupIds) { + if (seen == null) { + seen = seenGroupIds.seenGroupIds(bigArrays); + } + } + + protected final void trackGroupId(int groupId) { + if (trackingGroupIds()) { + seen.set(groupId); + } + } + + protected final boolean trackingGroupIds() { + return seen != null; + } + + @Override + public void close() { + Releasables.close(seen); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/Aggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/Aggregator.java new file mode 100644 index 0000000000000..3466de65021bd --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/Aggregator.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasable; + +import java.util.function.Supplier; + +public class Aggregator implements Releasable { + + public static final Object[] EMPTY_PARAMS = new Object[] {}; + + private final AggregatorFunction aggregatorFunction; + + private final AggregatorMode mode; + + public interface Factory extends Supplier, Describable {} + + public Aggregator(AggregatorFunction aggregatorFunction, AggregatorMode mode) { + this.aggregatorFunction = aggregatorFunction; + this.mode = mode; + } + + /** The number of Blocks required for evaluation. */ + public int evaluateBlockCount() { + return mode.isOutputPartial() ? aggregatorFunction.intermediateBlockCount() : 1; + } + + public void processPage(Page page) { + if (mode.isInputPartial()) { + aggregatorFunction.addIntermediateInput(page); + } else { + aggregatorFunction.addRawInput(page); + } + } + + public void evaluate(Block[] blocks, int offset) { + if (mode.isOutputPartial()) { + aggregatorFunction.evaluateIntermediate(blocks, offset); + } else { + aggregatorFunction.evaluateFinal(blocks, offset); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("aggregatorFunction=").append(aggregatorFunction).append(", "); + sb.append("mode=").append(mode); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + aggregatorFunction.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunction.java new file mode 100644 index 0000000000000..69ef8df616c8e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunction.java @@ -0,0 +1,26 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasable; + +public interface AggregatorFunction extends Releasable { + + void addRawInput(Page page); + + void addIntermediateInput(Page page); + + void evaluateIntermediate(Block[] blocks, int offset); + + void evaluateFinal(Block[] blocks, int offset); + + /** The number of blocks used by intermediate state. */ + int intermediateBlockCount(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunctionSupplier.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunctionSupplier.java new file mode 100644 index 0000000000000..957b100da01f4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorFunctionSupplier.java @@ -0,0 +1,48 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.operator.DriverContext; + +/** + * Builds aggregation implementations, closing over any state required to do so. + */ +public interface AggregatorFunctionSupplier extends Describable { + AggregatorFunction aggregator(); + + GroupingAggregatorFunction groupingAggregator(); + + default Aggregator.Factory aggregatorFactory(AggregatorMode mode) { + return new Aggregator.Factory() { + @Override + public Aggregator get() { + return new Aggregator(aggregator(), mode); + } + + @Override + public String describe() { + return AggregatorFunctionSupplier.this.describe(); + } + }; + } + + default GroupingAggregator.Factory groupingAggregatorFactory(AggregatorMode mode) { + return new GroupingAggregator.Factory() { + @Override + public GroupingAggregator apply(DriverContext driverContext) { + return new GroupingAggregator(groupingAggregator(), mode); + } + + @Override + public String describe() { + return AggregatorFunctionSupplier.this.describe(); + } + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorMode.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorMode.java new file mode 100644 index 0000000000000..22b20a445c196 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorMode.java @@ -0,0 +1,37 @@ +/* + * 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.compute.aggregation; + +public enum AggregatorMode { + + INITIAL(false, true), + + INTERMEDIATE(true, true), + + FINAL(true, false), + + // most useful for testing + SINGLE(false, false); + + private final boolean inputPartial; + + private final boolean outputPartial; + + AggregatorMode(boolean inputPartial, boolean outputPartial) { + this.inputPartial = inputPartial; + this.outputPartial = outputPartial; + } + + public boolean isInputPartial() { + return inputPartial; + } + + public boolean isOutputPartial() { + return outputPartial; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorState.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorState.java new file mode 100644 index 0000000000000..d0a644215a759 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/AggregatorState.java @@ -0,0 +1,17 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.core.Releasable; + +public interface AggregatorState extends Releasable { + + /** Extracts an intermediate view of the contents of this state. */ + void toIntermediate(Block[] blocks, int offset); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountAggregatorFunction.java new file mode 100644 index 0000000000000..25ff4a2a3ab6a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountAggregatorFunction.java @@ -0,0 +1,108 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; + +public class CountAggregatorFunction implements AggregatorFunction { + public static AggregatorFunctionSupplier supplier(BigArrays bigArrays, List channels) { + return new AggregatorFunctionSupplier() { + @Override + public AggregatorFunction aggregator() { + return CountAggregatorFunction.create(channels); + } + + @Override + public GroupingAggregatorFunction groupingAggregator() { + return CountGroupingAggregatorFunction.create(bigArrays, channels); + } + + @Override + public String describe() { + return "count"; + } + }; + } + + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("count", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) + ); + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + private final LongState state; + private final List channels; + + public static CountAggregatorFunction create(List inputChannels) { + return new CountAggregatorFunction(inputChannels, new LongState()); + } + + private CountAggregatorFunction(List channels, LongState state) { + this.channels = channels; + this.state = state; + } + + @Override + public int intermediateBlockCount() { + return intermediateStateDesc().size(); + } + + @Override + public void addRawInput(Page page) { + Block block = page.getBlock(channels.get(0)); + LongState state = this.state; + state.longValue(state.longValue() + block.getTotalValueCount()); + } + + @Override + public void addIntermediateInput(Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + LongVector count = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert count.getPositionCount() == 1; + assert count.getPositionCount() == seen.getPositionCount(); + state.longValue(state.longValue() + count.getLong(0)); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset) { + state.toIntermediate(blocks, offset); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset) { + blocks[offset] = LongBlock.newConstantBlockWith(state.longValue(), 1); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregator.java new file mode 100644 index 0000000000000..64dcb9910b897 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregator.java @@ -0,0 +1,152 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.core.Releasables; + +@Aggregator({ @IntermediateState(name = "fbit", type = "BOOLEAN"), @IntermediateState(name = "tbit", type = "BOOLEAN") }) +@GroupingAggregator +public class CountDistinctBooleanAggregator { + private static final byte BIT_FALSE = 0b01; + private static final byte BIT_TRUE = 0b10; + + public static SingleState initSingle() { + return new SingleState(); + } + + public static void combine(SingleState current, boolean v) { + current.bits |= v ? BIT_TRUE : BIT_FALSE; + } + + public static void combineStates(SingleState current, SingleState state) { + current.bits |= state.bits; + } + + public static void combineIntermediate(SingleState current, boolean fbit, boolean tbit) { + if (fbit) current.bits |= BIT_FALSE; + if (tbit) current.bits |= BIT_TRUE; + } + + public static Block evaluateFinal(SingleState state) { + long result = ((state.bits & BIT_TRUE) >> 1) + (state.bits & BIT_FALSE); + return LongBlock.newConstantBlockWith(result, 1); + } + + public static GroupingState initGrouping(BigArrays bigArrays) { + return new GroupingState(bigArrays); + } + + public static void combine(GroupingState current, int groupId, boolean v) { + current.collect(groupId, v); + } + + public static void combineStates(GroupingState current, int currentGroupId, GroupingState state, int statePosition) { + current.combineStates(currentGroupId, state); + } + + public static void combineIntermediate(GroupingState current, int groupId, boolean fbit, boolean tbit) { + if (fbit) current.bits.set(groupId * 2); + if (tbit) current.bits.set(groupId * 2 + 1); + } + + public static Block evaluateFinal(GroupingState state, IntVector selected) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + long count = (state.bits.get(2 * group) ? 1 : 0) + (state.bits.get(2 * group + 1) ? 1 : 0); + builder.appendLong(count); + } + return builder.build(); + } + + /** + * State contains a byte variable where we set two bits. Bit 0 is set when a boolean false + * value is collected. Bit 1 is set when a boolean true value is collected. + */ + static class SingleState implements AggregatorState { + + byte bits; + + SingleState() {} + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 2; + blocks[offset + 0] = BooleanBlock.newConstantBlockWith((bits & BIT_FALSE) != 0, 1); + blocks[offset + 1] = BooleanBlock.newConstantBlockWith((bits & BIT_TRUE) != 0, 1); + } + + @Override + public void close() {} + } + + /** + * Grouping state uses as a {@link BitArray} and stores two bits for each groupId. + * First bit is set if boolean false value is collected and second bit is set + * if boolean true value is collected. + * This means that false values for a groupId are stored at bits[2*groupId] and + * true values for a groupId are stored at bits[2*groupId + 1] + */ + static class GroupingState extends AbstractArrayState implements GroupingAggregatorState { + + final BitArray bits; + + GroupingState(BigArrays bigArrays) { + super(bigArrays); + boolean success = false; + try { + this.bits = new BitArray(2, bigArrays); // Start with two bits for a single groupId + success = true; + } finally { + if (success == false) { + close(); + } + } + } + + void collect(int groupId, boolean v) { + bits.set(groupId * 2 + (v ? 1 : 0)); + trackGroupId(groupId); + } + + void combineStates(int currentGroupId, GroupingState state) { + bits.or(state.bits); + trackGroupId(currentGroupId); + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 2; + var fbitBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + var tbitBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + fbitBuilder.appendBoolean(bits.get(2 * group + 0)); + tbitBuilder.appendBoolean(bits.get(2 * group + 1)); + } + blocks[offset + 0] = fbitBuilder.build(); + blocks[offset + 1] = tbitBuilder.build(); + } + + @Override + public void close() { + Releasables.close(bits, super::close); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregator.java new file mode 100644 index 0000000000000..92cf5cf38354a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregator.java @@ -0,0 +1,74 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; + +@Aggregator({ @IntermediateState(name = "hll", type = "BYTES_REF") }) +@GroupingAggregator +public class CountDistinctBytesRefAggregator { + + public static HllStates.SingleState initSingle(BigArrays bigArrays, int precision) { + return new HllStates.SingleState(bigArrays, precision); + } + + public static void combine(HllStates.SingleState current, BytesRef v) { + current.collect(v); + } + + public static void combineStates(HllStates.SingleState current, HllStates.SingleState state) { + current.merge(0, state.hll, 0); + } + + public static void combineIntermediate(HllStates.SingleState current, BytesRef inValue) { + current.merge(0, inValue, 0); + } + + public static Block evaluateFinal(HllStates.SingleState state) { + long result = state.cardinality(); + return LongBlock.newConstantBlockWith(result, 1); + } + + public static HllStates.GroupingState initGrouping(BigArrays bigArrays, int precision) { + return new HllStates.GroupingState(bigArrays, precision); + } + + public static void combine(HllStates.GroupingState current, int groupId, BytesRef v) { + current.collect(groupId, v); + } + + public static void combineIntermediate(HllStates.GroupingState current, int groupId, BytesRef inValue) { + current.merge(groupId, inValue, 0); + } + + public static void combineStates( + HllStates.GroupingState current, + int currentGroupId, + HllStates.GroupingState state, + int statePosition + ) { + current.merge(currentGroupId, state.hll, statePosition); + } + + public static Block evaluateFinal(HllStates.GroupingState state, IntVector selected) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + long count = state.cardinality(group); + builder.appendLong(count); + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregator.java new file mode 100644 index 0000000000000..cde6bbcf5783a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregator.java @@ -0,0 +1,74 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; + +@Aggregator({ @IntermediateState(name = "hll", type = "BYTES_REF") }) +@GroupingAggregator +public class CountDistinctDoubleAggregator { + + public static HllStates.SingleState initSingle(BigArrays bigArrays, int precision) { + return new HllStates.SingleState(bigArrays, precision); + } + + public static void combine(HllStates.SingleState current, double v) { + current.collect(v); + } + + public static void combineStates(HllStates.SingleState current, HllStates.SingleState state) { + current.merge(0, state.hll, 0); + } + + public static void combineIntermediate(HllStates.SingleState current, BytesRef inValue) { + current.merge(0, inValue, 0); + } + + public static Block evaluateFinal(HllStates.SingleState state) { + long result = state.cardinality(); + return LongBlock.newConstantBlockWith(result, 1); + } + + public static HllStates.GroupingState initGrouping(BigArrays bigArrays, int precision) { + return new HllStates.GroupingState(bigArrays, precision); + } + + public static void combine(HllStates.GroupingState current, int groupId, double v) { + current.collect(groupId, v); + } + + public static void combineIntermediate(HllStates.GroupingState current, int groupId, BytesRef inValue) { + current.merge(groupId, inValue, 0); + } + + public static void combineStates( + HllStates.GroupingState current, + int currentGroupId, + HllStates.GroupingState state, + int statePosition + ) { + current.merge(currentGroupId, state.hll, statePosition); + } + + public static Block evaluateFinal(HllStates.GroupingState state, IntVector selected) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + long count = state.cardinality(group); + builder.appendLong(count); + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregator.java new file mode 100644 index 0000000000000..e2f69b44eafc2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregator.java @@ -0,0 +1,74 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; + +@Aggregator({ @IntermediateState(name = "hll", type = "BYTES_REF") }) +@GroupingAggregator +public class CountDistinctIntAggregator { + + public static HllStates.SingleState initSingle(BigArrays bigArrays, int precision) { + return new HllStates.SingleState(bigArrays, precision); + } + + public static void combine(HllStates.SingleState current, int v) { + current.collect(v); + } + + public static void combineStates(HllStates.SingleState current, HllStates.SingleState state) { + current.merge(0, state.hll, 0); + } + + public static void combineIntermediate(HllStates.SingleState current, BytesRef inValue) { + current.merge(0, inValue, 0); + } + + public static Block evaluateFinal(HllStates.SingleState state) { + long result = state.cardinality(); + return LongBlock.newConstantBlockWith(result, 1); + } + + public static HllStates.GroupingState initGrouping(BigArrays bigArrays, int precision) { + return new HllStates.GroupingState(bigArrays, precision); + } + + public static void combine(HllStates.GroupingState current, int groupId, int v) { + current.collect(groupId, v); + } + + public static void combineIntermediate(HllStates.GroupingState current, int groupId, BytesRef inValue) { + current.merge(groupId, inValue, 0); + } + + public static void combineStates( + HllStates.GroupingState current, + int currentGroupId, + HllStates.GroupingState state, + int statePosition + ) { + current.merge(currentGroupId, state.hll, statePosition); + } + + public static Block evaluateFinal(HllStates.GroupingState state, IntVector selected) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + long count = state.cardinality(group); + builder.appendLong(count); + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregator.java new file mode 100644 index 0000000000000..290ced4ceaded --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregator.java @@ -0,0 +1,74 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; + +@Aggregator({ @IntermediateState(name = "hll", type = "BYTES_REF") }) +@GroupingAggregator +public class CountDistinctLongAggregator { + + public static HllStates.SingleState initSingle(BigArrays bigArrays, int precision) { + return new HllStates.SingleState(bigArrays, precision); + } + + public static void combine(HllStates.SingleState current, long v) { + current.collect(v); + } + + public static void combineStates(HllStates.SingleState current, HllStates.SingleState state) { + current.merge(0, state.hll, 0); + } + + public static void combineIntermediate(HllStates.SingleState current, BytesRef inValue) { + current.merge(0, inValue, 0); + } + + public static Block evaluateFinal(HllStates.SingleState state) { + long result = state.cardinality(); + return LongBlock.newConstantBlockWith(result, 1); + } + + public static HllStates.GroupingState initGrouping(BigArrays bigArrays, int precision) { + return new HllStates.GroupingState(bigArrays, precision); + } + + public static void combine(HllStates.GroupingState current, int groupId, long v) { + current.collect(groupId, v); + } + + public static void combineIntermediate(HllStates.GroupingState current, int groupId, BytesRef inValue) { + current.merge(groupId, inValue, 0); + } + + public static void combineStates( + HllStates.GroupingState current, + int currentGroupId, + HllStates.GroupingState state, + int statePosition + ) { + current.merge(currentGroupId, state.hll, statePosition); + } + + public static Block evaluateFinal(HllStates.GroupingState state, IntVector selected) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + long count = state.cardinality(group); + builder.appendLong(count); + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunction.java new file mode 100644 index 0000000000000..fc40faa0e08c4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunction.java @@ -0,0 +1,197 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; + +import java.util.List; + +public class CountGroupingAggregatorFunction implements GroupingAggregatorFunction { + + private static final List INTERMEDIATE_STATE_DESC = List.of( + new IntermediateStateDesc("count", ElementType.LONG), + new IntermediateStateDesc("seen", ElementType.BOOLEAN) + ); + + private final LongArrayState state; + private final List channels; + + public static CountGroupingAggregatorFunction create(BigArrays bigArrays, List inputChannels) { + return new CountGroupingAggregatorFunction(inputChannels, new LongArrayState(bigArrays, 0)); + } + + public static List intermediateStateDesc() { + return INTERMEDIATE_STATE_DESC; + } + + private CountGroupingAggregatorFunction(List channels, LongArrayState state) { + this.channels = channels; + this.state = state; + } + + @Override + public int intermediateBlockCount() { + return intermediateStateDesc().size(); + } + + @Override + public AddInput prepareProcessPage(SeenGroupIds seenGroupIds, Page page) { + Block valuesBlock = page.getBlock(channels.get(0)); + if (valuesBlock.areAllValuesNull()) { + state.enableGroupIdTracking(seenGroupIds); + return new AddInput() { // TODO return null meaning "don't collect me" and skip those + @Override + public void add(int positionOffset, LongBlock groupIds) {} + + @Override + public void add(int positionOffset, LongVector groupIds) {} + }; + } + Vector valuesVector = valuesBlock.asVector(); + if (valuesVector == null) { + if (valuesBlock.mayHaveNulls()) { + state.enableGroupIdTracking(seenGroupIds); + } + return new AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(positionOffset, groupIds, valuesBlock); + } + }; + } + return new AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + addRawInput(groupIds); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + addRawInput(groupIds); + } + }; + } + + private void addRawInput(int positionOffset, LongVector groups, Block values) { + int position = positionOffset; + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++, position++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + if (values.isNull(position)) { + continue; + } + state.increment(groupId, values.getValueCount(position)); + } + } + + private void addRawInput(int positionOffset, LongBlock groups, Block values) { + int position = positionOffset; + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++, position++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + if (values.isNull(position)) { + continue; + } + state.increment(groupId, values.getValueCount(position)); + } + } + } + + private void addRawInput(LongVector groups) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + int groupId = Math.toIntExact(groups.getLong(groupPosition)); + state.increment(groupId, 1); + } + } + + private void addRawInput(LongBlock groups) { + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + if (groups.isNull(groupPosition)) { + continue; + } + int groupStart = groups.getFirstValueIndex(groupPosition); + int groupEnd = groupStart + groups.getValueCount(groupPosition); + for (int g = groupStart; g < groupEnd; g++) { + int groupId = Math.toIntExact(groups.getLong(g)); + state.increment(groupId, 1); + } + } + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groups, Page page) { + assert channels.size() == intermediateBlockCount(); + assert page.getBlockCount() >= channels.get(0) + intermediateStateDesc().size(); + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + LongVector count = page.getBlock(channels.get(0)).asVector(); + BooleanVector seen = page.getBlock(channels.get(1)).asVector(); + assert count.getPositionCount() == seen.getPositionCount(); + for (int groupPosition = 0; groupPosition < groups.getPositionCount(); groupPosition++) { + state.increment(Math.toIntExact(groups.getLong(groupPosition)), count.getLong(groupPosition + positionOffset)); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + if (input.getClass() != getClass()) { + throw new IllegalArgumentException("expected " + getClass() + "; got " + input.getClass()); + } + final LongArrayState inState = ((CountGroupingAggregatorFunction) input).state; + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + if (inState.hasValue(position)) { + state.increment(groupId, inState.get(position)); + } + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + state.toIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + LongVector.Builder builder = LongVector.newVectorBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int si = selected.getInt(i); + builder.appendLong(state.hasValue(si) ? state.get(si) : 0); + } + blocks[offset] = builder.build().asBlock(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("channels=").append(channels); + sb.append("]"); + return sb.toString(); + } + + @Override + public void close() { + state.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregator.java new file mode 100644 index 0000000000000..e78033e08f903 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregator.java @@ -0,0 +1,93 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.core.Releasable; + +import java.util.function.Function; + +public class GroupingAggregator implements Releasable { + private final GroupingAggregatorFunction aggregatorFunction; + + private final AggregatorMode mode; + + public interface Factory extends Function, Describable {} + + public GroupingAggregator(GroupingAggregatorFunction aggregatorFunction, AggregatorMode mode) { + this.aggregatorFunction = aggregatorFunction; + this.mode = mode; + } + + /** The number of Blocks required for evaluation. */ + public int evaluateBlockCount() { + return mode.isOutputPartial() ? aggregatorFunction.intermediateBlockCount() : 1; + } + + /** + * Prepare to process a single page of results. + */ + public GroupingAggregatorFunction.AddInput prepareProcessPage(SeenGroupIds seenGroupIds, Page page) { + if (mode.isInputPartial()) { + return new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + throw new IllegalStateException("Intermediate group id must not have nulls"); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + aggregatorFunction.addIntermediateInput(positionOffset, groupIds, page); + } + }; + } else { + return aggregatorFunction.prepareProcessPage(seenGroupIds, page); + } + } + + /** + * Add the position-th row from the intermediate output of the given aggregator to this aggregator at the groupId position + */ + public void addIntermediateRow(int groupId, GroupingAggregator input, int position) { + aggregatorFunction.addIntermediateRowInput(groupId, input.aggregatorFunction, position); + } + + /** + * Build the results for this aggregation. + * @param selected the groupIds that have been selected to be included in + * the results. Always ascending. + */ + public void evaluate(Block[] blocks, int offset, IntVector selected) { + if (mode.isOutputPartial()) { + aggregatorFunction.evaluateIntermediate(blocks, offset, selected); + } else { + aggregatorFunction.evaluateFinal(blocks, offset, selected); + } + } + + @Override + public void close() { + aggregatorFunction.close(); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("aggregatorFunction=").append(aggregatorFunction).append(", "); + sb.append("mode=").append(mode); + sb.append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunction.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunction.java new file mode 100644 index 0000000000000..017300864a6a8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunction.java @@ -0,0 +1,105 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.core.Releasable; + +/** + * Applies some grouping function like {@code min} or {@code avg} to many values, + * grouped into buckets. + */ +public interface GroupingAggregatorFunction extends Releasable { + /** + * Consume group ids to cause the {@link GroupingAggregatorFunction} + * to group values at a particular position into a particular group. + */ + interface AddInput { + /** + * Send a batch of group ids to the aggregator. The {@code groupIds} + * may be offset from the start of the block to allow for sending chunks + * of group ids. + *

+ * Any single position may be collected into arbitrarily many group + * ids. Often it's just one, but it's quite possible for a single + * position to be collected into thousands or millions of group ids. + * The {@code positionOffset} controls the start of the chunk of group + * ids contained in {@code groupIds}. + *

+ *

+ * It is possible for an input position to be cut into more than one + * chunk. In other words, it's possible for this method to be called + * multiple times with the same {@code positionOffset} and a + * {@code groupIds} {@linkplain Block} that contains thousands of + * values at a single positions. + *

+ * @param positionOffset offset into the {@link Page} used to build this + * {@link AddInput} of these ids + * @param groupIds {@link Block} of group id, some of which may be null + * or multivalued + */ + void add(int positionOffset, LongBlock groupIds); + + /** + * Send a batch of group ids to the aggregator. The {@code groupIds} + * may be offset from the start of the block to allow for sending chunks + * of group ids. + *

+ * See {@link #add(int, LongBlock)} for discussion on the offset. This + * method can only be called with blocks contained in a {@link Vector} + * which only allows a single value per position. + *

+ * @param positionOffset offset into the {@link Page} used to build this + * {@link AddInput} of these ids + * @param groupIds {@link Vector} of group id, some of which may be null + * or multivalued + */ + void add(int positionOffset, LongVector groupIds); + } + + /** + * Prepare to process a single page of results. + *

+ * This should load the input {@link Block}s and check their types and + * select an optimal path and return that path as an {@link AddInput}. + *

+ */ + AddInput prepareProcessPage(SeenGroupIds seenGroupIds, Page page); // TODO allow returning null to opt out of the callback loop + + /** + * Add data produced by {@link #evaluateIntermediate}. + */ + void addIntermediateInput(int positionOffset, LongVector groupIdVector, Page page); + + /** + * Add the position-th row from the intermediate output of the given aggregator function to the groupId + */ + void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position); + + /** + * Build the intermediate results for this aggregation. + * @param selected the groupIds that have been selected to be included in + * the results. Always ascending. + */ + void evaluateIntermediate(Block[] blocks, int offset, IntVector selected); + + /** + * Build the final results for this aggregation. + * @param selected the groupIds that have been selected to be included in + * the results. Always ascending. + */ + void evaluateFinal(Block[] blocks, int offset, IntVector selected); + + /** The number of blocks used by intermediate state. */ + int intermediateBlockCount(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorState.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorState.java new file mode 100644 index 0000000000000..d159d15f83413 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/GroupingAggregatorState.java @@ -0,0 +1,19 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.core.Releasable; + +public interface GroupingAggregatorState extends Releasable { + + /** Extracts an intermediate view of the contents of this state. */ + void toIntermediate(Block[] blocks, int offset, IntVector selected); + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/HllStates.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/HllStates.java new file mode 100644 index 0000000000000..ade35259cbabf --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/HllStates.java @@ -0,0 +1,195 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.hppc.BitMixer; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.hash.MurmurHash3; +import org.elasticsearch.common.io.stream.ByteArrayStreamInput; +import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.aggregations.metrics.AbstractHyperLogLogPlusPlus; +import org.elasticsearch.search.aggregations.metrics.HyperLogLogPlusPlus; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +final class HllStates { + private HllStates() {} + + static BytesRef serializeHLL(int groupId, HyperLogLogPlusPlus hll) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(baos); + try { + hll.writeTo(groupId, out); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new BytesRef(baos.toByteArray()); + } + + static AbstractHyperLogLogPlusPlus deserializeHLL(BytesRef bytesRef) { + ByteArrayStreamInput in = new ByteArrayStreamInput(bytesRef.bytes); + in.reset(bytesRef.bytes, bytesRef.offset, bytesRef.length); + try { + return HyperLogLogPlusPlus.readFrom(in, BigArrays.NON_RECYCLING_INSTANCE); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Copies the content of the BytesReference to an array of bytes. The byte[] must + * have enough space to fit the bytesReference object, otherwise an + * {@link ArrayIndexOutOfBoundsException} will be thrown. + * + * @return number of bytes copied + */ + static int copyToArray(BytesReference bytesReference, byte[] arr, int offset) { + int origOffset = offset; + final BytesRefIterator iterator = bytesReference.iterator(); + try { + BytesRef slice; + while ((slice = iterator.next()) != null) { + System.arraycopy(slice.bytes, slice.offset, arr, offset, slice.length); + offset += slice.length; + } + return offset - origOffset; + } catch (IOException e) { + throw new AssertionError(e); + } + } + + static class SingleState implements AggregatorState { + + private static final int SINGLE_BUCKET_ORD = 0; + final HyperLogLogPlusPlus hll; + private final MurmurHash3.Hash128 hash = new MurmurHash3.Hash128(); + + SingleState(BigArrays bigArrays, int precision) { + this.hll = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.precisionFromThreshold(precision), bigArrays, 1); + } + + void collect(long v) { + doCollect(BitMixer.mix64(v)); + } + + void collect(int v) { + doCollect(BitMixer.mix64(v)); + } + + void collect(double v) { + doCollect(BitMixer.mix64(Double.doubleToLongBits(v))); + } + + void collect(BytesRef bytes) { + MurmurHash3.hash128(bytes.bytes, bytes.offset, bytes.length, 0, hash); + collect(hash.h1); + } + + private void doCollect(long hash) { + hll.collect(SINGLE_BUCKET_ORD, hash); + } + + long cardinality() { + return hll.cardinality(SINGLE_BUCKET_ORD); + } + + void merge(int groupId, AbstractHyperLogLogPlusPlus other, int otherGroup) { + hll.merge(groupId, other, otherGroup); + } + + void merge(int groupId, BytesRef other, int otherGroup) { + hll.merge(groupId, deserializeHLL(other), otherGroup); + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 1; + blocks[offset] = new ConstantBytesRefVector(serializeHLL(SINGLE_BUCKET_ORD, hll), 1).asBlock(); + } + + @Override + public void close() { + Releasables.close(hll); + } + } + + static class GroupingState implements GroupingAggregatorState { + + private final MurmurHash3.Hash128 hash = new MurmurHash3.Hash128(); + + final HyperLogLogPlusPlus hll; + + GroupingState(BigArrays bigArrays, int precision) { + this.hll = new HyperLogLogPlusPlus(HyperLogLogPlusPlus.precisionFromThreshold(precision), bigArrays, 1); + } + + void enableGroupIdTracking(SeenGroupIds seenGroupIds) { + // Nothing to do + } + + void collect(int groupId, long v) { + doCollect(groupId, BitMixer.mix64(v)); + } + + void collect(int groupId, int v) { + doCollect(groupId, BitMixer.mix64(v)); + } + + void collect(int groupId, double v) { + doCollect(groupId, BitMixer.mix64(Double.doubleToLongBits(v))); + } + + void collect(int groupId, BytesRef bytes) { + MurmurHash3.hash128(bytes.bytes, bytes.offset, bytes.length, 0, hash); + collect(groupId, hash.h1); + } + + private void doCollect(int groupId, long hash) { + hll.collect(groupId, hash); + } + + long cardinality(int groupId) { + return hll.cardinality(groupId); + } + + void merge(int groupId, AbstractHyperLogLogPlusPlus other, int otherGroup) { + hll.merge(groupId, other, otherGroup); + } + + void merge(int groupId, BytesRef other, int otherGroup) { + hll.merge(groupId, deserializeHLL(other), otherGroup); + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 1; + var builder = BytesRefBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + builder.appendBytesRef(serializeHLL(group, hll)); + } + blocks[offset] = builder.build(); + } + + @Override + public void close() { + Releasables.close(hll); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/IntermediateStateDesc.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/IntermediateStateDesc.java new file mode 100644 index 0000000000000..22766c36953c4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/IntermediateStateDesc.java @@ -0,0 +1,13 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.ElementType; + +/** Intermediate aggregation state descriptor. Intermediate state is a list of these. */ +public record IntermediateStateDesc(String name, ElementType type) {} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregator.java new file mode 100644 index 0000000000000..ee6555c4af67d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "max", type = "DOUBLE"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MaxDoubleAggregator { + + public static double init() { + return Double.MIN_VALUE; + } + + public static double combine(double current, double v) { + return Math.max(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxIntAggregator.java new file mode 100644 index 0000000000000..72646f3a8beac --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxIntAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "max", type = "INT"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MaxIntAggregator { + + public static int init() { + return Integer.MIN_VALUE; + } + + public static int combine(int current, int v) { + return Math.max(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxLongAggregator.java new file mode 100644 index 0000000000000..747d449db44ac --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MaxLongAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "max", type = "LONG"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MaxLongAggregator { + + public static long init() { + return Long.MIN_VALUE; + } + + public static long combine(long current, long v) { + return Math.max(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregator.java new file mode 100644 index 0000000000000..b2b30521ee3a4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class MedianAbsoluteDeviationDoubleAggregator { + + public static QuantileStates.SingleState initSingle() { + return new QuantileStates.SingleState(QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.SingleState current, double v) { + current.add(v); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluateMedianAbsoluteDeviation(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays) { + return new QuantileStates.GroupingState(bigArrays, QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, double v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selected) { + return state.evaluateMedianAbsoluteDeviation(selected); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregator.java new file mode 100644 index 0000000000000..43032c057d769 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class MedianAbsoluteDeviationIntAggregator { + + public static QuantileStates.SingleState initSingle() { + return new QuantileStates.SingleState(QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.SingleState current, int v) { + current.add(v); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluateMedianAbsoluteDeviation(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays) { + return new QuantileStates.GroupingState(bigArrays, QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, int v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selected) { + return state.evaluateMedianAbsoluteDeviation(selected); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregator.java new file mode 100644 index 0000000000000..11c0218b1728e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class MedianAbsoluteDeviationLongAggregator { + + public static QuantileStates.SingleState initSingle() { + return new QuantileStates.SingleState(QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.SingleState current, long v) { + current.add(v); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluateMedianAbsoluteDeviation(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays) { + return new QuantileStates.GroupingState(bigArrays, QuantileStates.MEDIAN); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, long v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selected) { + return state.evaluateMedianAbsoluteDeviation(selected); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinDoubleAggregator.java new file mode 100644 index 0000000000000..521f4f73c0a73 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinDoubleAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "min", type = "DOUBLE"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MinDoubleAggregator { + + public static double init() { + return Double.POSITIVE_INFINITY; + } + + public static double combine(double current, double v) { + return Math.min(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinIntAggregator.java new file mode 100644 index 0000000000000..0edfc3b70a8b0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinIntAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "min", type = "INT"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MinIntAggregator { + + public static int init() { + return Integer.MAX_VALUE; + } + + public static int combine(int current, int v) { + return Math.min(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinLongAggregator.java new file mode 100644 index 0000000000000..4e5dd50a0e78b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/MinLongAggregator.java @@ -0,0 +1,26 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "min", type = "LONG"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class MinLongAggregator { + + public static long init() { + return Long.MAX_VALUE; + } + + public static long combine(long current, long v) { + return Math.min(current, v); + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregator.java new file mode 100644 index 0000000000000..23e82f27602e0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class PercentileDoubleAggregator { + + public static QuantileStates.SingleState initSingle(double percentile) { + return new QuantileStates.SingleState(percentile); + } + + public static void combine(QuantileStates.SingleState current, double v) { + current.add(v); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluatePercentile(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays, double percentile) { + return new QuantileStates.GroupingState(bigArrays, percentile); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, double v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selected) { + return state.evaluatePercentile(selected); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileIntAggregator.java new file mode 100644 index 0000000000000..7f319786454f3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileIntAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class PercentileIntAggregator { + + public static QuantileStates.SingleState initSingle(double percentile) { + return new QuantileStates.SingleState(percentile); + } + + public static void combine(QuantileStates.SingleState current, int v) { + current.add(v); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluatePercentile(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays, double percentile) { + return new QuantileStates.GroupingState(bigArrays, percentile); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, int v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selected) { + return state.evaluatePercentile(selected); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileLongAggregator.java new file mode 100644 index 0000000000000..4611153bc97bf --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/PercentileLongAggregator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; + +@Aggregator({ @IntermediateState(name = "quart", type = "BYTES_REF") }) +@GroupingAggregator +class PercentileLongAggregator { + + public static QuantileStates.SingleState initSingle(double percentile) { + return new QuantileStates.SingleState(percentile); + } + + public static void combine(QuantileStates.SingleState current, long v) { + current.add(v); + } + + public static void combineStates(QuantileStates.SingleState current, QuantileStates.SingleState state) { + current.add(state); + } + + public static void combineIntermediate(QuantileStates.SingleState state, BytesRef inValue) { + state.add(inValue); + } + + public static Block evaluateFinal(QuantileStates.SingleState state) { + return state.evaluatePercentile(); + } + + public static QuantileStates.GroupingState initGrouping(BigArrays bigArrays, double percentile) { + return new QuantileStates.GroupingState(bigArrays, percentile); + } + + public static void combine(QuantileStates.GroupingState state, int groupId, long v) { + state.add(groupId, v); + } + + public static void combineIntermediate(QuantileStates.GroupingState state, int groupId, BytesRef inValue) { + state.add(groupId, inValue); + } + + public static void combineStates( + QuantileStates.GroupingState current, + int currentGroupId, + QuantileStates.GroupingState state, + int statePosition + ) { + current.add(currentGroupId, state.get(statePosition)); + } + + public static Block evaluateFinal(QuantileStates.GroupingState state, IntVector selectedGroups) { + return state.evaluatePercentile(selectedGroups); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/QuantileStates.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/QuantileStates.java new file mode 100644 index 0000000000000..be47e346abc04 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/QuantileStates.java @@ -0,0 +1,218 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.ByteArrayStreamInput; +import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.search.aggregations.metrics.InternalMedianAbsoluteDeviation; +import org.elasticsearch.search.aggregations.metrics.TDigestState; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +public final class QuantileStates { + public static final double MEDIAN = 50.0; + static final double DEFAULT_COMPRESSION = 1000.0; + + private QuantileStates() {} + + private static Double percentileParam(double p) { + // Percentile must be a double between 0 and 100 inclusive + // If percentile parameter is wrong, the aggregation will return NULL + return 0 <= p && p <= 100 ? p : null; + } + + static BytesRef serializeDigest(TDigestState digest) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + OutputStreamStreamOutput out = new OutputStreamStreamOutput(baos); + try { + TDigestState.write(digest, out); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new BytesRef(baos.toByteArray()); + } + + static TDigestState deserializeDigest(BytesRef bytesRef) { + ByteArrayStreamInput in = new ByteArrayStreamInput(bytesRef.bytes); + in.reset(bytesRef.bytes, bytesRef.offset, bytesRef.length); + try { + return TDigestState.read(in); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + static class SingleState implements AggregatorState { + private TDigestState digest; + private final Double percentile; + + SingleState(double percentile) { + this.digest = TDigestState.create(DEFAULT_COMPRESSION); + this.percentile = percentileParam(percentile); + } + + @Override + public void close() {} + + void add(double v) { + digest.add(v); + } + + void add(SingleState other) { + digest.add(other.digest); + } + + void add(BytesRef other) { + digest.add(deserializeDigest(other)); + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 1; + blocks[offset] = new ConstantBytesRefVector(serializeDigest(this.digest), 1).asBlock(); + } + + Block evaluateMedianAbsoluteDeviation() { + assert percentile == MEDIAN : "Median must be 50th percentile [percentile = " + percentile + "]"; + if (digest.size() == 0) { + return Block.constantNullBlock(1); + } + double result = InternalMedianAbsoluteDeviation.computeMedianAbsoluteDeviation(digest); + return DoubleBlock.newConstantBlockWith(result, 1); + } + + Block evaluatePercentile() { + if (percentile == null) { + return DoubleBlock.newBlockBuilder(1).appendNull().build(); + } + if (digest.size() == 0) { + return Block.constantNullBlock(1); + } + double result = digest.quantile(percentile / 100); + return DoubleBlock.newConstantBlockWith(result, 1); + } + } + + static class GroupingState implements GroupingAggregatorState { + private long largestGroupId = -1; + private ObjectArray digests; + private final BigArrays bigArrays; + private final Double percentile; + + GroupingState(BigArrays bigArrays, double percentile) { + this.bigArrays = bigArrays; + this.digests = bigArrays.newObjectArray(1); + this.percentile = percentileParam(percentile); + } + + private TDigestState getOrAddGroup(int groupId) { + digests = bigArrays.grow(digests, groupId + 1); + TDigestState qs = digests.get(groupId); + if (qs == null) { + qs = TDigestState.create(DEFAULT_COMPRESSION); + digests.set(groupId, qs); + } + return qs; + } + + void add(int groupId, double v) { + getOrAddGroup(groupId).add(v); + } + + void add(int groupId, TDigestState other) { + if (other != null) { + getOrAddGroup(groupId).add(other); + } + } + + void enableGroupIdTracking(SeenGroupIds seenGroupIds) { + // We always enable. + } + + void add(int groupId, BytesRef other) { + getOrAddGroup(groupId).add(deserializeDigest(other)); + } + + TDigestState get(int position) { + return digests.get(position); + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 1; + var builder = BytesRefBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + TDigestState state; + if (group < digests.size()) { + state = get(group); + if (state == null) { + state = TDigestState.create(DEFAULT_COMPRESSION); + } + } else { + state = TDigestState.create(DEFAULT_COMPRESSION); + } + builder.appendBytesRef(serializeDigest(state)); + } + blocks[offset] = builder.build(); + } + + Block evaluateMedianAbsoluteDeviation(IntVector selected) { + assert percentile == MEDIAN : "Median must be 50th percentile [percentile = " + percentile + "]"; + final DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int si = selected.getInt(i); + if (si >= digests.size()) { + builder.appendNull(); + continue; + } + final TDigestState digest = digests.get(si); + if (digest != null && digest.size() > 0) { + builder.appendDouble(InternalMedianAbsoluteDeviation.computeMedianAbsoluteDeviation(digest)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + Block evaluatePercentile(IntVector selected) { + final DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int si = selected.getInt(i); + if (si >= digests.size()) { + builder.appendNull(); + continue; + } + final TDigestState digest = digests.get(si); + if (percentile != null && digest != null && digest.size() > 0) { + builder.appendDouble(digest.quantile(percentile / 100)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + @Override + public void close() { + digests.close(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SeenGroupIds.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SeenGroupIds.java new file mode 100644 index 0000000000000..e57bec85d59ba --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SeenGroupIds.java @@ -0,0 +1,38 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; + +public interface SeenGroupIds { + /** + * The grouping ids that have been seen already. This {@link BitArray} is + * kept and mutated by the caller so make a copy if it's something you + * need your own copy of it. + */ + BitArray seenGroupIds(BigArrays bigArrays); + + record Empty() implements SeenGroupIds { + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new BitArray(1, bigArrays); + } + } + + record Range(int from, int to) implements SeenGroupIds { + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + BitArray seen = new BitArray(to - from, bigArrays); + for (int i = from; i < to; i++) { + seen.set(i); + } + return seen; + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumDoubleAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumDoubleAggregator.java new file mode 100644 index 0000000000000..6ea83850f7876 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumDoubleAggregator.java @@ -0,0 +1,213 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +@Aggregator( + { + @IntermediateState(name = "value", type = "DOUBLE"), + @IntermediateState(name = "delta", type = "DOUBLE"), + @IntermediateState(name = "seen", type = "BOOLEAN") } +) +@GroupingAggregator +class SumDoubleAggregator { + + public static SumState initSingle() { + return new SumState(); + } + + public static void combine(SumState current, double v) { + current.add(v); + } + + public static void combine(SumState current, double value, double delta) { + current.add(value, delta); + } + + public static void combineStates(SumState current, SumState state) { + current.add(state.value(), state.delta()); + } + + public static void combineIntermediate(SumState state, double inValue, double inDelta, boolean seen) { + if (seen) { + combine(state, inValue, inDelta); + state.seen(true); + } + } + + public static void evaluateIntermediate(SumState state, Block[] blocks, int offset) { + assert blocks.length >= offset + 3; + blocks[offset + 0] = new ConstantDoubleVector(state.value(), 1).asBlock(); + blocks[offset + 1] = new ConstantDoubleVector(state.delta(), 1).asBlock(); + blocks[offset + 2] = new ConstantBooleanVector(state.seen, 1).asBlock(); + } + + public static Block evaluateFinal(SumState state) { + double result = state.value(); + return DoubleBlock.newConstantBlockWith(result, 1); + } + + public static GroupingSumState initGrouping(BigArrays bigArrays) { + return new GroupingSumState(bigArrays); + } + + public static void combine(GroupingSumState current, int groupId, double v) { + current.add(v, groupId); + } + + public static void combineStates(GroupingSumState current, int groupId, GroupingSumState state, int statePosition) { + if (state.hasValue(statePosition)) { + current.add(state.values.get(statePosition), state.deltas.get(statePosition), groupId); + } + } + + public static void combineIntermediate(GroupingSumState current, int groupId, double inValue, double inDelta, boolean seen) { + if (seen) { + current.add(inValue, inDelta, groupId); + } + } + + public static void evaluateIntermediate(GroupingSumState state, Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 3; + var valuesBuilder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + var deltaBuilder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + var seenBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (group < state.values.size()) { + valuesBuilder.appendDouble(state.values.get(group)); + deltaBuilder.appendDouble(state.deltas.get(group)); + } else { + valuesBuilder.appendDouble(0); + deltaBuilder.appendDouble(0); + } + seenBuilder.appendBoolean(state.hasValue(group)); + } + blocks[offset + 0] = valuesBuilder.build(); + blocks[offset + 1] = deltaBuilder.build(); + blocks[offset + 2] = seenBuilder.build(); + } + + public static Block evaluateFinal(GroupingSumState state, IntVector selected) { + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int si = selected.getInt(i); + if (state.hasValue(si) && si < state.values.size()) { + builder.appendDouble(state.values.get(si)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + static class SumState extends CompensatedSum implements AggregatorState { + + private boolean seen; + + SumState() { + this(0, 0); + } + + SumState(double value, double delta) { + super(value, delta); + } + + @Override + public void toIntermediate(Block[] blocks, int offset) { + SumDoubleAggregator.evaluateIntermediate(this, blocks, offset); + } + + @Override + public void close() {} + + public boolean seen() { + return seen; + } + + public void seen(boolean seen) { + this.seen = seen; + } + } + + static class GroupingSumState extends AbstractArrayState implements GroupingAggregatorState { + static final long BYTES_SIZE = Double.BYTES + Double.BYTES; + + DoubleArray values; + DoubleArray deltas; + + GroupingSumState(BigArrays bigArrays) { + super(bigArrays); + boolean success = false; + try { + this.values = bigArrays.newDoubleArray(1); + this.deltas = bigArrays.newDoubleArray(1); + success = true; + } finally { + if (success == false) { + close(); + } + } + } + + void add(double valueToAdd, int groupId) { + add(valueToAdd, 0d, groupId); + } + + void add(double valueToAdd, double deltaToAdd, int groupId) { + ensureCapacity(groupId); + + // If the value is Inf or NaN, just add it to the running tally to "convert" to + // Inf/NaN. This keeps the behavior bwc from before kahan summing + if (Double.isFinite(valueToAdd) == false) { + values.increment(groupId, valueToAdd); + return; + } + + double value = values.get(groupId); + if (Double.isFinite(value) == false) { + // It isn't going to get any more infinite. + return; + } + double delta = deltas.get(groupId); + double correctedSum = valueToAdd + (delta + deltaToAdd); + double updatedValue = value + correctedSum; + deltas.set(groupId, correctedSum - (updatedValue - value)); + values.set(groupId, updatedValue); + trackGroupId(groupId); + } + + private void ensureCapacity(int groupId) { + values = bigArrays.grow(values, groupId + 1); + deltas = bigArrays.grow(deltas, groupId + 1); + } + + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + SumDoubleAggregator.evaluateIntermediate(this, blocks, offset, selected); + } + + @Override + public void close() { + Releasables.close(values, deltas, () -> super.close()); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumIntAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumIntAggregator.java new file mode 100644 index 0000000000000..5fb91aa7d7ff8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumIntAggregator.java @@ -0,0 +1,29 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "sum", type = "LONG"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class SumIntAggregator { + + public static long init() { + return 0; + } + + public static long combine(long current, int v) { + return Math.addExact(current, v); + } + + public static long combine(long current, long v) { + return Math.addExact(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumLongAggregator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumLongAggregator.java new file mode 100644 index 0000000000000..cd6a94e518be8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/SumLongAggregator.java @@ -0,0 +1,25 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.ann.Aggregator; +import org.elasticsearch.compute.ann.GroupingAggregator; +import org.elasticsearch.compute.ann.IntermediateState; + +@Aggregator({ @IntermediateState(name = "sum", type = "LONG"), @IntermediateState(name = "seen", type = "BOOLEAN") }) +@GroupingAggregator +class SumLongAggregator { + + public static long init() { + return 0; + } + + public static long combine(long current, long v) { + return Math.addExact(current, v); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-ArrayState.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-ArrayState.java.st new file mode 100644 index 0000000000000..45aaa5881683c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-ArrayState.java.st @@ -0,0 +1,124 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.$Type$Array; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +$if(long)$ +import org.elasticsearch.compute.data.IntVector; +$endif$ +import org.elasticsearch.compute.data.$Type$Block; +import org.elasticsearch.compute.data.$Type$Vector; +$if(double)$ +import org.elasticsearch.compute.data.IntVector; +$endif$ +import org.elasticsearch.core.Releasables; + +/** + * Aggregator state for an array of $type$s. It is created in a mode where it + * won't track the {@code groupId}s that are sent to it and it is the + * responsibility of the caller to only fetch values for {@code groupId}s + * that it has sent using the {@code selected} parameter when building the + * results. This is fine when there are no {@code null} values in the input + * data. But once there are null values in the input data it is + * much more convenient to only send non-null values and + * the tracking built into the grouping code can't track that. In that case + * call {@link #enableGroupIdTracking} to transition the state into a mode + * where it'll track which {@code groupIds} have been written. + *

+ * This class is generated. Do not edit it. + *

+ */ +final class $Type$ArrayState extends AbstractArrayState implements GroupingAggregatorState { + private final $type$ init; + + private $Type$Array values; + + $Type$ArrayState(BigArrays bigArrays, $type$ init) { + super(bigArrays); + this.values = bigArrays.new$Type$Array(1, false); + this.values.set(0, init); + this.init = init; + } + + $type$ get(int groupId) { + return values.get(groupId); + } + + $type$ getOrDefault(int groupId) { + return groupId < values.size() ? values.get(groupId) : init; + } + + void set(int groupId, $type$ value) { + ensureCapacity(groupId); + values.set(groupId, value); + trackGroupId(groupId); + } + +$if(long)$ + void increment(int groupId, long value) { + ensureCapacity(groupId); + values.increment(groupId, value); + trackGroupId(groupId); + } +$endif$ + + Block toValuesBlock(org.elasticsearch.compute.data.IntVector selected) { + if (false == trackingGroupIds()) { + $Type$Vector.Builder builder = $Type$Vector.newVectorBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + builder.append$Type$(values.get(selected.getInt(i))); + } + return builder.build().asBlock(); + } + $Type$Block.Builder builder = $Type$Block.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (hasValue(group)) { + builder.append$Type$(values.get(group)); + } else { + builder.appendNull(); + } + } + return builder.build(); + } + + private void ensureCapacity(int groupId) { + if (groupId >= values.size()) { + long prevSize = values.size(); + values = bigArrays.grow(values, groupId + 1); + values.fill(prevSize, values.size(), init); + } + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset, IntVector selected) { + assert blocks.length >= offset + 2; + var valuesBuilder = $Type$Block.newBlockBuilder(selected.getPositionCount()); + var hasValueBuilder = BooleanBlock.newBlockBuilder(selected.getPositionCount()); + for (int i = 0; i < selected.getPositionCount(); i++) { + int group = selected.getInt(i); + if (group < values.size()) { + valuesBuilder.append$Type$(values.get(group)); + } else { + valuesBuilder.append$Type$(0); // TODO can we just use null? + } + hasValueBuilder.appendBoolean(hasValue(group)); + } + blocks[offset + 0] = valuesBuilder.build(); + blocks[offset + 1] = hasValueBuilder.build(); + } + + @Override + public void close() { + Releasables.close(values, super::close); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-State.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-State.java.st new file mode 100644 index 0000000000000..2bcee35b48b4d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/X-State.java.st @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.Constant$Type$Vector; + +/** + * Aggregator state for a single $type$. + * This class is generated. Do not edit it. + */ +final class $Type$State implements AggregatorState { + private $type$ value; + private boolean seen; + + $Type$State() { + this(0); + } + + $Type$State($type$ init) { + this.value = init; + } + + $type$ $type$Value() { + return value; + } + + void $type$Value($type$ value) { + this.value = value; + } + + boolean seen() { + return seen; + } + + void seen(boolean seen) { + this.seen = seen; + } + + /** Extracts an intermediate view of the contents of this state. */ + @Override + public void toIntermediate(Block[] blocks, int offset) { + assert blocks.length >= offset + 2; + blocks[offset + 0] = new Constant$Type$Vector(value, 1).asBlock(); + blocks[offset + 1] = new ConstantBooleanVector(seen, 1).asBlock(); + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java new file mode 100644 index 0000000000000..9106508f7e262 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java @@ -0,0 +1,121 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.common.util.LongLongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.core.Releasable; + +import java.util.List; + +/** + * A specialized hash table implementation maps values of a {@link Block} to ids (in longs). + * This class delegates to {@link LongHash} or {@link BytesRefHash}. + * + * @see LongHash + * @see BytesRefHash + */ +public abstract sealed class BlockHash implements Releasable, SeenGroupIds // + permits BooleanBlockHash, BytesRefBlockHash, DoubleBlockHash, IntBlockHash, LongBlockHash,// + PackedValuesBlockHash, BytesRefLongBlockHash, LongLongBlockHash { + + /** + * Add all values for the "group by" columns in the page to the hash and + * pass the ordinals to the provided {@link GroupingAggregatorFunction.AddInput}. + */ + public abstract void add(Page page, GroupingAggregatorFunction.AddInput addInput); + + /** + * Returns a {@link Block} that contains all the keys that are inserted by {@link #add}. + */ + public abstract Block[] getKeys(); + + /** + * The grouping ids that are not empty. We use this because some block hashes reserve + * space for grouping ids and then don't end up using them. For example, + * {@link BooleanBlockHash} does this by always assigning {@code false} to {@code 0} + * and {@code true} to {@code 1}. It's only after collection when we + * know if there actually were any {@code true} or {@code false} values received. + */ + public abstract IntVector nonEmpty(); + + // TODO merge with nonEmpty + @Override + public abstract BitArray seenGroupIds(BigArrays bigArrays); + + /** + * Creates a specialized hash table that maps one or more {@link Block}s to ids. + * @param emitBatchSize maximum batch size to be emitted when handling combinatorial + * explosion of groups caused by multivalued fields + */ + public static BlockHash build(List groups, BigArrays bigArrays, int emitBatchSize) { + if (groups.size() == 1) { + return newForElementType(groups.get(0).channel(), groups.get(0).elementType(), bigArrays); + } + if (groups.size() == 2) { + var g1 = groups.get(0); + var g2 = groups.get(1); + if (g1.elementType() == ElementType.LONG && g2.elementType() == ElementType.LONG) { + return new LongLongBlockHash(bigArrays, g1.channel(), g2.channel(), emitBatchSize); + } + if (g1.elementType() == ElementType.BYTES_REF && g2.elementType() == ElementType.LONG) { + return new BytesRefLongBlockHash(bigArrays, g1.channel(), g2.channel(), false, emitBatchSize); + } + if (g1.elementType() == ElementType.LONG && g2.elementType() == ElementType.BYTES_REF) { + return new BytesRefLongBlockHash(bigArrays, g2.channel(), g1.channel(), true, emitBatchSize); + } + } + return new PackedValuesBlockHash(groups, bigArrays, emitBatchSize); + } + + /** + * Creates a specialized hash table that maps a {@link Block} of the given input element type to ids. + */ + private static BlockHash newForElementType(int channel, ElementType type, BigArrays bigArrays) { + return switch (type) { + case BOOLEAN -> new BooleanBlockHash(channel); + case INT -> new IntBlockHash(channel, bigArrays); + case LONG -> new LongBlockHash(channel, bigArrays); + case DOUBLE -> new DoubleBlockHash(channel, bigArrays); + case BYTES_REF -> new BytesRefBlockHash(channel, bigArrays); + default -> throw new IllegalArgumentException("unsupported grouping element type [" + type + "]"); + }; + } + + /** + * Convert the result of calling {@link LongHash} or {@link LongLongHash} + * or {@link BytesRefHash} or similar to a group ordinal. These hashes + * return negative numbers if the value that was added has already been + * seen. We don't use that and convert it back to the positive ord. + */ + public static long hashOrdToGroup(long ord) { + if (ord < 0) { // already seen + return -1 - ord; + } + return ord; + } + + /** + * Convert the result of calling {@link LongHash} or {@link LongLongHash} + * or {@link BytesRefHash} or similar to a group ordinal, reserving {@code 0} + * for null. + */ + public static long hashOrdToGroupNullReserved(long ord) { + return hashOrdToGroup(ord) + 1; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java new file mode 100644 index 0000000000000..277a201cfb54a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BooleanBlockHash.java @@ -0,0 +1,114 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.MultivalueDedupeBoolean; + +import static org.elasticsearch.compute.operator.MultivalueDedupeBoolean.FALSE_ORD; +import static org.elasticsearch.compute.operator.MultivalueDedupeBoolean.NULL_ORD; +import static org.elasticsearch.compute.operator.MultivalueDedupeBoolean.TRUE_ORD; + +/** + * Maps a {@link BooleanBlock} column to group ids. Assigns group + * {@code 0} to {@code false} and group {@code 1} to {@code true}. + */ +final class BooleanBlockHash extends BlockHash { + private final int channel; + private final boolean[] everSeen = new boolean[TRUE_ORD + 1]; + + BooleanBlockHash(int channel) { + this.channel = channel; + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + BooleanBlock block = page.getBlock(channel); + BooleanVector vector = block.asVector(); + if (vector == null) { + addInput.add(0, add(block)); + } else { + addInput.add(0, add(vector)); + } + } + + private LongVector add(BooleanVector vector) { + long[] groups = new long[vector.getPositionCount()]; + for (int i = 0; i < vector.getPositionCount(); i++) { + groups[i] = MultivalueDedupeBoolean.hashOrd(everSeen, vector.getBoolean(i)); + } + return new LongArrayVector(groups, groups.length); + } + + private LongBlock add(BooleanBlock block) { + return new MultivalueDedupeBoolean(block).hash(everSeen); + } + + @Override + public BooleanBlock[] getKeys() { + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(everSeen.length); + if (everSeen[NULL_ORD]) { + builder.appendNull(); + } + if (everSeen[FALSE_ORD]) { + builder.appendBoolean(false); + } + if (everSeen[TRUE_ORD]) { + builder.appendBoolean(true); + } + return new BooleanBlock[] { builder.build() }; + } + + @Override + public IntVector nonEmpty() { + IntVector.Builder builder = IntVector.newVectorBuilder(everSeen.length); + for (int i = 0; i < everSeen.length; i++) { + if (everSeen[i]) { + builder.appendInt(i); + } + } + return builder.build(); + } + + public BitArray seenGroupIds(BigArrays bigArrays) { + BitArray seen = new BitArray(everSeen.length, bigArrays); + for (int i = 0; i < everSeen.length; i++) { + if (everSeen[i]) { + seen.set(i); + } + } + return seen; + } + + @Override + public void close() { + // Nothing to close + } + + @Override + public String toString() { + return "BooleanBlockHash{channel=" + + channel + + ", seenFalse=" + + everSeen[FALSE_ORD] + + ", seenTrue=" + + everSeen[TRUE_ORD] + + ", seenNull=" + + everSeen[NULL_ORD] + + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java new file mode 100644 index 0000000000000..3d5ef461c4adc --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefBlockHash.java @@ -0,0 +1,137 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.elasticsearch.compute.operator.MultivalueDedupeBytesRef; + +import java.io.IOException; + +/** + * Maps a {@link BytesRefBlock} column to group ids. + */ +final class BytesRefBlockHash extends BlockHash { + private final BytesRef bytes = new BytesRef(); + private final int channel; + private final BytesRefHash bytesRefHash; + + /** + * Have we seen any {@code null} values? + *

+ * We reserve the 0 ordinal for the {@code null} key so methods like + * {@link #nonEmpty} need to skip 0 if we haven't seen any null values. + *

+ */ + private boolean seenNull; + + BytesRefBlockHash(int channel, BigArrays bigArrays) { + this.channel = channel; + this.bytesRefHash = new BytesRefHash(1, bigArrays); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + BytesRefBlock block = page.getBlock(channel); + BytesRefVector vector = block.asVector(); + if (vector == null) { + addInput.add(0, add(block)); + } else { + addInput.add(0, add(vector)); + } + } + + private LongVector add(BytesRefVector vector) { + long[] groups = new long[vector.getPositionCount()]; + for (int i = 0; i < vector.getPositionCount(); i++) { + groups[i] = hashOrdToGroupNullReserved(bytesRefHash.add(vector.getBytesRef(i, bytes))); + } + return new LongArrayVector(groups, vector.getPositionCount()); + } + + private LongBlock add(BytesRefBlock block) { + MultivalueDedupe.HashResult result = new MultivalueDedupeBytesRef(block).hash(bytesRefHash); + seenNull |= result.sawNull(); + return result.ords(); + } + + @Override + public BytesRefBlock[] getKeys() { + /* + * Create an un-owned copy of the data so we can close our BytesRefHash + * without and still read from the block. + */ + // TODO replace with takeBytesRefsOwnership ?! + + if (seenNull) { + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(Math.toIntExact(bytesRefHash.size() + 1)); + builder.appendNull(); + BytesRef spare = new BytesRef(); + for (long i = 0; i < bytesRefHash.size(); i++) { + builder.appendBytesRef(bytesRefHash.get(i, spare)); + } + return new BytesRefBlock[] { builder.build() }; + } + + final int size = Math.toIntExact(bytesRefHash.size()); + try (BytesStreamOutput out = new BytesStreamOutput()) { + bytesRefHash.getBytesRefs().writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + return new BytesRefBlock[] { + new BytesRefArrayVector(new BytesRefArray(in, BigArrays.NON_RECYCLING_INSTANCE), size).asBlock() }; + } + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(seenNull ? 0 : 1, Math.toIntExact(bytesRefHash.size() + 1)); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(bytesRefHash.size() + 1)).seenGroupIds(bigArrays); + } + + @Override + public void close() { + bytesRefHash.close(); + } + + @Override + public String toString() { + return "BytesRefBlockHash{channel=" + + channel + + ", entries=" + + bytesRefHash.size() + + ", size=" + + ByteSizeValue.ofBytes(bytesRefHash.ramBytesUsed()) + + ", seenNull=" + + seenNull + + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java new file mode 100644 index 0000000000000..aa90d4be69649 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BytesRefLongBlockHash.java @@ -0,0 +1,203 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongLongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasables; + +/** + * Maps a {@link LongBlock} column paired with a {@link BytesRefBlock} column to group ids. + */ +final class BytesRefLongBlockHash extends BlockHash { + private final int channel1; + private final int channel2; + private final boolean reverseOutput; + private final int emitBatchSize; + private final BytesRefHash bytesHash; + private final LongLongHash finalHash; + + BytesRefLongBlockHash(BigArrays bigArrays, int channel1, int channel2, boolean reverseOutput, int emitBatchSize) { + this.channel1 = channel1; + this.channel2 = channel2; + this.reverseOutput = reverseOutput; + this.emitBatchSize = emitBatchSize; + + boolean success = false; + BytesRefHash bytesHash = null; + LongLongHash longHash = null; + try { + bytesHash = new BytesRefHash(1, bigArrays); + longHash = new LongLongHash(1, bigArrays); + this.bytesHash = bytesHash; + this.finalHash = longHash; + success = true; + } finally { + if (success == false) { + Releasables.close(bytesHash); + } + } + } + + @Override + public void close() { + Releasables.close(bytesHash, finalHash); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + BytesRefBlock block1 = page.getBlock(channel1); + LongBlock block2 = page.getBlock(channel2); + BytesRefVector vector1 = block1.asVector(); + LongVector vector2 = block2.asVector(); + if (vector1 != null && vector2 != null) { + addInput.add(0, add(vector1, vector2)); + } else { + new AddBlock(block1, block2, addInput).add(); + } + } + + public LongVector add(BytesRefVector vector1, LongVector vector2) { + BytesRef scratch = new BytesRef(); + int positions = vector1.getPositionCount(); + final long[] ords = new long[positions]; + for (int i = 0; i < positions; i++) { + long hash1 = hashOrdToGroup(bytesHash.add(vector1.getBytesRef(i, scratch))); + ords[i] = hashOrdToGroup(finalHash.add(hash1, vector2.getLong(i))); + } + return new LongArrayVector(ords, positions); + } + + private static final long[] EMPTY = new long[0]; + + private class AddBlock extends LongLongBlockHash.AbstractAddBlock { + private final BytesRefBlock block1; + private final LongBlock block2; + + AddBlock(BytesRefBlock block1, LongBlock block2, GroupingAggregatorFunction.AddInput addInput) { + super(emitBatchSize, addInput); + this.block1 = block1; + this.block2 = block2; + } + + void add() { + BytesRef scratch = new BytesRef(); + int positions = block1.getPositionCount(); + long[] seen1 = EMPTY; + long[] seen2 = EMPTY; + for (int p = 0; p < positions; p++) { + if (block1.isNull(p) || block2.isNull(p)) { + ords.appendNull(); + addedValue(p); + continue; + } + // TODO use MultivalueDedupe + int start1 = block1.getFirstValueIndex(p); + int start2 = block2.getFirstValueIndex(p); + int count1 = block1.getValueCount(p); + int count2 = block2.getValueCount(p); + if (count1 == 1 && count2 == 1) { + long bytesOrd = hashOrdToGroup(bytesHash.add(block1.getBytesRef(start1, scratch))); + ords.appendLong(hashOrdToGroup(finalHash.add(bytesOrd, block2.getLong(start2)))); + addedValue(p); + continue; + } + int end = start1 + count1; + if (seen1.length < count1) { + seen1 = new long[ArrayUtil.oversize(count1, Long.BYTES)]; + } + int seenSize1 = 0; + for (int i = start1; i < end; i++) { + long bytesOrd = bytesHash.add(block1.getBytesRef(i, scratch)); + if (bytesOrd < 0) { // already seen + seenSize1 = LongLongBlockHash.add(seen1, seenSize1, -1 - bytesOrd); + } else { + seen1[seenSize1++] = bytesOrd; + } + } + if (seen2.length < count2) { + seen2 = new long[ArrayUtil.oversize(count2, Long.BYTES)]; + } + int seenSize2 = 0; + end = start2 + count2; + for (int i = start2; i < end; i++) { + seenSize2 = LongLongBlockHash.add(seen2, seenSize2, block2.getLong(i)); + } + if (seenSize1 == 1 && seenSize2 == 1) { + ords.appendLong(hashOrdToGroup(finalHash.add(seen1[0], seen2[0]))); + addedValue(p); + continue; + } + ords.beginPositionEntry(); + for (int s1 = 0; s1 < seenSize1; s1++) { + for (int s2 = 0; s2 < seenSize2; s2++) { + ords.appendLong(hashOrdToGroup(finalHash.add(seen1[s1], seen2[s2]))); + addedValueInMultivaluePosition(p); + } + } + ords.endPositionEntry(); + } + emitOrds(); + } + } + + @Override + public Block[] getKeys() { + int positions = (int) finalHash.size(); + BytesRefVector.Builder keys1 = BytesRefVector.newVectorBuilder(positions); + LongVector.Builder keys2 = LongVector.newVectorBuilder(positions); + BytesRef scratch = new BytesRef(); + for (long i = 0; i < positions; i++) { + keys2.appendLong(finalHash.getKey2(i)); + long h1 = finalHash.getKey1(i); + keys1.appendBytesRef(bytesHash.get(h1, scratch)); + } + if (reverseOutput) { + return new Block[] { keys2.build().asBlock(), keys1.build().asBlock() }; + } else { + return new Block[] { keys1.build().asBlock(), keys2.build().asBlock() }; + } + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(0, Math.toIntExact(finalHash.size())).seenGroupIds(bigArrays); + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(0, Math.toIntExact(finalHash.size())); + } + + @Override + public String toString() { + return "BytesRefLongBlockHash{keys=[BytesRefKey[channel=" + + channel1 + + "], LongKey[channel=" + + channel2 + + "]], entries=" + + finalHash.size() + + ", size=" + + bytesHash.ramBytesUsed() + + "b}"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java new file mode 100644 index 0000000000000..79c748e7901a5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/DoubleBlockHash.java @@ -0,0 +1,118 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.elasticsearch.compute.operator.MultivalueDedupeDouble; + +import java.util.BitSet; + +/** + * Maps a {@link DoubleBlock} column to group ids. + */ +final class DoubleBlockHash extends BlockHash { + private final int channel; + private final LongHash longHash; + + /** + * Have we seen any {@code null} values? + *

+ * We reserve the 0 ordinal for the {@code null} key so methods like + * {@link #nonEmpty} need to skip 0 if we haven't seen any null values. + *

+ */ + private boolean seenNull; + + DoubleBlockHash(int channel, BigArrays bigArrays) { + this.channel = channel; + this.longHash = new LongHash(1, bigArrays); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + DoubleBlock block = page.getBlock(channel); + DoubleVector vector = block.asVector(); + if (vector == null) { + addInput.add(0, add(block)); + } else { + addInput.add(0, add(vector)); + } + } + + private LongVector add(DoubleVector vector) { + long[] groups = new long[vector.getPositionCount()]; + for (int i = 0; i < vector.getPositionCount(); i++) { + groups[i] = hashOrdToGroupNullReserved(longHash.add(Double.doubleToLongBits(vector.getDouble(i)))); + } + return new LongArrayVector(groups, groups.length); + } + + private LongBlock add(DoubleBlock block) { + MultivalueDedupe.HashResult result = new MultivalueDedupeDouble(block).hash(longHash); + seenNull |= result.sawNull(); + return result.ords(); + } + + @Override + public DoubleBlock[] getKeys() { + if (seenNull) { + final int size = Math.toIntExact(longHash.size() + 1); + final double[] keys = new double[size]; + for (int i = 1; i < size; i++) { + keys[i] = Double.longBitsToDouble(longHash.get(i - 1)); + } + BitSet nulls = new BitSet(1); + nulls.set(0); + return new DoubleBlock[] { new DoubleArrayBlock(keys, keys.length, null, nulls, Block.MvOrdering.ASCENDING) }; + } + + final int size = Math.toIntExact(longHash.size()); + final double[] keys = new double[size]; + for (int i = 0; i < size; i++) { + keys[i] = Double.longBitsToDouble(longHash.get(i)); + } + + // TODO claim the array and wrap? + return new DoubleBlock[] { new DoubleArrayVector(keys, keys.length).asBlock() }; + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)).seenGroupIds(bigArrays); + } + + @Override + public void close() { + longHash.close(); + } + + @Override + public String toString() { + return "DoubleBlockHash{channel=" + channel + ", entries=" + longHash.size() + ", seenNull=" + seenNull + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java new file mode 100644 index 0000000000000..b4e991cebbe47 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/IntBlockHash.java @@ -0,0 +1,113 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.elasticsearch.compute.operator.MultivalueDedupeInt; + +import java.util.BitSet; + +/** + * Maps a {@link IntBlock} column to group ids. + */ +final class IntBlockHash extends BlockHash { + private final int channel; + private final LongHash longHash; + /** + * Have we seen any {@code null} values? + *

+ * We reserve the 0 ordinal for the {@code null} key so methods like + * {@link #nonEmpty} need to skip 0 if we haven't seen any null values. + *

+ */ + private boolean seenNull; + + IntBlockHash(int channel, BigArrays bigArrays) { + this.channel = channel; + this.longHash = new LongHash(1, bigArrays); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + IntBlock block = page.getBlock(channel); + IntVector vector = block.asVector(); + if (vector == null) { + addInput.add(0, add(block)); + } else { + addInput.add(0, add(vector)); + } + } + + private LongVector add(IntVector vector) { + long[] groups = new long[vector.getPositionCount()]; + for (int i = 0; i < vector.getPositionCount(); i++) { + groups[i] = hashOrdToGroupNullReserved(longHash.add(vector.getInt(i))); + } + return new LongArrayVector(groups, groups.length); + } + + private LongBlock add(IntBlock block) { + MultivalueDedupe.HashResult result = new MultivalueDedupeInt(block).hash(longHash); + seenNull |= result.sawNull(); + return result.ords(); + } + + @Override + public IntBlock[] getKeys() { + if (seenNull) { + final int size = Math.toIntExact(longHash.size() + 1); + final int[] keys = new int[size]; + for (int i = 1; i < size; i++) { + keys[i] = (int) longHash.get(i - 1); + } + BitSet nulls = new BitSet(1); + nulls.set(0); + return new IntBlock[] { new IntArrayBlock(keys, keys.length, null, nulls, Block.MvOrdering.ASCENDING) }; + } + final int size = Math.toIntExact(longHash.size()); + final int[] keys = new int[size]; + for (int i = 0; i < size; i++) { + keys[i] = (int) longHash.get(i); + } + return new IntBlock[] { new IntArrayVector(keys, keys.length).asBlock() }; + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)).seenGroupIds(bigArrays); + } + + @Override + public void close() { + longHash.close(); + } + + @Override + public String toString() { + return "IntBlockHash{channel=" + channel + ", entries=" + longHash.size() + ", seenNull=" + seenNull + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java new file mode 100644 index 0000000000000..d5e57171e9c71 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongBlockHash.java @@ -0,0 +1,115 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.elasticsearch.compute.operator.MultivalueDedupeLong; + +import java.util.BitSet; + +/** + * Maps {@link LongBlock} to group ids. + */ +final class LongBlockHash extends BlockHash { + private final int channel; + private final LongHash longHash; + + /** + * Have we seen any {@code null} values? + *

+ * We reserve the 0 ordinal for the {@code null} key so methods like + * {@link #nonEmpty} need to skip 0 if we haven't seen any null values. + *

+ */ + private boolean seenNull; + + LongBlockHash(int channel, BigArrays bigArrays) { + this.channel = channel; + this.longHash = new LongHash(1, bigArrays); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + LongBlock block = page.getBlock(channel); + LongVector vector = block.asVector(); + if (vector == null) { + addInput.add(0, add(block)); + } else { + addInput.add(0, add(vector)); + } + } + + private LongVector add(LongVector vector) { + long[] groups = new long[vector.getPositionCount()]; + for (int i = 0; i < vector.getPositionCount(); i++) { + groups[i] = hashOrdToGroupNullReserved(longHash.add(vector.getLong(i))); + } + return new LongArrayVector(groups, groups.length); + } + + private LongBlock add(LongBlock block) { + MultivalueDedupe.HashResult result = new MultivalueDedupeLong(block).hash(longHash); + seenNull |= result.sawNull(); + return result.ords(); + } + + @Override + public LongBlock[] getKeys() { + if (seenNull) { + final int size = Math.toIntExact(longHash.size() + 1); + final long[] keys = new long[size]; + for (int i = 1; i < size; i++) { + keys[i] = longHash.get(i - 1); + } + BitSet nulls = new BitSet(1); + nulls.set(0); + return new LongBlock[] { new LongArrayBlock(keys, keys.length, null, nulls, Block.MvOrdering.ASCENDING) }; + } + + final int size = Math.toIntExact(longHash.size()); + final long[] keys = new long[size]; + for (int i = 0; i < size; i++) { + keys[i] = longHash.get(i); + } + + // TODO call something like takeKeyOwnership to claim the keys array directly + return new LongBlock[] { new LongArrayVector(keys, keys.length).asBlock() }; + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(longHash.size() + 1)).seenGroupIds(bigArrays); + } + + @Override + public void close() { + longHash.close(); + } + + @Override + public String toString() { + return "LongBlockHash{channel=" + channel + ", entries=" + longHash.size() + ", seenNull=" + seenNull + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java new file mode 100644 index 0000000000000..e20373fff0a65 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/LongLongBlockHash.java @@ -0,0 +1,209 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.LongLongHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasables; + +/** + * Maps two {@link LongBlock} columns to group ids. + */ +final class LongLongBlockHash extends BlockHash { + private final int channel1; + private final int channel2; + private final int emitBatchSize; + private final LongLongHash hash; + + LongLongBlockHash(BigArrays bigArrays, int channel1, int channel2, int emitBatchSize) { + this.channel1 = channel1; + this.channel2 = channel2; + this.emitBatchSize = emitBatchSize; + this.hash = new LongLongHash(1, bigArrays); + } + + @Override + public void close() { + Releasables.close(hash); + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + LongBlock block1 = page.getBlock(channel1); + LongBlock block2 = page.getBlock(channel2); + LongVector vector1 = block1.asVector(); + LongVector vector2 = block2.asVector(); + if (vector1 != null && vector2 != null) { + addInput.add(0, add(vector1, vector2)); + } else { + new AddBlock(block1, block2, addInput).add(); + } + } + + private LongVector add(LongVector vector1, LongVector vector2) { + int positions = vector1.getPositionCount(); + final long[] ords = new long[positions]; + for (int i = 0; i < positions; i++) { + ords[i] = hashOrdToGroup(hash.add(vector1.getLong(i), vector2.getLong(i))); + } + return new LongArrayVector(ords, positions); + } + + private static final long[] EMPTY = new long[0]; + + private class AddBlock extends AbstractAddBlock { + private final LongBlock block1; + private final LongBlock block2; + + AddBlock(LongBlock block1, LongBlock block2, GroupingAggregatorFunction.AddInput addInput) { + super(emitBatchSize, addInput); + this.block1 = block1; + this.block2 = block2; + } + + void add() { + int positions = block1.getPositionCount(); + long[] seen1 = EMPTY; + long[] seen2 = EMPTY; + for (int p = 0; p < positions; p++) { + if (block1.isNull(p) || block2.isNull(p)) { + ords.appendNull(); + addedValue(p); + continue; + } + // TODO use MultivalueDedupe + int start1 = block1.getFirstValueIndex(p); + int start2 = block2.getFirstValueIndex(p); + int count1 = block1.getValueCount(p); + int count2 = block2.getValueCount(p); + if (count1 == 1 && count2 == 1) { + ords.appendLong(hashOrdToGroup(hash.add(block1.getLong(start1), block2.getLong(start2)))); + addedValue(p); + continue; + } + int end = start1 + count1; + if (seen1.length < count1) { + seen1 = new long[ArrayUtil.oversize(count1, Long.BYTES)]; + } + int seenSize1 = 0; + for (int i = start1; i < end; i++) { + seenSize1 = LongLongBlockHash.add(seen1, seenSize1, block1.getLong(i)); + } + if (seen2.length < count2) { + seen2 = new long[ArrayUtil.oversize(count2, Long.BYTES)]; + } + int seenSize2 = 0; + end = start2 + count2; + for (int i = start2; i < end; i++) { + seenSize2 = LongLongBlockHash.add(seen2, seenSize2, block2.getLong(i)); + } + if (seenSize1 == 1 && seenSize2 == 1) { + ords.appendLong(hashOrdToGroup(hash.add(seen1[0], seen2[0]))); + addedValue(p); + continue; + } + ords.beginPositionEntry(); + for (int s1 = 0; s1 < seenSize1; s1++) { + for (int s2 = 0; s2 < seenSize2; s2++) { + ords.appendLong(hashOrdToGroup(hash.add(seen1[s1], seen2[s2]))); + addedValueInMultivaluePosition(p); + } + } + ords.endPositionEntry(); + } + emitOrds(); + } + } + + static class AbstractAddBlock { + private final int emitBatchSize; + private final GroupingAggregatorFunction.AddInput addInput; + + private int positionOffset = 0; + private int added = 0; + protected LongBlock.Builder ords; + + AbstractAddBlock(int emitBatchSize, GroupingAggregatorFunction.AddInput addInput) { + this.emitBatchSize = emitBatchSize; + this.addInput = addInput; + + this.ords = LongBlock.newBlockBuilder(emitBatchSize); + } + + protected final void addedValue(int position) { + if (++added % emitBatchSize == 0) { + rollover(position + 1); + } + } + + protected final void addedValueInMultivaluePosition(int position) { + if (++added % emitBatchSize == 0) { + ords.endPositionEntry(); + rollover(position); + ords.beginPositionEntry(); + } + } + + protected final void emitOrds() { + addInput.add(positionOffset, ords.build()); + } + + private void rollover(int position) { + emitOrds(); + positionOffset = position; + ords = LongBlock.newBlockBuilder(emitBatchSize); // TODO add a clear method to the builder? + } + } + + static int add(long[] seen, int nextSeen, long v) { + for (int c = 0; c < nextSeen; c++) { + if (seen[c] == v) { + return nextSeen; + } + } + seen[nextSeen] = v; + return nextSeen + 1; + } + + @Override + public Block[] getKeys() { + int positions = (int) hash.size(); + LongVector.Builder keys1 = LongVector.newVectorBuilder(positions); + LongVector.Builder keys2 = LongVector.newVectorBuilder(positions); + for (long i = 0; i < positions; i++) { + keys1.appendLong(hash.getKey1(i)); + keys2.appendLong(hash.getKey2(i)); + } + return new Block[] { keys1.build().asBlock(), keys2.build().asBlock() }; + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(0, Math.toIntExact(hash.size())); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(0, Math.toIntExact(hash.size())).seenGroupIds(bigArrays); + } + + @Override + public String toString() { + return "LongLongBlockHash{channels=[" + channel1 + "," + channel2 + "], entries=" + hash.size() + "}"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java new file mode 100644 index 0000000000000..92b9be552e86b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/PackedValuesBlockHash.java @@ -0,0 +1,286 @@ +/* + * 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.compute.aggregation.blockhash; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.BatchEncoder; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.MultivalueDedupe; + +import java.util.Arrays; +import java.util.List; + +/** + * Maps any number of columns to a group ids with every unique combination resulting + * in a unique group id. Works by uniqing the values of each column and concatenating + * the combinatorial explosion of all values into a byte array and then hashing each + * byte array. If the values are + *
{@code
+ *     a=(1, 2, 3) b=(2, 3) c=(4, 5, 5)
+ * }
+ * Then you get these grouping keys: + *
{@code
+ *     1, 2, 4
+ *     1, 2, 5
+ *     1, 3, 4
+ *     1, 3, 5
+ *     2, 2, 4
+ *     2, 2, 5
+ *     2, 3, 4
+ *     2, 3, 5
+ *     3, 2, 4
+ *     3, 3, 5
+ * }
+ */ +final class PackedValuesBlockHash extends BlockHash { + private static final Logger logger = LogManager.getLogger(PackedValuesBlockHash.class); + static final int DEFAULT_BATCH_SIZE = Math.toIntExact(ByteSizeValue.ofKb(10).getBytes()); + + private final List groups; + private final int emitBatchSize; + private final BytesRefHash bytesRefHash; + private final int nullTrackingBytes; + + PackedValuesBlockHash(List groups, BigArrays bigArrays, int emitBatchSize) { + this.groups = groups; + this.emitBatchSize = emitBatchSize; + this.bytesRefHash = new BytesRefHash(1, bigArrays); + this.nullTrackingBytes = groups.size() / 8 + 1; + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + add(page, addInput, DEFAULT_BATCH_SIZE); + } + + void add(Page page, GroupingAggregatorFunction.AddInput addInput, int batchSize) { + new AddWork(page, addInput, batchSize).add(); + } + + class AddWork extends LongLongBlockHash.AbstractAddBlock { + final BatchEncoder[] encoders = new BatchEncoder[groups.size()]; + final int[] positionOffsets = new int[groups.size()]; + final int[] valueOffsets = new int[groups.size()]; + final BytesRef[] scratches = new BytesRef[groups.size()]; + final BytesRefBuilder bytes = new BytesRefBuilder(); + final int positionCount; + + int position; + int count; + long bufferedGroup; + + AddWork(Page page, GroupingAggregatorFunction.AddInput addInput, int batchSize) { + super(emitBatchSize, addInput); + for (int g = 0; g < groups.size(); g++) { + encoders[g] = MultivalueDedupe.batchEncoder(page.getBlock(groups.get(g).channel()), batchSize); + scratches[g] = new BytesRef(); + } + bytes.grow(nullTrackingBytes); + this.positionCount = page.getPositionCount(); + } + + /** + * Encodes one permutation of the keys at time into {@link #bytes}. The encoding is + * mostly provided by {@link BatchEncoder} with nulls living in a bit mask at the + * front of the bytes. + */ + void add() { + for (position = 0; position < positionCount; position++) { + if (logger.isTraceEnabled()) { + logger.trace("position {}", position); + } + // Make sure all encoders have encoded the current position and the offsets are queued to it's start + for (int g = 0; g < encoders.length; g++) { + positionOffsets[g]++; + while (positionOffsets[g] >= encoders[g].positionCount()) { + encoders[g].encodeNextBatch(); + positionOffsets[g] = 0; + valueOffsets[g] = 0; + } + } + + count = 0; + Arrays.fill(bytes.bytes(), 0, nullTrackingBytes, (byte) 0); + bytes.setLength(nullTrackingBytes); + addPosition(0); + switch (count) { + case 0 -> throw new IllegalStateException("didn't find any values"); + case 1 -> { + ords.appendLong(bufferedGroup); + addedValue(position); + } + default -> ords.endPositionEntry(); + } + for (int g = 0; g < encoders.length; g++) { + valueOffsets[g] += encoders[g].valueCount(positionOffsets[g]); + } + } + emitOrds(); + } + + private void addPosition(int g) { + if (g == groups.size()) { + addBytes(); + return; + } + int start = bytes.length(); + int count = encoders[g].valueCount(positionOffsets[g]); + assert count > 0; + int valueOffset = valueOffsets[g]; + BytesRef v = encoders[g].read(valueOffset++, scratches[g]); + if (logger.isTraceEnabled()) { + logger.trace("\t".repeat(g + 1) + v); + } + if (v.length == 0) { + assert count == 1 : "null value in non-singleton list"; + int nullByte = g / 8; + int nullShift = g % 8; + bytes.bytes()[nullByte] |= (byte) (1 << nullShift); + } + bytes.setLength(start); + bytes.append(v); + addPosition(g + 1); // TODO stack overflow protection + for (int i = 1; i < count; i++) { + v = encoders[g].read(valueOffset++, scratches[g]); + if (logger.isTraceEnabled()) { + logger.trace("\t".repeat(g + 1) + v); + } + assert v.length > 0 : "null value after the first position"; + bytes.setLength(start); + bytes.append(v); + addPosition(g + 1); + } + } + + private void addBytes() { + long group = hashOrdToGroup(bytesRefHash.add(bytes.get())); + switch (count) { + case 0 -> bufferedGroup = group; + case 1 -> { + ords.beginPositionEntry(); + ords.appendLong(bufferedGroup); + addedValueInMultivaluePosition(position); + ords.appendLong(group); + addedValueInMultivaluePosition(position); + } + default -> { + ords.appendLong(group); + addedValueInMultivaluePosition(position); + } + } + count++; + if (logger.isTraceEnabled()) { + logger.trace("{} = {}", bytes.get(), group); + } + } + } + + @Override + public Block[] getKeys() { + int size = Math.toIntExact(bytesRefHash.size()); + BatchEncoder.Decoder[] decoders = new BatchEncoder.Decoder[groups.size()]; + Block.Builder[] builders = new Block.Builder[groups.size()]; + for (int g = 0; g < builders.length; g++) { + ElementType elementType = groups.get(g).elementType(); + decoders[g] = BatchEncoder.decoder(elementType); + builders[g] = elementType.newBlockBuilder(size); + } + + BytesRef values[] = new BytesRef[(int) Math.min(100, bytesRefHash.size())]; + BytesRef nulls[] = new BytesRef[values.length]; + for (int offset = 0; offset < values.length; offset++) { + values[offset] = new BytesRef(); + nulls[offset] = new BytesRef(); + nulls[offset].length = nullTrackingBytes; + } + int offset = 0; + for (int i = 0; i < bytesRefHash.size(); i++) { + values[offset] = bytesRefHash.get(i, values[offset]); + + // Reference the null bytes in the nulls array and values in the values + nulls[offset].bytes = values[offset].bytes; + nulls[offset].offset = values[offset].offset; + values[offset].offset += nullTrackingBytes; + values[offset].length -= nullTrackingBytes; + + offset++; + if (offset == values.length) { + readKeys(decoders, builders, nulls, values, offset); + offset = 0; + } + } + if (offset > 0) { + readKeys(decoders, builders, nulls, values, offset); + } + + Block[] keyBlocks = new Block[groups.size()]; + for (int g = 0; g < keyBlocks.length; g++) { + keyBlocks[g] = builders[g].build(); + } + return keyBlocks; + } + + private void readKeys(BatchEncoder.Decoder[] decoders, Block.Builder[] builders, BytesRef[] nulls, BytesRef[] values, int count) { + for (int g = 0; g < builders.length; g++) { + int nullByte = g / 8; + int nullShift = g % 8; + byte nullTest = (byte) (1 << nullShift); + BatchEncoder.IsNull isNull = offset -> { + BytesRef n = nulls[offset]; + return (n.bytes[n.offset + nullByte] & nullTest) != 0; + }; + decoders[g].decode(builders[g], isNull, values, count); + } + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(0, Math.toIntExact(bytesRefHash.size())); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(0, Math.toIntExact(bytesRefHash.size())).seenGroupIds(bigArrays); + } + + @Override + public void close() { + bytesRefHash.close(); + } + + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append("PackedValuesBlockHash{groups=["); + boolean first = true; + for (HashAggregationOperator.GroupSpec spec : groups) { + if (first) { + first = false; + } else { + b.append(", "); + } + b.append(spec.channel()).append(':').append(spec.elementType()); + } + b.append("], entries=").append(bytesRefHash.size()); + b.append(", size=").append(ByteSizeValue.ofBytes(bytesRefHash.ramBytesUsed())); + return b.append("}").toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java new file mode 100644 index 0000000000000..8fb91e4a07a5c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractArrayBlock.java @@ -0,0 +1,56 @@ +/* + * 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.compute.data; + +import org.elasticsearch.core.Nullable; + +import java.util.BitSet; + +abstract class AbstractArrayBlock extends AbstractBlock { + + private final MvOrdering mvOrdering; + + /** + * @param positionCount the number of values in this block + */ + protected AbstractArrayBlock(int positionCount, MvOrdering mvOrdering) { + super(positionCount); + this.mvOrdering = mvOrdering; + } + + /** + * @param positionCount the number of values in this block + */ + protected AbstractArrayBlock(int positionCount, @Nullable int[] firstValueIndexes, @Nullable BitSet nullsMask, MvOrdering mvOrdering) { + super(positionCount, firstValueIndexes, nullsMask); + this.mvOrdering = mvOrdering; + } + + @Override + public boolean mayHaveMultivaluedFields() { + /* + * This could return a false positive if all the indices are one away from + * each other. But we will try to avoid that. + */ + return firstValueIndexes != null; + } + + @Override + public final MvOrdering mvOrdering() { + return mvOrdering; + } + + protected BitSet shiftNullsToExpandedPositions() { + BitSet expanded = new BitSet(getTotalValueCount()); + int next = -1; + while ((next = nullsMask.nextSetBit(next + 1)) != -1) { + expanded.set(getFirstValueIndex(next)); + } + return expanded; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java new file mode 100644 index 0000000000000..2d8d75f6c3972 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlock.java @@ -0,0 +1,88 @@ +/* + * 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.compute.data; + +import org.elasticsearch.core.Nullable; + +import java.util.BitSet; + +abstract class AbstractBlock implements Block { + + private final int positionCount; + + @Nullable + protected final int[] firstValueIndexes; + + @Nullable + protected final BitSet nullsMask; + + /** + * @param positionCount the number of values in this block + */ + protected AbstractBlock(int positionCount) { + assert positionCount >= 0; + this.positionCount = positionCount; + this.firstValueIndexes = null; + this.nullsMask = null; + } + + /** + * @param positionCount the number of values in this block + */ + protected AbstractBlock(int positionCount, @Nullable int[] firstValueIndexes, @Nullable BitSet nullsMask) { + assert positionCount >= 0; + this.positionCount = positionCount; + this.firstValueIndexes = firstValueIndexes; + this.nullsMask = nullsMask == null || nullsMask.isEmpty() ? null : nullsMask; + assert (firstValueIndexes == null && this.nullsMask == null) == false; + } + + @Override + public int getTotalValueCount() { + if (firstValueIndexes == null) { + return positionCount - nullValuesCount(); + } + return firstValueIndexes[positionCount] - nullValuesCount(); + } + + @Override + public final int getPositionCount() { + return positionCount; + } + + /** Gets the index of the first value for the given position. */ + public int getFirstValueIndex(int position) { + return firstValueIndexes == null ? position : firstValueIndexes[position]; + } + + /** Gets the number of values for the given position, possibly 0. */ + @Override + public int getValueCount(int position) { + return isNull(position) ? 0 : firstValueIndexes == null ? 1 : firstValueIndexes[position + 1] - firstValueIndexes[position]; + } + + @Override + public boolean isNull(int position) { + return mayHaveNulls() && nullsMask.get(position); + } + + @Override + public boolean mayHaveNulls() { + return nullsMask != null; + } + + @Override + public int nullValuesCount() { + return mayHaveNulls() ? nullsMask.cardinality() : 0; + } + + @Override + public boolean areAllValuesNull() { + return nullValuesCount() == getPositionCount(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlockBuilder.java new file mode 100644 index 0000000000000..95de2a05e4145 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractBlockBuilder.java @@ -0,0 +1,124 @@ +/* + * 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.compute.data; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.stream.IntStream; + +abstract class AbstractBlockBuilder implements Block.Builder { + + protected int[] firstValueIndexes; // lazily initialized, if multi-values + + protected BitSet nullsMask; // lazily initialized, if sparse + + protected int valueCount; + + protected int positionCount; + + protected boolean positionEntryIsOpen; + + protected boolean hasNonNullValue; + + protected Block.MvOrdering mvOrdering = Block.MvOrdering.UNORDERED; + + protected AbstractBlockBuilder() {} + + @Override + public AbstractBlockBuilder appendNull() { + if (positionEntryIsOpen) { + endPositionEntry(); + } + ensureCapacity(); + if (nullsMask == null) { + nullsMask = new BitSet(); + } + nullsMask.set(positionCount); + if (firstValueIndexes != null) { + setFirstValue(positionCount, valueCount); + } + positionCount++; + writeNullValue(); + valueCount++; + return this; + } + + protected void writeNullValue() {} // default is a no-op for array backed builders - since they have default value. + + /** The length of the internal values array. */ + protected abstract int valuesLength(); + + @Override + public AbstractBlockBuilder beginPositionEntry() { + if (firstValueIndexes == null) { + firstValueIndexes = new int[positionCount + 1]; + IntStream.range(0, positionCount).forEach(i -> firstValueIndexes[i] = i); + } + if (positionEntryIsOpen) { + endPositionEntry(); + } + positionEntryIsOpen = true; + setFirstValue(positionCount, valueCount); + return this; + } + + public AbstractBlockBuilder endPositionEntry() { + positionCount++; + positionEntryIsOpen = false; + return this; + } + + protected final boolean isDense() { + return nullsMask == null; + } + + protected final boolean singleValued() { + return firstValueIndexes == null; + } + + protected final void updatePosition() { + if (positionEntryIsOpen == false) { + if (firstValueIndexes != null) { + setFirstValue(positionCount, valueCount - 1); + } + positionCount++; + } + } + + protected final void finish() { + if (positionEntryIsOpen) { + endPositionEntry(); + } + if (firstValueIndexes != null) { + setFirstValue(positionCount, valueCount); + } + } + + protected abstract void growValuesArray(int newSize); + + protected final void ensureCapacity() { + int valuesLength = valuesLength(); + if (valueCount < valuesLength) { + return; + } + int newSize = calculateNewArraySize(valuesLength); + growValuesArray(newSize); + } + + static int calculateNewArraySize(int currentSize) { + // trivially, grows array by 50% + return currentSize + (currentSize >> 1); + } + + private void setFirstValue(int position, int value) { + if (position >= firstValueIndexes.length) { + firstValueIndexes = Arrays.copyOf(firstValueIndexes, position + 1); + } + firstValueIndexes[position] = value; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterBlock.java new file mode 100644 index 0000000000000..6ab1ea2063722 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterBlock.java @@ -0,0 +1,118 @@ +/* + * 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.compute.data; + +import java.util.Arrays; + +abstract class AbstractFilterBlock implements Block { + + protected final int[] positions; + + private final Block block; + + AbstractFilterBlock(Block block, int[] positions) { + this.positions = positions; + this.block = block; + } + + @Override + public ElementType elementType() { + return block.elementType(); + } + + @Override + public boolean isNull(int position) { + return block.isNull(mapPosition(position)); + } + + @Override + public boolean mayHaveNulls() { + return block.mayHaveNulls(); + } + + @Override + public boolean areAllValuesNull() { + return block.areAllValuesNull(); + } + + @Override + public boolean mayHaveMultivaluedFields() { + /* + * This could return a false positive. The block may have multivalued + * fields, but we're not pointing to any of them. That's acceptable. + */ + return block.mayHaveMultivaluedFields(); + } + + @Override + public final int nullValuesCount() { + if (mayHaveNulls() == false) { + return 0; + } else if (areAllValuesNull()) { + return getPositionCount(); + } else { + int nulls = 0; + for (int i = 0; i < getPositionCount(); i++) { + if (isNull(i)) { + nulls++; + } + } + return nulls; + } + } + + @Override + public final int getTotalValueCount() { + if (positions.length == block.getPositionCount()) { + // All the positions are still in the block, just jumbled. + return block.getTotalValueCount(); + } + // TODO this is expensive. maybe cache or something. + int total = 0; + for (int p = 0; p < positions.length; p++) { + total += getValueCount(p); + } + return total; + } + + @Override + public final int getValueCount(int position) { + return block.getValueCount(mapPosition(position)); + } + + @Override + public final int getPositionCount() { + return positions.length; + } + + @Override + public final int getFirstValueIndex(int position) { + return block.getFirstValueIndex(mapPosition(position)); + } + + @Override + public MvOrdering mvOrdering() { + return block.mvOrdering(); + } + + private int mapPosition(int position) { + assert assertPosition(position); + return positions[position]; + } + + @Override + public String toString() { + return "FilteredBlock{" + "positions=" + Arrays.toString(positions) + ", block=" + block + '}'; + } + + protected final boolean assertPosition(int position) { + assert (position >= 0 || position < getPositionCount()) + : "illegal position, " + position + ", position count:" + getPositionCount(); + return true; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterVector.java new file mode 100644 index 0000000000000..03e73224564d9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractFilterVector.java @@ -0,0 +1,28 @@ +/* + * 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.compute.data; + +/** + * Wraps another vector and only allows access to positions that have not been filtered out. + * + * To ensure fast access, the filter is implemented as an array of positions that map positions in + * the filtered block to positions in the wrapped vector. + */ +abstract class AbstractFilterVector extends AbstractVector { + + private final int[] positions; + + protected AbstractFilterVector(int[] positions) { + super(positions.length); + this.positions = positions; + } + + protected int mapPosition(int position) { + return positions[position]; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVector.java new file mode 100644 index 0000000000000..6b7ef080ae5a3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVector.java @@ -0,0 +1,29 @@ +/* + * 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.compute.data; + +/** + * A dense Vector of single values. + */ +abstract class AbstractVector implements Vector { + + private final int positionCount; + + protected AbstractVector(int positionCount) { + this.positionCount = positionCount; + } + + public final int getPositionCount() { + return positionCount; + } + + @Override + public final Vector getRow(int position) { + return filter(position); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBlock.java new file mode 100644 index 0000000000000..c95a4cfa52757 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBlock.java @@ -0,0 +1,62 @@ +/* + * 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.compute.data; + +/** + * A Block view of a Vector. + */ +abstract class AbstractVectorBlock extends AbstractBlock { + + AbstractVectorBlock(int positionCount) { + super(positionCount); + } + + @Override + public int getFirstValueIndex(int position) { + return position; + } + + public int getValueCount(int position) { + return 1; + } + + @Override + public boolean isNull(int position) { + return false; + } + + @Override + public int nullValuesCount() { + return 0; + } + + @Override + public boolean mayHaveNulls() { + return false; + } + + @Override + public boolean areAllValuesNull() { + return false; + } + + @Override + public boolean mayHaveMultivaluedFields() { + return false; + } + + @Override + public final MvOrdering mvOrdering() { + return MvOrdering.UNORDERED; + } + + @Override + public final Block expand() { + return this; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBuilder.java new file mode 100644 index 0000000000000..08b7e0d5dc10f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/AbstractVectorBuilder.java @@ -0,0 +1,31 @@ +/* + * 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.compute.data; + +abstract class AbstractVectorBuilder { + protected int valueCount; + + /** The length of the internal values array. */ + protected abstract int valuesLength(); + + protected abstract void growValuesArray(int newSize); + + protected final void ensureCapacity() { + int valuesLength = valuesLength(); + if (valueCount < valuesLength) { + return; + } + int newSize = calculateNewArraySize(valuesLength); + growValuesArray(newSize); + } + + static int calculateNewArraySize(int currentSize) { + // trivially, grows array by 50% + return currentSize + (currentSize >> 1); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java new file mode 100644 index 0000000000000..5f1b961547cd3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java @@ -0,0 +1,187 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; + +import java.util.List; + +/** + * A Block is a columnar representation of homogenous data. It has a position (row) count, and + * various data retrieval methods for accessing the underlying data that is stored at a given + * position. + * + *

Blocks can represent various shapes of underlying data. A Block can represent either sparse + * or dense data. A Block can represent either single or multi valued data. A Block that represents + * dense single-valued data can be viewed as a {@link Vector}. + * + * TODO: update comment + *

All Blocks share the same set of data retrieval methods, but actual concrete implementations + * effectively support a subset of these, throwing {@code UnsupportedOperationException} where a + * particular data retrieval method is not supported. For example, a Block of primitive longs may + * not support retrieval as an integer, {code getInt}. This greatly simplifies Block usage and + * avoids cumbersome use-site casting. + * + *

Block are immutable and can be passed between threads. + */ +public interface Block extends NamedWriteable { + + /** + * {@return an efficient dense single-value view of this block}. + * Null, if the block is not dense single-valued. That is, if + * mayHaveNulls returns true, or getTotalValueCount is not equal to getPositionCount. + */ + Vector asVector(); + + /** {@return The total number of values in this block not counting nulls.} */ + int getTotalValueCount(); + + /** {@return The number of positions in this block.} */ + int getPositionCount(); + + /** Gets the index of the first value for the given position. */ + int getFirstValueIndex(int position); + + /** Gets the number of values for the given position, possibly 0. */ + int getValueCount(int position); + + /** + * {@return the element type of this block} + */ + ElementType elementType(); + + /** + * Returns true if the value stored at the given position is null, false otherwise. + * + * @param position the position + * @return true or false + */ + boolean isNull(int position); + + /** + * @return the number of null values in this block. + */ + int nullValuesCount(); + + /** + * @return true if some values might be null. False, if all values are guaranteed to be not null. + */ + boolean mayHaveNulls(); + + /** + * @return true if all values in this block are guaranteed to be null. + */ + boolean areAllValuesNull(); + + /** + * Can this block have multivalued fields? Blocks that return {@code false} + * will never return more than one from {@link #getValueCount}. + */ + boolean mayHaveMultivaluedFields(); + + /** + * Creates a new block that only exposes the positions provided. Materialization of the selected positions is avoided. + * @param positions the positions to retain + * @return a filtered block + */ + Block filter(int... positions); + + /** + * How are multivalued fields ordered? + *

Note that there isn't a {@code DESCENDING} because we don't have + * anything that makes descending fields.

+ */ + enum MvOrdering { + ASCENDING, + UNORDERED; + } + + /** + * How are multivalued fields ordered? + */ + MvOrdering mvOrdering(); + + /** + * Expand multivalued fields into one row per value. Returns the + * block if there aren't any multivalued fields to expand. + */ + Block expand(); + + /** + * {@return a constant null block with the given number of positions}. + */ + static Block constantNullBlock(int positions) { + return new ConstantNullBlock(positions); + } + + interface Builder { + + /** + * Appends a null value to the block. + */ + Builder appendNull(); + + /** + * Begins a multivalued entry. Calling this for the first time will put + * the builder into a mode that generates Blocks that return {@code true} + * from {@link Block#mayHaveMultivaluedFields} which can force less + * optimized code paths. So don't call this unless you are sure you are + * emitting more than one value for this position. + */ + Builder beginPositionEntry(); + + /** + * Ends the current multi-value entry. + */ + Builder endPositionEntry(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + /** + * How are multivalued fields ordered? This defaults to {@link Block.MvOrdering#UNORDERED} + * but when you set it to {@link Block.MvOrdering#ASCENDING} some operators can optimize + * themselves. This is a promise that is never checked. If you set this + * to anything other than {@link Block.MvOrdering#UNORDERED} be sure the values are in + * that order or other operators will make mistakes. The actual ordering isn't checked + * at runtime. + */ + Builder mvOrdering(Block.MvOrdering mvOrdering); + + /** + * Builds the block. This method can be called multiple times. + */ + Block build(); + } + + static List getNamedWriteables() { + return List.of( + IntBlock.ENTRY, + LongBlock.ENTRY, + DoubleBlock.ENTRY, + BytesRefBlock.ENTRY, + BooleanBlock.ENTRY, + IntVectorBlock.ENTRY, + LongVectorBlock.ENTRY, + DoubleVectorBlock.ENTRY, + BytesRefVectorBlock.ENTRY, + BooleanVectorBlock.ENTRY, + ConstantNullBlock.ENTRY + ); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java new file mode 100644 index 0000000000000..fd7cd7ba0d120 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java @@ -0,0 +1,269 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.BytesRefs; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.function.Consumer; + +import static org.elasticsearch.common.lucene.BytesRefs.toBytesRef; +import static org.elasticsearch.compute.data.Block.constantNullBlock; + +public final class BlockUtils { + + public static final Block[] NO_BLOCKS = new Block[0]; + + private BlockUtils() {} + + public record BuilderWrapper(Block.Builder builder, Consumer append) { + public BuilderWrapper(Block.Builder builder, Consumer append) { + this.builder = builder; + this.append = o -> { + if (o == null) { + builder.appendNull(); + return; + } + if (o instanceof List l) { + builder.beginPositionEntry(); + for (Object v : l) { + append.accept(v); + } + builder.endPositionEntry(); + return; + } + append.accept(o); + }; + } + } + + public static Block[] fromArrayRow(Object... row) { + return fromListRow(Arrays.asList(row)); + } + + public static Block[] fromListRow(List row) { + return fromListRow(row, 1); + } + + public static Block[] fromListRow(List row, int blockSize) { + if (row.isEmpty()) { + return NO_BLOCKS; + } + + var size = row.size(); + Block[] blocks = new Block[size]; + for (int i = 0; i < size; i++) { + Object object = row.get(i); + if (object instanceof Integer intVal) { + blocks[i] = IntBlock.newConstantBlockWith(intVal, blockSize); + } else if (object instanceof Long longVal) { + blocks[i] = LongBlock.newConstantBlockWith(longVal, blockSize); + } else if (object instanceof Double doubleVal) { + blocks[i] = DoubleBlock.newConstantBlockWith(doubleVal, blockSize); + } else if (object instanceof BytesRef bytesRefVal) { + blocks[i] = BytesRefBlock.newConstantBlockWith(bytesRefVal, blockSize); + } else if (object instanceof Boolean booleanVal) { + blocks[i] = BooleanBlock.newConstantBlockWith(booleanVal, blockSize); + } else if (object instanceof List listVal) { + BuilderWrapper wrapper = wrapperFor(listVal.get(0).getClass(), 1); + wrapper.append.accept(listVal); + if (isAscending(listVal)) { + wrapper.builder.mvOrdering(Block.MvOrdering.ASCENDING); + } + blocks[i] = wrapper.builder.build(); + } else if (object == null) { + blocks[i] = constantNullBlock(blockSize); + } else { + throw new UnsupportedOperationException("can't make a block out of [" + object + "/" + object.getClass() + "]"); + } + } + return blocks; + } + + /** + * Detect blocks with ascending fields. This is *mostly* useful for + * exercising the specialized ascending implementations. + */ + private static boolean isAscending(List values) { + Comparable prev = null; + for (Object o : values) { + @SuppressWarnings("unchecked") + Comparable val = (Comparable) o; + if (prev == null) { + prev = val; + continue; + } + if (prev.compareTo(val) > 0) { + return false; + } + prev = val; + } + return true; + } + + public static Block[] fromList(List> list) { + var size = list.size(); + if (size == 0) { + return NO_BLOCKS; + } + if (size == 1) { + return fromListRow(list.get(0)); + } + + var wrappers = new BuilderWrapper[list.get(0).size()]; + + for (int i = 0; i < wrappers.length; i++) { + wrappers[i] = wrapperFor(type(list, i), size); + } + for (List values : list) { + for (int j = 0, vSize = values.size(); j < vSize; j++) { + wrappers[j].append.accept(values.get(j)); + } + } + return Arrays.stream(wrappers).map(b -> b.builder.build()).toArray(Block[]::new); + } + + private static Class type(List> list, int i) { + int p = 0; + while (p < list.size()) { + Object v = list.get(p++).get(i); + if (v == null) { + continue; + } + if (v instanceof List l) { + if (l.isEmpty()) { + continue; + } + return l.get(0).getClass(); + } + return v.getClass(); + } + return null; + } + + public static BuilderWrapper wrapperFor(Class type, int size) { + BuilderWrapper builder; + if (type == Integer.class) { + var b = IntBlock.newBlockBuilder(size); + builder = new BuilderWrapper(b, o -> b.appendInt((int) o)); + } else if (type == Long.class) { + var b = LongBlock.newBlockBuilder(size); + builder = new BuilderWrapper(b, o -> b.appendLong((long) o)); + } else if (type == Double.class) { + var b = DoubleBlock.newBlockBuilder(size); + builder = new BuilderWrapper(b, o -> b.appendDouble((double) o)); + } else if (type == BytesRef.class) { + var b = BytesRefBlock.newBlockBuilder(size); + builder = new BuilderWrapper(b, o -> b.appendBytesRef(BytesRefs.toBytesRef(o))); + } else if (type == Boolean.class) { + var b = BooleanBlock.newBlockBuilder(size); + builder = new BuilderWrapper(b, o -> b.appendBoolean((boolean) o)); + } else if (type == null) { + var b = new Block.Builder() { + @Override + public Block.Builder appendNull() { + return this; + } + + @Override + public Block.Builder beginPositionEntry() { + return this; + } + + @Override + public Block.Builder endPositionEntry() { + return this; + } + + @Override + public Block.Builder copyFrom(Block block, int beginInclusive, int endExclusive) { + return this; + } + + @Override + public Block.Builder mvOrdering(Block.MvOrdering mvOrdering) { + throw new UnsupportedOperationException(); + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + throw new UnsupportedOperationException(); + } + + @Override + public Block build() { + return constantNullBlock(size); + } + }; + builder = new BuilderWrapper(b, o -> {}); + } else { + throw new UnsupportedOperationException("Unrecognized type " + type); + } + return builder; + } + + public static void appendValue(Block.Builder builder, Object val, ElementType type) { + if (val == null) { + builder.appendNull(); + return; + } + switch (type) { + case LONG -> ((LongBlock.Builder) builder).appendLong((Long) val); + case INT -> ((IntBlock.Builder) builder).appendInt((Integer) val); + case BYTES_REF -> ((BytesRefBlock.Builder) builder).appendBytesRef(toBytesRef(val)); + case DOUBLE -> ((DoubleBlock.Builder) builder).appendDouble((Double) val); + case BOOLEAN -> ((BooleanBlock.Builder) builder).appendBoolean((Boolean) val); + default -> throw new UnsupportedOperationException("unsupported element type [" + type + "]"); + } + } + + /** + * Returned by {@link #toJavaObject} for "doc" type blocks. + */ + public record Doc(int shard, int segment, int doc) {} + + /** + * Read all values from a positions into a java object. This is not fast + * but fine to call in the "fold" path. + */ + public static Object toJavaObject(Block block, int position) { + if (block.isNull(position)) { + return null; + } + int count = block.getValueCount(position); + int start = block.getFirstValueIndex(position); + if (count == 1) { + return valueAtOffset(block, start); + } + int end = start + count; + List result = new ArrayList<>(count); + for (int i = start; i < end; i++) { + result.add(valueAtOffset(block, i)); + } + return result; + } + + private static Object valueAtOffset(Block block, int offset) { + return switch (block.elementType()) { + case BOOLEAN -> ((BooleanBlock) block).getBoolean(offset); + case BYTES_REF -> ((BytesRefBlock) block).getBytesRef(offset, new BytesRef()); + case DOUBLE -> ((DoubleBlock) block).getDouble(offset); + case INT -> ((IntBlock) block).getInt(offset); + case LONG -> ((LongBlock) block).getLong(offset); + case NULL -> null; + case DOC -> { + DocVector v = ((DocBlock) block).asVector(); + yield new Doc(v.shards().getInt(offset), v.segments().getInt(offset), v.docs().getInt(offset)); + } + case UNKNOWN -> throw new IllegalArgumentException("can't read values from [" + block + "]"); + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java new file mode 100644 index 0000000000000..5d5f65eb7ab1e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java @@ -0,0 +1,158 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +/** + * Block implementation representing a constant null value. + */ +public final class ConstantNullBlock extends AbstractBlock { + + ConstantNullBlock(int positionCount) { + super(positionCount); + } + + @Override + public Vector asVector() { + return null; + } + + @Override + public boolean isNull(int position) { + return true; + } + + @Override + public int nullValuesCount() { + return getPositionCount(); + } + + @Override + public boolean areAllValuesNull() { + return true; + } + + @Override + public boolean mayHaveNulls() { + return true; + } + + @Override + public boolean mayHaveMultivaluedFields() { + return false; + } + + @Override + public ElementType elementType() { + return ElementType.NULL; + } + + @Override + public Block filter(int... positions) { + return new ConstantNullBlock(positions.length); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "ConstantNullBlock", + ConstantNullBlock::of + ); + + @Override + public String getWriteableName() { + return "ConstantNullBlock"; + } + + static ConstantNullBlock of(StreamInput in) throws IOException { + return new ConstantNullBlock(in.readVInt()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(getPositionCount()); + } + + @Override + public MvOrdering mvOrdering() { + return MvOrdering.UNORDERED; + } + + @Override + public Block expand() { + return this; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ConstantNullBlock that) { + return this.getPositionCount() == that.getPositionCount(); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(getPositionCount()); + } + + @Override + public String toString() { + return "ConstantNullBlock[positions=" + getPositionCount() + "]"; + } + + static class Builder implements Block.Builder { + private int positionCount; + + @Override + public Builder appendNull() { + positionCount++; + return this; + } + + @Override + public Builder beginPositionEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public Builder endPositionEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public Builder copyFrom(Block block, int beginInclusive, int endExclusive) { + for (int i = beginInclusive; i < endExclusive; i++) { + if (false == block.isNull(i)) { + throw new UnsupportedOperationException("can't append non-null values to a null block"); + } + } + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + return appendNull(); + } + + @Override + public Block.Builder mvOrdering(MvOrdering mvOrdering) { + throw new UnsupportedOperationException(); + } + + @Override + public Block build() { + return new ConstantNullBlock(positionCount); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java new file mode 100644 index 0000000000000..7d14241801352 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java @@ -0,0 +1,125 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Wrapper around {@link DocVector} to make a valid {@link Block}. + */ +public class DocBlock extends AbstractVectorBlock implements Block { + private final DocVector vector; + + DocBlock(DocVector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public String getWriteableName() { + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public DocVector asVector() { + return vector; + } + + @Override + public ElementType elementType() { + return ElementType.DOC; + } + + @Override + public Block filter(int... positions) { + return new DocBlock(asVector().filter(positions)); + } + + /** + * A builder the for {@link DocBlock}. + */ + public static Builder newBlockBuilder(int estimatedSize) { + return new Builder(estimatedSize); + } + + public static class Builder implements Block.Builder { + private final IntVector.Builder shards; + private final IntVector.Builder segments; + private final IntVector.Builder docs; + + private Builder(int estimatedSize) { + shards = IntVector.newVectorBuilder(estimatedSize); + segments = IntVector.newVectorBuilder(estimatedSize); + docs = IntVector.newVectorBuilder(estimatedSize); + } + + public Builder appendShard(int shard) { + shards.appendInt(shard); + return this; + } + + public Builder appendSegment(int segment) { + segments.appendInt(segment); + return this; + } + + public Builder appendDoc(int doc) { + docs.appendInt(doc); + return this; + } + + @Override + public Builder appendNull() { + throw new UnsupportedOperationException("doc blocks can't contain null"); + } + + @Override + public Builder beginPositionEntry() { + throw new UnsupportedOperationException("doc blocks only contain one value per position"); + } + + @Override + public Builder endPositionEntry() { + throw new UnsupportedOperationException("doc blocks only contain one value per position"); + } + + @Override + public Builder copyFrom(Block block, int beginInclusive, int endExclusive) { + DocVector docVector = ((DocBlock) block).asVector(); + for (int i = beginInclusive; i < endExclusive; i++) { + shards.appendInt(docVector.shards().getInt(i)); + segments.appendInt(docVector.segments().getInt(i)); + docs.appendInt(docVector.docs().getInt(i)); + } + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + throw new UnsupportedOperationException("DocBlock doesn't support appendBlockAndMerge"); + } + + @Override + public Block.Builder mvOrdering(MvOrdering mvOrdering) { + throw new UnsupportedOperationException("doc blocks only contain one value per position"); + } + + @Override + public DocBlock build() { + // Pass null for singleSegmentNonDecreasing so we calculate it when we first need it. + return new DocVector(shards.build(), segments.build(), docs.build(), null).asBlock(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java new file mode 100644 index 0000000000000..a17ab3d64a706 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -0,0 +1,181 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.IntroSorter; + +/** + * {@link Vector} where each entry references a lucene document. + */ +public class DocVector extends AbstractVector implements Vector { + /** + * Per position memory cost to build the shard segment doc map required + * to load fields out of order. + */ + public static final int SHARD_SEGMENT_DOC_MAP_PER_ROW_OVERHEAD = Integer.BYTES * 2; + + private final IntVector shards; + private final IntVector segments; + private final IntVector docs; + + /** + * Are the docs in this vector all in one segment and non-decreasing? If + * so we can load doc values via a fast path. + */ + private Boolean singleSegmentNonDecreasing; + + /** + * Maps the vector positions to ascending docs per-shard and per-segment. + */ + private int[] shardSegmentDocMapForwards; + + /** + * Reverse of {@link #shardSegmentDocMapForwards}. + */ + private int[] shardSegmentDocMapBackwards; + + public DocVector(IntVector shards, IntVector segments, IntVector docs, Boolean singleSegmentNonDecreasing) { + super(shards.getPositionCount()); + this.shards = shards; + this.segments = segments; + this.docs = docs; + this.singleSegmentNonDecreasing = singleSegmentNonDecreasing; + if (shards.getPositionCount() != segments.getPositionCount()) { + throw new IllegalArgumentException( + "invalid position count [" + shards.getPositionCount() + " != " + segments.getPositionCount() + "]" + ); + } + if (shards.getPositionCount() != docs.getPositionCount()) { + throw new IllegalArgumentException( + "invalid position count [" + shards.getPositionCount() + " != " + docs.getPositionCount() + "]" + ); + } + } + + public IntVector shards() { + return shards; + } + + public IntVector segments() { + return segments; + } + + public IntVector docs() { + return docs; + } + + public boolean singleSegmentNonDecreasing() { + if (singleSegmentNonDecreasing == null) { + singleSegmentNonDecreasing = checkIfSingleSegmentNonDecreasing(); + } + return singleSegmentNonDecreasing; + } + + private boolean checkIfSingleSegmentNonDecreasing() { + if (getPositionCount() < 2) { + return true; + } + if (shards.isConstant() == false || segments.isConstant() == false) { + return false; + } + int prev = docs.getInt(0); + int p = 1; + while (p < getPositionCount()) { + int v = docs.getInt(p++); + if (prev > v) { + return false; + } + prev = v; + } + return true; + + } + + /** + * Map from the positions in this page to the positions in lucene's native order for + * loading doc values. + */ + public int[] shardSegmentDocMapForwards() { + buildShardSegmentDocMapIfMissing(); + return shardSegmentDocMapForwards; + } + + /** + * Reverse of {@link #shardSegmentDocMapForwards}. If you load doc values in the "forward" + * order then you can call {@link Block#filter} on the loaded values with this array to + * put them in the same order as this {@link Page}. + */ + public int[] shardSegmentDocMapBackwards() { + buildShardSegmentDocMapIfMissing(); + return shardSegmentDocMapBackwards; + } + + private void buildShardSegmentDocMapIfMissing() { + if (shardSegmentDocMapForwards != null) { + return; + } + + int[] forwards = shardSegmentDocMapForwards = new int[shards.getPositionCount()]; + for (int p = 0; p < forwards.length; p++) { + forwards[p] = p; + } + new IntroSorter() { + int pivot; + + @Override + protected void setPivot(int i) { + pivot = forwards[i]; + } + + @Override + protected int comparePivot(int j) { + int cmp = Integer.compare(shards.getInt(pivot), shards.getInt(forwards[j])); + if (cmp != 0) { + return cmp; + } + cmp = Integer.compare(segments.getInt(pivot), segments.getInt(forwards[j])); + if (cmp != 0) { + return cmp; + } + return Integer.compare(docs.getInt(pivot), docs.getInt(forwards[j])); + } + + @Override + protected void swap(int i, int j) { + int tmp = forwards[i]; + forwards[i] = forwards[j]; + forwards[j] = tmp; + } + }.sort(0, forwards.length); + + int[] backwards = shardSegmentDocMapBackwards = new int[forwards.length]; + for (int p = 0; p < forwards.length; p++) { + backwards[forwards[p]] = p; + } + } + + @Override + public DocBlock asBlock() { + return new DocBlock(this); + } + + @Override + public DocVector filter(int... positions) { + return new DocVector(shards.filter(positions), segments.filter(positions), docs.filter(positions), null); + } + + @Override + public ElementType elementType() { + return ElementType.DOC; + } + + @Override + public boolean isConstant() { + return shards.isConstant() && segments.isConstant() && docs.isConstant(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java new file mode 100644 index 0000000000000..d9f78747ff665 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java @@ -0,0 +1,49 @@ +/* + * 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.compute.data; + +import java.util.function.IntFunction; + +/** + * The type of elements in {@link Block} and {@link Vector} + */ +public enum ElementType { + BOOLEAN(BooleanBlock::newBlockBuilder), + INT(IntBlock::newBlockBuilder), + LONG(LongBlock::newBlockBuilder), + DOUBLE(DoubleBlock::newBlockBuilder), + /** + * Blocks containing only null values. + */ + NULL(estimatedSize -> new ConstantNullBlock.Builder()), + + BYTES_REF(BytesRefBlock::newBlockBuilder), + + /** + * Blocks that reference individual lucene documents. + */ + DOC(DocBlock::newBlockBuilder), + + /** + * Intermediate blocks which don't support retrieving elements. + */ + UNKNOWN(estimatedSize -> { throw new UnsupportedOperationException("can't build null blocks"); }); + + private final IntFunction builder; + + ElementType(IntFunction builder) { + this.builder = builder; + } + + /** + * Create a new {@link Block.Builder} for blocks of this type. + */ + public Block.Builder newBlockBuilder(int estimatedSize) { + return builder.apply(estimatedSize); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Page.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Page.java new file mode 100644 index 0000000000000..42998770e2d84 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Page.java @@ -0,0 +1,219 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.Assertions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Objects; + +/** + * A page is a column-oriented data abstraction that allows data to be passed between operators in + * batches. + * + *

A page has a fixed number of positions (or rows), exposed via {@link #getPositionCount()}. + * It is further composed of a number of {@link Block}s, which represent the columnar data. + * The number of blocks can be retrieved via {@link #getBlockCount()}, and the respective + * blocks can be retrieved via their index {@link #getBlock(int)}. + * + *

Pages are immutable and can be passed between threads. + */ +public final class Page implements Writeable { + + private final Block[] blocks; + + private final int positionCount; + + /** + * Creates a new page with the given blocks. Every block has the same number of positions. + * + * @param blocks the blocks + * @throws IllegalArgumentException if all blocks do not have the same number of positions + */ + public Page(Block... blocks) { + this(true, determinePositionCount(blocks), blocks); + } + + /** + * Creates a new page with the given positionCount and blocks. Assumes that every block has the + * same number of positions as the positionCount that's passed in - there is no validation of + * this. + * + * @param positionCount the block position count + * @param blocks the blocks + */ + public Page(int positionCount, Block... blocks) { + this(true, positionCount, blocks); + } + + private Page(boolean copyBlocks, int positionCount, Block[] blocks) { + Objects.requireNonNull(blocks, "blocks is null"); + // assert assertPositionCount(blocks); + this.positionCount = positionCount; + this.blocks = copyBlocks ? blocks.clone() : blocks; + if (Assertions.ENABLED) { + for (Block b : blocks) { + assert b.getPositionCount() == positionCount : "expected positionCount=" + positionCount + " but was " + b; + } + } + } + + public Page(StreamInput in) throws IOException { + int positionCount = in.readVInt(); + int blockPositions = in.readVInt(); + Block[] blocks = new Block[blockPositions]; + for (int blockIndex = 0; blockIndex < blockPositions; blockIndex++) { + blocks[blockIndex] = in.readNamedWriteable(Block.class); + } + this.positionCount = positionCount; + this.blocks = blocks; + } + + private static int determinePositionCount(Block... blocks) { + Objects.requireNonNull(blocks, "blocks is null"); + if (blocks.length == 0) { + throw new IllegalArgumentException("blocks is empty"); + } + return blocks[0].getPositionCount(); + } + + /** + * Returns the block at the given block index. + * + * @param blockIndex the block index + * @return the block + */ + public B getBlock(int blockIndex) { + @SuppressWarnings("unchecked") + B block = (B) blocks[blockIndex]; + return block; + } + + /** + * Creates a new page, appending the given block to the existing blocks in this Page. + * + * @param block the block to append + * @return a new Page with the block appended + * @throws IllegalArgumentException if the given block does not have the same number of + * positions as the blocks in this Page + */ + public Page appendBlock(Block block) { + if (positionCount != block.getPositionCount()) { + throw new IllegalArgumentException("Block does not have same position count"); + } + + Block[] newBlocks = Arrays.copyOf(blocks, blocks.length + 1); + newBlocks[blocks.length] = block; + return new Page(false, positionCount, newBlocks); + } + + /** + * Creates a new page, appending the given blocks to the existing blocks in this Page. + * + * @param toAdd the blocks to append + * @return a new Page with the block appended + * @throws IllegalArgumentException if one of the given blocks does not have the same number of + * positions as the blocks in this Page + */ + public Page appendBlocks(Block[] toAdd) { + for (Block block : toAdd) { + if (positionCount != block.getPositionCount()) { + throw new IllegalArgumentException("Block does not have same position count"); + } + } + + Block[] newBlocks = Arrays.copyOf(blocks, blocks.length + toAdd.length); + for (int i = 0; i < toAdd.length; i++) { + newBlocks[blocks.length + i] = toAdd[i]; + } + return new Page(false, positionCount, newBlocks); + } + + /** + * Creates a new page, appending the blocks of the given block to the existing blocks in this Page. + * + * @param toAdd the page to append + * @return a new Page + * @throws IllegalArgumentException if any blocks of the given page does not have the same number of + * positions as the blocks in this Page + */ + public Page appendPage(Page toAdd) { + return appendBlocks(toAdd.blocks); + } + + @Override + public int hashCode() { + int result = Objects.hash(positionCount); + for (int i = 0; i < blocks.length; i++) { + result = 31 * result + Objects.hashCode(blocks[i]); + } + return result; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Page page = (Page) o; + return positionCount == page.positionCount + && (positionCount == 0 || Arrays.equals(blocks, 0, blocks.length, page.blocks, 0, page.blocks.length)); + } + + @Override + public String toString() { + return "Page{" + "blocks=" + Arrays.toString(blocks) + '}'; + } + + /** + * Returns the number of positions (rows) in this page. + * + * @return the number of positions + */ + public int getPositionCount() { + return positionCount; + } + + /** + * Returns the number of blocks in this page. Blocks can then be retrieved via + * {@link #getBlock(int)} where channel ranges from 0 to {@code getBlockCount}. + * + * @return the number of blocks in this page + */ + public int getBlockCount() { + return blocks.length; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(positionCount); + out.writeVInt(getBlockCount()); + for (Block block : blocks) { + out.writeNamedWriteable(block); + } + } + + public static class PageWriter implements Writeable.Writer { + + @Override + public void write(StreamOutput out, Page value) throws IOException { + value.writeTo(out); + } + } + + public static class PageReader implements Writeable.Reader { + + @Override + public Page read(StreamInput in) throws IOException { + return new Page(in); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Vector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Vector.java new file mode 100644 index 0000000000000..7954834a0debc --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Vector.java @@ -0,0 +1,53 @@ +/* + * 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.compute.data; + +/** + * A dense Vector of single values. + */ +public interface Vector { + + /** + * {@return Returns a Block view over this vector.} + */ + Block asBlock(); + + /** + * The number of positions in this vector. + * + * @return the number of positions + */ + int getPositionCount(); + + // TODO: improve implementation not to waste as much space + Vector getRow(int position); + + /** + * Creates a new vector that only exposes the positions provided. Materialization of the selected positions is avoided. + * @param positions the positions to retain + * @return a filtered vector + */ + Vector filter(int... positions); + + /** + * {@return the element type of this vector} + */ + ElementType elementType(); + + /** + * {@return true iff this vector is a constant vector - returns the same constant value for every position} + */ + boolean isConstant(); + + interface Builder { + /** + * Builds the block. This method can be called multiple times. + */ + Vector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st new file mode 100644 index 0000000000000..3f4b348185796 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayBlock.java.st @@ -0,0 +1,109 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefArray; + +$else$ +import java.util.Arrays; +$endif$ +import java.util.BitSet; +import java.util.stream.IntStream; + +/** + * Block implementation that stores an array of $type$. + * This class is generated. Do not edit it. + */ +public final class $Type$ArrayBlock extends AbstractArrayBlock implements $Type$Block { + +$if(BytesRef)$ + private final BytesRefArray values; + +$else$ + private final $type$[] values; +$endif$ + +$if(BytesRef)$ + public $Type$ArrayBlock(BytesRefArray values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { +$else$ + public $Type$ArrayBlock($type$[] values, int positionCount, int[] firstValueIndexes, BitSet nulls, MvOrdering mvOrdering) { +$endif$ + super(positionCount, firstValueIndexes, nulls, mvOrdering); + this.values = values; + } + + @Override + public $Type$Vector asVector() { + return null; + } + + @Override +$if(BytesRef)$ + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return values.get(valueIndex, dest); +$else$ + public $type$ get$Type$(int valueIndex) { + return values[valueIndex]; +$endif$ + } + + @Override + public $Type$Block filter(int... positions) { + return new Filter$Type$Block(this, positions); + } + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public $Type$Block expand() { + if (firstValueIndexes == null) { + return this; + } + int end = firstValueIndexes[getPositionCount()]; + if (nullsMask == null) { + return new $Type$ArrayVector(values, end).asBlock(); + } + int[] firstValues = IntStream.range(0, end + 1).toArray(); + return new $Type$ArrayBlock(values, end, firstValues, shiftNullsToExpandedPositions(), MvOrdering.UNORDERED); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Block that) { + return $Type$Block.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Block.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + + "[positions=" + + getPositionCount() + + ", mvOrdering=" + + mvOrdering() +$if(BytesRef)$ + + ", values=" + + values.size() +$else$ + + ", values=" + + Arrays.toString(values) +$endif$ + + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayVector.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayVector.java.st new file mode 100644 index 0000000000000..4b3f234c05dc6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ArrayVector.java.st @@ -0,0 +1,94 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefArray; + +$else$ +import java.util.Arrays; +$endif$ + +/** + * Vector implementation that stores an array of $type$ values. + * This class is generated. Do not edit it. + */ +public final class $Type$ArrayVector extends AbstractVector implements $Type$Vector { + +$if(BytesRef)$ + private final BytesRefArray values; + +$else$ + private final $type$[] values; +$endif$ + +$if(BytesRef)$ + public $Type$ArrayVector(BytesRefArray values, int positionCount) { +$else$ + public $Type$ArrayVector($type$[] values, int positionCount) { +$endif$ + super(positionCount); + this.values = values; + } + + @Override + public $Type$Block asBlock() { + return new $Type$VectorBlock(this); + } + +$if(BytesRef)$ + @Override + public BytesRef getBytesRef(int position, BytesRef dest) { + return values.get(position, dest); + } + +$else$ + @Override + public $type$ get$Type$(int position) { + return values[position]; + } +$endif$ + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public $Type$Vector filter(int... positions) { + return new Filter$Type$Vector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Vector that) { + return $Type$Vector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Vector.hash(this); + } + + @Override + public String toString() { +$if(BytesRef)$ + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ']'; +$else$ + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + Arrays.toString(values) + ']'; +$endif$ + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayVector.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayVector.java.st new file mode 100644 index 0000000000000..09566bed63dc3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BigArrayVector.java.st @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.$if(boolean)$Bit$else$$Type$$endif$Array; +import org.elasticsearch.core.Releasable; + +/** + * Vector implementation that defers to an enclosed $Type$Array. + * This class is generated. Do not edit it. + */ +public final class $Type$BigArrayVector extends AbstractVector implements $Type$Vector, Releasable { + + private final $if(boolean)$Bit$else$$Type$$endif$Array values; + + public $Type$BigArrayVector($if(boolean)$Bit$else$$Type$$endif$Array values, int positionCount) { + super(positionCount); + this.values = values; + } + + @Override + public $Type$Block asBlock() { + return new $Type$VectorBlock(this); + } + + @Override + public $type$ get$Type$(int position) { + return values.get(position); + } + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public boolean isConstant() { + return false; + } + + @Override + public $Type$Vector filter(int... positions) { + return new Filter$Type$Vector(this, positions); + } + + @Override + public void close() { + values.close(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Vector that) { + return $Type$Vector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Vector.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", values=" + values + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st new file mode 100644 index 0000000000000..61fa0ddd26de3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st @@ -0,0 +1,236 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$else$ +$endif$ +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block that stores $type$ values. + * This class is generated. Do not edit it. + */ +public sealed interface $Type$Block extends Block permits Filter$Type$Block, $Type$ArrayBlock, $Type$VectorBlock { + + /** + * Retrieves the $type$ value stored at the given value index. + * + *

Values for a given position are between getFirstValueIndex(position) (inclusive) and + * getFirstValueIndex(position) + getValueCount(position) (exclusive). + * + * @param valueIndex the value index +$if(BytesRef)$ + * @param dest the destination +$endif$ + * @return the data value (as a $type$) + */ +$if(BytesRef)$ + BytesRef getBytesRef(int valueIndex, BytesRef dest); + +$else$ + $type$ get$Type$(int valueIndex); +$endif$ + + @Override + $Type$Vector asVector(); + + @Override + $Type$Block filter(int... positions); + + NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Block.class, "$Type$Block", $Type$Block::of); + + @Override + default String getWriteableName() { + return "$Type$Block"; + } + + static $Type$Block of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + var builder = newBlockBuilder(positions); + for (int i = 0; i < positions; i++) { + if (in.readBoolean()) { + builder.appendNull(); + } else { + final int valueCount = in.readVInt(); + builder.beginPositionEntry(); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + builder.append$Type$(in.read$Type$()); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + @Override + default void writeTo(StreamOutput out) throws IOException { + final int positions = getPositionCount(); + out.writeVInt(positions); + for (int pos = 0; pos < positions; pos++) { + if (isNull(pos)) { + out.writeBoolean(true); + } else { + out.writeBoolean(false); + final int valueCount = getValueCount(pos); + out.writeVInt(valueCount); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { +$if(BytesRef)$ + out.write$Type$(get$Type$(getFirstValueIndex(pos) + valueIndex, new BytesRef())); +$else$ + out.write$Type$(get$Type$(getFirstValueIndex(pos) + valueIndex)); +$endif$ + } + } + } + } + + /** + * Compares the given object with this block for equality. Returns {@code true} if and only if the + * given object is a $Type$Block, and both blocks are {@link #equals($Type$Block, $Type$Block) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this block, as defined by {@link #hash($Type$Block)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the $Type$Block interface. + */ + static boolean equals($Type$Block block1, $Type$Block block2) { + final int positions = block1.getPositionCount(); + if (positions != block2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { + if (block1.isNull(pos) || block2.isNull(pos)) { + if (block1.isNull(pos) != block2.isNull(pos)) { + return false; + } + } else { + final int valueCount = block1.getValueCount(pos); + if (valueCount != block2.getValueCount(pos)) { + return false; + } + final int b1ValueIdx = block1.getFirstValueIndex(pos); + final int b2ValueIdx = block2.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { +$if(BytesRef)$ + if (block1.getBytesRef(b1ValueIdx + valueIndex, new BytesRef()) + .equals(block2.getBytesRef(b2ValueIdx + valueIndex, new BytesRef())) == false) { +$else$ + if (block1.get$Type$(b1ValueIdx + valueIndex) != block2.get$Type$(b2ValueIdx + valueIndex)) { +$endif$ + return false; + } + } + } + } + return true; + } + + /** + * Generates the hash code for the given block. The hash code is computed from the block's values. + * This ensures that {@code block1.equals(block2)} implies that {@code block1.hashCode()==block2.hashCode()} + * for any two blocks, {@code block1} and {@code block2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash($Type$Block block) { + final int positions = block.getPositionCount(); + int result = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + result = 31 * result - 1; + } else { + final int valueCount = block.getValueCount(pos); + result = 31 * result + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { +$if(BytesRef)$ + result = 31 * result + block.getBytesRef(firstValueIdx + valueIndex, new BytesRef()).hashCode(); +$elseif(boolean)$ + result = 31 * result + Boolean.hashCode(block.getBoolean(firstValueIdx + valueIndex)); +$elseif(int)$ + result = 31 * result + block.getInt(firstValueIdx + valueIndex); +$elseif(long)$ + long element = block.getLong(firstValueIdx + valueIndex); + result = 31 * result + (int) (element ^ (element >>> 32)); +$elseif(double)$ + long element = Double.doubleToLongBits(block.getDouble(firstValueIdx + valueIndex)); + result = 31 * result + (int) (element ^ (element >>> 32)); +$endif$ + } + } + } + return result; + } + + static Builder newBlockBuilder(int estimatedSize) { + return new $Type$BlockBuilder(estimatedSize); + } + + static $Type$Block newConstantBlockWith($type$ value, int positions) { + return new Constant$Type$Vector(value, positions).asBlock(); + } + + sealed interface Builder extends Block.Builder permits $Type$BlockBuilder { + + /** + * Appends a $type$ to the current entry. + */ + Builder append$Type$($type$ value); + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + Builder copyFrom($Type$Block block, int beginInclusive, int endExclusive); + + @Override + Builder appendNull(); + + @Override + Builder beginPositionEntry(); + + @Override + Builder endPositionEntry(); + + @Override + Builder copyFrom(Block block, int beginInclusive, int endExclusive); + + @Override + Builder mvOrdering(Block.MvOrdering mvOrdering); + + // TODO boolean containsMvDups(); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition(Block block); + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + Builder appendAllValuesToCurrentPosition($Type$Block block); + + @Override + $Type$Block build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st new file mode 100644 index 0000000000000..ff517b1c85533 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-BlockBuilder.java.st @@ -0,0 +1,258 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; + +$else$ +import java.util.Arrays; +$endif$ + +/** + * Block build of $Type$Blocks. + * This class is generated. Do not edit it. + */ +final class $Type$BlockBuilder extends AbstractBlockBuilder implements $Type$Block.Builder { + +$if(BytesRef)$ + private static final BytesRef NULL_VALUE = new BytesRef(); + + private BytesRefArray values; + + BytesRefBlockBuilder(int estimatedSize) { + this(estimatedSize, BigArrays.NON_RECYCLING_INSTANCE); + } + + BytesRefBlockBuilder(int estimatedSize, BigArrays bigArrays) { + values = new BytesRefArray(Math.max(estimatedSize, 2), bigArrays); + } + +$else$ + private $type$[] values; + + $Type$BlockBuilder(int estimatedSize) { + values = new $type$[Math.max(estimatedSize, 2)]; + } +$endif$ + + @Override + public $Type$BlockBuilder append$Type$($type$ value) { + ensureCapacity(); +$if(BytesRef)$ + values.append(value); +$else$ + values[valueCount] = value; +$endif$ + hasNonNullValue = true; + valueCount++; + updatePosition(); + return this; + } + + @Override + protected int valuesLength() { +$if(BytesRef)$ + return Integer.MAX_VALUE; // allow the BytesRefArray through its own append +$else$ + return values.length; +$endif$ + } + + @Override + protected void growValuesArray(int newSize) { +$if(BytesRef)$ + throw new AssertionError("should not reach here"); +$else$ + values = Arrays.copyOf(values, newSize); +$endif$ + } + + @Override + public $Type$BlockBuilder appendNull() { + super.appendNull(); + return this; + } + + @Override + public $Type$BlockBuilder beginPositionEntry() { + super.beginPositionEntry(); + return this; + } + + @Override + public $Type$BlockBuilder endPositionEntry() { + super.endPositionEntry(); + return this; + } + +$if(BytesRef)$ + @Override + protected void writeNullValue() { + values.append(NULL_VALUE); + } +$endif$ + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public $Type$BlockBuilder appendAllValuesToCurrentPosition(Block block) { + if (block.areAllValuesNull()) { + return appendNull(); + } + return appendAllValuesToCurrentPosition(($Type$Block) block); + } + + /** + * Appends the all values of the given block into a the current position + * in this builder. + */ + @Override + public $Type$BlockBuilder appendAllValuesToCurrentPosition($Type$Block block) { + final int positionCount = block.getPositionCount(); + if (positionCount == 0) { + return appendNull(); + } + final int totalValueCount = block.getTotalValueCount(); + if (totalValueCount == 0) { + return appendNull(); + } + if (totalValueCount > 1) { + beginPositionEntry(); + } +$if(BytesRef)$ + BytesRef scratch = new BytesRef(); +$endif$ + final $Type$Vector vector = block.asVector(); + if (vector != null) { + for (int p = 0; p < positionCount; p++) { +$if(BytesRef)$ + appendBytesRef(vector.getBytesRef(p, scratch)); +$else$ + append$Type$(vector.get$Type$(p)); +$endif$ + } + } else { + for (int p = 0; p < positionCount; p++) { + int count = block.getValueCount(p); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { +$if(BytesRef)$ + appendBytesRef(block.getBytesRef(i++, scratch)); +$else$ + append$Type$(block.get$Type$(i++)); +$endif$ + } + } + } + if (totalValueCount > 1) { + endPositionEntry(); + } + return this; + } + + @Override + public $Type$BlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + if (block.areAllValuesNull()) { + for (int p = beginInclusive; p < endExclusive; p++) { + appendNull(); + } + return this; + } + return copyFrom(($Type$Block) block, beginInclusive, endExclusive); + } + + /** + * Copy the values in {@code block} from {@code beginInclusive} to + * {@code endExclusive} into this builder. + */ + public $Type$BlockBuilder copyFrom($Type$Block block, int beginInclusive, int endExclusive) { + if (endExclusive > block.getPositionCount()) { + throw new IllegalArgumentException("can't copy past the end [" + endExclusive + " > " + block.getPositionCount() + "]"); + } + $Type$Vector vector = block.asVector(); + if (vector != null) { + copyFromVector(vector, beginInclusive, endExclusive); + } else { + copyFromBlock(block, beginInclusive, endExclusive); + } + return this; + } + + private void copyFromBlock($Type$Block block, int beginInclusive, int endExclusive) { +$if(BytesRef)$ + BytesRef scratch = new BytesRef(); +$endif$ + for (int p = beginInclusive; p < endExclusive; p++) { + if (block.isNull(p)) { + appendNull(); + continue; + } + int count = block.getValueCount(p); + if (count > 1) { + beginPositionEntry(); + } + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { +$if(BytesRef)$ + appendBytesRef(block.getBytesRef(i++, scratch)); +$else$ + append$Type$(block.get$Type$(i++)); +$endif$ + } + if (count > 1) { + endPositionEntry(); + } + } + } + + private void copyFromVector($Type$Vector vector, int beginInclusive, int endExclusive) { +$if(BytesRef)$ + BytesRef scratch = new BytesRef(); +$endif$ + for (int p = beginInclusive; p < endExclusive; p++) { +$if(BytesRef)$ + appendBytesRef(vector.getBytesRef(p, scratch)); +$else$ + append$Type$(vector.get$Type$(p)); +$endif$ + } + } + + @Override + public $Type$BlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + this.mvOrdering = mvOrdering; + return this; + } + + @Override + public $Type$Block build() { + finish(); + if (hasNonNullValue && positionCount == 1 && valueCount == 1) { +$if(BytesRef)$ + return new ConstantBytesRefVector(values.get(0, new BytesRef()), 1).asBlock(); + } else { +$else$ + return new Constant$Type$Vector(values[0], 1).asBlock(); + } else { + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } +$endif$ + if (isDense() && singleValued()) { + return new $Type$ArrayVector(values, positionCount).asBlock(); + } else { + return new $Type$ArrayBlock(values, positionCount, firstValueIndexes, nullsMask, mvOrdering); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ConstantVector.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ConstantVector.java.st new file mode 100644 index 0000000000000..3915c0c0f7fbc --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-ConstantVector.java.st @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$endif$ + +/** + * Vector implementation that stores a constant $type$ value. + * This class is generated. Do not edit it. + */ +public final class Constant$Type$Vector extends AbstractVector implements $Type$Vector { + + private final $type$ value; + + public Constant$Type$Vector($type$ value, int positionCount) { + super(positionCount); + this.value = value; + } + + @Override +$if(BytesRef)$ + public BytesRef getBytesRef(int position, BytesRef ignore) { +$else$ + public $type$ get$Type$(int position) { +$endif$ + return value; + } + + @Override + public $Type$Block asBlock() { + return new $Type$VectorBlock(this); + } + + @Override + public $Type$Vector filter(int... positions) { + return new Constant$Type$Vector(value, positions.length); + } + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public boolean isConstant() { + return true; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Vector that) { + return $Type$Vector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Vector.hash(this); + } + + public String toString() { + return getClass().getSimpleName() + "[positions=" + getPositionCount() + ", value=" + value + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterBlock.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterBlock.java.st new file mode 100644 index 0000000000000..097dfef0c6864 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterBlock.java.st @@ -0,0 +1,139 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$endif$ + +/** + * Filter block for $Type$Blocks. + * This class is generated. Do not edit it. + */ +final class Filter$Type$Block extends AbstractFilterBlock implements $Type$Block { + + private final $Type$Block block; + + Filter$Type$Block($Type$Block block, int... positions) { + super(block, positions); + this.block = block; + } + + @Override + public $Type$Vector asVector() { + return null; + } + + @Override +$if(BytesRef)$ + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return block.getBytesRef(valueIndex, dest); +$else$ + public $type$ get$Type$(int valueIndex) { + return block.get$Type$(valueIndex); +$endif$ + } + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public $Type$Block filter(int... positions) { + return new Filter$Type$Block(this, positions); + } + + @Override + public $Type$Block expand() { + if (false == block.mayHaveMultivaluedFields()) { + return this; + } + /* + * Build a copy of the target block, selecting only the positions + * we've been assigned and expanding all multivalued fields + * into single valued fields. + */ + $Type$Block.Builder builder = $Type$Block.newBlockBuilder(positions.length); +$if(BytesRef)$ + BytesRef scratch = new BytesRef(); +$endif$ + for (int p : positions) { + if (block.isNull(p)) { + builder.appendNull(); + continue; + } + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { +$if(BytesRef)$ + BytesRef v = block.getBytesRef(i, scratch); + builder.appendBytesRef(v); +$else$ + builder.append$Type$(block.get$Type$(i)); +$endif$ + } + } + return builder.build(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Block that) { + return $Type$Block.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Block.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int p = 0; p < positions; p++) { + if (p > 0) { + sb.append(", "); + } + int start = getFirstValueIndex(p); + int count = getValueCount(p); + if (count == 1) { +$if(BytesRef)$ + sb.append(get$Type$(start, new BytesRef())); +$else$ + sb.append(get$Type$(start)); +$endif$ + continue; + } + sb.append('['); + int end = start + count; + for (int i = start; i < end; i++) { + if (i > start) { + sb.append(", "); + } +$if(BytesRef)$ + sb.append(get$Type$(i, new BytesRef())); +$else$ + sb.append(get$Type$(i)); +$endif$ + } + sb.append(']'); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterVector.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterVector.java.st new file mode 100644 index 0000000000000..0f7c69805f406 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-FilterVector.java.st @@ -0,0 +1,93 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$endif$ + +/** + * Filter vector for $Type$Vectors. + * This class is generated. Do not edit it. + */ +public final class Filter$Type$Vector extends AbstractFilterVector implements $Type$Vector { + + private final $Type$Vector vector; + + Filter$Type$Vector($Type$Vector vector, int... positions) { + super(positions); + this.vector = vector; + } + + @Override +$if(BytesRef)$ + public BytesRef getBytesRef(int position, BytesRef dest) { + return vector.getBytesRef(mapPosition(position), dest); +$else$ + public $type$ get$Type$(int position) { + return vector.get$Type$(mapPosition(position)); +$endif$ + } + + @Override + public $Type$Block asBlock() { + return new $Type$VectorBlock(this); + } + + @Override + public ElementType elementType() { + return ElementType.$TYPE$; + } + + @Override + public boolean isConstant() { + return vector.isConstant(); + } + + @Override + public $Type$Vector filter(int... positions) { + return new Filter$Type$Vector(this, positions); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Vector that) { + return $Type$Vector.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Vector.hash(this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()); + sb.append("[positions=" + getPositionCount() + ", values=["); + appendValues(sb); + sb.append("]]"); + return sb.toString(); + } + + private void appendValues(StringBuilder sb) { + final int positions = getPositionCount(); + for (int i = 0; i < positions; i++) { + if (i > 0) { + sb.append(", "); + } +$if(BytesRef)$ + sb.append(get$Type$(i, new BytesRef())); +$else$ + sb.append(get$Type$(i)); +$endif$ + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Vector.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Vector.java.st new file mode 100644 index 0000000000000..85cc558b3f5f3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Vector.java.st @@ -0,0 +1,128 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$endif$ + +/** + * Vector that stores $type$ values. + * This class is generated. Do not edit it. + */ +$if(BytesRef)$ +public sealed interface $Type$Vector extends Vector permits Constant$Type$Vector, Filter$Type$Vector, $Type$ArrayVector { +$elseif(boolean)$ +public sealed interface $Type$Vector extends Vector permits Constant$Type$Vector, Filter$Type$Vector, $Type$ArrayVector, + $Type$BigArrayVector { +$elseif(double)$ +public sealed interface $Type$Vector extends Vector permits Constant$Type$Vector, Filter$Type$Vector, $Type$ArrayVector, + $Type$BigArrayVector { +$else$ +public sealed interface $Type$Vector extends Vector permits Constant$Type$Vector, Filter$Type$Vector, $Type$ArrayVector, $Type$BigArrayVector { +$endif$ + +$if(BytesRef)$ + BytesRef getBytesRef(int position, BytesRef dest); + +$else$ + $type$ get$Type$(int position); +$endif$ + + @Override + $Type$Block asBlock(); + + @Override + $Type$Vector filter(int... positions); + + /** + * Compares the given object with this vector for equality. Returns {@code true} if and only if the + * given object is a $Type$Vector, and both vectors are {@link #equals($Type$Vector, $Type$Vector) equal}. + */ + @Override + boolean equals(Object obj); + + /** Returns the hash code of this vector, as defined by {@link #hash($Type$Vector)}. */ + @Override + int hashCode(); + + /** + * Returns {@code true} if the given vectors are equal to each other, otherwise {@code false}. + * Two vectors are considered equal if they have the same position count, and contain the same + * values in the same order. This definition ensures that the equals method works properly + * across different implementations of the $Type$Vector interface. + */ + static boolean equals($Type$Vector vector1, $Type$Vector vector2) { + final int positions = vector1.getPositionCount(); + if (positions != vector2.getPositionCount()) { + return false; + } + for (int pos = 0; pos < positions; pos++) { +$if(BytesRef)$ + if (vector1.getBytesRef(pos, new BytesRef()).equals(vector2.getBytesRef(pos, new BytesRef())) == false) { +$else$ + if (vector1.get$Type$(pos) != vector2.get$Type$(pos)) { +$endif$ + return false; + } + } + return true; + } + + /** + * Generates the hash code for the given vector. The hash code is computed from the vector's values. + * This ensures that {@code vector1.equals(vector2)} implies that {@code vector1.hashCode()==vector2.hashCode()} + * for any two vectors, {@code vector1} and {@code vector2}, as required by the general contract of + * {@link Object#hashCode}. + */ + static int hash($Type$Vector vector) { + final int len = vector.getPositionCount(); + int result = 1; + for (int pos = 0; pos < len; pos++) { +$if(BytesRef)$ + result = 31 * result + vector.getBytesRef(pos, new BytesRef()).hashCode(); +$elseif(boolean)$ + result = 31 * result + Boolean.hashCode(vector.getBoolean(pos)); +$elseif(int)$ + result = 31 * result + vector.getInt(pos); +$elseif(long)$ + long element = vector.getLong(pos); + result = 31 * result + (int) (element ^ (element >>> 32)); +$elseif(double)$ + long element = Double.doubleToLongBits(vector.getDouble(pos)); + result = 31 * result + (int) (element ^ (element >>> 32)); +$endif$ + } + return result; + } + + static Builder newVectorBuilder(int estimatedSize) { + return new $Type$VectorBuilder(estimatedSize); + } + +$if(int)$ + /** Create a vector for a range of ints. */ + static IntVector range(int startInclusive, int endExclusive) { + int[] values = new int[endExclusive - startInclusive]; + for (int i = 0; i < values.length; i++) { + values[i] = startInclusive + i; + } + return new IntArrayVector(values, values.length); + } +$endif$ + + sealed interface Builder extends Vector.Builder permits $Type$VectorBuilder { + /** + * Appends a $type$ to the current entry. + */ + Builder append$Type$($type$ value); + + @Override + $Type$Vector build(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBlock.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBlock.java.st new file mode 100644 index 0000000000000..d323465ebb1a0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBlock.java.st @@ -0,0 +1,128 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +$else$ +$endif$ +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Block view of a $Type$Vector. + * This class is generated. Do not edit it. + */ +public final class $Type$VectorBlock extends AbstractVectorBlock implements $Type$Block { + + private final $Type$Vector vector; + + $Type$VectorBlock($Type$Vector vector) { + super(vector.getPositionCount()); + this.vector = vector; + } + + @Override + public $Type$Vector asVector() { + return vector; + } + + @Override +$if(BytesRef)$ + public BytesRef getBytesRef(int valueIndex, BytesRef dest) { + return vector.getBytesRef(valueIndex, dest); +$else$ + public $type$ get$Type$(int valueIndex) { + return vector.get$Type$(valueIndex); +$endif$ + } + + @Override + public int getTotalValueCount() { + return vector.getPositionCount(); + } + + @Override + public ElementType elementType() { + return vector.elementType(); + } + + @Override + public $Type$Block filter(int... positions) { + return new Filter$Type$Vector(vector, positions).asBlock(); + } + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Block.class, + "$Type$VectorBlock", + $Type$VectorBlock::of + ); + + @Override + public String getWriteableName() { + return "$Type$VectorBlock"; + } + + static $Type$VectorBlock of(StreamInput in) throws IOException { + final int positions = in.readVInt(); + final boolean constant = in.readBoolean(); + if (constant && positions > 0) { + return new $Type$VectorBlock(new Constant$Type$Vector(in.read$Type$(), positions)); + } else { + var builder = $Type$Vector.newVectorBuilder(positions); + for (int i = 0; i < positions; i++) { + builder.append$Type$(in.read$Type$()); + } + return new $Type$VectorBlock(builder.build()); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final $Type$Vector vector = this.vector; + final int positions = vector.getPositionCount(); + out.writeVInt(positions); + out.writeBoolean(vector.isConstant()); + if (vector.isConstant() && positions > 0) { +$if(BytesRef)$ + out.write$Type$(get$Type$(0, new BytesRef())); +$else$ + out.write$Type$(get$Type$(0)); +$endif$ + } else { + for (int i = 0; i < positions; i++) { +$if(BytesRef)$ + out.write$Type$(get$Type$(i, new BytesRef())); +$else$ + out.write$Type$(get$Type$(i)); +$endif$ + } + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof $Type$Block that) { + return $Type$Block.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return $Type$Block.hash(this); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[vector=" + vector + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBuilder.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBuilder.java.st new file mode 100644 index 0000000000000..4c4747e949bff --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-VectorBuilder.java.st @@ -0,0 +1,89 @@ +/* + * 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.compute.data; + +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; + +$else$ +import java.util.Arrays; +$endif$ + +/** + * Block build of $Type$Blocks. + * This class is generated. Do not edit it. + */ +final class $Type$VectorBuilder extends AbstractVectorBuilder implements $Type$Vector.Builder { + +$if(BytesRef)$ + private BytesRefArray values; + + BytesRefVectorBuilder(int estimatedSize) { + this(estimatedSize, BigArrays.NON_RECYCLING_INSTANCE); + } + + BytesRefVectorBuilder(int estimatedSize, BigArrays bigArrays) { + values = new BytesRefArray(Math.max(estimatedSize, 2), bigArrays); + } + +$else$ + private $type$[] values; + + $Type$VectorBuilder(int estimatedSize) { + values = new $type$[Math.max(estimatedSize, 2)]; + } +$endif$ + + @Override + public $Type$VectorBuilder append$Type$($type$ value) { + ensureCapacity(); +$if(BytesRef)$ + values.append(value); +$else$ + values[valueCount] = value; +$endif$ + valueCount++; + return this; + } + + @Override + protected int valuesLength() { +$if(BytesRef)$ + return Integer.MAX_VALUE; // allow the BytesRefArray through its own append +$else$ + return values.length; +$endif$ + } + + @Override + protected void growValuesArray(int newSize) { +$if(BytesRef)$ + throw new AssertionError("should not reach here"); +$else$ + values = Arrays.copyOf(values, newSize); +$endif$ + } + + @Override + public $Type$Vector build() { + if (valueCount == 1) { +$if(BytesRef)$ + return new ConstantBytesRefVector(values.get(0, new BytesRef()), 1); + } +$else$ + return new Constant$Type$Vector(values[0], 1); + } + if (values.length - valueCount > 1024 || valueCount < (values.length / 2)) { + values = Arrays.copyOf(values, valueCount); + } +$endif$ + return new $Type$ArrayVector(values, valueCount); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java new file mode 100644 index 0000000000000..4290075b05ae8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java @@ -0,0 +1,698 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.index.fielddata.FieldData; +import org.elasticsearch.index.fielddata.NumericDoubleValues; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; + +import java.io.IOException; + +/** + * A reader that supports reading doc-values from a Lucene segment in Block fashion. + */ +public abstract class BlockDocValuesReader { + + protected final Thread creationThread; + + public BlockDocValuesReader() { + this.creationThread = Thread.currentThread(); + } + + /** + * Returns the current doc that this reader is on. + */ + public abstract int docID(); + + /** + * The {@link Block.Builder} for data of this type. + */ + public abstract Block.Builder builder(int positionCount); + + /** + * Reads the values of the given documents specified in the input block + */ + public abstract Block readValues(IntVector docs) throws IOException; + + /** + * Reads the values of the given document into the builder + */ + public abstract void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException; + + /** + * Checks if the reader can be used to read a range documents starting with the given docID by the current thread. + */ + public static boolean canReuse(BlockDocValuesReader reader, int startingDocID) { + return reader != null && reader.creationThread == Thread.currentThread() && reader.docID() <= startingDocID; + } + + public static BlockDocValuesReader createBlockReader( + ValuesSource valuesSource, + ValuesSourceType valuesSourceType, + ElementType elementType, + LeafReaderContext leafReaderContext + ) throws IOException { + if (valuesSourceType instanceof UnsupportedValueSourceType) { + final UnsupportedValueSource bytesVS = (UnsupportedValueSource) valuesSource; + final SortedBinaryDocValues bytesValues = bytesVS.bytesValues(leafReaderContext); + return new BytesValuesReader(bytesValues); + } + if (CoreValuesSourceType.NUMERIC.equals(valuesSourceType) || CoreValuesSourceType.DATE.equals(valuesSourceType)) { + ValuesSource.Numeric numericVS = (ValuesSource.Numeric) valuesSource; + if (numericVS.isFloatingPoint()) { + if (elementType != ElementType.DOUBLE) { + throw new UnsupportedOperationException("can't extract [" + elementType + "] from floating point fields"); + } + final SortedNumericDoubleValues doubleValues = numericVS.doubleValues(leafReaderContext); + final NumericDoubleValues singleton = FieldData.unwrapSingleton(doubleValues); + if (singleton != null) { + return new DoubleSingletonValuesReader(singleton); + } + return new DoubleValuesReader(doubleValues); + } else { + final SortedNumericDocValues longValues = numericVS.longValues(leafReaderContext); + final NumericDocValues singleton = DocValues.unwrapSingleton(longValues); + if (singleton != null) { + return switch (elementType) { + case LONG -> new LongSingletonValuesReader(singleton); + case INT -> new IntSingletonValuesReader(singleton); + default -> throw new UnsupportedOperationException("can't extract [" + elementType + "] from integer fields"); + }; + } + return switch (elementType) { + case LONG -> new LongValuesReader(longValues); + case INT -> new IntValuesReader(longValues); + default -> throw new UnsupportedOperationException("can't extract [" + elementType + "] from integer fields"); + }; + } + } + if (CoreValuesSourceType.KEYWORD.equals(valuesSourceType) || CoreValuesSourceType.IP.equals(valuesSourceType)) { + if (elementType != ElementType.BYTES_REF) { + throw new UnsupportedOperationException("can't extract [" + elementType + "] from keywords"); + } + final ValuesSource.Bytes bytesVS = (ValuesSource.Bytes) valuesSource; + final SortedBinaryDocValues bytesValues = bytesVS.bytesValues(leafReaderContext); + return new BytesValuesReader(bytesValues); + } + if (CoreValuesSourceType.BOOLEAN.equals(valuesSourceType)) { + if (elementType != ElementType.BOOLEAN) { + throw new UnsupportedOperationException("can't extract [" + elementType + "] from booleans"); + } + ValuesSource.Numeric numericVS = (ValuesSource.Numeric) valuesSource; + final SortedNumericDocValues longValues = numericVS.longValues(leafReaderContext); + final NumericDocValues singleton = DocValues.unwrapSingleton(longValues); + if (singleton != null) { + return new BooleanSingletonValuesReader(singleton); + } + return new BooleanValuesReader(longValues); + } + if (valuesSourceType instanceof NullValueSourceType) { + return new NullValuesReader(); + } + throw new IllegalArgumentException("Field type [" + valuesSourceType.typeName() + "] is not supported"); + } + + private static class LongSingletonValuesReader extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + LongSingletonValuesReader(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public LongBlock.Builder builder(int positionCount) { + return LongBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public LongBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + int lastDoc = -1; + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendLong(numericDocValues.longValue()); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + LongBlock.Builder blockBuilder = (LongBlock.Builder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendLong(numericDocValues.longValue()); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return "LongSingletonValuesReader"; + } + } + + private static class LongValuesReader extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + LongValuesReader(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public LongBlock.Builder builder(int positionCount) { + return LongBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public LongBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + read(docId, (LongBlock.Builder) builder); + } + + private void read(int doc, LongBlock.Builder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendLong(numericDocValues.nextValue()); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendLong(numericDocValues.nextValue()); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return "LongValuesReader"; + } + } + + private static class IntSingletonValuesReader extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + IntSingletonValuesReader(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public IntBlock.Builder builder(int positionCount) { + return IntBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public IntBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + int lastDoc = -1; + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + IntBlock.Builder blockBuilder = (IntBlock.Builder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return "LongSingletonValuesReader"; + } + } + + private static class IntValuesReader extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + IntValuesReader(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public IntBlock.Builder builder(int positionCount) { + return IntBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public IntBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < this.docID) { + // TODO this may not be true after sorting many docs in a single segment. + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + read(docId, (IntBlock.Builder) builder); + } + + private void read(int doc, IntBlock.Builder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return "LongValuesReader"; + } + } + + private static class DoubleSingletonValuesReader extends BlockDocValuesReader { + private final NumericDoubleValues numericDocValues; + private int docID = -1; + + DoubleSingletonValuesReader(NumericDoubleValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public DoubleBlock.Builder builder(int positionCount) { + return DoubleBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public DoubleBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + int lastDoc = -1; + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendDouble(numericDocValues.doubleValue()); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + this.docID = doc; + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + this.docID = docId; + DoubleBlock.Builder blockBuilder = (DoubleBlock.Builder) builder; + if (numericDocValues.advanceExact(this.docID)) { + blockBuilder.appendDouble(numericDocValues.doubleValue()); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "DoubleSingletonValuesReader"; + } + } + + private static class DoubleValuesReader extends BlockDocValuesReader { + private final SortedNumericDoubleValues numericDocValues; + private int docID = -1; + + DoubleValuesReader(SortedNumericDoubleValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public DoubleBlock.Builder builder(int positionCount) { + return DoubleBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public DoubleBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + read(docId, (DoubleBlock.Builder) builder); + } + + private void read(int doc, DoubleBlock.Builder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendDouble(numericDocValues.nextValue()); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendDouble(numericDocValues.nextValue()); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "DoubleValuesReader"; + } + } + + private static class BytesValuesReader extends BlockDocValuesReader { + private final SortedBinaryDocValues binaryDV; + private int docID = -1; + + BytesValuesReader(SortedBinaryDocValues binaryDV) { + this.binaryDV = binaryDV; + } + + @Override + public BytesRefBlock.Builder builder(int positionCount) { + return BytesRefBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public BytesRefBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + for (int i = 0; i < docs.getPositionCount(); i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + read(docId, (BytesRefBlock.Builder) builder); + } + + private void read(int doc, BytesRefBlock.Builder builder) throws IOException { + this.docID = doc; + if (false == binaryDV.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = binaryDV.docValueCount(); + if (count == 1) { + builder.appendBytesRef(binaryDV.nextValue()); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendBytesRef(binaryDV.nextValue()); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "BytesValuesReader"; + } + } + + private static class BooleanSingletonValuesReader extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + BooleanSingletonValuesReader(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public BooleanBlock.Builder builder(int positionCount) { + return BooleanBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public BooleanBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + int lastDoc = -1; + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendBoolean(numericDocValues.longValue() != 0); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + BooleanBlock.Builder blockBuilder = (BooleanBlock.Builder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendBoolean(numericDocValues.longValue() != 0); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } + } + + private static class BooleanValuesReader extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + BooleanValuesReader(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public BooleanBlock.Builder builder(int positionCount) { + return BooleanBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.ASCENDING); + } + + @Override + public BooleanBlock readValues(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + var blockBuilder = builder(positionCount); + for (int i = 0; i < positionCount; i++) { + int doc = docs.getInt(i); + // docs within same block must be in order + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder.build(); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { + read(docId, (BooleanBlock.Builder) builder); + } + + private void read(int doc, BooleanBlock.Builder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendBoolean(numericDocValues.nextValue() != 0); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendBoolean(numericDocValues.nextValue() != 0); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } + } + + private static class NullValuesReader extends BlockDocValuesReader { + private int docID = -1; + + @Override + public Block.Builder builder(int positionCount) { + return ElementType.NULL.newBlockBuilder(positionCount); + } + + @Override + public Block readValues(IntVector docs) throws IOException { + return Block.constantNullBlock(docs.getPositionCount()); + } + + @Override + public void readValuesFromSingleDoc(int docId, Block.Builder builder) { + this.docID = docId; + builder.appendNull(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return getClass().getSimpleName(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockOrdinalsReader.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockOrdinalsReader.java new file mode 100644 index 0000000000000..d55540a37b8a6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockOrdinalsReader.java @@ -0,0 +1,58 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.SortedSetDocValues; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; + +import java.io.IOException; + +public final class BlockOrdinalsReader { + private final SortedSetDocValues sortedSetDocValues; + private final Thread creationThread; + + public BlockOrdinalsReader(SortedSetDocValues sortedSetDocValues) { + this.sortedSetDocValues = sortedSetDocValues; + this.creationThread = Thread.currentThread(); + } + + public LongBlock readOrdinals(IntVector docs) throws IOException { + final int positionCount = docs.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int doc = docs.getInt(p); + if (false == sortedSetDocValues.advanceExact(doc)) { + builder.appendNull(); + continue; + } + int count = sortedSetDocValues.docValueCount(); + if (count == 1) { + builder.appendLong(sortedSetDocValues.nextOrd()); + continue; + } + builder.beginPositionEntry(); + for (int i = 0; i < count; i++) { + builder.appendLong(sortedSetDocValues.nextOrd()); + } + builder.endPositionEntry(); + } + return builder.build(); + } + + public int docID() { + return sortedSetDocValues.docID(); + } + + /** + * Checks if the reader can be used to read a range documents starting with the given docID by the current thread. + */ + public static boolean canReuse(BlockOrdinalsReader reader, int startingDocID) { + return reader != null && reader.creationThread == Thread.currentThread() && reader.docID() <= startingDocID; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/DataPartitioning.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/DataPartitioning.java new file mode 100644 index 0000000000000..926b9e08d2e08 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/DataPartitioning.java @@ -0,0 +1,17 @@ +/* + * 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.compute.lucene; + +public enum DataPartitioning { + + SHARD, + + SEGMENT, + + DOC, +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdFieldIndexFieldData.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdFieldIndexFieldData.java new file mode 100644 index 0000000000000..d91c758ab3bd9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdFieldIndexFieldData.java @@ -0,0 +1,129 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.SortField; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; +import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.script.field.DocValuesScriptFieldFactory; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.MultiValueMode; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.sort.BucketedSort; +import org.elasticsearch.search.sort.SortOrder; + +import java.io.IOException; +import java.util.Set; + +public class IdFieldIndexFieldData implements IndexFieldData { + + private static final String FIELD_NAME = IdFieldMapper.NAME; + private final ValuesSourceType valuesSourceType; + private final StoredFieldLoader loader; + + protected IdFieldIndexFieldData(ValuesSourceType valuesSourceType) { + this.valuesSourceType = valuesSourceType; + this.loader = StoredFieldLoader.create(false, Set.of(FIELD_NAME)); + } + + @Override + public String getFieldName() { + return FIELD_NAME; + } + + @Override + public ValuesSourceType getValuesSourceType() { + return valuesSourceType; + } + + @Override + public final IdFieldLeafFieldData load(LeafReaderContext context) { + try { + return loadDirect(context); + } catch (Exception e) { + throw ExceptionsHelper.convertToElastic(e); + } + } + + @Override + public final IdFieldLeafFieldData loadDirect(LeafReaderContext context) throws Exception { + return new IdFieldLeafFieldData(loader.getLoader(context, null)); + } + + @Override + public SortField sortField(Object missingValue, MultiValueMode sortMode, XFieldComparatorSource.Nested nested, boolean reverse) { + throw new IllegalArgumentException("not supported for stored field fallback"); + } + + @Override + public BucketedSort newBucketedSort( + BigArrays bigArrays, + Object missingValue, + MultiValueMode sortMode, + XFieldComparatorSource.Nested nested, + SortOrder sortOrder, + DocValueFormat format, + int bucketSize, + BucketedSort.ExtraData extra + ) { + throw new IllegalArgumentException("not supported for stored field fallback"); + } + + class IdFieldLeafFieldData implements LeafFieldData { + private final LeafStoredFieldLoader loader; + + protected IdFieldLeafFieldData(LeafStoredFieldLoader loader) { + this.loader = loader; + } + + @Override + public DocValuesScriptFieldFactory getScriptFieldFactory(String name) { + throw new IllegalArgumentException("not supported for _id field"); + } + + @Override + public long ramBytesUsed() { + return 0L; + } + + @Override + public void close() {} + + @Override + public SortedBinaryDocValues getBytesValues() { + return new SortedBinaryDocValues() { + private String id; + + @Override + public boolean advanceExact(int doc) throws IOException { + loader.advanceTo(doc); + id = loader.id(); + return id != null; + } + + @Override + public int docValueCount() { + return 1; + } + + @Override + public BytesRef nextValue() throws IOException { + return new BytesRef(id); + } + }; + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdValueSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdValueSource.java new file mode 100644 index 0000000000000..906d6a0932806 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/IdValueSource.java @@ -0,0 +1,26 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +public class IdValueSource extends ValuesSource.Bytes { + + private final IdFieldIndexFieldData indexFieldData; + + public IdValueSource(IdFieldIndexFieldData indexFieldData) { + this.indexFieldData = indexFieldData; + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext leafReaderContext) { + return indexFieldData.load(leafReaderContext).getBytesValues(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java new file mode 100644 index 0000000000000..35ccb7daca1a4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -0,0 +1,442 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Weight; +import org.elasticsearch.common.Strings; +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.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +public abstract class LuceneOperator extends SourceOperator { + + public static final int NO_LIMIT = Integer.MAX_VALUE; + + private static final int MAX_DOCS_PER_SLICE = 250_000; // copied from IndexSearcher + private static final int MAX_SEGMENTS_PER_SLICE = 5; // copied from IndexSearcher + + @Nullable + final IndexReader indexReader; + final int shardId; + @Nullable + final Query query; + final List leaves; + final int maxPageSize; + final int minPageSize; + + Weight weight; + + int currentLeaf = 0; + LuceneSourceOperator.PartialLeafReaderContext currentLeafReaderContext = null; + BulkScorer currentScorer = null; + private Thread createdScorerThread = null; + + int currentPagePos; + int currentScorerPos; + int pagesEmitted; + + LuceneOperator(IndexReader reader, int shardId, Query query, int maxPageSize) { + this.indexReader = reader; + this.shardId = shardId; + this.leaves = reader.leaves().stream().map(PartialLeafReaderContext::new).collect(Collectors.toList()); + this.query = query; + this.maxPageSize = maxPageSize; + this.minPageSize = Math.max(1, maxPageSize / 2); + } + + LuceneOperator(Weight weight, int shardId, List leaves, int maxPageSize) { + this.indexReader = null; + this.shardId = shardId; + this.leaves = leaves; + this.query = null; + this.weight = weight; + this.maxPageSize = maxPageSize; + this.minPageSize = maxPageSize / 2; + } + + abstract LuceneOperator docSliceLuceneOperator(List slice); + + abstract LuceneOperator segmentSliceLuceneOperator(IndexSearcher.LeafSlice leafSlice); + + public abstract static class LuceneOperatorFactory implements SourceOperatorFactory { + + final Function queryFunction; + + final DataPartitioning dataPartitioning; + + final int maxPageSize; + + final List searchContexts; + + final int taskConcurrency; + + final int limit; + + private Iterator iterator; + + public LuceneOperatorFactory( + List searchContexts, + Function queryFunction, + DataPartitioning dataPartitioning, + int taskConcurrency, + int maxPageSize, + int limit + ) { + this.searchContexts = searchContexts; + this.queryFunction = queryFunction; + this.dataPartitioning = dataPartitioning; + this.taskConcurrency = taskConcurrency; + this.maxPageSize = maxPageSize; + this.limit = limit; + } + + abstract LuceneOperator luceneOperatorForShard(int shardIndex); + + Iterator sourceOperatorIterator() { + final List luceneOperators = new ArrayList<>(); + for (int shardIndex = 0; shardIndex < searchContexts.size(); shardIndex++) { + LuceneOperator queryOperator = luceneOperatorForShard(shardIndex); + switch (dataPartitioning) { + case SHARD -> luceneOperators.add(queryOperator); + case SEGMENT -> luceneOperators.addAll(queryOperator.segmentSlice()); + case DOC -> luceneOperators.addAll(queryOperator.docSlice(taskConcurrency)); + default -> throw new UnsupportedOperationException(); + } + } + return luceneOperators.iterator(); + } + + @Override + public final SourceOperator get(DriverContext driverContext) { + if (iterator == null) { + iterator = sourceOperatorIterator(); + } + if (iterator.hasNext()) { + return iterator.next(); + } else { + throw new IllegalStateException("Lucene operator factory exhausted"); + } + } + + public int size() { + return Math.toIntExact( + StreamSupport.stream(Spliterators.spliteratorUnknownSize(sourceOperatorIterator(), Spliterator.ORDERED), false).count() + ); + } + + public int maxPageSize() { + return maxPageSize; + } + + public int limit() { + return limit; + } + } + + /** + * Split this source operator into a given number of slices + */ + public List docSlice(int numSlices) { + if (weight != null) { + throw new IllegalStateException("can only call slice method once"); + } + initializeWeightIfNecessary(); + + List operators = new ArrayList<>(); + for (List slice : docSlices(indexReader, numSlices)) { + operators.add(docSliceLuceneOperator(slice)); + } + return operators; + } + + static final List> docSlices(IndexReader indexReader, int numSlices) { + final int totalDocCount = indexReader.maxDoc(); + final int normalMaxDocsPerSlice = totalDocCount / numSlices; + final int extraDocsInFirstSlice = totalDocCount % numSlices; + final List> slices = new ArrayList<>(); + int docsAllocatedInCurrentSlice = 0; + List currentSlice = null; + int maxDocsPerSlice = normalMaxDocsPerSlice + extraDocsInFirstSlice; + for (LeafReaderContext ctx : indexReader.leaves()) { + final int numDocsInLeaf = ctx.reader().maxDoc(); + int minDoc = 0; + while (minDoc < numDocsInLeaf) { + int numDocsToUse = Math.min(maxDocsPerSlice - docsAllocatedInCurrentSlice, numDocsInLeaf - minDoc); + if (numDocsToUse <= 0) { + break; + } + if (currentSlice == null) { + currentSlice = new ArrayList<>(); + } + currentSlice.add(new PartialLeafReaderContext(ctx, minDoc, minDoc + numDocsToUse)); + minDoc += numDocsToUse; + docsAllocatedInCurrentSlice += numDocsToUse; + if (docsAllocatedInCurrentSlice == maxDocsPerSlice) { + slices.add(currentSlice); + maxDocsPerSlice = normalMaxDocsPerSlice; // once the first slice with the extra docs is added, no need for extra docs + currentSlice = null; + docsAllocatedInCurrentSlice = 0; + } + } + } + if (currentSlice != null) { + slices.add(currentSlice); + } + if (numSlices < totalDocCount && slices.size() != numSlices) { + throw new IllegalStateException("wrong number of slices, expected " + numSlices + " but got " + slices.size()); + } + if (slices.stream() + .flatMapToInt( + l -> l.stream().mapToInt(partialLeafReaderContext -> partialLeafReaderContext.maxDoc - partialLeafReaderContext.minDoc) + ) + .sum() != totalDocCount) { + throw new IllegalStateException("wrong doc count"); + } + return slices; + } + + /** + * Uses Lucene's own slicing method, which creates per-segment level slices + */ + public List segmentSlice() { + if (weight != null) { + throw new IllegalStateException("can only call slice method once"); + } + initializeWeightIfNecessary(); + List operators = new ArrayList<>(); + for (IndexSearcher.LeafSlice leafSlice : segmentSlices(indexReader)) { + operators.add(segmentSliceLuceneOperator(leafSlice)); + } + return operators; + } + + static IndexSearcher.LeafSlice[] segmentSlices(IndexReader indexReader) { + return IndexSearcher.slices(indexReader.leaves(), MAX_DOCS_PER_SLICE, MAX_SEGMENTS_PER_SLICE); + } + + @Override + public void finish() { + throw new UnsupportedOperationException(); + } + + void initializeWeightIfNecessary() { + if (weight == null) { + try { + IndexSearcher indexSearcher = new IndexSearcher(indexReader); + weight = indexSearcher.createWeight(indexSearcher.rewrite(new ConstantScoreQuery(query)), ScoreMode.COMPLETE_NO_SCORES, 1); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + boolean maybeReturnEarlyOrInitializeScorer() { + // Reset the Scorer if the operator is run by a different thread + if (currentLeafReaderContext != null && createdScorerThread != Thread.currentThread()) { + try { + currentScorer = weight.bulkScorer(currentLeafReaderContext.leafReaderContext); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + createdScorerThread = Thread.currentThread(); + return false; + } + if (currentLeafReaderContext == null) { + assert currentScorer == null : "currentScorer wasn't reset"; + do { + currentLeafReaderContext = leaves.get(currentLeaf); + currentScorerPos = currentLeafReaderContext.minDoc; + try { + currentScorer = weight.bulkScorer(currentLeafReaderContext.leafReaderContext); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + if (currentScorer == null) { + // doesn't match anything; move to the next leaf or abort if finished + currentLeaf++; + if (doneCollecting()) { + return true; + } + } + } while (currentScorer == null); + createdScorerThread = Thread.currentThread(); + } + return false; + } + + protected abstract boolean doneCollecting(); + + @Override + public void close() { + + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("shardId=").append(shardId); + sb.append(", maxPageSize=").append(maxPageSize); + sb.append("]"); + return sb.toString(); + } + + static class PartialLeafReaderContext { + + final LeafReaderContext leafReaderContext; + final int minDoc; // incl + final int maxDoc; // excl + + PartialLeafReaderContext(LeafReaderContext leafReaderContext, int minDoc, int maxDoc) { + this.leafReaderContext = leafReaderContext; + this.minDoc = minDoc; + this.maxDoc = maxDoc; + } + + PartialLeafReaderContext(LeafReaderContext leafReaderContext) { + this(leafReaderContext, 0, leafReaderContext.reader().maxDoc()); + } + + } + + @Override + public Operator.Status status() { + return new Status(this); + } + + public static class Status implements Operator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "lucene_source", + Status::new + ); + + private final int currentLeaf; + private final int totalLeaves; + private final int pagesEmitted; + private final int leafPosition; + private final int leafSize; + + private Status(LuceneOperator operator) { + currentLeaf = operator.currentLeaf; + totalLeaves = operator.leaves.size(); + leafPosition = operator.currentScorerPos; + LuceneOperator.PartialLeafReaderContext ctx = operator.currentLeafReaderContext; + leafSize = ctx == null ? 0 : ctx.maxDoc - ctx.minDoc; + pagesEmitted = operator.pagesEmitted; + } + + Status(int currentLeaf, int totalLeaves, int pagesEmitted, int leafPosition, int leafSize) { + this.currentLeaf = currentLeaf; + this.totalLeaves = totalLeaves; + this.leafPosition = leafPosition; + this.leafSize = leafSize; + this.pagesEmitted = pagesEmitted; + } + + Status(StreamInput in) throws IOException { + currentLeaf = in.readVInt(); + totalLeaves = in.readVInt(); + leafPosition = in.readVInt(); + leafSize = in.readVInt(); + pagesEmitted = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(currentLeaf); + out.writeVInt(totalLeaves); + out.writeVInt(leafPosition); + out.writeVInt(leafSize); + out.writeVInt(pagesEmitted); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public int currentLeaf() { + return currentLeaf; + } + + public int totalLeaves() { + return totalLeaves; + } + + public int pagesEmitted() { + return pagesEmitted; + } + + public int leafPosition() { + return leafPosition; + } + + public int leafSize() { + return leafSize; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("current_leaf", currentLeaf); + builder.field("total_leaves", totalLeaves); + builder.field("leaf_position", leafPosition); + builder.field("leaf_size", leafSize); + builder.field("pages_emitted", pagesEmitted); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return currentLeaf == status.currentLeaf + && totalLeaves == status.totalLeaves + && pagesEmitted == status.pagesEmitted + && leafPosition == status.leafPosition + && leafSize == status.leafSize; + } + + @Override + public int hashCode() { + return Objects.hash(currentLeaf, totalLeaves, pagesEmitted, leafPosition, leafSize); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java new file mode 100644 index 0000000000000..467ca03ea4b21 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java @@ -0,0 +1,180 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.Weight; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Source operator that incrementally runs Lucene searches + */ +public class LuceneSourceOperator extends LuceneOperator { + + private int numCollectedDocs = 0; + + private final int maxCollectedDocs; + + private IntVector.Builder currentDocsBuilder; + + public static class LuceneSourceOperatorFactory extends LuceneOperatorFactory { + + public LuceneSourceOperatorFactory( + List searchContexts, + Function queryFunction, + DataPartitioning dataPartitioning, + int taskConcurrency, + int maxPageSize, + int limit + ) { + super(searchContexts, queryFunction, dataPartitioning, taskConcurrency, maxPageSize, limit); + } + + @Override + LuceneOperator luceneOperatorForShard(int shardIndex) { + final SearchContext ctx = searchContexts.get(shardIndex); + final Query query = queryFunction.apply(ctx); + return new LuceneSourceOperator(ctx.getSearchExecutionContext().getIndexReader(), shardIndex, query, maxPageSize, limit); + } + + @Override + public String describe() { + return "LuceneSourceOperator[dataPartitioning = " + + dataPartitioning + + ", maxPageSize = " + + maxPageSize + + ", limit = " + + limit + + "]"; + } + } + + public LuceneSourceOperator(IndexReader reader, int shardId, Query query, int maxPageSize, int limit) { + super(reader, shardId, query, maxPageSize); + this.currentDocsBuilder = IntVector.newVectorBuilder(maxPageSize); + this.maxCollectedDocs = limit; + } + + LuceneSourceOperator(Weight weight, int shardId, List leaves, int maxPageSize, int maxCollectedDocs) { + super(weight, shardId, leaves, maxPageSize); + this.currentDocsBuilder = IntVector.newVectorBuilder(maxPageSize); + this.maxCollectedDocs = maxCollectedDocs; + } + + @Override + LuceneOperator docSliceLuceneOperator(List slice) { + return new LuceneSourceOperator(weight, shardId, slice, maxPageSize, maxCollectedDocs); + } + + @Override + LuceneOperator segmentSliceLuceneOperator(IndexSearcher.LeafSlice leafSlice) { + return new LuceneSourceOperator( + weight, + shardId, + Arrays.asList(leafSlice.leaves).stream().map(PartialLeafReaderContext::new).collect(Collectors.toList()), + maxPageSize, + maxCollectedDocs + ); + } + + @Override + protected boolean doneCollecting() { + return currentLeaf >= leaves.size() || numCollectedDocs >= maxCollectedDocs; + } + + @Override + public boolean isFinished() { + return doneCollecting(); + } + + @Override + public Page getOutput() { + if (isFinished()) { + return null; + } + + // initialize weight if not done yet + initializeWeightIfNecessary(); + + // if there are documents matching, initialize currentLeafReaderContext, currentScorer, and currentScorerPos when we switch + // to a new leaf reader, otherwise return + if (maybeReturnEarlyOrInitializeScorer()) { + return null; + } + + Page page = null; + + try { + currentScorerPos = currentScorer.score(new LeafCollector() { + @Override + public void setScorer(Scorable scorer) { + // ignore + } + + @Override + public void collect(int doc) { + if (numCollectedDocs < maxCollectedDocs) { + currentDocsBuilder.appendInt(doc); + numCollectedDocs++; + currentPagePos++; + } + } + }, + currentLeafReaderContext.leafReaderContext.reader().getLiveDocs(), + currentScorerPos, + // Note: if (maxPageSize - currentPagePos) is a small "remaining" interval, this could lead to slow collection with a + // highly selective filter. Having a large "enough" difference between max- and minPageSize (and thus currentPagePos) + // alleviates this issue. + Math.min(currentLeafReaderContext.maxDoc, currentScorerPos + maxPageSize - currentPagePos) + ); + + if (currentPagePos >= minPageSize + || currentScorerPos >= currentLeafReaderContext.maxDoc + || numCollectedDocs >= maxCollectedDocs) { + page = new Page( + currentPagePos, + new DocVector( + IntBlock.newConstantBlockWith(shardId, currentPagePos).asVector(), + IntBlock.newConstantBlockWith(currentLeafReaderContext.leafReaderContext.ord, currentPagePos).asVector(), + currentDocsBuilder.build(), + true + ).asBlock() + ); + currentDocsBuilder = IntVector.newVectorBuilder(maxPageSize); + currentPagePos = 0; + } + + if (currentScorerPos >= currentLeafReaderContext.maxDoc) { + currentLeaf++; + currentLeafReaderContext = null; + currentScorer = null; + currentScorerPos = 0; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + pagesEmitted++; + return page; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java new file mode 100644 index 0000000000000..c2725596adb92 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java @@ -0,0 +1,292 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.ReaderUtil; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.CollectorManager; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.search.Weight; +import org.elasticsearch.common.Strings; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.SortAndFormats; +import org.elasticsearch.search.sort.SortBuilder; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Source operator that builds Pages out of the output of a TopFieldCollector (aka TopN) + */ +public class LuceneTopNSourceOperator extends LuceneOperator { + + private Thread currentThread; + + private final TopFieldCollector topFieldCollector;// this should only be created via the collector manager + + private LeafCollector currentLeafCollector; + + private final List leafReaderContexts; + + private final CollectorManager collectorManager;// one for each shard + + private LeafReaderContext previousLeafReaderContext; + + /** + * Collected docs. {@code null} until we're {@link #doneCollecting}. + */ + private ScoreDoc[] scoreDocs; + /** + * The offset in {@link #scoreDocs} of the next page. + */ + private int offset = 0; + + public LuceneTopNSourceOperator(IndexReader reader, int shardId, Sort sort, Query query, int maxPageSize, int limit) { + super(reader, shardId, query, maxPageSize); + this.leafReaderContexts = reader.leaves(); + this.collectorManager = TopFieldCollector.createSharedManager(sort, limit, null, 0); + try { + this.topFieldCollector = collectorManager.newCollector(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + this.currentThread = Thread.currentThread(); + } + + private LuceneTopNSourceOperator( + Weight weight, + int shardId, + List leaves, + List leafReaderContexts, + CollectorManager collectorManager, + Thread currentThread, + int maxPageSize + ) { + super(weight, shardId, leaves, maxPageSize); + this.leafReaderContexts = leafReaderContexts; + this.collectorManager = collectorManager; + try { + this.topFieldCollector = collectorManager.newCollector(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + this.currentThread = currentThread; + } + + public static class LuceneTopNSourceOperatorFactory extends LuceneOperatorFactory { + + private final List> sorts; + + public LuceneTopNSourceOperatorFactory( + List searchContexts, + Function queryFunction, + DataPartitioning dataPartitioning, + int taskConcurrency, + int maxPageSize, + int limit, + List> sorts + ) { + super(searchContexts, queryFunction, dataPartitioning, taskConcurrency, maxPageSize, limit); + assert sorts != null; + this.sorts = sorts; + } + + @Override + LuceneOperator luceneOperatorForShard(int shardIndex) { + final SearchContext ctx = searchContexts.get(shardIndex); + final Query query = queryFunction.apply(ctx); + Sort sort = null; + try { + Optional optionalSort = SortBuilder.buildSort(sorts, ctx.getSearchExecutionContext()); + if (optionalSort.isPresent()) { + sort = optionalSort.get().sort; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return new LuceneTopNSourceOperator( + ctx.getSearchExecutionContext().getIndexReader(), + shardIndex, + sort, + query, + maxPageSize, + limit + ); + } + + @Override + public String describe() { + String notPrettySorts = sorts.stream().map(s -> Strings.toString(s)).collect(Collectors.joining(",")); + return "LuceneTopNSourceOperator[dataPartitioning = " + + dataPartitioning + + ", maxPageSize = " + + maxPageSize + + ", limit = " + + limit + + ", sorts = [" + + notPrettySorts + + "]]"; + } + + } + + @Override + LuceneOperator docSliceLuceneOperator(List slice) { + return new LuceneTopNSourceOperator(weight, shardId, slice, leafReaderContexts, collectorManager, currentThread, maxPageSize); + } + + @Override + LuceneOperator segmentSliceLuceneOperator(IndexSearcher.LeafSlice leafSlice) { + return new LuceneTopNSourceOperator( + weight, + shardId, + Arrays.asList(leafSlice.leaves).stream().map(PartialLeafReaderContext::new).collect(Collectors.toList()), + leafReaderContexts, + collectorManager, + currentThread, + maxPageSize + ); + } + + @Override + void initializeWeightIfNecessary() { + if (weight == null) { + try { + IndexSearcher indexSearcher = new IndexSearcher(indexReader); + weight = indexSearcher.createWeight(indexSearcher.rewrite(new ConstantScoreQuery(query)), ScoreMode.TOP_DOCS, 1); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + @Override + protected boolean doneCollecting() { + return currentLeaf >= leaves.size(); + } + + private boolean doneEmitting() { + /* + * If there aren't any leaves then we never initialize scoreDocs. + */ + return leaves.isEmpty() || offset >= scoreDocs.length; + } + + @Override + public boolean isFinished() { + return doneCollecting() && doneEmitting(); + } + + @Override + public Page getOutput() { + if (doneCollecting()) { + return emit(); + } + return collect(); + } + + private Page collect() { + assert false == doneCollecting(); + // initialize weight if not done yet + initializeWeightIfNecessary(); + + // if there are documents matching, initialize currentLeafReaderContext and currentScorer when we switch to a new group in the slice + if (maybeReturnEarlyOrInitializeScorer()) { + // if there are no more documents matching and we reached the final slice, build the Page + scoreDocs = topFieldCollector.topDocs().scoreDocs; + return emit(); + } + + try { + // one leaf collector per thread and per segment/leaf + if (currentLeafCollector == null + || currentThread.equals(Thread.currentThread()) == false + || previousLeafReaderContext != currentLeafReaderContext.leafReaderContext) { + currentLeafCollector = topFieldCollector.getLeafCollector(currentLeafReaderContext.leafReaderContext); + currentThread = Thread.currentThread(); + previousLeafReaderContext = currentLeafReaderContext.leafReaderContext; + } + + try { + currentScorerPos = currentScorer.score( + currentLeafCollector, + currentLeafReaderContext.leafReaderContext.reader().getLiveDocs(), + currentScorerPos, + Math.min(currentLeafReaderContext.maxDoc, currentScorerPos + maxPageSize) + ); + } catch (CollectionTerminatedException cte) { + // Lucene terminated early the collection (doing topN for an index that's sorted and the topN uses the same sorting) + currentScorerPos = currentLeafReaderContext.maxDoc; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + if (currentScorerPos >= currentLeafReaderContext.maxDoc) { + // move to the next leaf if we are done reading from the current leaf (current scorer position reached the final doc) + currentLeaf++; + currentLeafReaderContext = null; + currentScorer = null; + currentScorerPos = 0; + } + if (doneCollecting()) { + // we reached the final leaf in this slice/operator, build the single Page this operator should create + scoreDocs = topFieldCollector.topDocs().scoreDocs; + return emit(); + } + return null; + } + + private Page emit() { + assert doneCollecting(); + if (doneEmitting()) { + return null; + } + int size = Math.min(maxPageSize, scoreDocs.length - offset); + IntVector.Builder currentSegmentBuilder = IntVector.newVectorBuilder(size); + IntVector.Builder currentDocsBuilder = IntVector.newVectorBuilder(size); + + int start = offset; + offset += size; + for (int i = start; i < offset; i++) { + int doc = scoreDocs[i].doc; + int segment = ReaderUtil.subIndex(doc, leafReaderContexts); + currentSegmentBuilder.appendInt(segment); + currentDocsBuilder.appendInt(doc - leafReaderContexts.get(segment).docBase); // the offset inside the segment + } + + pagesEmitted++; + return new Page( + size, + new DocVector( + IntBlock.newConstantBlockWith(shardId, size).asVector(), + currentSegmentBuilder.build(), + currentDocsBuilder.build(), + null + ).asBlock() + ); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java new file mode 100644 index 0000000000000..fc9807b2e2410 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java @@ -0,0 +1,53 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.index.fielddata.DocValueBits; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +import java.io.IOException; +import java.util.function.Function; + +public class NullValueSource extends ValuesSource { + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + + return new SortedBinaryDocValues() { + @Override + public boolean advanceExact(int doc) throws IOException { + return true; + } + + @Override + public int docValueCount() { + return 1; + } + + @Override + public BytesRef nextValue() throws IOException { + return null; + } + }; + } + + @Override + public DocValueBits docsWithValue(LeafReaderContext context) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + protected Function roundingPreparer(AggregationContext context) throws IOException { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java new file mode 100644 index 0000000000000..fd354bd9e1a0b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java @@ -0,0 +1,50 @@ +/* + * 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.compute.lucene; + +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; + +public class NullValueSourceType implements ValuesSourceType { + + @Override + public ValuesSource getEmpty() { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource getField(FieldContext fieldContext, AggregationScript.LeafFactory script) { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource replaceMissing( + ValuesSource valuesSource, + Object rawMissing, + DocValueFormat docValueFormat, + AggregationContext context + ) { + throw new UnsupportedOperationException(); + } + + @Override + public String typeName() { + return null; + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TextValueSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TextValueSource.java new file mode 100644 index 0000000000000..04dbcd91c18c8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TextValueSource.java @@ -0,0 +1,42 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.script.field.TextDocValuesField; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +public class TextValueSource extends ValuesSource.Bytes { + + private final IndexFieldData indexFieldData; + + public TextValueSource(IndexFieldData indexFieldData) { + this.indexFieldData = indexFieldData; + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext leafReaderContext) { + String fieldName = indexFieldData.getFieldName(); + LeafFieldData fieldData = indexFieldData.load(leafReaderContext); + return ((TextDocValuesFieldWrapper) fieldData.getScriptFieldFactory(fieldName)).bytesValues(); + } + + /** Wrapper around TextDocValuesField that provides access to the SortedBinaryDocValues. */ + static final class TextDocValuesFieldWrapper extends TextDocValuesField { + TextDocValuesFieldWrapper(SortedBinaryDocValues input, String name) { + super(input, name); + } + + SortedBinaryDocValues bytesValues() { + return input; + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSource.java new file mode 100644 index 0000000000000..d3ed8da1a17b0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSource.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.index.fielddata.DocValueBits; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +import java.io.IOException; +import java.util.function.Function; + +public class UnsupportedValueSource extends ValuesSource { + + public static final String UNSUPPORTED_OUTPUT = ""; + private static final BytesRef result = new BytesRef(UNSUPPORTED_OUTPUT); + private final ValuesSource originalSource; + + public UnsupportedValueSource(ValuesSource originalSource) { + this.originalSource = originalSource; + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + if (originalSource != null) { + try { + return originalSource.bytesValues(context); + } catch (Exception e) { + // ignore and fall back to UNSUPPORTED_OUTPUT + } + } + return new SortedBinaryDocValues() { + @Override + public boolean advanceExact(int doc) throws IOException { + return true; + } + + @Override + public int docValueCount() { + return 1; + } + + @Override + public BytesRef nextValue() throws IOException { + return result; + } + }; + } + + @Override + public DocValueBits docsWithValue(LeafReaderContext context) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + protected Function roundingPreparer(AggregationContext context) throws IOException { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSourceType.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSourceType.java new file mode 100644 index 0000000000000..ead3d9b46d6a5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/UnsupportedValueSourceType.java @@ -0,0 +1,57 @@ +/* + * 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.compute.lucene; + +import org.elasticsearch.script.AggregationScript; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; + +// just a placeholder class for unsupported data types +public class UnsupportedValueSourceType implements ValuesSourceType { + + private final String typeName; + + public UnsupportedValueSourceType(String typeName) { + this.typeName = typeName; + } + + @Override + public ValuesSource getEmpty() { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource getField(FieldContext fieldContext, AggregationScript.LeafFactory script) { + throw new UnsupportedOperationException(); + } + + @Override + public ValuesSource replaceMissing( + ValuesSource valuesSource, + Object rawMissing, + DocValueFormat docValueFormat, + AggregationContext context + ) { + throw new UnsupportedOperationException(); + } + + @Override + public String typeName() { + return typeName; + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java new file mode 100644 index 0000000000000..e4dffdfe72c4d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java @@ -0,0 +1,15 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.IndexReader; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; + +public record ValueSourceInfo(ValuesSourceType type, ValuesSource source, ElementType elementType, IndexReader reader) {} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java new file mode 100644 index 0000000000000..6cc13fd383ef3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java @@ -0,0 +1,132 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.index.fielddata.FieldDataContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.SourceValueFetcherSortedBinaryIndexFieldData; +import org.elasticsearch.index.fielddata.StoredFieldSortedBinaryIndexFieldData; +import org.elasticsearch.index.mapper.IdFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.SourceValueFetcher; +import org.elasticsearch.index.mapper.TextFieldMapper; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.internal.SearchContext; + +import java.util.ArrayList; +import java.util.List; + +public final class ValueSources { + + private ValueSources() {} + + public static List sources( + List searchContexts, + String fieldName, + boolean asUnsupportedSource, + ElementType elementType + ) { + List sources = new ArrayList<>(searchContexts.size()); + + for (SearchContext searchContext : searchContexts) { + SearchExecutionContext ctx = searchContext.getSearchExecutionContext(); + var fieldType = ctx.getFieldType(fieldName); + if (fieldType == null) { + sources.add(new ValueSourceInfo(new NullValueSourceType(), new NullValueSource(), elementType, ctx.getIndexReader())); + continue; // the field does not exist in this context + } + + // MatchOnlyTextFieldMapper class lives in the mapper-extras module. We use string equality + // for the field type name to avoid adding a dependency to the module + if (fieldType instanceof TextFieldMapper.TextFieldType || "match_only_text".equals(fieldType.typeName())) { + ValuesSource vs = textValueSource(ctx, fieldType); + sources.add(new ValueSourceInfo(CoreValuesSourceType.KEYWORD, vs, elementType, ctx.getIndexReader())); + continue; + } + + if (IdFieldMapper.NAME.equals(fieldType.name())) { + ValuesSource vs = new IdValueSource(new IdFieldIndexFieldData(CoreValuesSourceType.KEYWORD)); + sources.add(new ValueSourceInfo(CoreValuesSourceType.KEYWORD, vs, elementType, ctx.getIndexReader())); + continue; + } + + IndexFieldData fieldData; + try { + fieldData = ctx.getForField(fieldType, MappedFieldType.FielddataOperation.SEARCH); + } catch (IllegalArgumentException e) { + if (asUnsupportedSource) { + sources.add( + new ValueSourceInfo( + new UnsupportedValueSourceType(fieldType.typeName()), + new UnsupportedValueSource(null), + elementType, + ctx.getIndexReader() + ) + ); + continue; + } else { + throw e; + } + } + var fieldContext = new FieldContext(fieldName, fieldData, fieldType); + var vsType = fieldData.getValuesSourceType(); + var vs = vsType.getField(fieldContext, null); + + if (asUnsupportedSource) { + sources.add( + new ValueSourceInfo( + new UnsupportedValueSourceType(fieldType.typeName()), + new UnsupportedValueSource(vs), + elementType, + ctx.getIndexReader() + ) + ); + } else { + sources.add(new ValueSourceInfo(vsType, vs, elementType, ctx.getIndexReader())); + } + } + + return sources; + } + + private static TextValueSource textValueSource(SearchExecutionContext ctx, MappedFieldType fieldType) { + if (fieldType.isStored()) { + IndexFieldData fieldData = new StoredFieldSortedBinaryIndexFieldData( + fieldType.name(), + CoreValuesSourceType.KEYWORD, + TextValueSource.TextDocValuesFieldWrapper::new + ) { + @Override + protected BytesRef storedToBytesRef(Object stored) { + return new BytesRef((String) stored); + } + }; + return new TextValueSource(fieldData); + } + + FieldDataContext fieldDataContext = new FieldDataContext( + ctx.getFullyQualifiedIndex().getName(), + () -> ctx.lookup().forkAndTrackFieldReferences(fieldType.name()), + ctx::sourcePath, + MappedFieldType.FielddataOperation.SEARCH + ); + IndexFieldData fieldData = new SourceValueFetcherSortedBinaryIndexFieldData.Builder( + fieldType.name(), + CoreValuesSourceType.KEYWORD, + SourceValueFetcher.toString(fieldDataContext.sourcePathsLookup().apply(fieldType.name())), + fieldDataContext.lookupSupplier().get(), + TextValueSource.TextDocValuesFieldWrapper::new + ).build(null, null); // Neither cache nor breakerService are used by SourceValueFetcherSortedBinaryIndexFieldData builder + return new TextValueSource(fieldData); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java new file mode 100644 index 0000000000000..3d20cd069e164 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -0,0 +1,205 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.common.Strings; +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.DocBlock; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AbstractPageMappingOperator; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +/** + * Operator that extracts doc_values from a Lucene index out of pages that have been produced by {@link LuceneSourceOperator} + * and outputs them to a new column. The operator leverages the {@link ValuesSource} infrastructure for extracting + * field values. This allows for a more uniform way of extracting data compared to deciding the correct doc_values + * loader for different field types. + */ +public class ValuesSourceReaderOperator extends AbstractPageMappingOperator { + /** + * Creates a new extractor that uses ValuesSources load data + * @param sources the value source, type and index readers to use for extraction + * @param docChannel the channel containing the shard, leaf/segment and doc id + * @param field the lucene field being loaded + */ + public record ValuesSourceReaderOperatorFactory(List sources, int docChannel, String field) + implements + OperatorFactory { + @Override + public Operator get(DriverContext driverContext) { + return new ValuesSourceReaderOperator(sources, docChannel, field); + } + + @Override + public String describe() { + return "ValuesSourceReaderOperator[field = " + field + "]"; + } + } + + private final List sources; + private final int docChannel; + private final String field; + + private BlockDocValuesReader lastReader; + private int lastShard = -1; + private int lastSegment = -1; + + private final Map readersBuilt = new TreeMap<>(); + + /** + * Creates a new extractor + * @param sources the value source, type and index readers to use for extraction + * @param docChannel the channel containing the shard, leaf/segment and doc id + * @param field the lucene field being loaded + */ + public ValuesSourceReaderOperator(List sources, int docChannel, String field) { + this.sources = sources; + this.docChannel = docChannel; + this.field = field; + } + + @Override + protected Page process(Page page) { + DocVector docVector = page.getBlock(docChannel).asVector(); + + try { + if (docVector.singleSegmentNonDecreasing()) { + return page.appendBlock(loadFromSingleLeaf(docVector)); + } + return page.appendBlock(loadFromManyLeaves(docVector)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private Block loadFromSingleLeaf(DocVector docVector) throws IOException { + setupReader(docVector.shards().getInt(0), docVector.segments().getInt(0), docVector.docs().getInt(0)); + return lastReader.readValues(docVector.docs()); + } + + private Block loadFromManyLeaves(DocVector docVector) throws IOException { + int[] forwards = docVector.shardSegmentDocMapForwards(); + int doc = docVector.docs().getInt(forwards[0]); + setupReader(docVector.shards().getInt(forwards[0]), docVector.segments().getInt(forwards[0]), doc); + Block.Builder builder = lastReader.builder(forwards.length); + lastReader.readValuesFromSingleDoc(doc, builder); + for (int i = 1; i < forwards.length; i++) { + int shard = docVector.shards().getInt(forwards[i]); + int segment = docVector.segments().getInt(forwards[i]); + doc = docVector.docs().getInt(forwards[i]); + if (segment != lastSegment || shard != lastShard) { + setupReader(shard, segment, doc); + } + lastReader.readValuesFromSingleDoc(doc, builder); + } + // TODO maybe it's better for downstream consumers if we perform a copy here. + return builder.build().filter(docVector.shardSegmentDocMapBackwards()); + } + + private void setupReader(int shard, int segment, int doc) throws IOException { + if (lastSegment == segment && lastShard == shard && BlockDocValuesReader.canReuse(lastReader, doc)) { + return; + } + var info = sources.get(shard); + LeafReaderContext leafReaderContext = info.reader().leaves().get(segment); + lastReader = BlockDocValuesReader.createBlockReader(info.source(), info.type(), info.elementType(), leafReaderContext); + lastShard = shard; + lastSegment = segment; + readersBuilt.compute(lastReader.toString(), (k, v) -> v == null ? 1 : v + 1); + } + + @Override + public String toString() { + return "ValuesSourceReaderOperator[field = " + field + "]"; + } + + @Override + protected Status status(int pagesProcessed) { + return new Status(new TreeMap<>(readersBuilt), pagesProcessed); + } + + public static class Status extends AbstractPageMappingOperator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "values_source_reader", + Status::new + ); + + private final Map readersBuilt; + + Status(Map readersBuilt, int pagesProcessed) { + super(pagesProcessed); + this.readersBuilt = readersBuilt; + } + + Status(StreamInput in) throws IOException { + super(in); + readersBuilt = in.readOrderedMap(StreamInput::readString, StreamInput::readVInt); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeMap(readersBuilt, StreamOutput::writeString, StreamOutput::writeVInt); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public Map readersBuilt() { + return readersBuilt; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.startObject("readers_built"); + for (Map.Entry e : readersBuilt.entrySet()) { + builder.field(e.getKey(), e.getValue()); + } + builder.endObject(); + builder.field("pages_processed", pagesProcessed()); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return pagesProcessed() == status.pagesProcessed() && readersBuilt.equals(status.readersBuilt); + } + + @Override + public int hashCode() { + return Objects.hash(readersBuilt, pagesProcessed()); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AbstractPageMappingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AbstractPageMappingOperator.java new file mode 100644 index 0000000000000..c32a45cb1407b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AbstractPageMappingOperator.java @@ -0,0 +1,140 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +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.Page; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +/** + * Abstract superclass for operators that accept a single page, modify it, and then return it. + */ +public abstract class AbstractPageMappingOperator implements Operator { + private Page prev; + private boolean finished = false; + + /** + * Count of pages that have been processed by this operator. + */ + private int pagesProcessed; + + protected abstract Page process(Page page); + + @Override + public abstract String toString(); + + @Override + public final boolean needsInput() { + return prev == null && finished == false; + } + + @Override + public final void addInput(Page page) { + assert prev == null : "has pending input page"; + prev = page; + } + + @Override + public final void finish() { + finished = true; + } + + @Override + public final boolean isFinished() { + return finished && prev == null; + } + + @Override + public final Page getOutput() { + if (prev == null) { + return null; + } + if (prev.getPositionCount() == 0) { + return prev; + } + pagesProcessed++; + Page p = process(prev); + prev = null; + return p; + } + + @Override + public final Status status() { + return status(pagesProcessed); + } + + protected Status status(int pagesProcessed) { + return new Status(pagesProcessed); + } + + @Override + public final void close() {} + + public static class Status implements Operator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "page_mapping", + Status::new + ); + + private final int pagesProcessed; + + protected Status(int pagesProcessed) { + this.pagesProcessed = pagesProcessed; + } + + protected Status(StreamInput in) throws IOException { + pagesProcessed = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(pagesProcessed); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public int pagesProcessed() { + return pagesProcessed; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("pages_processed", pagesProcessed); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return pagesProcessed == status.pagesProcessed; + } + + @Override + public int hashCode() { + return Objects.hash(pagesProcessed); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AggregationOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AggregationOperator.java new file mode 100644 index 0000000000000..d2cc1b3322cce --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AggregationOperator.java @@ -0,0 +1,132 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.aggregation.Aggregator; +import org.elasticsearch.compute.aggregation.Aggregator.Factory; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasables; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +/** + * Blocking aggregation operator. An aggregation operator aggregates its input with one or more + * aggregator functions, e.g. avg, max, etc, and outputs a Page containing the results of those + * aggregations. + * + * The operator is blocking in the sense that it only produces output once all possible input has + * been added, that is, when the {@link #finish} method has been called. + */ +public class AggregationOperator implements Operator { + + private boolean finished; + private Page output; + private final List aggregators; + + public record AggregationOperatorFactory(List aggregators, AggregatorMode mode) implements OperatorFactory { + @Override + public Operator get(DriverContext driverContext) { + return new AggregationOperator(aggregators.stream().map(Factory::get).toList()); + } + + @Override + public String toString() { + return describe(); + } + + @Override + public String describe() { + return "AggregationOperator[mode = " + + mode + + ", aggs = " + + aggregators.stream().map(Factory::describe).collect(joining(", ")) + + "]"; + } + } + + public AggregationOperator(List aggregators) { + Objects.requireNonNull(aggregators); + checkNonEmpty(aggregators); + this.aggregators = aggregators; + } + + @Override + public boolean needsInput() { + return finished == false; + } + + @Override + public void addInput(Page page) { + checkState(needsInput(), "Operator is already finishing"); + requireNonNull(page, "page is null"); + for (Aggregator aggregator : aggregators) { + aggregator.processPage(page); + } + } + + @Override + public Page getOutput() { + Page p = output; + this.output = null; + return p; + } + + @Override + public void finish() { + if (finished) { + return; + } + finished = true; + int[] aggBlockCounts = aggregators.stream().mapToInt(Aggregator::evaluateBlockCount).toArray(); + Block[] blocks = new Block[Arrays.stream(aggBlockCounts).sum()]; + int offset = 0; + for (int i = 0; i < aggregators.size(); i++) { + var aggregator = aggregators.get(i); + aggregator.evaluate(blocks, offset); + offset += aggBlockCounts[i]; + } + output = new Page(blocks); + } + + @Override + public boolean isFinished() { + return finished && output == null; + } + + @Override + public void close() { + Releasables.close(aggregators); + } + + private static void checkState(boolean condition, String msg) { + if (condition == false) { + throw new IllegalArgumentException(msg); + } + } + + private static void checkNonEmpty(List list) { + if (list.size() < 1) { + throw new IllegalArgumentException("empty list"); + } + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("aggregators=").append(aggregators).append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AsyncOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AsyncOperator.java new file mode 100644 index 0000000000000..1258e2b4c7177 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AsyncOperator.java @@ -0,0 +1,175 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.tasks.TaskCancelledException; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * {@link AsyncOperator} performs an external computation specified in {@link #performAsync(Page, ActionListener)}. + * This operator acts as a client and operates on a per-page basis to reduce communication overhead. + * @see #performAsync(Page, ActionListener) + */ +public abstract class AsyncOperator implements Operator { + + private volatile ListenableActionFuture blockedFuture; + + private final Map buffers = ConcurrentCollections.newConcurrentMap(); + private final AtomicReference failure = new AtomicReference<>(); + + private final int maxOutstandingRequests; + private boolean finished = false; + + /* + * The checkpoint tracker is used to maintain the order of emitted pages after passing through this async operator. + * - Generates a new sequence number for each incoming page + * - Uses the processed checkpoint for pages that have completed this computation + * - Uses the persisted checkpoint for pages that have already been emitted to the next operator + */ + private final LocalCheckpointTracker checkpoint = new LocalCheckpointTracker( + SequenceNumbers.NO_OPS_PERFORMED, + SequenceNumbers.NO_OPS_PERFORMED + ); + + /** + * Create an operator that performs an external computation + * + * @param maxOutstandingRequests the maximum number of outstanding requests + */ + public AsyncOperator(int maxOutstandingRequests) { + this.maxOutstandingRequests = maxOutstandingRequests; + } + + @Override + public boolean needsInput() { + final long outstandingPages = checkpoint.getMaxSeqNo() - checkpoint.getPersistedCheckpoint(); + return outstandingPages < maxOutstandingRequests; + } + + @Override + public void addInput(Page input) { + checkFailure(); + final long seqNo = checkpoint.generateSeqNo(); + performAsync(input, ActionListener.wrap(output -> { + buffers.put(seqNo, output); + onSeqNoCompleted(seqNo); + }, e -> { + onFailure(e); + onSeqNoCompleted(seqNo); + })); + } + + /** + * Performs an external computation and notify the listener when the result is ready. + * + * @param inputPage the input page + * @param listener the listener + */ + protected abstract void performAsync(Page inputPage, ActionListener listener); + + private void onFailure(Exception e) { + failure.getAndUpdate(first -> { + if (first == null) { + return e; + } + // ignore subsequent TaskCancelledException exceptions as they don't provide useful info. + if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) { + return first; + } + if (ExceptionsHelper.unwrap(first, TaskCancelledException.class) != null) { + return e; + } + if (ExceptionsHelper.unwrapCause(first) != ExceptionsHelper.unwrapCause(e)) { + first.addSuppressed(e); + } + return first; + }); + } + + private void onSeqNoCompleted(long seqNo) { + checkpoint.markSeqNoAsProcessed(seqNo); + if (checkpoint.getPersistedCheckpoint() < checkpoint.getProcessedCheckpoint()) { + notifyIfBlocked(); + } + } + + private void notifyIfBlocked() { + if (blockedFuture != null) { + final ListenableActionFuture future; + synchronized (this) { + future = blockedFuture; + this.blockedFuture = null; + } + if (future != null) { + future.onResponse(null); + } + } + } + + private void checkFailure() { + Exception e = failure.get(); + if (e != null) { + throw ExceptionsHelper.convertToElastic(e); + } + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean isFinished() { + checkFailure(); + return finished && checkpoint.getPersistedCheckpoint() == checkpoint.getMaxSeqNo(); + } + + @Override + public Page getOutput() { + checkFailure(); + long persistedCheckpoint = checkpoint.getPersistedCheckpoint(); + if (persistedCheckpoint < checkpoint.getProcessedCheckpoint()) { + persistedCheckpoint++; + Page page = buffers.remove(persistedCheckpoint); + checkpoint.markSeqNoAsPersisted(persistedCheckpoint); + return page; + } else { + return null; + } + } + + @Override + public ListenableActionFuture isBlocked() { + if (finished) { + return Operator.NOT_BLOCKED; + } + long persistedCheckpoint = checkpoint.getPersistedCheckpoint(); + if (persistedCheckpoint == checkpoint.getMaxSeqNo() || persistedCheckpoint < checkpoint.getProcessedCheckpoint()) { + return Operator.NOT_BLOCKED; + } + synchronized (this) { + persistedCheckpoint = checkpoint.getPersistedCheckpoint(); + if (persistedCheckpoint == checkpoint.getMaxSeqNo() || persistedCheckpoint < checkpoint.getProcessedCheckpoint()) { + return Operator.NOT_BLOCKED; + } + if (blockedFuture == null) { + blockedFuture = new ListenableActionFuture<>(); + } + return blockedFuture; + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/BatchEncoder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/BatchEncoder.java new file mode 100644 index 0000000000000..a0a88c0d8e0b9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/BatchEncoder.java @@ -0,0 +1,473 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; + +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; +import java.nio.ByteOrder; + +public abstract class BatchEncoder implements Accountable { + /** + * Checks if an offset is {@code null}. + */ + public interface IsNull { + boolean isNull(int offset); + } + + /** + * Decodes values encoded by {@link BatchEncoder}. + */ + public interface Decoder { + void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count); + } + + /** + * Get a {@link Decoder} for the provided {@link ElementType}. + */ + public static Decoder decoder(ElementType elementType) { + return switch (elementType) { + case INT -> new IntsDecoder(); + case LONG -> new LongsDecoder(); + case DOUBLE -> new DoublesDecoder(); + case BYTES_REF -> new BytesRefsDecoder(); + case BOOLEAN -> new BooleansDecoder(); + default -> throw new IllegalArgumentException("can't encode " + elementType); + }; + } + + private static long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BatchEncoder.class); + + /** + * Buffer into which we encode values. + */ + protected final BytesRefBuilder bytes = new BytesRefBuilder(); + + /** + * Count of values at each position. + */ + private int[] counts = new int[ArrayUtil.oversize(10, Integer.BYTES)]; + + /** + * Offsets into the {@link #bytes} for each value. + */ + private int[] valueOffsets = new int[ArrayUtil.oversize(10, Integer.BYTES)]; + + /** + * The first position in the current batch. + */ + private int firstPosition; + + /** + * The number of positions in the current batch. It's also the maximum index into + * {@link #counts} that has an meaning. + */ + private int positionCount; + + /** + * The value being encoded right now. + */ + private int currentValue; + + /** + * Build the encoder. + * @param batchSize The number of bytes in a batch. We'll allocate this much memory for the + * encoder and only expand the allocation if the first entry in a batch + * doesn't fit into the buffer. + */ + BatchEncoder(int batchSize) { + bytes.grow(batchSize); + } + + /** + * The first position in the current batch. + */ + public int firstPosition() { + return firstPosition; + } + + /** + * The number of positions in the current batch. + */ + public int positionCount() { + return positionCount; + } + + /** + * The number of values at the position with this offset in the batch. + * The actual position in the block we're encoding is {@code positionOffset + firstPosition()}. + */ + public int valueCount(int positionOffset) { + if (positionOffset >= positionCount) { + throw new IllegalArgumentException("wanted " + positionOffset + " but only have " + positionCount); + } + return counts[positionOffset]; + } + + /** + * Read the value at the specified index. Values at the first position + * start at index {@code 0} and advance one per value. So the values + * at position n start at {@code (0..n-1).sum(valueCount)}. There is + * no random-access way to get the first index for a position. + */ + public final BytesRef read(int index, BytesRef scratch) { + scratch.bytes = bytes.bytes(); + scratch.offset = valueOffsets[index]; + scratch.length = valueOffsets[index + 1] - scratch.offset; + return scratch; + } + + /** + * Encodes the next batch of entries. This will encode values until the next + * value doesn't fit into the buffer. Callers should iterate on the values + * that have been encoded and then call this again for the next batch. + *

+ * It's possible for this batch to be empty if there isn't room for the + * first entry in the buffer. If so, call again to force the buffer to + * expand and encode that entry. + *

+ */ + public final void encodeNextBatch() { + bytes.clear(); + firstPosition += positionCount; + positionCount = 0; + currentValue = 0; + readNextBatch(); + } + + @Override + public long ramBytesUsed() { + return SHALLOW_SIZE + RamUsageEstimator.sizeOf(counts) + RamUsageEstimator.sizeOf(valueOffsets); + } + + /** + * Encodes the next batch of values. See {@link #encodeNextBatch()}. + */ + protected abstract void readNextBatch(); + + /** + * Implementations of {@link #readNextBatch} should call this before any + * values at the current position. + */ + protected final void startPosition() { + counts = ArrayUtil.grow(counts, positionCount + 1); + counts[positionCount] = 0; + } + + /** + * Implementations of {@link #readNextBatch} should call this before adding + * each value to the current position to mark its start. + */ + protected final void addingValue() { + counts[positionCount]++; + valueOffsets = ArrayUtil.grow(valueOffsets, currentValue + 1); + valueOffsets[currentValue++] = bytes.length(); + } + + /** + * Implementations of {@link #readNextBatch} should call this to end + * the current position. + */ + protected final void endPosition() { + valueOffsets = ArrayUtil.grow(valueOffsets, currentValue + 1); + valueOffsets[currentValue] = bytes.length(); + positionCount++; + } + + /** + * Implementations of {@link #readNextBatch} should call this to encode + * an entirely null position. + */ + protected final void encodeNull() { + startPosition(); + addingValue(); + endPosition(); + } + + /** + * The number of bytes in all entries in the batch. + */ + final int bytesLength() { + return bytes.length(); + } + + /** + * The maximum batch size. This starts the same as the constructor parameter + * but will grow if a single entry doesn't fit into the batch. + */ + final int bytesCapacity() { + return bytes.bytes().length; + } + + private static final VarHandle intHandle = MethodHandles.byteArrayViewVarHandle(int[].class, ByteOrder.nativeOrder()); + + protected abstract static class Ints extends BatchEncoder { + protected Ints(int batchSize) { + super(batchSize); + } + + /** + * Is there capacity for this many {@code int}s? + */ + protected final boolean hasCapacity(int count) { + return bytes.length() + count * Integer.BYTES <= bytesCapacity(); + } + + /** + * Make sure there is capacity for this many {@code int}s, growing + * the buffer if needed. + */ + protected final void ensureCapacity(int count) { + // TODO some protection against growing to gigabytes or whatever + bytes.grow(count * Integer.BYTES); + } + + /** + * Encode an {@code int} into the current position and advance + * to the next position. + */ + protected final void encode(int v) { + addingValue(); + intHandle.set(bytes.bytes(), bytes.length(), v); + bytes.setLength(bytes.length() + Integer.BYTES); + } + } + + private static class IntsDecoder implements Decoder { + @Override + public void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count) { + IntBlock.Builder b = (IntBlock.Builder) builder; + for (int i = 0; i < count; i++) { + if (isNull.isNull(i)) { + b.appendNull(); + } else { + BytesRef e = encoded[i]; + b.appendInt((int) intHandle.get(e.bytes, e.offset)); + e.offset += Integer.BYTES; + e.length -= Integer.BYTES; + } + } + } + } + + private static final VarHandle longHandle = MethodHandles.byteArrayViewVarHandle(long[].class, ByteOrder.nativeOrder()); + + protected abstract static class Longs extends BatchEncoder { + protected Longs(int batchSize) { + super(batchSize); + } + + /** + * Is there capacity for this many {@code long}s? + */ + protected final boolean hasCapacity(int count) { + return bytes.length() + count * Long.BYTES <= bytesCapacity(); + } + + /** + * Make sure there is capacity for this many {@code long}s, growing + * the buffer if needed. + */ + protected final void ensureCapacity(int count) { + // TODO some protection against growing to gigabytes or whatever + bytes.grow(count * Long.BYTES); + } + + /** + * Encode a {@code long} and advance to the next position. + */ + protected final void encode(long v) { + addingValue(); + longHandle.set(bytes.bytes(), bytes.length(), v); + bytes.setLength(bytes.length() + Long.BYTES); + } + } + + private static class LongsDecoder implements Decoder { + @Override + public void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count) { + LongBlock.Builder b = (LongBlock.Builder) builder; + for (int i = 0; i < count; i++) { + if (isNull.isNull(i)) { + b.appendNull(); + } else { + BytesRef e = encoded[i]; + b.appendLong((long) longHandle.get(e.bytes, e.offset)); + e.offset += Long.BYTES; + e.length -= Long.BYTES; + } + } + } + + } + + private static final VarHandle doubleHandle = MethodHandles.byteArrayViewVarHandle(double[].class, ByteOrder.nativeOrder()); + + protected abstract static class Doubles extends BatchEncoder { + protected Doubles(int batchSize) { + super(batchSize); + } + + /** + * Is there capacity for this many {@code double}s? + */ + protected final boolean hasCapacity(int count) { + return bytes.length() + count * Double.BYTES <= bytesCapacity(); + } + + /** + * Make sure there is capacity for this many {@code double}s, growing + * the buffer if needed. + */ + protected final void ensureCapacity(int count) { + // TODO some protection against growing to gigabytes or whatever + bytes.grow(count * Double.BYTES); + } + + /** + * Encode a {@code double} and advance to the next position. + */ + protected final void encode(double v) { + addingValue(); + doubleHandle.set(bytes.bytes(), bytes.length(), v); + bytes.setLength(bytes.length() + Double.BYTES); + } + } + + private static class DoublesDecoder implements Decoder { + @Override + public void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count) { + DoubleBlock.Builder b = (DoubleBlock.Builder) builder; + for (int i = 0; i < count; i++) { + if (isNull.isNull(i)) { + b.appendNull(); + } else { + BytesRef e = encoded[i]; + b.appendDouble((double) doubleHandle.get(e.bytes, e.offset)); + e.offset += Double.BYTES; + e.length -= Double.BYTES; + } + } + } + } + + protected abstract static class Booleans extends BatchEncoder { + protected Booleans(int batchSize) { + super(batchSize); + } + + /** + * Is there capacity for this many {@code booleans}s? + */ + protected final boolean hasCapacity(int count) { + return bytes.length() + count <= bytesCapacity(); + } + + /* + * There isn't an ensureCapacity here because the only user presently + * deduplicates values and there are only two possible boolean values. + * Which will always fit into any reasonable sized buffer. + */ + + /** + * Encode a {@code boolean} and advance to the next position. + */ + protected final void encode(boolean v) { + addingValue(); + bytes.bytes()[bytes.length()] = (byte) (v ? 1 : 0); + bytes.setLength(bytes.length() + 1); + } + } + + private static class BooleansDecoder implements Decoder { + @Override + public void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count) { + BooleanBlock.Builder b = (BooleanBlock.Builder) builder; + for (int i = 0; i < count; i++) { + if (isNull.isNull(i)) { + b.appendNull(); + } else { + BytesRef e = encoded[i]; + b.appendBoolean(e.bytes[e.offset] == 1); + e.offset++; + e.length--; + } + } + } + } + + protected abstract static class BytesRefs extends BatchEncoder { + protected BytesRefs(int batchSize) { + super(batchSize); + } + + /** + * Is there capacity for {@code totalBytes} and spread across + * {@code #count} {@link BytesRef}s? You could call this with something + * like {@code hasCapacity(Arrays.stream(bytes).mapToInt(b -> b.length).sum(), bytes.length)}. + */ + protected final boolean hasCapacity(int totalBytes, int count) { + return bytes.length() + totalBytes + count * Integer.BYTES <= bytesCapacity(); + } + + /** + * Make sure there is capacity for {@code totalBytes} and spread across + * {@code #count} {@link BytesRef}s? You could call this with something + * like {@code ensureCapacity(Arrays.stream(bytes).mapToInt(b -> b.length).sum(), bytes.length)}. + */ + protected final void ensureCapacity(int totalBytes, int count) { + // TODO some protection against growing to gigabytes or whatever + bytes.grow(totalBytes + count * Integer.BYTES); + } + + /** + * Encode a {@link BytesRef} and advance to the next position. + */ + protected final void encode(BytesRef v) { + addingValue(); + intHandle.set(bytes.bytes(), bytes.length(), v.length); + bytes.setLength(bytes.length() + Integer.BYTES); + bytes.append(v); + } + } + + private static class BytesRefsDecoder implements Decoder { + @Override + public void decode(Block.Builder builder, IsNull isNull, BytesRef[] encoded, int count) { + BytesRef scratch = new BytesRef(); + BytesRefBlock.Builder b = (BytesRefBlock.Builder) builder; + for (int i = 0; i < count; i++) { + if (isNull.isNull(i)) { + b.appendNull(); + } else { + BytesRef e = encoded[i]; + scratch.bytes = e.bytes; + scratch.length = (int) intHandle.get(e.bytes, e.offset); + e.offset += Integer.BYTES; + e.length -= Integer.BYTES; + scratch.offset = e.offset; + b.appendBytesRef(scratch); + e.offset += scratch.length; + e.length -= scratch.length; + } + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ColumnExtractOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ColumnExtractOperator.java new file mode 100644 index 0000000000000..8e5244cb75226 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ColumnExtractOperator.java @@ -0,0 +1,93 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +import java.util.function.Supplier; + +public class ColumnExtractOperator extends AbstractPageMappingOperator { + + public record Factory( + ElementType[] types, + Supplier inputEvalSupplier, + Supplier evaluatorSupplier + ) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new ColumnExtractOperator(types, inputEvalSupplier.get(), evaluatorSupplier.get()); + } + + @Override + public String describe() { + return "ColumnExtractOperator[evaluator=" + evaluatorSupplier.get() + "]"; + } + } + + private final ElementType[] types; + private final EvalOperator.ExpressionEvaluator inputEvaluator; + private final ColumnExtractOperator.Evaluator evaluator; + + public ColumnExtractOperator( + ElementType[] types, + EvalOperator.ExpressionEvaluator inputEvaluator, + ColumnExtractOperator.Evaluator evaluator + ) { + this.types = types; + this.inputEvaluator = inputEvaluator; + this.evaluator = evaluator; + } + + @Override + protected Page process(Page page) { + int rowsCount = page.getPositionCount(); + + Block.Builder[] blockBuilders = new Block.Builder[types.length]; + for (int i = 0; i < types.length; i++) { + blockBuilders[i] = types[i].newBlockBuilder(rowsCount); + } + + BytesRefBlock input = (BytesRefBlock) inputEvaluator.eval(page); + BytesRef spare = new BytesRef(); + for (int row = 0; row < rowsCount; row++) { + if (input.isNull(row)) { + for (int i = 0; i < blockBuilders.length; i++) { + blockBuilders[i].appendNull(); + } + continue; + } + evaluator.computeRow(input, row, blockBuilders, spare); + } + + Block[] blocks = new Block[blockBuilders.length]; + for (int i = 0; i < blockBuilders.length; i++) { + blocks[i] = blockBuilders[i].build(); + } + return page.appendBlocks(blocks); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("evaluator="); + sb.append(evaluator.toString()); + sb.append("]"); + return sb.toString(); + } + + public interface Evaluator { + void computeRow(BytesRefBlock input, int row, Block.Builder[] target, BytesRef spare); + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java new file mode 100644 index 0000000000000..4808094e116be --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -0,0 +1,331 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.TaskCancelledException; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * A driver operates single-threadedly on a simple chain of {@link Operator}s, passing + * {@link Page}s from one operator to the next. It also controls the lifecycle of the + * operators. + * The operator chain typically starts with a source operator (i.e. an operator that purely produces pages) + * and ends with a sink operator (i.e. an operator that purely consumes pages). + * + * More details on how this integrates with other components can be found in the package documentation of + * {@link org.elasticsearch.compute} + */ + +public class Driver implements Releasable, Describable { + public static final TimeValue DEFAULT_TIME_BEFORE_YIELDING = TimeValue.timeValueMinutes(5); + public static final int DEFAULT_MAX_ITERATIONS = 10_000; + + private final String sessionId; + private final DriverContext driverContext; + private final Supplier description; + private final List activeOperators; + private final Releasable releasable; + + private final AtomicReference cancelReason = new AtomicReference<>(); + private final AtomicReference> blocked = new AtomicReference<>(); + private final AtomicReference status; + + /** + * Creates a new driver with a chain of operators. + * @param sessionId session Id + * @param driverContext the driver context + * @param source source operator + * @param intermediateOperators the chain of operators to execute + * @param sink sink operator + * @param releasable a {@link Releasable} to invoked once the chain of operators has run to completion + */ + public Driver( + String sessionId, + DriverContext driverContext, + Supplier description, + SourceOperator source, + List intermediateOperators, + SinkOperator sink, + Releasable releasable + ) { + this.sessionId = sessionId; + this.driverContext = driverContext; + this.description = description; + this.activeOperators = new ArrayList<>(); + this.activeOperators.add(source); + this.activeOperators.addAll(intermediateOperators); + this.activeOperators.add(sink); + this.releasable = releasable; + this.status = new AtomicReference<>(new DriverStatus(sessionId, System.currentTimeMillis(), DriverStatus.Status.QUEUED, List.of())); + } + + /** + * Creates a new driver with a chain of operators. + * @param driverContext the driver context + * @param source source operator + * @param intermediateOperators the chain of operators to execute + * @param sink sink operator + * @param releasable a {@link Releasable} to invoked once the chain of operators has run to completion + */ + public Driver( + DriverContext driverContext, + SourceOperator source, + List intermediateOperators, + SinkOperator sink, + Releasable releasable + ) { + this("unset", driverContext, () -> null, source, intermediateOperators, sink, releasable); + } + + public DriverContext driverContext() { + return driverContext; + } + + /** + * Runs computations on the chain of operators for a given maximum amount of time or iterations. + * Returns a blocked future when the chain of operators is blocked, allowing the caller + * thread to do other work instead of blocking or busy-spinning on the blocked operator. + */ + private ListenableActionFuture run(TimeValue maxTime, int maxIterations) { + long maxTimeNanos = maxTime.nanos(); + long startTime = System.nanoTime(); + int iter = 0; + while (isFinished() == false) { + ListenableActionFuture fut = runSingleLoopIteration(); + if (fut.isDone() == false) { + return fut; + } + if (++iter >= maxIterations) { + break; + } + long now = System.nanoTime(); + if (now - startTime > maxTimeNanos) { + break; + } + } + if (isFinished()) { + status.set(updateStatus(DriverStatus.Status.DONE)); // Report status for the tasks API + driverContext.finish(); + releasable.close(); + } else { + status.set(updateStatus(DriverStatus.Status.RUNNING)); // Report status for the tasks API + } + return Operator.NOT_BLOCKED; + } + + /** + * Whether the driver has run the chain of operators to completion. + */ + public boolean isFinished() { + return activeOperators.isEmpty(); + } + + @Override + public void close() { + drainAndCloseOperators(null); + } + + private ListenableActionFuture runSingleLoopIteration() { + ensureNotCancelled(); + boolean movedPage = false; + + for (int i = 0; i < activeOperators.size() - 1; i++) { + Operator op = activeOperators.get(i); + Operator nextOp = activeOperators.get(i + 1); + + // skip blocked operator + if (op.isBlocked().isDone() == false) { + continue; + } + + if (op.isFinished() == false && nextOp.needsInput()) { + Page page = op.getOutput(); + if (page != null && page.getPositionCount() != 0) { + nextOp.addInput(page); + movedPage = true; + } + } + + if (op.isFinished()) { + nextOp.finish(); + } + } + + for (int index = activeOperators.size() - 1; index >= 0; index--) { + if (activeOperators.get(index).isFinished()) { + /* + * Close and remove this operator and all source operators in the + * most paranoid possible way. Closing operators shouldn't throw, + * but if it does, this will make sure we don't try to close any + * that succeed twice. + */ + List finishedOperators = this.activeOperators.subList(0, index + 1); + Iterator itr = finishedOperators.iterator(); + while (itr.hasNext()) { + itr.next().close(); + itr.remove(); + } + + // Finish the next operator, which is now the first operator. + if (activeOperators.isEmpty() == false) { + Operator newRootOperator = activeOperators.get(0); + newRootOperator.finish(); + } + break; + } + } + + if (movedPage == false) { + return oneOf( + activeOperators.stream().map(Operator::isBlocked).filter(laf -> laf.isDone() == false).collect(Collectors.toList()) + ); + } + return Operator.NOT_BLOCKED; + } + + public void cancel(String reason) { + if (cancelReason.compareAndSet(null, reason)) { + synchronized (this) { + ListenableActionFuture fut = this.blocked.get(); + if (fut != null) { + fut.onFailure(new TaskCancelledException(reason)); + } + } + } + } + + private boolean isCancelled() { + return cancelReason.get() != null; + } + + private void ensureNotCancelled() { + String reason = cancelReason.get(); + if (reason != null) { + throw new TaskCancelledException(reason); + } + } + + public static void start(Executor executor, Driver driver, int maxIterations, ActionListener listener) { + driver.status.set(driver.updateStatus(DriverStatus.Status.STARTING)); // Report status for the tasks API + schedule(DEFAULT_TIME_BEFORE_YIELDING, maxIterations, executor, driver, listener); + } + + // Drains all active operators and closes them. + private void drainAndCloseOperators(@Nullable Exception e) { + Iterator itr = activeOperators.iterator(); + while (itr.hasNext()) { + try { + Releasables.closeWhileHandlingException(itr.next()); + } catch (Exception x) { + if (e != null) { + e.addSuppressed(x); + } + } + itr.remove(); + } + driverContext.finish(); + Releasables.closeWhileHandlingException(releasable); + } + + private static void schedule(TimeValue maxTime, int maxIterations, Executor executor, Driver driver, ActionListener listener) { + executor.execute(new AbstractRunnable() { + @Override + protected void doRun() { + if (driver.isFinished()) { + listener.onResponse(null); + return; + } + ListenableActionFuture fut = driver.run(maxTime, maxIterations); + if (fut.isDone()) { + schedule(maxTime, maxIterations, executor, driver, listener); + } else { + synchronized (driver) { + if (driver.isCancelled() == false) { + driver.blocked.set(fut); + } + } + fut.addListener( + ActionListener.wrap(ignored -> schedule(maxTime, maxIterations, executor, driver, listener), this::onFailure) + ); + } + } + + @Override + public void onFailure(Exception e) { + driver.drainAndCloseOperators(e); + listener.onFailure(e); + } + }); + } + + private static ListenableActionFuture oneOf(List> futures) { + if (futures.isEmpty()) { + return Operator.NOT_BLOCKED; + } + if (futures.size() == 1) { + return futures.get(0); + } + ListenableActionFuture oneOf = new ListenableActionFuture<>(); + for (ListenableActionFuture fut : futures) { + fut.addListener(oneOf); + } + return oneOf; + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + "[activeOperators=" + activeOperators + "]"; + } + + @Override + public String describe() { + return description.get(); + } + + public String sessionId() { + return sessionId; + } + + /** + * Get the last status update from the driver. These updates are made + * when the driver is queued and after every + * processing {@link #run(TimeValue, int) batch}. + */ + public DriverStatus status() { + return status.get(); + } + + /** + * Update the status. + * @param status the status of the overall driver request + */ + private DriverStatus updateStatus(DriverStatus.Status status) { + return new DriverStatus( + sessionId, + System.currentTimeMillis(), + status, + activeOperators.stream().map(o -> new DriverStatus.OperatorStatus(o.toString(), o.status())).toList() + ); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverContext.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverContext.java new file mode 100644 index 0000000000000..6512c417b91ca --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverContext.java @@ -0,0 +1,102 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.core.Releasable; + +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * A driver-local context that is shared across operators. + * + * Operators in the same driver pipeline are executed in a single threaded fashion. A driver context + * has a set of mutating methods that can be used to store and share values across these operators, + * or even outside the Driver. When the Driver is finished, it finishes the context. Finishing the + * context effectively takes a snapshot of the driver context values so that they can be exposed + * outside the Driver. The net result of this is that the driver context can be mutated freely, + * without contention, by the thread executing the pipeline of operators until it is finished. + * The context must be finished by the thread running the Driver, when the Driver is finished. + * + * Releasables can be added and removed to the context by operators in the same driver pipeline. + * This allows to "transfer ownership" of a shared resource across operators (and even across + * Drivers), while ensuring that the resource can be correctly released when no longer needed. + * + * Currently only supports releasables, but additional driver-local context can be added. + */ +public class DriverContext { + + // Working set. Only the thread executing the driver will update this set. + Set workingSet = Collections.newSetFromMap(new IdentityHashMap<>()); + + private final AtomicReference snapshot = new AtomicReference<>(); + + /** A snapshot of the driver context. */ + public record Snapshot(Set releasables) {} + + /** + * Adds a releasable to this context. Releasables are identified by Object identity. + * @return true if the releasable was added, otherwise false (if already present) + */ + public boolean addReleasable(Releasable releasable) { + return workingSet.add(releasable); + } + + /** + * Removes a releasable from this context. Releasables are identified by Object identity. + * @return true if the releasable was removed, otherwise false (if not present) + */ + public boolean removeReleasable(Releasable releasable) { + return workingSet.remove(releasable); + } + + /** + * Retrieves the snapshot of the driver context after it has been finished. + * @return the snapshot + */ + public Snapshot getSnapshot() { + ensureFinished(); + // should be called by the DriverRunner + return snapshot.get(); + } + + /** + * Tells whether this context is finished. Can be invoked from any thread. + */ + public boolean isFinished() { + return snapshot.get() != null; + } + + /** + * Finishes this context. Further mutating operations should not be performed. + */ + public void finish() { + if (isFinished()) { + return; + } + // must be called by the thread executing the driver. + // no more updates to this context. + var itr = workingSet.iterator(); + workingSet = null; + Set releasableSet = Collections.newSetFromMap(new IdentityHashMap<>()); + while (itr.hasNext()) { + var r = itr.next(); + releasableSet.add(r); + itr.remove(); + } + snapshot.compareAndSet(null, new Snapshot(releasableSet)); + } + + private void ensureFinished() { + if (isFinished() == false) { + throw new IllegalStateException("not finished"); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverRunner.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverRunner.java new file mode 100644 index 0000000000000..9ab40b15e4623 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverRunner.java @@ -0,0 +1,122 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Run a set of drivers to completion. + */ +public abstract class DriverRunner { + /** + * Start a driver. + */ + protected abstract void start(Driver driver, ActionListener driverListener); + + /** + * Run all drivers to completion asynchronously. + */ + public void runToCompletion(List drivers, ActionListener listener) { + AtomicReference failure = new AtomicReference<>(); + CountDown counter = new CountDown(drivers.size()); + for (Driver driver : drivers) { + ActionListener driverListener = new ActionListener<>() { + @Override + public void onResponse(Void unused) { + done(); + } + + @Override + public void onFailure(Exception e) { + failure.getAndUpdate(first -> { + if (first == null) { + return e; + } + if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) { + return first; + } else { + if (ExceptionsHelper.unwrap(first, TaskCancelledException.class) != null) { + return e; + } else { + if (first != e) { + first.addSuppressed(e); + } + return first; + } + } + }); + for (Driver d : drivers) { + if (driver != d) { + d.cancel("Driver [" + driver.sessionId() + "] was cancelled or failed"); + } + } + done(); + } + + private void done() { + if (counter.countDown()) { + for (Driver d : drivers) { + if (d.status().status() == DriverStatus.Status.QUEUED) { + d.close(); + } else { + Releasables.close(d.driverContext().getSnapshot().releasables()); + } + } + Exception error = failure.get(); + if (error != null) { + listener.onFailure(error); + } else { + listener.onResponse(null); + } + } + } + }; + + start(driver, driverListener); + } + } + + /** + * Run all the of the listed drivers in the supplier {@linkplain ThreadPool}. + * @return the headers added to the context while running the drivers + */ + public static Map> runToCompletion(ThreadPool threadPool, int maxIterations, List drivers) { + DriverRunner runner = new DriverRunner() { + @Override + protected void start(Driver driver, ActionListener driverListener) { + Driver.start(threadPool.executor("esql"), driver, maxIterations, driverListener); + } + }; + AtomicReference>> responseHeaders = new AtomicReference<>(); + PlainActionFuture future = new PlainActionFuture<>(); + runner.runToCompletion(drivers, new ActionListener<>() { + @Override + public void onResponse(Void unused) { + responseHeaders.set(threadPool.getThreadContext().getResponseHeaders()); + future.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + future.onFailure(e); + } + }); + future.actionGet(); + return responseHeaders.get(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java new file mode 100644 index 0000000000000..36bd6fc8cc53f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java @@ -0,0 +1,220 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.xcontent.ToXContentFragment; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Objects; + +/** + * {@link Task.Status} reported from a {@link Driver} to be reported by the tasks api. + */ +public class DriverStatus implements Task.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Task.Status.class, + "driver", + DriverStatus::new + ); + + /** + * The session for this driver. + */ + private final String sessionId; + /** + * When this status was generated. + */ + private final long lastUpdated; + /** + * The state of the overall driver - queue, starting, running, finished. + */ + private final Status status; + /** + * Status of each {@link Operator} in the driver. + */ + private final List activeOperators; + + DriverStatus(String sessionId, long lastUpdated, Status status, List activeOperators) { + this.sessionId = sessionId; + this.lastUpdated = lastUpdated; + this.status = status; + this.activeOperators = activeOperators; + } + + DriverStatus(StreamInput in) throws IOException { + this(in.readString(), in.readLong(), Status.valueOf(in.readString()), in.readImmutableList(OperatorStatus::new)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(sessionId); + out.writeLong(lastUpdated); + out.writeString(status.toString()); + out.writeList(activeOperators); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + /** + * The session for this driver. + */ + public String sessionId() { + return sessionId; + } + + /** + * When this status was generated. + */ + public long lastUpdated() { + return lastUpdated; + } + + /** + * The state of the overall driver - queue, starting, running, finished. + */ + public Status status() { + return status; + } + + /** + * Status of each {@link Operator} in the driver. + */ + public List activeOperators() { + return activeOperators; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("sessionId", sessionId); + builder.field("last_updated", DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(lastUpdated)); + builder.field("status", status.toString().toLowerCase(Locale.ROOT)); + builder.startArray("active_operators"); + for (OperatorStatus active : activeOperators) { + builder.value(active); + } + builder.endArray(); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DriverStatus that = (DriverStatus) o; + return sessionId.equals(that.sessionId) + && lastUpdated == that.lastUpdated + && status == that.status + && activeOperators.equals(that.activeOperators); + } + + @Override + public int hashCode() { + return Objects.hash(sessionId, lastUpdated, status, activeOperators); + } + + @Override + public String toString() { + return Strings.toString(this); + } + + /** + * Status of an {@link Operator}. + */ + public static class OperatorStatus implements Writeable, ToXContentObject { + /** + * String representation of the {@link Operator}. Literally just the + * {@link Object#toString()} of it. + */ + private final String operator; + /** + * Status as reported by the {@link Operator}. + */ + @Nullable + private final Operator.Status status; + + OperatorStatus(String operator, Operator.Status status) { + this.operator = operator; + this.status = status; + } + + private OperatorStatus(StreamInput in) throws IOException { + operator = in.readString(); + status = in.readOptionalNamedWriteable(Operator.Status.class); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(operator); + out.writeOptionalNamedWriteable(status); + } + + public String operator() { + return operator; + } + + public Operator.Status status() { + return status; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("operator", operator); + if (status != null) { + builder.field("status", status); + } + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + OperatorStatus that = (OperatorStatus) o; + return operator.equals(that.operator) && Objects.equals(status, that.status); + } + + @Override + public int hashCode() { + return Objects.hash(operator, status); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } + + public enum Status implements ToXContentFragment { + QUEUED, + STARTING, + RUNNING, + DONE; + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.value(toString().toLowerCase(Locale.ROOT)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverTaskRunner.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverTaskRunner.java new file mode 100644 index 0000000000000..6643321dbfea7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverTaskRunner.java @@ -0,0 +1,132 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Executor; + +/** + * A {@link DriverRunner} that executes {@link Driver} with a child task so that we can retrieve the progress with the Task API. + */ +public class DriverTaskRunner { + public static final String ACTION_NAME = "internal:data/read/esql/compute"; + private final TransportService transportService; + + public DriverTaskRunner(TransportService transportService, Executor executor) { + this.transportService = transportService; + transportService.registerRequestHandler(ACTION_NAME, ThreadPool.Names.SAME, DriverRequest::new, new DriverRequestHandler(executor)); + } + + public void executeDrivers(Task parentTask, List drivers, ActionListener listener) { + var runner = new DriverRunner() { + @Override + protected void start(Driver driver, ActionListener driverListener) { + transportService.sendChildRequest( + transportService.getLocalNode(), + ACTION_NAME, + new DriverRequest(driver), + parentTask, + TransportRequestOptions.EMPTY, + new TransportResponseHandler.Empty() { + @Override + public Executor executor(ThreadPool threadPool) { + return TRANSPORT_WORKER; + } + + @Override + public void handleResponse() { + driverListener.onResponse(null); + } + + @Override + public void handleException(TransportException exp) { + driverListener.onFailure(exp); + } + } + ); + } + }; + runner.runToCompletion(drivers, listener); + } + + private static class DriverRequest extends ActionRequest { + private final Driver driver; + + DriverRequest(Driver driver) { + this.driver = driver; + } + + DriverRequest(StreamInput in) { + throw new UnsupportedOperationException("Driver request should never leave the current node"); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + throw new UnsupportedOperationException("Driver request should never leave the current node"); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + if (parentTaskId.isSet() == false) { + assert false : "DriverRequest must have a parent task"; + throw new IllegalStateException("DriverRequest must have a parent task"); + } + return new CancellableTask(id, type, action, "", parentTaskId, headers) { + @Override + protected void onCancelled() { + String reason = Objects.requireNonNullElse(getReasonCancelled(), "cancelled"); + driver.cancel(reason); + } + + @Override + public String getDescription() { + return driver.describe(); + } + + @Override + public Status getStatus() { + return driver.status(); + } + }; + } + } + + private record DriverRequestHandler(Executor executor) implements TransportRequestHandler { + @Override + public void messageReceived(DriverRequest request, TransportChannel channel, Task task) { + var listener = new ChannelActionListener(channel); + Driver.start(executor, request.driver, Driver.DEFAULT_MAX_ITERATIONS, listener.map(unused -> TransportResponse.Empty.INSTANCE)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EmptySourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EmptySourceOperator.java new file mode 100644 index 0000000000000..58496bc16a53e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EmptySourceOperator.java @@ -0,0 +1,48 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; + +/** + * An empty source operator, which is already finished and never emits any output. + */ +public final class EmptySourceOperator extends SourceOperator { + + public static class Factory implements SourceOperatorFactory { + @Override + public String describe() { + return "EmptySourceOperator[]"; + } + + @Override + public SourceOperator get(DriverContext driverContext) { + return new EmptySourceOperator(); + } + } + + @Override + public void finish() { + + } + + @Override + public boolean isFinished() { + return true; + } + + @Override + public Page getOutput() { + return null; + } + + @Override + public void close() { + + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EvalOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EvalOperator.java new file mode 100644 index 0000000000000..c2ecb94550769 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EvalOperator.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.function.Supplier; + +/** + * Evaluates a tree of functions for every position in the block, resulting in a + * new block which is appended to the page. + */ +public class EvalOperator extends AbstractPageMappingOperator { + + public record EvalOperatorFactory(Supplier evaluator) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new EvalOperator(evaluator.get()); + } + + @Override + public String describe() { + return "EvalOperator[evaluator=" + evaluator.get() + "]"; + } + } + + private final ExpressionEvaluator evaluator; + + public EvalOperator(ExpressionEvaluator evaluator) { + this.evaluator = evaluator; + } + + @Override + protected Page process(Page page) { + return page.appendBlock(evaluator.eval(page)); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "[evaluator=" + evaluator + "]"; + } + + public interface ExpressionEvaluator { + Block eval(Page page); + } + + public static final ExpressionEvaluator CONSTANT_NULL = page -> Block.constantNullBlock(page.getPositionCount()); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FilterOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FilterOperator.java new file mode 100644 index 0000000000000..61e7c25d1000b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FilterOperator.java @@ -0,0 +1,82 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.Page; + +import java.util.Arrays; +import java.util.function.Supplier; + +public class FilterOperator extends AbstractPageMappingOperator { + + private final EvalOperator.ExpressionEvaluator evaluator; + + public record FilterOperatorFactory(Supplier evaluatorSupplier) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new FilterOperator(evaluatorSupplier.get()); + } + + @Override + public String describe() { + return "FilterOperator[evaluator=" + evaluatorSupplier.get() + "]"; + } + } + + public FilterOperator(EvalOperator.ExpressionEvaluator evaluator) { + this.evaluator = evaluator; + } + + @Override + protected Page process(Page page) { + int rowCount = 0; + int[] positions = new int[page.getPositionCount()]; + + Block uncastTest = evaluator.eval(page); + if (uncastTest.areAllValuesNull()) { + // All results are null which is like false. No values selected. + return null; + } + BooleanBlock test = (BooleanBlock) uncastTest; + // TODO we can detect constant true or false from the type + // TODO or we could make a new method in bool-valued evaluators that returns a list of numbers + for (int p = 0; p < page.getPositionCount(); p++) { + if (test.isNull(p) || test.getValueCount(p) != 1) { + // Null is like false + // And, for now, multivalued results are like false too + continue; + } + if (test.getBoolean(test.getFirstValueIndex(p))) { + positions[rowCount++] = p; + } + } + + if (rowCount == 0) { + return null; + } + if (rowCount == page.getPositionCount()) { + return page; + } + positions = Arrays.copyOf(positions, rowCount); + + Block[] filteredBlocks = new Block[page.getBlockCount()]; + for (int i = 0; i < page.getBlockCount(); i++) { + filteredBlocks[i] = page.getBlock(i).filter(positions); + } + + return new Page(filteredBlocks); + } + + @Override + public String toString() { + return "FilterOperator[" + "evaluator=" + evaluator + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java new file mode 100644 index 0000000000000..1910cc4ec590c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java @@ -0,0 +1,177 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.aggregation.GroupingAggregator; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasables; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +public class HashAggregationOperator implements Operator { + + public record GroupSpec(int channel, ElementType elementType) {} + + public record HashAggregationOperatorFactory( + List groups, + List aggregators, + int maxPageSize, + BigArrays bigArrays + ) implements OperatorFactory { + @Override + public Operator get(DriverContext driverContext) { + return new HashAggregationOperator(aggregators, () -> BlockHash.build(groups, bigArrays, maxPageSize), driverContext); + } + + @Override + public String describe() { + return "HashAggregationOperator[mode = " + + "" + + ", aggs = " + + aggregators.stream().map(Describable::describe).collect(joining(", ")) + + "]"; + } + } + + private boolean finished; + private Page output; + + private final BlockHash blockHash; + + private final List aggregators; + + public HashAggregationOperator( + List aggregators, + Supplier blockHash, + DriverContext driverContext + ) { + this.aggregators = new ArrayList<>(aggregators.size()); + boolean success = false; + try { + this.blockHash = blockHash.get(); + for (GroupingAggregator.Factory a : aggregators) { + this.aggregators.add(a.apply(driverContext)); + } + success = true; + } finally { + if (success == false) { + close(); + } + } + } + + @Override + public boolean needsInput() { + return finished == false; + } + + @Override + public void addInput(Page page) { + checkState(needsInput(), "Operator is already finishing"); + requireNonNull(page, "page is null"); + + GroupingAggregatorFunction.AddInput[] prepared = new GroupingAggregatorFunction.AddInput[aggregators.size()]; + for (int i = 0; i < prepared.length; i++) { + prepared[i] = aggregators.get(i).prepareProcessPage(blockHash, page); + } + + blockHash.add(wrapPage(page), new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + LongVector groupIdsVector = groupIds.asVector(); + if (groupIdsVector != null) { + add(positionOffset, groupIdsVector); + } else { + for (GroupingAggregatorFunction.AddInput p : prepared) { + p.add(positionOffset, groupIds); + } + } + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + for (GroupingAggregatorFunction.AddInput p : prepared) { + p.add(positionOffset, groupIds); + } + } + }); + } + + @Override + public Page getOutput() { + Page p = output; + output = null; + return p; + } + + @Override + public void finish() { + if (finished) { + return; + } + finished = true; + Block[] keys = blockHash.getKeys(); + IntVector selected = blockHash.nonEmpty(); + + int[] aggBlockCounts = aggregators.stream().mapToInt(GroupingAggregator::evaluateBlockCount).toArray(); + Block[] blocks = new Block[keys.length + Arrays.stream(aggBlockCounts).sum()]; + System.arraycopy(keys, 0, blocks, 0, keys.length); + int offset = keys.length; + for (int i = 0; i < aggregators.size(); i++) { + var aggregator = aggregators.get(i); + aggregator.evaluate(blocks, offset, selected); + offset += aggBlockCounts[i]; + } + output = new Page(blocks); + } + + @Override + public boolean isFinished() { + return finished && output == null; + } + + @Override + public void close() { + Releasables.close(blockHash, () -> Releasables.close(aggregators)); + } + + protected static void checkState(boolean condition, String msg) { + if (condition == false) { + throw new IllegalArgumentException(msg); + } + } + + protected Page wrapPage(Page page) { + return page; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("blockHash=").append(blockHash).append(", "); + sb.append("aggregators=").append(aggregators); + sb.append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LimitOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LimitOperator.java new file mode 100644 index 0000000000000..99fb410122d4e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LimitOperator.java @@ -0,0 +1,219 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +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.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +public class LimitOperator implements Operator { + /** + * Total number of position that are emitted by this operator. + */ + private final int limit; + + /** + * Remaining number of positions that will be emitted by this operator. + */ + private int limitRemaining; + + /** + * Count of pages that have been processed by this operator. + */ + private int pagesProcessed; + + private Page lastInput; + + private boolean finished; + + public LimitOperator(int limit) { + this.limit = this.limitRemaining = limit; + } + + public record Factory(int limit) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new LimitOperator(limit); + } + + @Override + public String describe() { + return "LimitOperator[limit = " + limit + "]"; + } + } + + @Override + public boolean needsInput() { + return finished == false && lastInput == null; + } + + @Override + public void addInput(Page page) { + assert lastInput == null : "has pending input page"; + lastInput = page; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean isFinished() { + return finished && lastInput == null; + } + + @Override + public Page getOutput() { + if (lastInput == null) { + return null; + } + + Page result; + if (lastInput.getPositionCount() <= limitRemaining) { + result = lastInput; + limitRemaining -= lastInput.getPositionCount(); + } else { + int[] filter = new int[limitRemaining]; + for (int i = 0; i < limitRemaining; i++) { + filter[i] = i; + } + Block[] blocks = new Block[lastInput.getBlockCount()]; + for (int b = 0; b < blocks.length; b++) { + blocks[b] = lastInput.getBlock(b).filter(filter); + } + result = new Page(blocks); + limitRemaining = 0; + } + if (limitRemaining == 0) { + finished = true; + } + lastInput = null; + pagesProcessed++; + + return result; + } + + @Override + public Status status() { + return new Status(limit, limitRemaining, pagesProcessed); + } + + @Override + public void close() { + + } + + @Override + public String toString() { + return "LimitOperator[limit = " + limitRemaining + "/" + limit + "]"; + } + + public static class Status implements Operator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "limit", + Status::new + ); + + /** + * Total number of position that are emitted by this operator. + */ + private final int limit; + + /** + * Remaining number of positions that will be emitted by this operator. + */ + private final int limitRemaining; + + /** + * Count of pages that have been processed by this operator. + */ + private final int pagesProcessed; + + protected Status(int limit, int limitRemaining, int pagesProcessed) { + this.limit = limit; + this.limitRemaining = limitRemaining; + this.pagesProcessed = pagesProcessed; + } + + protected Status(StreamInput in) throws IOException { + limit = in.readVInt(); + limitRemaining = in.readVInt(); + pagesProcessed = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(limit); + out.writeVInt(limitRemaining); + out.writeVInt(pagesProcessed); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + /** + * Total number of position that are emitted by this operator. + */ + public int limit() { + return limit; + } + + /** + * Count of pages that have been processed by this operator. + */ + public int limitRemaining() { + return limitRemaining; + } + + /** + * Count of pages that have been processed by this operator. + */ + public int pagesProcessed() { + return pagesProcessed; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("limit", limit); + builder.field("limit_remaining", limitRemaining); + builder.field("pages_processed", pagesProcessed); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return limit == status.limit && limitRemaining == status.limitRemaining && pagesProcessed == status.pagesProcessed; + } + + @Override + public int hashCode() { + return Objects.hash(limit, limitRemaining, pagesProcessed); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LocalSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LocalSourceOperator.java new file mode 100644 index 0000000000000..b5d1b817d5005 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/LocalSourceOperator.java @@ -0,0 +1,85 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.fromList; +import static org.elasticsearch.compute.data.BlockUtils.fromListRow; + +public class LocalSourceOperator extends SourceOperator { + + public record LocalSourceFactory(Supplier factory) implements SourceOperatorFactory { + + @Override + public SourceOperator get(DriverContext driverContext) { + return factory().get(); + } + + @Override + public String describe() { + return "LocalSourceOperator[" + factory + "]"; + } + } + + public interface ObjectSupplier extends Supplier> {} + + public interface ListSupplier extends Supplier>> {} + + public interface BlockSupplier extends Supplier {} + + public interface PageSupplier extends Supplier {} + + protected final PageSupplier supplier; + + boolean finished; + + public LocalSourceOperator(ObjectSupplier objectSupplier) { + this(() -> fromListRow(objectSupplier.get())); + } + + public LocalSourceOperator(ListSupplier listSupplier) { + this(() -> fromList(listSupplier.get())); + } + + public LocalSourceOperator(BlockSupplier blockSupplier) { + this(() -> { + var blocks = blockSupplier.get(); + return CollectionUtils.isEmpty(blocks) ? new Page(0, blocks) : new Page(blocks); + }); + } + + public LocalSourceOperator(PageSupplier pageSupplier) { + this.supplier = pageSupplier; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean isFinished() { + return finished; + } + + @Override + public Page getOutput() { + var page = supplier.get(); + finished = true; + return page; + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupe.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupe.java new file mode 100644 index 0000000000000..cb51dc53c1840 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupe.java @@ -0,0 +1,162 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; + +import java.util.function.Supplier; + +/** + * Utilities to remove duplicates from multivalued fields. + */ +public final class MultivalueDedupe { + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public static Block dedupeToBlockAdaptive(Block block) { + return switch (block.elementType()) { + case BOOLEAN -> new MultivalueDedupeBoolean((BooleanBlock) block).dedupeToBlock(); + case BYTES_REF -> new MultivalueDedupeBytesRef((BytesRefBlock) block).dedupeToBlockAdaptive(); + case INT -> new MultivalueDedupeInt((IntBlock) block).dedupeToBlockAdaptive(); + case LONG -> new MultivalueDedupeLong((LongBlock) block).dedupeToBlockAdaptive(); + case DOUBLE -> new MultivalueDedupeDouble((DoubleBlock) block).dedupeToBlockAdaptive(); + default -> throw new IllegalArgumentException(); + }; + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public static Block dedupeToBlockUsingCopyMissing(Block block) { + return switch (block.elementType()) { + case BOOLEAN -> new MultivalueDedupeBoolean((BooleanBlock) block).dedupeToBlock(); + case BYTES_REF -> new MultivalueDedupeBytesRef((BytesRefBlock) block).dedupeToBlockUsingCopyMissing(); + case INT -> new MultivalueDedupeInt((IntBlock) block).dedupeToBlockUsingCopyMissing(); + case LONG -> new MultivalueDedupeLong((LongBlock) block).dedupeToBlockUsingCopyMissing(); + case DOUBLE -> new MultivalueDedupeDouble((DoubleBlock) block).dedupeToBlockUsingCopyMissing(); + default -> throw new IllegalArgumentException(); + }; + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public static Block dedupeToBlockUsingCopyAndSort(Block block) { + return switch (block.elementType()) { + case BOOLEAN -> new MultivalueDedupeBoolean((BooleanBlock) block).dedupeToBlock(); + case BYTES_REF -> new MultivalueDedupeBytesRef((BytesRefBlock) block).dedupeToBlockUsingCopyAndSort(); + case INT -> new MultivalueDedupeInt((IntBlock) block).dedupeToBlockUsingCopyAndSort(); + case LONG -> new MultivalueDedupeLong((LongBlock) block).dedupeToBlockUsingCopyAndSort(); + case DOUBLE -> new MultivalueDedupeDouble((DoubleBlock) block).dedupeToBlockUsingCopyAndSort(); + default -> throw new IllegalArgumentException(); + }; + } + + /** + * Build and {@link EvalOperator.ExpressionEvaluator} that deduplicates values + * using an adaptive algorithm based on the size of the input list. + */ + public static Supplier evaluator( + ElementType elementType, + Supplier nextSupplier + ) { + return switch (elementType) { + case BOOLEAN -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return new MultivalueDedupeBoolean((BooleanBlock) field.eval(page)).dedupeToBlock(); + } + }; + case BYTES_REF -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return new MultivalueDedupeBytesRef((BytesRefBlock) field.eval(page)).dedupeToBlockAdaptive(); + } + }; + case INT -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return new MultivalueDedupeInt((IntBlock) field.eval(page)).dedupeToBlockAdaptive(); + } + }; + case LONG -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return new MultivalueDedupeLong((LongBlock) field.eval(page)).dedupeToBlockAdaptive(); + } + }; + case DOUBLE -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return new MultivalueDedupeDouble((DoubleBlock) field.eval(page)).dedupeToBlockAdaptive(); + } + }; + case NULL -> () -> new MvDedupeEvaluator(nextSupplier.get()) { + @Override + public Block eval(Page page) { + return field.eval(page); // The page is all nulls and when you dedupe that it's still all nulls + } + }; + default -> throw new IllegalArgumentException("unsupported type [" + elementType + "]"); + }; + } + + /** + * Result of calling "hash" on a multivalue dedupe. + */ + public record HashResult(LongBlock ords, boolean sawNull) {} + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public static BatchEncoder batchEncoder(Block block, int batchSize) { + // TODO collect single-valued block handling here. And maybe vector. And maybe all null? + // TODO check for for unique multivalued fields and for ascending multivalue fields. + return switch (block.elementType()) { + case BOOLEAN -> new MultivalueDedupeBoolean((BooleanBlock) block).batchEncoder(batchSize); + case BYTES_REF -> new MultivalueDedupeBytesRef((BytesRefBlock) block).batchEncoder(batchSize); + case INT -> new MultivalueDedupeInt((IntBlock) block).batchEncoder(batchSize); + case LONG -> new MultivalueDedupeLong((LongBlock) block).batchEncoder(batchSize); + case DOUBLE -> new MultivalueDedupeDouble((DoubleBlock) block).batchEncoder(batchSize); + default -> throw new IllegalArgumentException(); + }; + } + + private abstract static class MvDedupeEvaluator implements EvalOperator.ExpressionEvaluator { + protected final EvalOperator.ExpressionEvaluator field; + + private MvDedupeEvaluator(EvalOperator.ExpressionEvaluator field) { + this.field = field; + } + + @Override + public String toString() { + return "MvDedupe[field=" + field + "]"; + } + } + + private MultivalueDedupe() {} +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupeBoolean.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupeBoolean.java new file mode 100644 index 0000000000000..b4e7dd8914eb8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupeBoolean.java @@ -0,0 +1,196 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.LongBlock; + +/** + * Removes duplicate values from multivalued positions. + */ +public class MultivalueDedupeBoolean { + /** + * Ordinal assigned to {@code null}. + */ + public static final int NULL_ORD = 0; + /** + * Ordinal assigned to {@code false}. + */ + public static final int FALSE_ORD = 1; + /** + * Ordinal assigned to {@code true}. + */ + public static final int TRUE_ORD = 2; + + private final BooleanBlock block; + private boolean seenTrue; + private boolean seenFalse; + + public MultivalueDedupeBoolean(BooleanBlock block) { + this.block = block; + } + + /** + * Dedupe values using an adaptive algorithm based on the size of the input list. + */ + public BooleanBlock dedupeToBlock() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendBoolean(block.getBoolean(first)); + default -> { + readValues(first, count); + writeValues(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + * @param everSeen array tracking if the values {@code false} and {@code true} are ever seen + */ + public LongBlock hash(boolean[] everSeen) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + everSeen[NULL_ORD] = true; + builder.appendLong(NULL_ORD); + } + case 1 -> builder.appendLong(hashOrd(everSeen, block.getBoolean(first))); + default -> { + readValues(first, count); + hashValues(everSeen, builder); + } + } + } + return builder.build(); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.Booleans(Math.max(2, batchSize)) { + @Override + protected void readNextBatch() { + for (int position = firstPosition(); position < block.getPositionCount(); position++) { + if (hasCapacity(2) == false) { + return; + } + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { + boolean v = block.getBoolean(first); + startPosition(); + encode(v); + endPosition(); + } + default -> { + readValues(first, count); + startPosition(); + encodeUniquedWork(this); + endPosition(); + } + } + } + } + }; + } + + private void readValues(int first, int count) { + int end = first + count; + + seenFalse = false; + seenTrue = false; + for (int i = first; i < end; i++) { + if (block.getBoolean(i)) { + seenTrue = true; + if (seenFalse) { + break; + } + } else { + seenFalse = true; + if (seenTrue) { + break; + } + } + } + } + + private void writeValues(BooleanBlock.Builder builder) { + if (seenFalse) { + if (seenTrue) { + builder.beginPositionEntry(); + builder.appendBoolean(false); + builder.appendBoolean(true); + builder.endPositionEntry(); + } else { + builder.appendBoolean(false); + } + } else if (seenTrue) { + builder.appendBoolean(true); + } else { + throw new IllegalStateException("didn't see true of false but counted values"); + } + } + + private void hashValues(boolean[] everSeen, LongBlock.Builder builder) { + if (seenFalse) { + if (seenTrue) { + builder.beginPositionEntry(); + builder.appendLong(hashOrd(everSeen, false)); + builder.appendLong(hashOrd(everSeen, true)); + builder.endPositionEntry(); + } else { + builder.appendLong(hashOrd(everSeen, false)); + } + } else if (seenTrue) { + builder.appendLong(hashOrd(everSeen, true)); + } else { + throw new IllegalStateException("didn't see true of false but counted values"); + } + } + + private void encodeUniquedWork(BatchEncoder.Booleans encoder) { + if (seenFalse) { + encoder.encode(false); + } + if (seenTrue) { + encoder.encode(true); + } + } + + /** + * Convert the boolean to an ordinal and track if it's been seen in {@code everSeen}. + */ + public static long hashOrd(boolean[] everSeen, boolean b) { + if (b) { + everSeen[TRUE_ORD] = true; + return TRUE_ORD; + } + everSeen[FALSE_ORD] = true; + return FALSE_ORD; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MvExpandOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MvExpandOperator.java new file mode 100644 index 0000000000000..f6156507dffa2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MvExpandOperator.java @@ -0,0 +1,163 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +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.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Objects; + +/** + * "Expands" multivalued blocks by duplicating all the other columns for each value. + *
+ *     [0, 1, 2] | 2 | "foo"
+ * 
+ * becomes + *
+ *     0 | 2 | "foo"
+ *     1 | 2 | "foo"
+ *     2 | 2 | "foo"
+ * 
+ */ +public class MvExpandOperator extends AbstractPageMappingOperator { + public record Factory(int channel) implements OperatorFactory { + @Override + public Operator get(DriverContext driverContext) { + return new MvExpandOperator(channel); + } + + @Override + public String describe() { + return "MvExpandOperator[channel=" + channel + "]"; + } + } + + private final int channel; + + private int noops; + + public MvExpandOperator(int channel) { + this.channel = channel; + } + + @Override + protected Page process(Page page) { + Block expandingBlock = page.getBlock(channel); + Block expandedBlock = expandingBlock.expand(); + if (expandedBlock == expandingBlock) { + noops++; + return page; + } + if (page.getBlockCount() == 1) { + assert channel == 0; + return new Page(expandedBlock); + } + + int[] duplicateFilter = buildDuplicateExpandingFilter(expandingBlock, expandedBlock.getPositionCount()); + + Block[] result = new Block[page.getBlockCount()]; + for (int b = 0; b < result.length; b++) { + result[b] = b == channel ? expandedBlock : page.getBlock(b).filter(duplicateFilter); + } + return new Page(result); + } + + private int[] buildDuplicateExpandingFilter(Block expandingBlock, int newPositions) { + int[] duplicateFilter = new int[newPositions]; + int n = 0; + for (int p = 0; p < expandingBlock.getPositionCount(); p++) { + int count = expandingBlock.getValueCount(p); + int positions = count == 0 ? 1 : count; + Arrays.fill(duplicateFilter, n, n + positions, p); + n += positions; + } + return duplicateFilter; + } + + @Override + protected AbstractPageMappingOperator.Status status(int pagesProcessed) { + return new Status(pagesProcessed, noops); + } + + @Override + public String toString() { + return "MvExpandOperator[channel=" + channel + "]"; + } + + public static final class Status extends AbstractPageMappingOperator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "mv_expand", + Status::new + ); + + private final int noops; + + Status(int pagesProcessed, int noops) { + super(pagesProcessed); + this.noops = noops; + } + + Status(StreamInput in) throws IOException { + super(in); + noops = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVInt(noops); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("pages_processed", pagesProcessed()); + builder.field("noops", noops); + return builder.endObject(); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public int noops() { + return noops; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Status status = (Status) o; + return noops == status.noops && pagesProcessed() == status.pagesProcessed(); + } + + @Override + public int hashCode() { + return Objects.hash(noops, pagesProcessed()); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java new file mode 100644 index 0000000000000..1e61dc8010070 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Operator.java @@ -0,0 +1,115 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.xcontent.ToXContentObject; + +/** + * Operator is low-level building block that consumes, transforms and produces data. + * An operator can have state, and assumes single-threaded access. + * Data is processed in smaller batches (termed {@link Page}s) that are passed to + * (see {@link #addInput(Page)}) or retrieved from (see {@link #getOutput()} operators. + * The component that's in charge of passing data between operators is the {@link Driver}. + * + * More details on how this integrates with other components can be found in the package documentation of + * {@link org.elasticsearch.compute} + */ +public interface Operator extends Releasable { + /** + * Target number of bytes in a page. By default we'll try and size pages + * so that they contain this many bytes. + */ + int TARGET_PAGE_SIZE = Math.toIntExact(ByteSizeValue.ofKb(256).getBytes()); + + /** + * The minimum number of positions for a {@link SourceOperator} to + * target generating. This isn't 1 because {@link Block}s have + * non-trivial overhead and it's just not worth building even + * smaller blocks without under normal circumstances. + */ + int MIN_TARGET_PAGE_SIZE = 10; + + /** + * whether the given operator can accept more input pages + */ + boolean needsInput(); + + /** + * adds an input page to the operator. only called when needsInput() == true and isFinished() == false + * @throws UnsupportedOperationException if the operator is a {@link SourceOperator} + */ + void addInput(Page page); + + /** + * notifies the operator that it won't receive any more input pages + */ + void finish(); + + /** + * whether the operator has finished processing all input pages and made the corresponding output pages available + */ + boolean isFinished(); + + /** + * returns non-null if output page available. Only called when isFinished() == false + * @throws UnsupportedOperationException if the operator is a {@link SinkOperator} + */ + Page getOutput(); + + /** + * notifies the operator that it won't be used anymore (i.e. none of the other methods called), + * and its resources can be cleaned up + */ + @Override + void close(); + + /** + * The status of the operator. + */ + default Status status() { + return null; + } + + /** + * An operator can be blocked on some action (e.g. waiting for some resources to become available). + * If so, it returns a future that completes when the operator becomes unblocked. + * If the operator is not blocked, this method returns {@link #NOT_BLOCKED} which is an already + * completed future. + */ + default ListenableActionFuture isBlocked() { + return NOT_BLOCKED; + } + + ListenableActionFuture NOT_BLOCKED = newCompletedFuture(); + + static ListenableActionFuture newCompletedFuture() { + ListenableActionFuture fut = new ListenableActionFuture<>(); + fut.onResponse(null); + return fut; + } + + /** + * A factory for creating intermediate operators. + */ + interface OperatorFactory extends Describable { + /** Creates a new intermediate operator. */ + Operator get(DriverContext driverContext); + } + + /** + * Status of an {@link Operator} to be returned by the tasks API. + */ + interface Status extends ToXContentObject, NamedWriteable {} +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java new file mode 100644 index 0000000000000..3a1cf5fee3512 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -0,0 +1,451 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.aggregation.GroupingAggregator; +import org.elasticsearch.compute.aggregation.GroupingAggregator.Factory; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.BlockOrdinalsReader; +import org.elasticsearch.compute.lucene.ValueSourceInfo; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.HashAggregationOperator.GroupSpec; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.aggregations.support.ValuesSource; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; + +/** + * Unlike {@link HashAggregationOperator}, this hash operator also extracts values or ordinals of the input documents. + */ +public class OrdinalsGroupingOperator implements Operator { + public record OrdinalsGroupingOperatorFactory( + List sources, + int docChannel, + String groupingField, + List aggregators, + int maxPageSize, + BigArrays bigArrays + ) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new OrdinalsGroupingOperator(sources, docChannel, groupingField, aggregators, maxPageSize, bigArrays, driverContext); + } + + @Override + public String describe() { + return "OrdinalsGroupingOperator(aggs = " + aggregators.stream().map(Describable::describe).collect(joining(", ")) + ")"; + } + } + + private final List sources; + private final int docChannel; + private final String groupingField; + + private final List aggregatorFactories; + private final Map ordinalAggregators; + private final BigArrays bigArrays; + + private final DriverContext driverContext; + + private boolean finished = false; + + // used to extract and aggregate values + private final int maxPageSize; + private ValuesAggregator valuesAggregator; + + public OrdinalsGroupingOperator( + List sources, + int docChannel, + String groupingField, + List aggregatorFactories, + int maxPageSize, + BigArrays bigArrays, + DriverContext driverContext + ) { + Objects.requireNonNull(aggregatorFactories); + boolean bytesValues = sources.get(0).source() instanceof ValuesSource.Bytes; + for (int i = 1; i < sources.size(); i++) { + if (sources.get(i).source() instanceof ValuesSource.Bytes != bytesValues) { + throw new IllegalStateException("ValuesSources are mismatched"); + } + } + this.sources = sources; + this.docChannel = docChannel; + this.groupingField = groupingField; + this.aggregatorFactories = aggregatorFactories; + this.ordinalAggregators = new HashMap<>(); + this.maxPageSize = maxPageSize; + this.bigArrays = bigArrays; + this.driverContext = driverContext; + } + + @Override + public boolean needsInput() { + return finished == false; + } + + @Override + public void addInput(Page page) { + checkState(needsInput(), "Operator is already finishing"); + requireNonNull(page, "page is null"); + DocVector docVector = page.getBlock(docChannel).asVector(); + if (docVector.getPositionCount() == 0) { + return; + } + final int shardIndex = docVector.shards().getInt(0); + final var source = sources.get(shardIndex); + if (docVector.singleSegmentNonDecreasing() && source.source() instanceof ValuesSource.Bytes.WithOrdinals withOrdinals) { + final IntVector segmentIndexVector = docVector.segments(); + assert segmentIndexVector.isConstant(); + final OrdinalSegmentAggregator ordinalAggregator = this.ordinalAggregators.computeIfAbsent( + new SegmentID(shardIndex, segmentIndexVector.getInt(0)), + k -> { + final List groupingAggregators = createGroupingAggregators(); + boolean success = false; + try { + final LeafReaderContext leafReaderContext = source.reader().leaves().get(k.segmentIndex); + final OrdinalSegmentAggregator ordinalSegmentAggregator = new OrdinalSegmentAggregator( + groupingAggregators, + withOrdinals, + leafReaderContext, + bigArrays + ); + success = true; + return ordinalSegmentAggregator; + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + if (success == false) { + Releasables.close(groupingAggregators); + } + } + } + ); + ordinalAggregator.addInput(docVector.docs(), page); + } else { + if (valuesAggregator == null) { + int channelIndex = page.getBlockCount(); // extractor will append a new block at the end + valuesAggregator = new ValuesAggregator( + sources, + docChannel, + groupingField, + channelIndex, + aggregatorFactories, + maxPageSize, + bigArrays, + driverContext + ); + } + valuesAggregator.addInput(page); + } + } + + private List createGroupingAggregators() { + boolean success = false; + List aggregators = new ArrayList<>(aggregatorFactories.size()); + try { + for (GroupingAggregator.Factory aggregatorFactory : aggregatorFactories) { + aggregators.add(aggregatorFactory.apply(driverContext)); + } + success = true; + return aggregators; + } finally { + if (success == false) { + Releasables.close(aggregators); + } + } + } + + @Override + public Page getOutput() { + if (finished == false) { + return null; + } + if (valuesAggregator != null) { + try { + return valuesAggregator.getOutput(); + } finally { + final ValuesAggregator aggregator = this.valuesAggregator; + this.valuesAggregator = null; + Releasables.close(aggregator); + } + } + if (ordinalAggregators.isEmpty() == false) { + try { + return mergeOrdinalsSegmentResults(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } finally { + Releasables.close(() -> Releasables.close(ordinalAggregators.values()), ordinalAggregators::clear); + } + } + return null; + } + + @Override + public void finish() { + finished = true; + if (valuesAggregator != null) { + valuesAggregator.finish(); + } + } + + private Page mergeOrdinalsSegmentResults() throws IOException { + // TODO: Should we also combine from the results from ValuesAggregator + final PriorityQueue pq = new PriorityQueue<>(ordinalAggregators.size()) { + @Override + protected boolean lessThan(AggregatedResultIterator a, AggregatedResultIterator b) { + return a.currentTerm.compareTo(b.currentTerm) < 0; + } + }; + final List aggregators = createGroupingAggregators(); + try { + for (OrdinalSegmentAggregator agg : ordinalAggregators.values()) { + final AggregatedResultIterator it = agg.getResultIterator(); + if (it.next()) { + pq.add(it); + } + } + int position = -1; + final BytesRefBuilder lastTerm = new BytesRefBuilder(); + var blockBuilder = BytesRefBlock.newBlockBuilder(1); + while (pq.size() > 0) { + final AggregatedResultIterator top = pq.top(); + if (position == -1 || lastTerm.get().equals(top.currentTerm) == false) { + position++; + lastTerm.copyBytes(top.currentTerm); + blockBuilder.appendBytesRef(top.currentTerm); + } + for (int i = 0; i < top.aggregators.size(); i++) { + aggregators.get(i).addIntermediateRow(position, top.aggregators.get(i), top.currentPosition()); + } + if (top.next()) { + pq.updateTop(); + } else { + pq.pop(); + } + } + int[] aggBlockCounts = aggregators.stream().mapToInt(GroupingAggregator::evaluateBlockCount).toArray(); + Block[] blocks = new Block[1 + Arrays.stream(aggBlockCounts).sum()]; + blocks[0] = blockBuilder.build(); + IntVector selected = IntVector.range(0, blocks[0].getPositionCount()); + int offset = 1; + for (int i = 0; i < aggregators.size(); i++) { + aggregators.get(i).evaluate(blocks, offset, selected); + offset += aggBlockCounts[i]; + } + return new Page(blocks); + } finally { + Releasables.close(() -> Releasables.close(aggregators)); + } + } + + @Override + public boolean isFinished() { + return finished && valuesAggregator == null && ordinalAggregators.isEmpty(); + } + + @Override + public void close() { + Releasables.close(() -> Releasables.close(ordinalAggregators.values()), valuesAggregator); + } + + private static void checkState(boolean condition, String msg) { + if (condition == false) { + throw new IllegalArgumentException(msg); + } + } + + @Override + public String toString() { + return this.getClass().getSimpleName() + "[" + "aggregators=" + aggregatorFactories + "]"; + } + + record SegmentID(int shardIndex, int segmentIndex) { + + } + + static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds { + private final List aggregators; + private final ValuesSource.Bytes.WithOrdinals withOrdinals; + private final LeafReaderContext leafReaderContext; + private final BitArray visitedOrds; + private BlockOrdinalsReader currentReader; + + OrdinalSegmentAggregator( + List aggregators, + ValuesSource.Bytes.WithOrdinals withOrdinals, + LeafReaderContext leafReaderContext, + BigArrays bigArrays + ) throws IOException { + boolean success = false; + try { + this.aggregators = aggregators; + this.withOrdinals = withOrdinals; + this.leafReaderContext = leafReaderContext; + final SortedSetDocValues sortedSetDocValues = withOrdinals.ordinalsValues(leafReaderContext); + this.currentReader = new BlockOrdinalsReader(sortedSetDocValues); + this.visitedOrds = new BitArray(sortedSetDocValues.getValueCount(), bigArrays); + success = true; + } finally { + if (success == false) { + close(); + } + } + } + + void addInput(IntVector docs, Page page) { + try { + GroupingAggregatorFunction.AddInput[] prepared = new GroupingAggregatorFunction.AddInput[aggregators.size()]; + for (int i = 0; i < prepared.length; i++) { + prepared[i] = aggregators.get(i).prepareProcessPage(this, page); + } + + if (BlockOrdinalsReader.canReuse(currentReader, docs.getInt(0)) == false) { + currentReader = new BlockOrdinalsReader(withOrdinals.ordinalsValues(leafReaderContext)); + } + final LongBlock ordinals = currentReader.readOrdinals(docs); + for (int p = 0; p < ordinals.getPositionCount(); p++) { + if (ordinals.isNull(p)) { + continue; + } + int start = ordinals.getFirstValueIndex(p); + int end = start + ordinals.getValueCount(p); + for (int i = start; i < end; i++) { + long ord = ordinals.getLong(i); + visitedOrds.set(ord); + } + } + for (GroupingAggregator aggregator : aggregators) { + aggregator.prepareProcessPage(this, page).add(0, ordinals); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + AggregatedResultIterator getResultIterator() throws IOException { + return new AggregatedResultIterator(aggregators, visitedOrds, withOrdinals.ordinalsValues(leafReaderContext)); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + BitArray seen = new BitArray(0, bigArrays); + seen.or(visitedOrds); + return seen; + } + + @Override + public void close() { + Releasables.close(visitedOrds, () -> Releasables.close(aggregators)); + } + } + + private static class AggregatedResultIterator { + private BytesRef currentTerm; + private long currentOrd = -1; + private final List aggregators; + private final BitArray ords; + private final SortedSetDocValues dv; + + AggregatedResultIterator(List aggregators, BitArray ords, SortedSetDocValues dv) { + this.aggregators = aggregators; + this.ords = ords; + this.dv = dv; + } + + int currentPosition() { + assert currentOrd != Long.MAX_VALUE : "Must not read position when iterator is exhausted"; + return Math.toIntExact(currentOrd); + } + + boolean next() throws IOException { + currentOrd = ords.nextSetBit(currentOrd + 1); + if (currentOrd < Long.MAX_VALUE) { + currentTerm = dv.lookupOrd(currentOrd); + return true; + } else { + currentTerm = null; + return false; + } + } + } + + private static class ValuesAggregator implements Releasable { + private final ValuesSourceReaderOperator extractor; + private final HashAggregationOperator aggregator; + + ValuesAggregator( + List sources, + int docChannel, + String groupingField, + int channelIndex, + List aggregatorFactories, + int maxPageSize, + BigArrays bigArrays, + DriverContext driverContext + ) { + this.extractor = new ValuesSourceReaderOperator(sources, docChannel, groupingField); + this.aggregator = new HashAggregationOperator( + aggregatorFactories, + () -> BlockHash.build(List.of(new GroupSpec(channelIndex, sources.get(0).elementType())), bigArrays, maxPageSize), + driverContext + ); + } + + void addInput(Page page) { + extractor.addInput(page); + Page out = extractor.getOutput(); + if (out != null) { + aggregator.addInput(out); + } + } + + void finish() { + aggregator.finish(); + } + + Page getOutput() { + return aggregator.getOutput(); + } + + @Override + public void close() { + Releasables.close(extractor, aggregator); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OutputOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OutputOperator.java new file mode 100644 index 0000000000000..47ee5bb1b6a15 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OutputOperator.java @@ -0,0 +1,85 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.function.Consumer; +import java.util.function.Function; + +import static java.util.stream.Collectors.joining; + +/** + * Sink operator that calls a given listener for each page received. The listener receives both the page as well as schema information, + * i.e. the names of the rows that are outputted. + */ +public class OutputOperator extends SinkOperator { + + private final List columns; + private final Consumer pageConsumer; + private final Function mapper; + + public record OutputOperatorFactory(List columns, Function mapper, Consumer pageConsumer) + implements + SinkOperatorFactory { + + @Override + public SinkOperator get(DriverContext driverContext) { + return new OutputOperator(columns, mapper, pageConsumer); + } + + @Override + public String describe() { + return "OutputOperator[columns = " + columns.stream().collect(joining(", ")) + "]"; + } + } + + public OutputOperator(List columns, Function mapper, Consumer pageConsumer) { + this.columns = columns; + this.mapper = mapper; + this.pageConsumer = pageConsumer; + } + + boolean finished = false; + + @Override + public boolean isFinished() { + return finished; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean needsInput() { + return finished == false; + } + + @Override + public void addInput(Page page) { + pageConsumer.accept(mapper.apply(page)); + } + + @Override + public void close() { + + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("columns=").append(columns).append(", "); + sb.append("pageConsumer=").append(pageConsumer); + sb.append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/PageConsumerOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/PageConsumerOperator.java new file mode 100644 index 0000000000000..3d53a09856c1f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/PageConsumerOperator.java @@ -0,0 +1,60 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; + +import java.util.function.Consumer; + +/** + * Sink operator that's useful for passing off pages to a {@link Consumer}. + */ +public class PageConsumerOperator extends SinkOperator { + + private final Consumer pageConsumer; + + public PageConsumerOperator(Consumer pageConsumer) { + this.pageConsumer = pageConsumer; + } + + boolean finished = false; + + @Override + public boolean isFinished() { + return finished; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean needsInput() { + return finished == false; + } + + @Override + public void addInput(Page page) { + pageConsumer.accept(page); + } + + @Override + public void close() { + + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("pageConsumer=").append(pageConsumer); + sb.append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ProjectOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ProjectOperator.java new file mode 100644 index 0000000000000..4192bfd570bd4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ProjectOperator.java @@ -0,0 +1,66 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.Arrays; +import java.util.BitSet; + +public class ProjectOperator extends AbstractPageMappingOperator { + + private final BitSet bs; + private Block[] blocks; + + public record ProjectOperatorFactory(BitSet mask) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new ProjectOperator(mask); + } + + @Override + public String describe() { + return "ProjectOperator[mask = " + mask + "]"; + } + } + + /** + * Creates a project that applies the given mask (as a bitset). + * + * @param mask bitset mask for enabling/disabling blocks / columns inside a Page + */ + public ProjectOperator(BitSet mask) { + this.bs = mask; + } + + @Override + protected Page process(Page page) { + if (page.getBlockCount() == 0) { + return page; + } + if (blocks == null) { + blocks = new Block[bs.cardinality()]; + } + + Arrays.fill(blocks, null); + int b = 0; + int positionCount = page.getPositionCount(); + for (int i = bs.nextSetBit(0); i >= 0 && i < page.getBlockCount(); i = bs.nextSetBit(i + 1)) { + var block = page.getBlock(i); + blocks[b++] = block; + } + return new Page(positionCount, blocks); + } + + @Override + public String toString() { + return "ProjectOperator[mask = " + bs + ']'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/RowOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/RowOperator.java new file mode 100644 index 0000000000000..bff6d1c34fe48 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/RowOperator.java @@ -0,0 +1,45 @@ +/* + * 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.compute.operator; + +import java.util.List; +import java.util.Objects; + +import static java.util.stream.Collectors.joining; + +public class RowOperator extends LocalSourceOperator { + + private final List objects; + + public record RowOperatorFactory(List objects) implements SourceOperatorFactory { + + @Override + public SourceOperator get(DriverContext driverContext) { + return new RowOperator(objects); + } + + @Override + public String describe() { + return "RowOperator[objects = " + objects.stream().map(Objects::toString).collect(joining(",")) + "]"; + } + } + + public RowOperator(List objects) { + super(() -> objects); + this.objects = objects; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(this.getClass().getSimpleName()).append("["); + sb.append("objects=").append(objects); + sb.append("]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ShowOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ShowOperator.java new file mode 100644 index 0000000000000..3a8baad260c37 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ShowOperator.java @@ -0,0 +1,32 @@ +/* + * 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.compute.operator; + +import java.util.List; +import java.util.Objects; + +import static java.util.stream.Collectors.joining; + +public class ShowOperator extends LocalSourceOperator { + + public record ShowOperatorFactory(List> objects) implements SourceOperatorFactory { + @Override + public String describe() { + return "ShowOperator[objects = " + objects.stream().map(Objects::toString).collect(joining(",")) + "]"; + } + + @Override + public SourceOperator get(DriverContext driverContext) { + return new ShowOperator(() -> objects); + } + } + + public ShowOperator(ListSupplier listSupplier) { + super(listSupplier); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SinkOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SinkOperator.java new file mode 100644 index 0000000000000..f469906379595 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SinkOperator.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.compute.operator; + +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Page; + +/** + * A sink operator - accepts input, produces no output. + */ +public abstract class SinkOperator implements Operator { + + /** + * A sink operator produces no output - unconditionally throws UnsupportedOperationException + */ + @Override + public final Page getOutput() { + throw new UnsupportedOperationException(); + } + + /** + * A factory for creating sink operators. + */ + public interface SinkOperatorFactory extends Describable { + /** Creates a new sink operator. */ + SinkOperator get(DriverContext driverContext); + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SourceOperator.java new file mode 100644 index 0000000000000..3047f562ba026 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/SourceOperator.java @@ -0,0 +1,42 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Page; + +/** + * A source operator - produces output, accepts no input. + */ +public abstract class SourceOperator implements Operator { + /** + * A source operator needs no input - unconditionally returns false. + * @return false + */ + public final boolean needsInput() { + return false; + } + + /** + * A source operator does not accept input - unconditionally throws UnsupportedOperationException. + * @param page a page + */ + @Override + public final void addInput(Page page) { + throw new UnsupportedOperationException(); + } + + /** + * A factory for creating source operators. + */ + public interface SourceOperatorFactory extends OperatorFactory, Describable { + /** Creates a new source operator. */ + @Override + SourceOperator get(DriverContext driverContext); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/StringExtractOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/StringExtractOperator.java new file mode 100644 index 0000000000000..d512f40e0dcbb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/StringExtractOperator.java @@ -0,0 +1,142 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class StringExtractOperator extends AbstractPageMappingOperator { + + public record StringExtractOperatorFactory( + String[] fieldNames, + Supplier expressionEvaluator, + Supplier>> parserSupplier + ) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new StringExtractOperator(fieldNames, expressionEvaluator.get(), parserSupplier.get()); + } + + @Override + public String describe() { + return "StringExtractOperator[fields=[" + Arrays.stream(fieldNames).collect(Collectors.joining(", ")) + "]]"; + } + } + + private final String[] fieldNames; + private final EvalOperator.ExpressionEvaluator inputEvaluator; + private final Function> parser; // TODO parser should consume ByteRef instead of String + + public StringExtractOperator( + String[] fieldNames, + EvalOperator.ExpressionEvaluator inputEvaluator, + Function> parser + ) { + this.fieldNames = fieldNames; + this.inputEvaluator = inputEvaluator; + this.parser = parser; + } + + @Override + protected Page process(Page page) { + int rowsCount = page.getPositionCount(); + + BytesRefBlock.Builder[] blockBuilders = new BytesRefBlock.Builder[fieldNames.length]; + for (int i = 0; i < fieldNames.length; i++) { + blockBuilders[i] = BytesRefBlock.newBlockBuilder(rowsCount); + } + + BytesRefBlock input = (BytesRefBlock) inputEvaluator.eval(page); + BytesRef spare = new BytesRef(); + for (int row = 0; row < rowsCount; row++) { + if (input.isNull(row)) { + for (int i = 0; i < fieldNames.length; i++) { + blockBuilders[i].appendNull(); + } + continue; + } + + int position = input.getFirstValueIndex(row); + int valueCount = input.getValueCount(row); + if (valueCount == 1) { + Map items = parser.apply(input.getBytesRef(position, spare).utf8ToString()); + if (items == null) { + for (int i = 0; i < fieldNames.length; i++) { + blockBuilders[i].appendNull(); + } + continue; + } + for (int i = 0; i < fieldNames.length; i++) { + String val = items.get(fieldNames[i]); + BlockUtils.appendValue(blockBuilders[i], val, ElementType.BYTES_REF); + } + } else { + // multi-valued input + String[] firstValues = new String[fieldNames.length]; + boolean[] positionEntryOpen = new boolean[fieldNames.length]; + for (int c = 0; c < valueCount; c++) { + Map items = parser.apply(input.getBytesRef(position + c, spare).utf8ToString()); + if (items == null) { + continue; + } + for (int i = 0; i < fieldNames.length; i++) { + String val = items.get(fieldNames[i]); + if (val == null) { + continue; + } + if (firstValues[i] == null) { + firstValues[i] = val; + } else { + if (positionEntryOpen[i] == false) { + positionEntryOpen[i] = true; + blockBuilders[i].beginPositionEntry(); + BlockUtils.appendValue(blockBuilders[i], firstValues[i], ElementType.BYTES_REF); + } + BlockUtils.appendValue(blockBuilders[i], val, ElementType.BYTES_REF); + } + } + } + for (int i = 0; i < fieldNames.length; i++) { + if (positionEntryOpen[i]) { + blockBuilders[i].endPositionEntry(); + } else if (firstValues[i] == null) { + blockBuilders[i].appendNull(); + } else { + BlockUtils.appendValue(blockBuilders[i], firstValues[i], ElementType.BYTES_REF); + } + } + } + } + + Block[] blocks = new Block[blockBuilders.length]; + for (int i = 0; i < blockBuilders.length; i++) { + blocks[i] = blockBuilders[i].build(); + } + return page.appendBlocks(blocks); + } + + @Override + public String toString() { + return "StringExtractOperator[fields=[" + Arrays.stream(fieldNames).collect(Collectors.joining(", ")) + "]]"; + } + + public interface ExtractEvaluator { + Map computeRow(Page page, int position); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/TopNOperator.java new file mode 100644 index 0000000000000..aa95579bbd0e1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/TopNOperator.java @@ -0,0 +1,508 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +public class TopNOperator implements Operator { + + /** + * Internal row to be used in the PriorityQueue instead of the full blown Page. + * It mirrors somehow the Block build in the sense that it keeps around an array of offsets and a count of values (to account for + * multivalues) to reference each position in each block of the Page. + */ + static final class Row { + boolean[] booleans; + int[] ints; + long[] longs; + double[] doubles; + BytesRef[] byteRefs; + int[] docs; + boolean[] nullValues; + + int[] idToFirstValueIndex; // keeps the offset inside each of the arrays above where a specific block position starts from + ElementType[] idToType; + int[] numberOfValues; // keeps the count of values of each field in the specialized array + + boolean isNull(int i) { + return nullValues[i]; + } + + boolean getBoolean(int i, int offset) { + return booleans[idToFirstValueIndex[i] + offset]; + } + + int getInt(int i, int offset) { + return ints[idToFirstValueIndex[i] + offset]; + } + + long getLong(int i, int offset) { + return longs[idToFirstValueIndex[i] + offset]; + } + + double getDouble(int i, int offset) { + return doubles[idToFirstValueIndex[i] + offset]; + } + + BytesRef getBytesRef(int i, int offset) { + return byteRefs[idToFirstValueIndex[i] + offset]; + } + } + + static final class RowFactory { + + int size; + int nBooleans; + int nInts; + int nLongs; + int nDoubles; + int nByteRefs; + int nDocs; + + ElementType[] idToType; + + RowFactory(Page page) { + size = page.getBlockCount(); + idToType = new ElementType[size]; + for (int i = 0; i < size; i++) { + Block block = page.getBlock(i); + switch (block.elementType()) { + case LONG -> nLongs++; + case INT -> nInts++; + case DOUBLE -> nDoubles++; + case BYTES_REF -> nByteRefs++; + case BOOLEAN -> nBooleans++; + case DOC -> nDocs++; + case NULL -> { + } + case UNKNOWN -> { + assert false : "Must not occur here as TopN should never receive intermediate blocks"; + throw new UnsupportedOperationException("Block doesn't support retrieving elements"); + } + } + idToType[i] = block.elementType(); + } + } + + Row row(Page origin, int rowNum, Row spare) { + Row result; + if (spare == null) { + result = new Row(); + result.nullValues = new boolean[size]; + result.booleans = new boolean[nBooleans]; + result.ints = new int[nInts]; + result.longs = new long[nLongs]; + result.doubles = new double[nDoubles]; + result.byteRefs = new BytesRef[nByteRefs]; + for (int i = 0; i < nByteRefs; i++) { + result.byteRefs[i] = new BytesRef(); + } + result.idToFirstValueIndex = new int[size]; + result.idToType = idToType; + result.docs = new int[nDocs * 3]; + result.numberOfValues = new int[size]; + } else { + result = spare; + // idToType has to be set because different pages could have different block types due to different mappings + result.idToType = idToType; + Arrays.fill(result.nullValues, false); + } + + int lastLongFirstValueIndex = 0; + int lastIntFirstValueIndex = 0; + int lastDoubleFirstValueIndex = 0; + int lastBytesRefFirstValueIndex = 0; + int lastBooleanFirstValueIndex = 0; + int lastDocFirstValueIndex = 0; + + for (int i = 0; i < size; i++) { + Block block = origin.getBlock(i); + if (block.isNull(rowNum)) { + result.nullValues[i] = true; + } else { + int valuesCount = block.getValueCount(rowNum); + result.numberOfValues[i] = valuesCount; + switch (block.elementType()) { + case LONG -> { + int firstValueIndex = lastLongFirstValueIndex; + if (firstValueIndex + valuesCount > result.longs.length) { + result.longs = Arrays.copyOf(result.longs, firstValueIndex + valuesCount); + } + int start = block.getFirstValueIndex(rowNum); + int end = start + valuesCount; + for (int j = start, offset = 0; j < end; j++, offset++) { + result.longs[firstValueIndex + offset] = ((LongBlock) block).getLong(j); + } + result.idToFirstValueIndex[i] = firstValueIndex; + lastLongFirstValueIndex = firstValueIndex + valuesCount; + } + case INT -> { + int firstValueIndex = lastIntFirstValueIndex; + if (firstValueIndex + valuesCount > result.ints.length) { + result.ints = Arrays.copyOf(result.ints, firstValueIndex + valuesCount); + } + int start = block.getFirstValueIndex(rowNum); + int end = start + valuesCount; + for (int j = start, offset = 0; j < end; j++, offset++) { + result.ints[firstValueIndex + offset] = ((IntBlock) block).getInt(j); + } + result.idToFirstValueIndex[i] = firstValueIndex; + lastIntFirstValueIndex = firstValueIndex + valuesCount; + } + case DOUBLE -> { + int firstValueIndex = lastDoubleFirstValueIndex; + if (firstValueIndex + valuesCount > result.doubles.length) { + result.doubles = Arrays.copyOf(result.doubles, firstValueIndex + valuesCount); + } + int start = block.getFirstValueIndex(rowNum); + int end = start + valuesCount; + for (int j = start, offset = 0; j < end; j++, offset++) { + result.doubles[firstValueIndex + offset] = ((DoubleBlock) block).getDouble(j); + } + result.idToFirstValueIndex[i] = firstValueIndex; + lastDoubleFirstValueIndex = firstValueIndex + valuesCount; + } + case BYTES_REF -> { + int firstValueIndex = lastBytesRefFirstValueIndex; + if (firstValueIndex + valuesCount > result.byteRefs.length) { + int additionalSize = firstValueIndex + valuesCount - result.byteRefs.length; + result.byteRefs = Arrays.copyOf(result.byteRefs, firstValueIndex + valuesCount); + for (int j = 1; j <= additionalSize; j++) { + result.byteRefs[result.byteRefs.length - j] = new BytesRef(); + } + } + int start = block.getFirstValueIndex(rowNum); + int end = start + valuesCount; + for (int j = start, offset = 0; j < end; j++, offset++) { + BytesRef b = result.byteRefs[firstValueIndex + offset]; + b = ((BytesRefBlock) block).getBytesRef(j, b); + result.byteRefs[firstValueIndex + offset] = b; + } + result.idToFirstValueIndex[i] = firstValueIndex; + lastBytesRefFirstValueIndex = firstValueIndex + valuesCount; + } + case BOOLEAN -> { + int firstValueIndex = lastBooleanFirstValueIndex; + if (firstValueIndex + valuesCount > result.booleans.length) { + result.booleans = Arrays.copyOf(result.booleans, firstValueIndex + valuesCount); + } + int start = block.getFirstValueIndex(rowNum); + int end = start + valuesCount; + for (int j = start, offset = 0; j < end; j++, offset++) { + result.booleans[firstValueIndex + offset] = ((BooleanBlock) block).getBoolean(j); + } + result.idToFirstValueIndex[i] = firstValueIndex; + lastBooleanFirstValueIndex = firstValueIndex + valuesCount; + } + case DOC -> { + int firstValueIndex = lastDocFirstValueIndex; + if (firstValueIndex + 3 > result.docs.length) { + result.docs = Arrays.copyOf(result.docs, firstValueIndex + 3); + } + DocVector doc = ((DocBlock) block).asVector(); + result.docs[firstValueIndex] = doc.shards().getInt(rowNum); + result.docs[firstValueIndex + 1] = doc.segments().getInt(rowNum); + result.docs[firstValueIndex + 2] = doc.docs().getInt(rowNum); + + result.idToFirstValueIndex[i] = firstValueIndex; + lastDocFirstValueIndex = firstValueIndex + 3; + } + case NULL -> { + assert false : "Must not occur here as we check nulls above already"; + throw new UnsupportedOperationException("Block of nulls doesn't support comparison"); + } + default -> { + assert false : "Must not occur here as TopN should never receive intermediate blocks"; + throw new UnsupportedOperationException("Block doesn't support retrieving elements"); + } + + } + } + } + return result; + } + } + + public record SortOrder(int channel, boolean asc, boolean nullsFirst) {} + + public record TopNOperatorFactory(int topCount, List sortOrders, int maxPageSize) implements OperatorFactory { + + @Override + public Operator get(DriverContext driverContext) { + return new TopNOperator(topCount, sortOrders, maxPageSize); + } + + @Override + public String describe() { + return "TopNOperator[count = " + topCount + ", sortOrders = " + sortOrders + "]"; + } + } + + private final PriorityQueue inputQueue; + + private final int maxPageSize; + private RowFactory rowFactory; + + // these will be inferred at runtime: one input page might not contain all the information needed + // eg. it could be missing some fields in the mapping, so it could have NULL blocks as placeholders + private ElementType[] outputTypes; + + private Iterator output; + + public TopNOperator(int topCount, List sortOrders, int maxPageSize) { + this.maxPageSize = maxPageSize; + if (sortOrders.size() == 1) { + // avoid looping over sortOrders if there is only one order + SortOrder order = sortOrders.get(0); + this.inputQueue = new PriorityQueue<>(topCount) { + @Override + protected boolean lessThan(Row a, Row b) { + return comparePositions(order.asc, order.nullsFirst, a, b, order.channel) < 0; + } + + @Override + public String toString() { + return "count = " + size() + "/" + topCount + ", sortOrder = " + order; + } + }; + } else { + this.inputQueue = new PriorityQueue<>(topCount) { + @Override + protected boolean lessThan(Row a, Row b) { + return TopNOperator.compareTo(sortOrders, a, b) < 0; + } + + @Override + public String toString() { + return "count = " + size() + "/" + topCount + ", sortOrders = " + sortOrders; + } + }; + } + } + + private static int compareTo(List orders, Row a, Row b) { + for (SortOrder order : orders) { + int cmp = comparePositions(order.asc, order.nullsFirst, a, b, order.channel); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + + static int comparePositions(boolean asc, boolean nullsFirst, Row b1, Row b2, int position) { + boolean firstIsNull = b1.isNull(position); + boolean secondIsNull = b2.isNull(position); + if (firstIsNull || secondIsNull) { + return Boolean.compare(firstIsNull, secondIsNull) * (nullsFirst ? 1 : -1); + } + if (b1.idToType[position] != b2.idToType[position]) { + throw new IllegalStateException( + "Blocks have incompatible element types: " + b1.idToType[position] + " != " + b2.idToType[position] + ); + } + int cmp = switch (b1.idToType[position]) { + case INT -> Integer.compare(b1.getInt(position, 0), b2.getInt(position, 0)); + case LONG -> Long.compare(b1.getLong(position, 0), b2.getLong(position, 0)); + case DOUBLE -> Double.compare(b1.getDouble(position, 0), b2.getDouble(position, 0)); + case BOOLEAN -> Boolean.compare(b1.getBoolean(position, 0), b2.getBoolean(position, 0)); + case BYTES_REF -> b1.getBytesRef(position, 0).compareTo(b2.getBytesRef(position, 0)); + case DOC -> throw new UnsupportedOperationException("Block of nulls doesn't support comparison"); + case NULL -> { + assert false : "Must not occur here as we check nulls above already"; + throw new UnsupportedOperationException("Block of nulls doesn't support comparison"); + } + case UNKNOWN -> { + assert false : "Must not occur here as TopN should never receive intermediate blocks"; + throw new UnsupportedOperationException("Block doesn't support retrieving elements"); + } + }; + return asc ? -cmp : cmp; + } + + @Override + public boolean needsInput() { + return output == null; + } + + @Override + public void addInput(Page page) { + // rebuild for every page, since blocks can originate from different indices, with different mapping + rowFactory = new RowFactory(page); + if (outputTypes == null) { + outputTypes = Arrays.copyOf(rowFactory.idToType, rowFactory.idToType.length); + } else { + for (int i = 0; i < rowFactory.idToType.length; i++) { + if (outputTypes[i] == ElementType.NULL) { // the type could just be missing in the previous mappings + outputTypes[i] = rowFactory.idToType[i]; + } + } + } + + Row removed = null; + for (int i = 0; i < page.getPositionCount(); i++) { + Row x = rowFactory.row(page, i, removed); + removed = inputQueue.insertWithOverflow(x); + } + } + + @Override + public void finish() { + if (output == null) { + output = toPages(); + } + } + + private Iterator toPages() { + if (inputQueue.size() == 0) { + return Collections.emptyIterator(); + } + List list = new ArrayList<>(inputQueue.size()); + while (inputQueue.size() > 0) { + list.add(inputQueue.pop()); + } + Collections.reverse(list); + + List result = new ArrayList<>(); + Block.Builder[] builders = null; + int p = 0; + int size = 0; + for (int i = 0; i < list.size(); i++) { + if (builders == null) { + size = Math.min(maxPageSize, list.size() - i); + builders = new Block.Builder[rowFactory.size]; + for (int b = 0; b < builders.length; b++) { + builders[b] = outputTypes[b].newBlockBuilder(size); + } + p = 0; + } + + Row row = list.get(i); + for (int b = 0; b < builders.length; b++) { + if (row.isNull(b)) { + builders[b].appendNull(); + continue; + } + switch (outputTypes[b]) { + case BOOLEAN -> { + if (row.numberOfValues[b] > 1) { + ((BooleanBlock.Builder) builders[b]).beginPositionEntry(); + for (int j = 0; j < row.numberOfValues[b]; j++) { + ((BooleanBlock.Builder) builders[b]).appendBoolean(row.getBoolean(b, j)); + } + ((BooleanBlock.Builder) builders[b]).endPositionEntry(); + } else { + ((BooleanBlock.Builder) builders[b]).appendBoolean(row.getBoolean(b, 0)); + } + } + case INT -> { + if (row.numberOfValues[b] > 1) { + ((IntBlock.Builder) builders[b]).beginPositionEntry(); + for (int j = 0; j < row.numberOfValues[b]; j++) { + ((IntBlock.Builder) builders[b]).appendInt(row.getInt(b, j)); + } + ((IntBlock.Builder) builders[b]).endPositionEntry(); + } else { + ((IntBlock.Builder) builders[b]).appendInt(row.getInt(b, 0)); + } + } + case LONG -> { + if (row.numberOfValues[b] > 1) { + ((LongBlock.Builder) builders[b]).beginPositionEntry(); + for (int j = 0; j < row.numberOfValues[b]; j++) { + ((LongBlock.Builder) builders[b]).appendLong(row.getLong(b, j)); + } + ((LongBlock.Builder) builders[b]).endPositionEntry(); + } else { + ((LongBlock.Builder) builders[b]).appendLong(row.getLong(b, 0)); + } + } + case DOUBLE -> { + if (row.numberOfValues[b] > 1) { + ((DoubleBlock.Builder) builders[b]).beginPositionEntry(); + for (int j = 0; j < row.numberOfValues[b]; j++) { + ((DoubleBlock.Builder) builders[b]).appendDouble(row.getDouble(b, j)); + } + ((DoubleBlock.Builder) builders[b]).endPositionEntry(); + } else { + ((DoubleBlock.Builder) builders[b]).appendDouble(row.getDouble(b, 0)); + } + } + case BYTES_REF -> { + if (row.numberOfValues[b] > 1) { + ((BytesRefBlock.Builder) builders[b]).beginPositionEntry(); + for (int j = 0; j < row.numberOfValues[b]; j++) { + ((BytesRefBlock.Builder) builders[b]).appendBytesRef(row.getBytesRef(b, j)); + } + ((BytesRefBlock.Builder) builders[b]).endPositionEntry(); + } else { + ((BytesRefBlock.Builder) builders[b]).appendBytesRef(row.getBytesRef(b, 0)); + } + } + case DOC -> { + int dp = row.idToFirstValueIndex[b]; + int shard = row.docs[dp++]; + int segment = row.docs[dp++]; + int doc = row.docs[dp]; + ((DocBlock.Builder) builders[b]).appendShard(shard).appendSegment(segment).appendDoc(doc); + } + case NULL -> builders[b].appendNull(); + default -> throw new IllegalStateException("unsupported type [" + rowFactory.idToType[b] + "]"); + } + } + + p++; + if (p == size) { + result.add(new Page(Arrays.stream(builders).map(Block.Builder::build).toArray(Block[]::new))); + builders = null; + } + } + assert builders == null; + return result.iterator(); + } + + @Override + public boolean isFinished() { + return output != null && output.hasNext() == false; + } + + @Override + public Page getOutput() { + if (output != null && output.hasNext()) { + return output.next(); + } + return null; + } + + @Override + public void close() { + + } + + @Override + public String toString() { + return "TopNOperator[" + inputQueue + "]"; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/X-MultivalueDedupe.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/X-MultivalueDedupe.java.st new file mode 100644 index 0000000000000..7c4fdb7bebdec --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/X-MultivalueDedupe.java.st @@ -0,0 +1,498 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.ArrayUtil; +$if(BytesRef)$ +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BytesRefHash; +$else$ +import org.elasticsearch.common.util.LongHash; +$endif$ +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +$if(long)$ +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; + +$else$ +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.$Type$Block; +import org.elasticsearch.compute.data.LongBlock; +$endif$ + +import java.util.Arrays; + +/** + * Removes duplicate values from multivalued positions. + * This class is generated. Edit {@code X-MultivalueDedupe.java.st} instead. + */ +public class MultivalueDedupe$Type$ { + /** + * The number of entries before we switch from and {@code n^2} strategy + * with low overhead to an {@code n*log(n)} strategy with higher overhead. + * The choice of number has been experimentally derived. + */ +$if(BytesRef)$ + private static final int ALWAYS_COPY_MISSING = 20; // TODO BytesRef should try adding to the hash *first* and then comparing. +$elseif(double)$ + private static final int ALWAYS_COPY_MISSING = 110; +$elseif(int)$ + private static final int ALWAYS_COPY_MISSING = 300; +$elseif(long)$ + private static final int ALWAYS_COPY_MISSING = 300; +$endif$ + + private final $Type$Block block; + private $type$[] work = new $type$[ArrayUtil.oversize(2, $BYTES$)]; + private int w; + + public MultivalueDedupe$Type$($Type$Block block) { + this.block = block; +$if(BytesRef)$ + // TODO very large numbers might want a hash based implementation - and for BytesRef that might not be that big + fillWork(0, work.length); +$endif$ + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an adaptive algorithm based on the size of the input list. + */ + public $Type$Block dedupeToBlockAdaptive() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + $Type$Block.Builder builder = $Type$Block.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); +$if(BytesRef)$ + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); +$else$ + case 1 -> builder.append$Type$(block.get$Type$(first)); +$endif$ + default -> { + /* + * It's better to copyMissing when there are few unique values + * and better to copy and sort when there are many unique values. + * The more duplicate values there are the more comparatively worse + * copyAndSort is. But we don't know how many unique values there + * because our job is to find them. So we use the count of values + * as a proxy that is fast to test. It's not always going to be + * optimal but it has the nice property of being quite quick on + * short lists and not n^2 levels of terrible on long ones. + * + * It'd also be possible to make a truly hybrid mechanism that + * switches from copyMissing to copyUnique once it collects enough + * unique values. The trouble is that the switch is expensive and + * makes kind of a "hole" in the performance of that mechanism where + * you may as well have just gone with either of the two other + * strategies. So we just don't try it for now. + */ + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + writeUniquedWork(builder); + } else { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm with very low overhead but {@code n^2} + * case complexity for larger. Prefer {@link #dedupeToBlockAdaptive} + * which picks based on the number of elements at each position. + */ + public $Type$Block dedupeToBlockUsingCopyAndSort() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + $Type$Block.Builder builder = $Type$Block.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); +$if(BytesRef)$ + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); +$else$ + case 1 -> builder.append$Type$(block.get$Type$(first)); +$endif$ + default -> { + copyAndSort(first, count); + writeSortedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Remove duplicate values from each position and write the results to a + * {@link Block} using an algorithm that sorts all values. It has a higher + * overhead for small numbers of values at each position than + * {@link #dedupeToBlockUsingCopyMissing} for large numbers of values the + * performance is dominated by the {@code n*log n} sort. Prefer + * {@link #dedupeToBlockAdaptive} unless you need the results sorted. + */ + public $Type$Block dedupeToBlockUsingCopyMissing() { + if (false == block.mayHaveMultivaluedFields()) { + return block; + } + $Type$Block.Builder builder = $Type$Block.newBlockBuilder(block.getPositionCount()); + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> builder.appendNull(); +$if(BytesRef)$ + case 1 -> builder.appendBytesRef(block.getBytesRef(first, work[0])); +$else$ + case 1 -> builder.append$Type$(block.get$Type$(first)); +$endif$ + default -> { + copyMissing(first, count); + writeUniquedWork(builder); + } + } + } + return builder.build(); + } + + /** + * Dedupe values and build a {@link LongBlock} suitable for passing + * as the grouping block to a {@link GroupingAggregatorFunction}. + */ +$if(BytesRef)$ + public MultivalueDedupe.HashResult hash(BytesRefHash hash) { +$else$ + public MultivalueDedupe.HashResult hash(LongHash hash) { +$endif$ + LongBlock.Builder builder = LongBlock.newBlockBuilder(block.getPositionCount()); + boolean sawNull = false; + for (int p = 0; p < block.getPositionCount(); p++) { + int count = block.getValueCount(p); + int first = block.getFirstValueIndex(p); + switch (count) { + case 0 -> { + sawNull = true; + builder.appendLong(0); + } + case 1 -> { +$if(BytesRef)$ + BytesRef v = block.getBytesRef(first, work[0]); +$else$ + $type$ v = block.get$Type$(first); +$endif$ + hash(builder, hash, v); + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + hashUniquedWork(hash, builder); + } else { + copyAndSort(first, count); + hashSortedWork(hash, builder); + } + } + } + } + return new MultivalueDedupe.HashResult(builder.build(), sawNull); + } + + /** + * Build a {@link BatchEncoder} which deduplicates values at each position + * and then encodes the results into a {@link byte[]} which can be used for + * things like hashing many fields together. + */ + public BatchEncoder batchEncoder(int batchSize) { + return new BatchEncoder.$Type$s(batchSize) { + @Override + protected void readNextBatch() { + int position = firstPosition(); + if (w > 0) { + // The last block didn't fit so we have to *make* it fit +$if(BytesRef)$ + ensureCapacity(workSize(), w); +$else$ + ensureCapacity(w); +$endif$ + startPosition(); + encodeUniquedWork(this); + endPosition(); + position++; + } + for (; position < block.getPositionCount(); position++) { + int count = block.getValueCount(position); + int first = block.getFirstValueIndex(position); + switch (count) { + case 0 -> encodeNull(); + case 1 -> { +$if(BytesRef)$ + BytesRef v = block.getBytesRef(first, work[0]); + if (hasCapacity(v.length, 1)) { +$else$ + $type$ v = block.get$Type$(first); + if (hasCapacity(1)) { +$endif$ + startPosition(); + encode(v); + endPosition(); + } else { + work[0] = v; + w = 1; + return; + } + } + default -> { + if (count < ALWAYS_COPY_MISSING) { + copyMissing(first, count); + } else { + copyAndSort(first, count); + convertSortedWorkToUnique(); + } +$if(BytesRef)$ + if (hasCapacity(workSize(), w)) { +$else$ + if (hasCapacity(w)) { +$endif$ + startPosition(); + encodeUniquedWork(this); + endPosition(); + } else { + return; + } + } + } + } + } + +$if(BytesRef)$ + private int workSize() { + int size = 0; + for (int i = 0; i < w; i++) { + size += work[i].length; + } + return size; + } +$endif$ + }; + } + + /** + * Copy all value from the position into {@link #work} and then + * sorts it {@code n * log(n)}. + */ + private void copyAndSort(int first, int count) { + grow(count); + int end = first + count; + + w = 0; + for (int i = first; i < end; i++) { +$if(BytesRef)$ + work[w] = block.getBytesRef(i, work[w]); + w++; +$else$ + work[w++] = block.get$Type$(i); +$endif$ + } + + Arrays.sort(work, 0, w); + } + + /** + * Fill {@link #work} with the unique values in the position by scanning + * all fields already copied {@code n^2}. + */ + private void copyMissing(int first, int count) { + grow(count); + int end = first + count; + +$if(BytesRef)$ + work[0] = block.getBytesRef(first, work[0]); +$else$ + work[0] = block.get$Type$(first); +$endif$ + w = 1; + i: for (int i = first + 1; i < end; i++) { +$if(BytesRef)$ + $type$ v = block.getBytesRef(i, work[w]); +$else$ + $type$ v = block.get$Type$(i); +$endif$ + for (int j = 0; j < w; j++) { +$if(BytesRef)$ + if (v.equals(work[j])) { +$else$ + if (v == work[j]) { +$endif$ + continue i; + } + } + work[w++] = v; + } + } + + /** + * Writes an already deduplicated {@link #work} to a {@link $Type$Block.Builder}. + */ + private void writeUniquedWork($Type$Block.Builder builder) { + if (w == 1) { + builder.append$Type$(work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + builder.append$Type$(work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a {@link $Type$Block.Builder}, skipping duplicates. + */ + private void writeSortedWork($Type$Block.Builder builder) { + if (w == 1) { + builder.append$Type$(work[0]); + return; + } + builder.beginPositionEntry(); + $type$ prev = work[0]; + builder.append$Type$(prev); + for (int i = 1; i < w; i++) { +$if(BytesRef)$ + if (false == prev.equals(work[i])) { +$else$ + if (prev != work[i]) { +$endif$ + prev = work[i]; + builder.append$Type$(prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes an already deduplicated {@link #work} to a hash. + */ +$if(BytesRef)$ + private void hashUniquedWork(BytesRefHash hash, LongBlock.Builder builder) { +$else$ + private void hashUniquedWork(LongHash hash, LongBlock.Builder builder) { +$endif$ + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + for (int i = 0; i < w; i++) { + hash(builder, hash, work[i]); + } + builder.endPositionEntry(); + } + + /** + * Writes a sorted {@link #work} to a hash, skipping duplicates. + */ +$if(BytesRef)$ + private void hashSortedWork(BytesRefHash hash, LongBlock.Builder builder) { +$else$ + private void hashSortedWork(LongHash hash, LongBlock.Builder builder) { +$endif$ + if (w == 1) { + hash(builder, hash, work[0]); + return; + } + builder.beginPositionEntry(); + $type$ prev = work[0]; + hash(builder, hash, prev); + for (int i = 1; i < w; i++) { +$if(BytesRef)$ + if (false == prev.equals(work[i])) { +$else$ + if (prev != work[i]) { +$endif$ + prev = work[i]; + hash(builder, hash, prev); + } + } + builder.endPositionEntry(); + } + + /** + * Writes a deduplicated {@link #work} to a {@link BatchEncoder.$Type$s}. + */ + private void encodeUniquedWork(BatchEncoder.$Type$s encoder) { + for (int i = 0; i < w; i++) { + encoder.encode(work[i]); + } + } + + /** + * Converts {@link #work} from sorted array to a deduplicated array. + */ + private void convertSortedWorkToUnique() { + $type$ prev = work[0]; + int end = w; + w = 1; + for (int i = 1; i < end; i++) { +$if(BytesRef)$ + if (false == prev.equals(work[i])) { + prev = work[i]; + work[w].bytes = prev.bytes; + work[w].offset = prev.offset; + work[w].length = prev.length; + w++; + } +$else$ + if (prev != work[i]) { + prev = work[i]; + work[w++] = prev; + } +$endif$ + } + } + + private void grow(int size) { +$if(BytesRef)$ + int prev = work.length; + work = ArrayUtil.grow(work, size); + fillWork(prev, work.length); +$else$ + work = ArrayUtil.grow(work, size); +$endif$ + } + +$if(BytesRef)$ + private void fillWork(int from, int to) { + for (int i = from; i < to; i++) { + work[i] = new BytesRef(); + } + } +$endif$ + +$if(BytesRef)$ + private void hash(LongBlock.Builder builder, BytesRefHash hash, BytesRef v) { +$else$ + private void hash(LongBlock.Builder builder, LongHash hash, $type$ v) { +$endif$ +$if(double)$ + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(Double.doubleToLongBits(v)))); +$else$ + builder.appendLong(BlockHash.hashOrdToGroupNullReserved(hash.add(v))); +$endif$ + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeBuffer.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeBuffer.java new file mode 100644 index 0000000000000..5c2c8d4b79349 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeBuffer.java @@ -0,0 +1,145 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Operator; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; + +final class ExchangeBuffer { + + private final Queue queue = new ConcurrentLinkedQueue<>(); + // uses a separate counter for size for CAS; and ConcurrentLinkedQueue#size is not a constant time operation. + private final AtomicInteger queueSize = new AtomicInteger(); + private final int maxSize; + + private final Object notEmptyLock = new Object(); + private ListenableActionFuture notEmptyFuture = null; + + private final Object notFullLock = new Object(); + private ListenableActionFuture notFullFuture = null; + + private final ListenableActionFuture completionFuture = new ListenableActionFuture<>(); + + private volatile boolean noMoreInputs = false; + + ExchangeBuffer(int maxSize) { + if (maxSize < 1) { + throw new IllegalArgumentException("max_buffer_size must be at least one; got=" + maxSize); + } + this.maxSize = maxSize; + } + + void addPage(Page page) { + queue.add(page); + if (queueSize.incrementAndGet() == 1) { + notifyNotEmpty(); + } + } + + Page pollPage() { + final var page = queue.poll(); + if (page != null && queueSize.decrementAndGet() == maxSize - 1) { + notifyNotFull(); + } + if (page == null && noMoreInputs && queueSize.get() == 0) { + completionFuture.onResponse(null); + } + return page; + } + + private void notifyNotEmpty() { + final ListenableActionFuture toNotify; + synchronized (notEmptyLock) { + toNotify = notEmptyFuture; + notEmptyFuture = null; + } + if (toNotify != null) { + toNotify.onResponse(null); + } + } + + private void notifyNotFull() { + final ListenableActionFuture toNotify; + synchronized (notFullLock) { + toNotify = notFullFuture; + notFullFuture = null; + } + if (toNotify != null) { + toNotify.onResponse(null); + } + } + + ListenableActionFuture waitForWriting() { + // maxBufferSize check is not water-tight as more than one sink can pass this check at the same time. + if (queueSize.get() < maxSize || noMoreInputs) { + return Operator.NOT_BLOCKED; + } + synchronized (notFullLock) { + if (queueSize.get() < maxSize || noMoreInputs) { + return Operator.NOT_BLOCKED; + } + if (notFullFuture == null) { + notFullFuture = new ListenableActionFuture<>(); + } + return notFullFuture; + } + } + + ListenableActionFuture waitForReading() { + if (size() > 0 || noMoreInputs) { + return Operator.NOT_BLOCKED; + } + synchronized (notEmptyLock) { + if (size() > 0 || noMoreInputs) { + return Operator.NOT_BLOCKED; + } + if (notEmptyFuture == null) { + notEmptyFuture = new ListenableActionFuture<>(); + } + return notEmptyFuture; + } + } + + void finish(boolean drainingPages) { + noMoreInputs = true; + if (drainingPages) { + while (pollPage() != null) { + + } + } + notifyNotEmpty(); + if (drainingPages || queueSize.get() == 0) { + completionFuture.onResponse(null); + } + } + + boolean isFinished() { + return completionFuture.isDone(); + } + + boolean noMoreInputs() { + return noMoreInputs; + } + + int size() { + return queueSize.get(); + } + + /** + * Adds a listener that will be notified when this exchange buffer is finished. + */ + void addCompletionListener(ActionListener listener) { + completionFuture.addListener(listener); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeRequest.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeRequest.java new file mode 100644 index 0000000000000..6ed2cc7e587be --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeRequest.java @@ -0,0 +1,84 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.transport.TransportRequest; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +public final class ExchangeRequest extends TransportRequest { + private final String exchangeId; + private final boolean sourcesFinished; + + public ExchangeRequest(String exchangeId, boolean sourcesFinished) { + this.exchangeId = exchangeId; + this.sourcesFinished = sourcesFinished; + } + + public ExchangeRequest(StreamInput in) throws IOException { + super(in); + this.exchangeId = in.readString(); + this.sourcesFinished = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(exchangeId); + out.writeBoolean(sourcesFinished); + } + + /** + * True if the {@link ExchangeSourceHandler} has enough input. + * The corresponding {@link ExchangeSinkHandler} can drain pages and finish itself. + */ + public boolean sourcesFinished() { + return sourcesFinished; + } + + /** + * Returns the exchange ID. We don't use the parent task id because it can be overwritten by a proxy node. + */ + public String exchangeId() { + return exchangeId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ExchangeRequest that = (ExchangeRequest) o; + return sourcesFinished == that.sourcesFinished && exchangeId.equals(that.exchangeId); + } + + @Override + public int hashCode() { + return Objects.hash(exchangeId, sourcesFinished); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + if (parentTaskId.isSet() == false) { + assert false : "ExchangeRequest must have a parent task"; + throw new IllegalStateException("ExchangeRequest must have a parent task"); + } + return new CancellableTask(id, type, action, "", parentTaskId, headers) { + @Override + public String getDescription() { + return "exchange request id=" + exchangeId; + } + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeResponse.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeResponse.java new file mode 100644 index 0000000000000..ec7a66d213d5f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeResponse.java @@ -0,0 +1,68 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.transport.TransportResponse; + +import java.io.IOException; +import java.util.Objects; + +public final class ExchangeResponse extends TransportResponse { + private final Page page; + private final boolean finished; + + public ExchangeResponse(Page page, boolean finished) { + this.page = page; + this.finished = finished; + } + + public ExchangeResponse(StreamInput in) throws IOException { + super(in); + this.page = in.readOptionalWriteable(Page::new); + this.finished = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalWriteable(page); + out.writeBoolean(finished); + } + + /** + * Returns a page responded by {@link RemoteSink}. This can be null and out of order. + */ + @Nullable + public Page page() { + return page; + } + + /** + * Returns true if the {@link RemoteSink} is already completed. In this case, the {@link ExchangeSourceHandler} + * can stop polling pages and finish itself. + */ + public boolean finished() { + return finished; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ExchangeResponse response = (ExchangeResponse) o; + return finished == response.finished && Objects.equals(page, response.page); + } + + @Override + public int hashCode() { + return Objects.hash(page, finished); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java new file mode 100644 index 0000000000000..32f21c57c5e9d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java @@ -0,0 +1,300 @@ +/* + * 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.compute.operator.exchange; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.ResourceNotFoundException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.component.Lifecycle; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractAsyncTask; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +import java.util.Map; + +/** + * {@link ExchangeService} is responsible for exchanging pages between exchange sinks and sources on the same or different nodes. + * It holds a map of {@link ExchangeSourceHandler} and {@link ExchangeSinkHandler} instances for each node in the cluster. + * To connect exchange sources to exchange sinks, use the {@link ExchangeSourceHandler#addRemoteSink(RemoteSink, int)} method. + */ +public final class ExchangeService extends AbstractLifecycleComponent { + // TODO: Make this a child action of the data node transport to ensure that exchanges + // are accessed only by the user initialized the session. + public static final String EXCHANGE_ACTION_NAME = "internal:data/read/esql/exchange"; + + private static final String OPEN_EXCHANGE_ACTION_NAME = "internal:data/read/esql/open_exchange"; + + /** + * The time interval for an exchange sink handler to be considered inactive and subsequently + * removed from the exchange service if no sinks are attached (i.e., no computation uses that sink handler). + */ + public static final String INACTIVE_SINKS_INTERVAL_SETTING = "esql.exchange.sink_inactive_interval"; + + private static final Logger LOGGER = LogManager.getLogger(ExchangeService.class); + + private final ThreadPool threadPool; + + private final Map sinks = ConcurrentCollections.newConcurrentMap(); + private final Map sources = ConcurrentCollections.newConcurrentMap(); + + private final InactiveSinksReaper inactiveSinksReaper; + + public ExchangeService(Settings settings, ThreadPool threadPool) { + this.threadPool = threadPool; + final var inactiveInterval = settings.getAsTime(INACTIVE_SINKS_INTERVAL_SETTING, TimeValue.timeValueMinutes(5)); + this.inactiveSinksReaper = new InactiveSinksReaper(LOGGER, threadPool, inactiveInterval); + } + + public void registerTransportHandler(TransportService transportService) { + transportService.registerRequestHandler( + EXCHANGE_ACTION_NAME, + ThreadPool.Names.SAME, + ExchangeRequest::new, + new ExchangeTransportAction() + ); + transportService.registerRequestHandler( + OPEN_EXCHANGE_ACTION_NAME, + ThreadPool.Names.SAME, + OpenExchangeRequest::new, + new OpenExchangeRequestHandler() + ); + } + + /** + * Creates an {@link ExchangeSinkHandler} for the specified exchange id. + * + * @throws IllegalStateException if a sink handler for the given id already exists + */ + ExchangeSinkHandler createSinkHandler(String exchangeId, int maxBufferSize) { + ExchangeSinkHandler sinkHandler = new ExchangeSinkHandler(maxBufferSize, threadPool::relativeTimeInMillis); + if (sinks.putIfAbsent(exchangeId, sinkHandler) != null) { + throw new IllegalStateException("sink exchanger for id [" + exchangeId + "] already exists"); + } + return sinkHandler; + } + + /** + * Returns an exchange sink handler for the given id. + */ + public ExchangeSinkHandler getSinkHandler(String exchangeId) { + ExchangeSinkHandler sinkHandler = sinks.get(exchangeId); + if (sinkHandler == null) { + throw new ResourceNotFoundException("sink exchanger for id [{}] doesn't exist", exchangeId); + } + return sinkHandler; + } + + /** + * Removes the exchange sink handler associated with the given exchange id. + */ + public void finishSinkHandler(String exchangeId, Exception failure) { + final ExchangeSinkHandler sinkHandler = sinks.remove(exchangeId); + if (sinkHandler != null) { + if (failure != null) { + sinkHandler.onFailure(failure); + } + assert sinkHandler.isFinished() : "Exchange sink " + exchangeId + " wasn't finished yet"; + } + } + + /** + * Creates an {@link ExchangeSourceHandler} for the specified exchange id. + * + * @throws IllegalStateException if a source handler for the given id already exists + */ + public ExchangeSourceHandler createSourceHandler(String exchangeId, int maxBufferSize, String fetchExecutor) { + ExchangeSourceHandler sourceHandler = new ExchangeSourceHandler(maxBufferSize, threadPool.executor(fetchExecutor)); + if (sources.putIfAbsent(exchangeId, sourceHandler) != null) { + throw new IllegalStateException("source exchanger for id [" + exchangeId + "] already exists"); + } + sourceHandler.addCompletionListener(ActionListener.releasing(() -> sources.remove(exchangeId))); + return sourceHandler; + } + + /** + * Opens a remote sink handler on the remote node for the given session ID. + */ + public static void openExchange( + TransportService transportService, + DiscoveryNode targetNode, + String sessionId, + int exchangeBuffer, + ActionListener listener + ) { + transportService.sendRequest( + targetNode, + OPEN_EXCHANGE_ACTION_NAME, + new OpenExchangeRequest(sessionId, exchangeBuffer), + new ActionListenerResponseHandler<>( + listener.map(unused -> null), + in -> TransportResponse.Empty.INSTANCE, + TransportResponseHandler.TRANSPORT_WORKER + ) + ); + } + + private static class OpenExchangeRequest extends TransportRequest { + private final String sessionId; + private final int exchangeBuffer; + + OpenExchangeRequest(String sessionId, int exchangeBuffer) { + this.sessionId = sessionId; + this.exchangeBuffer = exchangeBuffer; + } + + OpenExchangeRequest(StreamInput in) throws IOException { + super(in); + this.sessionId = in.readString(); + this.exchangeBuffer = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sessionId); + out.writeVInt(exchangeBuffer); + } + } + + private class OpenExchangeRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(OpenExchangeRequest request, TransportChannel channel, Task task) throws Exception { + createSinkHandler(request.sessionId, request.exchangeBuffer); + channel.sendResponse(new TransportResponse.Empty()); + } + } + + private class ExchangeTransportAction implements TransportRequestHandler { + @Override + public void messageReceived(ExchangeRequest request, TransportChannel channel, Task task) { + final String exchangeId = request.exchangeId(); + ActionListener listener = new ChannelActionListener<>(channel); + final ExchangeSinkHandler sinkHandler = sinks.get(exchangeId); + if (sinkHandler == null) { + listener.onResponse(new ExchangeResponse(null, true)); + } else { + // the data-node request hasn't arrived yet; use the task framework to cancel the request if needed. + if (sinkHandler.hasData() == false) { + ((CancellableTask) task).addListener(() -> sinkHandler.onFailure(new TaskCancelledException("task cancelled"))); + } + sinkHandler.fetchPageAsync(request.sourcesFinished(), listener); + } + } + } + + private final class InactiveSinksReaper extends AbstractAsyncTask { + InactiveSinksReaper(Logger logger, ThreadPool threadPool, TimeValue interval) { + super(logger, threadPool, interval, true); + rescheduleIfNecessary(); + } + + @Override + protected boolean mustReschedule() { + Lifecycle.State state = lifecycleState(); + return state != Lifecycle.State.STOPPED && state != Lifecycle.State.CLOSED; + } + + @Override + protected void runInternal() { + final TimeValue maxInterval = getInterval(); + final long nowInMillis = threadPool.relativeTimeInMillis(); + for (Map.Entry e : sinks.entrySet()) { + ExchangeSinkHandler sink = e.getValue(); + if (sink.hasData() && sink.hasListeners()) { + continue; + } + long elapsed = nowInMillis - sink.lastUpdatedTimeInMillis(); + if (elapsed > maxInterval.millis()) { + finishSinkHandler( + e.getKey(), + new ElasticsearchTimeoutException( + "Exchange sink {} has been inactive for {}", + e.getKey(), + TimeValue.timeValueMillis(elapsed) + ) + ); + } + } + } + } + + /** + * Creates a new {@link RemoteSink} that fetches pages from an exchange sink located on the remote node. + * + * @param parentTask the parent task that initialized the ESQL request + * @param exchangeId the exchange ID + * @param transportService the transport service + * @param remoteNode the node where the remote exchange sink is located + */ + public RemoteSink newRemoteSink(Task parentTask, String exchangeId, TransportService transportService, DiscoveryNode remoteNode) { + return new TransportRemoteSink(transportService, remoteNode, parentTask, exchangeId); + } + + record TransportRemoteSink(TransportService transportService, DiscoveryNode node, Task parentTask, String exchangeId) + implements + RemoteSink { + + @Override + public void fetchPageAsync(boolean allSourcesFinished, ActionListener listener) { + transportService.sendChildRequest( + node, + EXCHANGE_ACTION_NAME, + new ExchangeRequest(exchangeId, allSourcesFinished), + parentTask, + TransportRequestOptions.EMPTY, + new ActionListenerResponseHandler<>(listener, ExchangeResponse::new, TransportResponseHandler.TRANSPORT_WORKER) + ); + } + } + + // For testing + public boolean isEmpty() { + return sources.isEmpty() && sinks.isEmpty(); + } + + @Override + protected void doStart() { + + } + + @Override + protected void doStop() { + inactiveSinksReaper.close(); + } + + @Override + protected void doClose() { + doStop(); + } + + @Override + public String toString() { + return "ExchangeService{" + "sinks=" + sinks.keySet() + ", sources=" + sources.keySet() + '}'; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSink.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSink.java new file mode 100644 index 0000000000000..80a941711dfb5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSink.java @@ -0,0 +1,37 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.compute.data.Page; + +/** + * Sink for exchanging data + * @see ExchangeSinkOperator + */ +public interface ExchangeSink { + /** + * adds a new page to this sink + */ + void addPage(Page page); + + /** + * called once all pages have been added (see {@link #addPage(Page)}). + */ + void finish(); + + /** + * Whether the sink has received all pages + */ + boolean isFinished(); + + /** + * Whether the sink is blocked on adding more pages + */ + ListenableActionFuture waitForWriting(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkHandler.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkHandler.java new file mode 100644 index 0000000000000..901419f23ed56 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkHandler.java @@ -0,0 +1,182 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.compute.data.Page; + +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongSupplier; + +/** + * An {@link ExchangeSinkHandler} receives pages and status from its {@link ExchangeSink}s, which are created using + * {@link #createExchangeSink()}} method. Pages and status can then be retrieved asynchronously by {@link ExchangeSourceHandler}s + * using the {@link #fetchPageAsync(boolean, ActionListener)} method. + * + * @see #createExchangeSink() + * @see #fetchPageAsync(boolean, ActionListener) + * @see ExchangeSourceHandler + */ +public final class ExchangeSinkHandler { + + private final ExchangeBuffer buffer; + private final Queue> listeners = new ConcurrentLinkedQueue<>(); + private final AtomicInteger outstandingSinks = new AtomicInteger(); + // listeners are notified by only one thread. + private final Semaphore promised = new Semaphore(1); + + private final ListenableActionFuture completionFuture; + private final LongSupplier nowInMillis; + private final AtomicLong lastUpdatedInMillis; + + public ExchangeSinkHandler(int maxBufferSize, LongSupplier nowInMillis) { + this.buffer = new ExchangeBuffer(maxBufferSize); + this.completionFuture = new ListenableActionFuture<>(); + this.buffer.addCompletionListener(completionFuture); + this.nowInMillis = nowInMillis; + this.lastUpdatedInMillis = new AtomicLong(nowInMillis.getAsLong()); + } + + private class LocalExchangeSink implements ExchangeSink { + boolean finished; + + LocalExchangeSink() { + onChanged(); + outstandingSinks.incrementAndGet(); + } + + @Override + public void addPage(Page page) { + buffer.addPage(page); + notifyListeners(); + } + + @Override + public void finish() { + if (finished == false) { + finished = true; + onChanged(); + if (outstandingSinks.decrementAndGet() == 0) { + buffer.finish(false); + notifyListeners(); + } + } + } + + @Override + public boolean isFinished() { + return finished || buffer.isFinished(); + } + + @Override + public ListenableActionFuture waitForWriting() { + return buffer.waitForWriting(); + } + } + + /** + * Fetches pages and the sink status asynchronously. + * + * @param sourceFinished if true, then this handler can finish as sources have enough pages. + * @param listener the listener that will be notified when pages are ready or this handler is finished + * @see RemoteSink + * @see ExchangeSourceHandler#addRemoteSink(RemoteSink, int) + */ + public void fetchPageAsync(boolean sourceFinished, ActionListener listener) { + if (sourceFinished) { + buffer.finish(true); + } + listeners.add(listener); + onChanged(); + notifyListeners(); + } + + /** + * Add a listener, which will be notified when this exchange sink handler is completed. An exchange sink + * handler is consider completed when all associated sinks are completed and the output pages are fetched. + */ + public void addCompletionListener(ActionListener listener) { + completionFuture.addListener(listener); + } + + boolean isFinished() { + return completionFuture.isDone(); + } + + /** + * Fails this sink exchange handler + */ + void onFailure(Exception failure) { + completionFuture.onFailure(failure); + buffer.finish(true); + notifyListeners(); + } + + private void notifyListeners() { + while (listeners.isEmpty() == false && (buffer.size() > 0 || buffer.noMoreInputs())) { + if (promised.tryAcquire() == false) { + break; + } + final ActionListener listener; + final ExchangeResponse response; + try { + // Use `poll` and recheck because `listeners.isEmpty()` might return true, while a listener is being added + listener = listeners.poll(); + if (listener == null) { + continue; + } + response = new ExchangeResponse(buffer.pollPage(), buffer.isFinished()); + } finally { + promised.release(); + } + onChanged(); + listener.onResponse(response); + } + } + + /** + * Create a new exchange sink for exchanging data + * + * @see ExchangeSinkOperator + */ + public ExchangeSink createExchangeSink() { + return new LocalExchangeSink(); + } + + /** + * Whether this sink handler has sinks attached or available pages + */ + boolean hasData() { + return outstandingSinks.get() > 0 || buffer.size() > 0; + } + + /** + * Whether this sink handler has listeners waiting for data + */ + boolean hasListeners() { + return listeners.isEmpty() == false; + } + + private void onChanged() { + lastUpdatedInMillis.accumulateAndGet(nowInMillis.getAsLong(), Math::max); + } + + /** + * The time in millis when this sink handler was updated. This timestamp is used to prune idle sinks. + * + * @see ExchangeService#INACTIVE_SINKS_INTERVAL_SETTING + */ + long lastUpdatedTimeInMillis() { + return lastUpdatedInMillis.get(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperator.java new file mode 100644 index 0000000000000..d1bbdd9e0afd2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperator.java @@ -0,0 +1,157 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.Strings; +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.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.SinkOperator; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Sink operator implementation that pushes data to an {@link ExchangeSink} + */ +public class ExchangeSinkOperator extends SinkOperator { + + private final ExchangeSink sink; + private final Function transformer; + private int pagesAccepted; + + public record ExchangeSinkOperatorFactory(Supplier exchangeSinks, Function transformer) + implements + SinkOperatorFactory { + + public ExchangeSinkOperatorFactory(Supplier exchangeSinks) { + this(exchangeSinks, Function.identity()); + } + + @Override + public SinkOperator get(DriverContext driverContext) { + return new ExchangeSinkOperator(exchangeSinks.get(), transformer); + } + + @Override + public String describe() { + return "ExchangeSinkOperator"; + } + } + + public ExchangeSinkOperator(ExchangeSink sink, Function transformer) { + this.sink = sink; + this.transformer = transformer; + } + + @Override + public boolean isFinished() { + return sink.isFinished(); + } + + @Override + public void finish() { + sink.finish(); + } + + @Override + public ListenableActionFuture isBlocked() { + return sink.waitForWriting(); + } + + @Override + public boolean needsInput() { + return isFinished() == false && isBlocked().isDone(); + } + + @Override + public void addInput(Page page) { + pagesAccepted++; + var newPage = transformer.apply(page); + sink.addPage(newPage); + } + + @Override + public void close() { + finish(); + } + + @Override + public String toString() { + return "ExchangeSinkOperator"; + } + + @Override + public Status status() { + return new Status(pagesAccepted); + } + + public static class Status implements Operator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "exchange_sink", + Status::new + ); + + private final int pagesAccepted; + + Status(int pagesAccepted) { + this.pagesAccepted = pagesAccepted; + } + + Status(StreamInput in) throws IOException { + pagesAccepted = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(pagesAccepted); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public int pagesAccepted() { + return pagesAccepted; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("pages_accepted", pagesAccepted); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return pagesAccepted == status.pagesAccepted; + } + + @Override + public int hashCode() { + return Objects.hash(pagesAccepted); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSource.java new file mode 100644 index 0000000000000..78249689e3bd8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSource.java @@ -0,0 +1,42 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.compute.data.Page; + +/** + * Source for exchanging data + * @see ExchangeSourceOperator + */ +public interface ExchangeSource { + /** + * Remove the page from this source if any; otherwise, returns null + */ + Page pollPage(); + + /** + * Called when the source has enough input pages + */ + void finish(); + + /** + * Whether the associated sinks are finished and pages are processed. + */ + boolean isFinished(); + + /** + * Returns the number of pages that are buffered in this exchange source + */ + int bufferSize(); + + /** + * Allows callers to stop reading from the source when it's blocked + */ + ListenableActionFuture waitForReading(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java new file mode 100644 index 0000000000000..d657e7a117dd1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java @@ -0,0 +1,272 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.tasks.TaskCancelledException; + +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +/** + * An {@link ExchangeSourceHandler} asynchronously fetches pages and status from multiple {@link RemoteSink}s + * and feeds them to its {@link ExchangeSource}, which are created using the {@link #createExchangeSource()}) method. + * {@link RemoteSink}s are added using the {@link #addRemoteSink(RemoteSink, int)}) method. + * + * @see #createExchangeSource() + * @see #addRemoteSink(RemoteSink, int) + */ +public final class ExchangeSourceHandler extends AbstractRefCounted { + private final ExchangeBuffer buffer; + private final Executor fetchExecutor; + + private final PendingInstances outstandingSinks = new PendingInstances(); + private final PendingInstances outstandingSources = new PendingInstances(); + private final AtomicReference failure = new AtomicReference<>(); + private final ListenableActionFuture completionFuture = new ListenableActionFuture<>(); + + public ExchangeSourceHandler(int maxBufferSize, Executor fetchExecutor) { + this.buffer = new ExchangeBuffer(maxBufferSize); + this.fetchExecutor = fetchExecutor; + } + + private class LocalExchangeSource implements ExchangeSource { + private boolean finished; + + LocalExchangeSource() { + outstandingSources.trackNewInstance(); + } + + private void checkFailure() { + Exception e = failure.get(); + if (e != null) { + throw ExceptionsHelper.convertToElastic(e); + } + } + + @Override + public Page pollPage() { + checkFailure(); + return buffer.pollPage(); + } + + @Override + public boolean isFinished() { + checkFailure(); + return finished || buffer.isFinished(); + } + + @Override + public ListenableActionFuture waitForReading() { + return buffer.waitForReading(); + } + + @Override + public void finish() { + if (finished == false) { + finished = true; + if (outstandingSources.finishInstance()) { + buffer.finish(true); + } + } + } + + @Override + public int bufferSize() { + return buffer.size(); + } + } + + /** + * Create a new {@link ExchangeSource} for exchanging data + * + * @see ExchangeSinkOperator + */ + public ExchangeSource createExchangeSource() { + return new LocalExchangeSource(); + } + + /** + * If we continue fetching pages using the same thread, we risk encountering a StackOverflow error. + * On the other hand, if we fork when receiving a reply on the same thread, we add unnecessary overhead + * from thread scheduling and context switching. LoopControl can be used to avoid these issues. + */ + private static class LoopControl { + enum Status { + RUNNING, + EXITING, + EXITED + } + + private final Thread startedThread; + private Status status = Status.RUNNING; + + LoopControl() { + this.startedThread = Thread.currentThread(); + } + + boolean isRunning() { + return status == Status.RUNNING; + } + + boolean tryResume() { + if (startedThread == Thread.currentThread() && status != Status.EXITED) { + status = Status.RUNNING; + return true; + } else { + return false; + } + } + + void exiting() { + status = Status.EXITING; + } + + void exited() { + status = Status.EXITED; + } + } + + /** + * Wraps {@link RemoteSink} with a fetch loop and error handling + */ + private final class RemoteSinkFetcher { + private volatile boolean finished = false; + private final RemoteSink remoteSink; + + RemoteSinkFetcher(RemoteSink remoteSink) { + outstandingSinks.trackNewInstance(); + this.remoteSink = remoteSink; + } + + void fetchPage() { + final LoopControl loopControl = new LoopControl(); + while (loopControl.isRunning()) { + loopControl.exiting(); + // finish other sinks if one of them failed or sources no longer need pages. + boolean toFinishSinks = buffer.noMoreInputs() || failure.get() != null; + remoteSink.fetchPageAsync(toFinishSinks, ActionListener.wrap(resp -> { + Page page = resp.page(); + if (page != null) { + buffer.addPage(page); + } + if (resp.finished()) { + onSinkComplete(); + } else { + ListenableActionFuture future = buffer.waitForWriting(); + if (future.isDone()) { + if (loopControl.tryResume() == false) { + fetchPage(); + } + } else { + future.addListener(ActionListener.wrap(unused -> { + if (loopControl.tryResume() == false) { + fetchPage(); + } + }, this::onSinkFailed)); + } + } + }, this::onSinkFailed)); + } + loopControl.exited(); + } + + void onSinkFailed(Exception e) { + failure.getAndUpdate(first -> { + if (first == null) { + return e; + } + // ignore subsequent TaskCancelledException exceptions as they don't provide useful info. + if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) { + return first; + } + if (ExceptionsHelper.unwrap(first, TaskCancelledException.class) != null) { + return e; + } + if (ExceptionsHelper.unwrapCause(first) != ExceptionsHelper.unwrapCause(e)) { + first.addSuppressed(e); + } + return first; + }); + onSinkComplete(); + } + + void onSinkComplete() { + if (finished == false) { + finished = true; + if (outstandingSinks.finishInstance()) { + buffer.finish(false); + } + } + } + } + + /** + * Add a remote sink as a new data source of this handler. The handler will start fetching data from this remote sink intermediately. + * + * @param remoteSink the remote sink + * @param instances the number of concurrent ``clients`` that this handler should use to fetch pages. More clients reduce latency, + * but add overhead. + * @see ExchangeSinkHandler#fetchPageAsync(boolean, ActionListener) + */ + public void addRemoteSink(RemoteSink remoteSink, int instances) { + for (int i = 0; i < instances; i++) { + var fetcher = new RemoteSinkFetcher(remoteSink); + fetchExecutor.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + fetcher.onSinkFailed(e); + } + + @Override + protected void doRun() { + fetcher.fetchPage(); + } + }); + } + } + + @Override + protected void closeInternal() { + Exception error = failure.get(); + if (error != null) { + completionFuture.onFailure(error); + } else { + completionFuture.onResponse(null); + } + } + + /** + * Add a listener, which will be notified when this exchange source handler is completed. An exchange source + * handler is consider completed when all exchange sources and sinks are completed and de-attached. + */ + public void addCompletionListener(ActionListener listener) { + completionFuture.addListener(listener); + } + + private final class PendingInstances { + private final AtomicInteger instances = new AtomicInteger(); + + void trackNewInstance() { + incRef(); + instances.incrementAndGet(); + } + + boolean finishInstance() { + decRef(); + return instances.decrementAndGet() == 0; + } + } + +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperator.java new file mode 100644 index 0000000000000..e9726e91c73bf --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperator.java @@ -0,0 +1,161 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.Strings; +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.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; +import java.util.function.Supplier; + +/** + * Source operator implementation that retrieves data from an {@link ExchangeSource} + */ +public class ExchangeSourceOperator extends SourceOperator { + + private final ExchangeSource source; + private ListenableActionFuture isBlocked = NOT_BLOCKED; + private int pagesEmitted; + + public record ExchangeSourceOperatorFactory(Supplier exchangeSources) implements SourceOperatorFactory { + + @Override + public SourceOperator get(DriverContext driverContext) { + return new ExchangeSourceOperator(exchangeSources.get()); + } + + @Override + public String describe() { + return "ExchangeSourceOperator[]"; + } + } + + public ExchangeSourceOperator(ExchangeSource source) { + this.source = source; + } + + @Override + public Page getOutput() { + final var page = source.pollPage(); + if (page != null) { + pagesEmitted++; + } + return page; + } + + @Override + public boolean isFinished() { + return source.isFinished(); + } + + @Override + public void finish() { + source.finish(); + } + + @Override + public ListenableActionFuture isBlocked() { + if (isBlocked.isDone()) { + isBlocked = source.waitForReading(); + if (isBlocked.isDone()) { + isBlocked = NOT_BLOCKED; + } + } + return isBlocked; + } + + @Override + public void close() { + finish(); + } + + @Override + public String toString() { + return "ExchangeSourceOperator"; + } + + @Override + public Status status() { + return new Status(source.bufferSize(), pagesEmitted); + } + + public static class Status implements Operator.Status { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Operator.Status.class, + "exchange_source", + Status::new + ); + + private final int pagesWaiting; + private final int pagesEmitted; + + Status(int pagesWaiting, int pagesEmitted) { + this.pagesWaiting = pagesWaiting; + this.pagesEmitted = pagesEmitted; + } + + Status(StreamInput in) throws IOException { + pagesWaiting = in.readVInt(); + pagesEmitted = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(pagesWaiting); + out.writeVInt(pagesEmitted); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + public int pagesWaiting() { + return pagesWaiting; + } + + public int pagesEmitted() { + return pagesEmitted; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("pages_waiting", pagesWaiting); + builder.field("pages_emitted", pagesEmitted); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Status status = (Status) o; + return pagesWaiting == status.pagesWaiting && pagesEmitted == status.pagesEmitted; + } + + @Override + public int hashCode() { + return Objects.hash(pagesWaiting, pagesEmitted); + } + + @Override + public String toString() { + return Strings.toString(this); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java new file mode 100644 index 0000000000000..7d81cd3f66600 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java @@ -0,0 +1,15 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.action.ActionListener; + +public interface RemoteSink { + + void fetchPageAsync(boolean allSourcesFinished, ActionListener listener); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/package-info.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/package-info.java new file mode 100644 index 0000000000000..fbba447d563f2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Exchanges provide the ability to split an execution into multiple pipelines. + * Pipelines can be executed by different threads on the same or different nodes, allowing parallel and distributed of processing of data. + */ +package org.elasticsearch.compute.operator.exchange; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/package-info.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/package-info.java new file mode 100644 index 0000000000000..75d918e4302cd --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/package-info.java @@ -0,0 +1,32 @@ +/* + * 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. + */ + +/** + *

This package exposes the core compute engine functionality.

+ * + * The {@link org.elasticsearch.compute.data.Page} class is the batched columnar representation of data + * that's passed around in the compute engine. Pages are immutable and thread-safe. + * The {@link org.elasticsearch.compute.operator.Operator} interface is the low-level building block that consumes, + * transforms and produces data in the compute engine. + * Each {@link org.elasticsearch.compute.operator.Driver} operates in single-threaded fashion on a simple chain of + * operators, passing pages from one operator to the next. + * + * Parallelization and distribution is achieved via data exchanges. An exchange connects sink and source operators from different drivers + * (see {@link org.elasticsearch.compute.operator.exchange.ExchangeSinkOperator} and + * {@link org.elasticsearch.compute.operator.exchange.ExchangeSourceOperator}). + * Exchanges can be thought of as simple FIFO queues of pages + * (see {@link org.elasticsearch.compute.operator.exchange.ExchangeSource}). + * Their classes are generally thread-safe due to concurrent access. + * Exchanges can be remote as well as local (only local implemented so far). + * They allow multi-plexing via an exchange, broadcasting one + * sink to multiple sources (e.g. partitioning the incoming data to multiple targets based on the value of a given field), or connecting + * multiple sinks to a single source (merging subcomputations). Even if no multiplexing is happening, exchanges allow pipeline processing + * (i.e. you can have two pipelines of operators that are connected via an exchange, allowing two drivers to work in parallel on each side + * of the exchange, even on the same node). Each driver does not require a new thread, however, so you could still schedule the two drivers + * to run with the same thread when resources are scarce. + */ +package org.elasticsearch.compute; diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java new file mode 100644 index 0000000000000..114576b7bed7e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java @@ -0,0 +1,557 @@ +/* + * 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.compute; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.store.BaseDirectoryWrapper; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.MockPageCacheRecycler; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.compute.aggregation.CountAggregatorFunction; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.LuceneOperator; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator; +import org.elasticsearch.compute.lucene.ValueSourceInfo; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.AbstractPageMappingOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.LimitOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OperatorTestCase; +import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.TopNOperator; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.plain.SortedNumericIndexFieldData; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ql.util.Holder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.LongUnaryOperator; + +import static org.elasticsearch.compute.aggregation.AggregatorMode.FINAL; +import static org.elasticsearch.compute.aggregation.AggregatorMode.INITIAL; +import static org.elasticsearch.compute.operator.OperatorTestCase.randomPageSize; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; + +public class OperatorTests extends ESTestCase { + + public void testLuceneOperatorsLimit() throws IOException { + final int numDocs = randomIntBetween(10_000, 100_000); + try (Directory dir = newDirectory(); RandomIndexWriter w = writeTestDocs(dir, numDocs, "value", null)) { + try (IndexReader reader = w.getReader()) { + AtomicInteger rowCount = new AtomicInteger(); + final int limit = randomIntBetween(1, numDocs * 2); + DriverContext driverContext = new DriverContext(); + try ( + Driver driver = new Driver( + driverContext, + new LuceneSourceOperator(reader, 0, new MatchAllDocsQuery(), randomPageSize(), limit), + Collections.emptyList(), + new PageConsumerOperator(page -> rowCount.addAndGet(page.getPositionCount())), + () -> {} + ) + ) { + OperatorTestCase.runDriver(driver); + } + assertEquals(Math.min(limit, numDocs), rowCount.get()); + assertDriverContext(driverContext); + } + } + } + + public void testLuceneTopNSourceOperator() throws IOException { + final int numDocs = randomIntBetween(10_000, 100_000); + final int pageSize = randomIntBetween(1_000, 100_000); + final int limit = randomIntBetween(1, pageSize); + String fieldName = "value"; + + try (Directory dir = newDirectory(); RandomIndexWriter w = writeTestDocs(dir, numDocs, fieldName, null)) { + ValuesSource vs = new ValuesSource.Numeric.FieldData( + new SortedNumericIndexFieldData( + fieldName, + IndexNumericFieldData.NumericType.LONG, + IndexNumericFieldData.NumericType.LONG.getValuesSourceType(), + null + ) + ); + try (IndexReader reader = w.getReader()) { + AtomicInteger rowCount = new AtomicInteger(); + Sort sort = new Sort(new SortField(fieldName, SortField.Type.LONG)); + Holder expectedValue = new Holder<>(0L); + DriverContext driverContext = new DriverContext(); + try ( + Driver driver = new Driver( + driverContext, + new LuceneTopNSourceOperator(reader, 0, sort, new MatchAllDocsQuery(), pageSize, limit), + List.of( + new ValuesSourceReaderOperator( + List.of(new ValueSourceInfo(CoreValuesSourceType.NUMERIC, vs, ElementType.LONG, reader)), + 0, + fieldName + ), + new TopNOperator(limit, List.of(new TopNOperator.SortOrder(1, true, true)), randomPageSize()) + ), + new PageConsumerOperator(page -> { + rowCount.addAndGet(page.getPositionCount()); + for (int i = 0; i < page.getPositionCount(); i++) { + LongBlock longValuesBlock = page.getBlock(1); + long expected = expectedValue.get(); + assertEquals(expected, longValuesBlock.getLong(i)); + expectedValue.set(expected + 1); + } + }), + () -> {} + ) + ) { + OperatorTestCase.runDriver(driver); + } + assertEquals(Math.min(limit, numDocs), rowCount.get()); + assertDriverContext(driverContext); + } + } + } + + public void testOperatorsWithLuceneSlicing() throws IOException { + final String fieldName = "value"; + final int numDocs = 100000; + try (Directory dir = newDirectory(); RandomIndexWriter w = writeTestDocs(dir, numDocs, fieldName, randomIntBetween(1, 10))) { + ValuesSource vs = new ValuesSource.Numeric.FieldData( + new SortedNumericIndexFieldData( + fieldName, + IndexNumericFieldData.NumericType.LONG, + IndexNumericFieldData.NumericType.LONG.getValuesSourceType(), + null + ) + ); + + try (IndexReader reader = w.getReader()) { + AtomicInteger rowCount = new AtomicInteger(); + + List drivers = new ArrayList<>(); + LuceneSourceOperator luceneOperator = new LuceneSourceOperator( + reader, + 0, + new MatchAllDocsQuery(), + randomPageSize(), + LuceneOperator.NO_LIMIT + ); + try { + for (LuceneOperator luceneSourceOperator : luceneOperator.docSlice(randomIntBetween(1, 10))) { + drivers.add( + new Driver( + new DriverContext(), + luceneSourceOperator, + List.of( + new ValuesSourceReaderOperator( + List.of(new ValueSourceInfo(CoreValuesSourceType.NUMERIC, vs, ElementType.LONG, reader)), + 0, + fieldName + ) + ), + new PageConsumerOperator(page -> rowCount.addAndGet(page.getPositionCount())), + () -> {} + ) + ); + } + OperatorTestCase.runDriver(drivers); + } finally { + Releasables.close(drivers); + } + assertEquals(numDocs, rowCount.get()); + drivers.stream().map(Driver::driverContext).forEach(OperatorTests::assertDriverContext); + } + } + } + + private static RandomIndexWriter writeTestDocs(Directory dir, int numDocs, String fieldName, Integer maxSegmentCount) + throws IOException { + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + Document doc = new Document(); + NumericDocValuesField docValuesField = new NumericDocValuesField(fieldName, 0); + for (int i = 0; i < numDocs; i++) { + doc.clear(); + docValuesField.setLongValue(i); + doc.add(docValuesField); + w.addDocument(doc); + } + if (maxSegmentCount != null && randomBoolean()) { + w.forceMerge(randomIntBetween(1, 10)); + } + w.commit(); + + return w; + } + + public void testQueryOperator() throws IOException { + Map docs = new HashMap<>(); + CheckedConsumer verifier = reader -> { + final long from = randomBoolean() ? Long.MIN_VALUE : randomLongBetween(0, 10000); + final long to = randomBoolean() ? Long.MAX_VALUE : randomLongBetween(from, from + 10000); + final Query query = LongPoint.newRangeQuery("pt", from, to); + final String partition = randomFrom("shard", "segment", "doc"); + final LuceneSourceOperator luceneOperator = new LuceneSourceOperator( + reader, + 0, + query, + randomPageSize(), + LuceneOperator.NO_LIMIT + ); + final List queryOperators = switch (partition) { + case "shard" -> List.of(luceneOperator); + case "segment" -> luceneOperator.segmentSlice(); + case "doc" -> luceneOperator.docSlice(randomIntBetween(1, 10)); + default -> throw new AssertionError("unknown partition [" + partition + "]"); + }; + List drivers = new ArrayList<>(); + try { + Set actualDocIds = Collections.newSetFromMap(ConcurrentCollections.newConcurrentMap()); + for (LuceneOperator queryOperator : queryOperators) { + PageConsumerOperator docCollector = new PageConsumerOperator(page -> { + DocVector docVector = page.getBlock(0).asVector(); + IntVector doc = docVector.docs(); + IntVector segment = docVector.segments(); + for (int i = 0; i < doc.getPositionCount(); i++) { + int docBase = reader.leaves().get(segment.getInt(i)).docBase; + int docId = docBase + doc.getInt(i); + assertTrue("duplicated docId=" + docId, actualDocIds.add(docId)); + } + }); + drivers.add(new Driver(new DriverContext(), queryOperator, List.of(), docCollector, () -> {})); + } + OperatorTestCase.runDriver(drivers); + Set expectedDocIds = searchForDocIds(reader, query); + assertThat("query=" + query + ", partition=" + partition, actualDocIds, equalTo(expectedDocIds)); + drivers.stream().map(Driver::driverContext).forEach(OperatorTests::assertDriverContext); + } finally { + Releasables.close(drivers); + } + }; + + try (Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir)) { + int numDocs = randomIntBetween(0, 10_000); + for (int i = 0; i < numDocs; i++) { + Document d = new Document(); + long point = randomLongBetween(0, 5000); + d.add(new LongPoint("pt", point)); + BytesRef id = Uid.encodeId("id-" + randomIntBetween(0, 5000)); + d.add(new Field("id", id, KeywordFieldMapper.Defaults.FIELD_TYPE)); + if (docs.put(id, point) != null) { + w.updateDocument(new Term("id", id), d); + } else { + w.addDocument(d); + } + } + try (DirectoryReader reader = w.getReader()) { + verifier.accept(reader); + } + } + } + + public void testGroupingWithOrdinals() throws Exception { + final String gField = "g"; + final int numDocs = between(100, 10000); + final Map expectedCounts = new HashMap<>(); + int keyLength = randomIntBetween(1, 10); + try (BaseDirectoryWrapper dir = newDirectory(); RandomIndexWriter writer = new RandomIndexWriter(random(), dir)) { + for (int i = 0; i < numDocs; i++) { + Document doc = new Document(); + BytesRef key = new BytesRef(randomByteArrayOfLength(keyLength)); + SortedSetDocValuesField docValuesField = new SortedSetDocValuesField(gField, key); + doc.add(docValuesField); + writer.addDocument(doc); + expectedCounts.compute(key, (k, v) -> v == null ? 1 : v + 1); + } + writer.commit(); + Map actualCounts = new HashMap<>(); + BigArrays bigArrays = bigArrays(); + boolean shuffleDocs = randomBoolean(); + Operator shuffleDocsOperator = new AbstractPageMappingOperator() { + @Override + protected Page process(Page page) { + if (shuffleDocs == false) { + return page; + } + DocVector docVector = (DocVector) page.getBlock(0).asVector(); + int positionCount = docVector.getPositionCount(); + IntVector shards = docVector.shards(); + if (randomBoolean()) { + IntVector.Builder builder = IntVector.newVectorBuilder(positionCount); + for (int i = 0; i < positionCount; i++) { + builder.appendInt(shards.getInt(i)); + } + shards = builder.build(); + } + IntVector segments = docVector.segments(); + if (randomBoolean()) { + IntVector.Builder builder = IntVector.newVectorBuilder(positionCount); + for (int i = 0; i < positionCount; i++) { + builder.appendInt(segments.getInt(i)); + } + segments = builder.build(); + } + IntVector docs = docVector.docs(); + if (randomBoolean()) { + List ids = new ArrayList<>(positionCount); + for (int i = 0; i < positionCount; i++) { + ids.add(docs.getInt(i)); + } + Collections.shuffle(ids, random()); + docs = new IntArrayVector(ids.stream().mapToInt(n -> n).toArray(), positionCount); + } + Block[] blocks = new Block[page.getBlockCount()]; + blocks[0] = new DocVector(shards, segments, docs, false).asBlock(); + for (int i = 1; i < blocks.length; i++) { + blocks[i] = page.getBlock(i); + } + return new Page(blocks); + } + + @Override + public String toString() { + return "ShuffleDocs"; + } + }; + + try (DirectoryReader reader = writer.getReader()) { + DriverContext driverContext = new DriverContext(); + Driver driver = new Driver( + driverContext, + new LuceneSourceOperator(reader, 0, new MatchAllDocsQuery(), randomPageSize(), LuceneOperator.NO_LIMIT), + List.of(shuffleDocsOperator, new AbstractPageMappingOperator() { + @Override + protected Page process(Page page) { + return page.appendBlock(IntBlock.newConstantBlockWith(1, page.getPositionCount())); + } + + @Override + public String toString() { + return "Add(1)"; + } + }, + new OrdinalsGroupingOperator( + List.of( + new ValueSourceInfo( + CoreValuesSourceType.KEYWORD, + randomBoolean() ? getOrdinalsValuesSource(gField) : getBytesValuesSource(gField), + ElementType.BYTES_REF, + reader + ) + ), + 0, + gField, + List.of(CountAggregatorFunction.supplier(bigArrays, List.of(1)).groupingAggregatorFactory(INITIAL)), + randomPageSize(), + bigArrays, + driverContext + ), + new HashAggregationOperator( + List.of(CountAggregatorFunction.supplier(bigArrays, List.of(1, 2)).groupingAggregatorFactory(FINAL)), + () -> BlockHash.build( + List.of(new HashAggregationOperator.GroupSpec(0, ElementType.BYTES_REF)), + bigArrays, + randomPageSize() + ), + driverContext + ) + ), + new PageConsumerOperator(page -> { + BytesRefBlock keys = page.getBlock(0); + LongBlock counts = page.getBlock(1); + for (int i = 0; i < keys.getPositionCount(); i++) { + BytesRef spare = new BytesRef(); + actualCounts.put(keys.getBytesRef(i, spare), counts.getLong(i)); + } + }), + () -> {} + ); + OperatorTestCase.runDriver(driver); + assertThat(actualCounts, equalTo(expectedCounts)); + assertDriverContext(driverContext); + } + } + } + + public void testLimitOperator() { + var positions = 100; + var limit = randomIntBetween(90, 101); + var values = randomList(positions, positions, ESTestCase::randomLong); + + var results = new ArrayList(); + DriverContext driverContext = new DriverContext(); + try ( + var driver = new Driver( + driverContext, + new SequenceLongBlockSourceOperator(values, 100), + List.of((new LimitOperator.Factory(limit)).get(driverContext)), + new PageConsumerOperator(page -> { + LongBlock block = page.getBlock(0); + for (int i = 0; i < page.getPositionCount(); i++) { + results.add(block.getLong(i)); + } + }), + () -> {} + ) + ) { + OperatorTestCase.runDriver(driver); + } + + assertThat(results, contains(values.stream().limit(limit).toArray())); + assertDriverContext(driverContext); + } + + private static Set searchForDocIds(IndexReader reader, Query query) throws IOException { + IndexSearcher searcher = new IndexSearcher(reader); + Set docIds = new HashSet<>(); + searcher.search(query, new Collector() { + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) { + return new LeafCollector() { + @Override + public void setScorer(Scorable scorer) { + + } + + @Override + public void collect(int doc) { + int docId = context.docBase + doc; + assertTrue(docIds.add(docId)); + } + }; + } + + @Override + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; + } + }); + return docIds; + } + + static ValuesSource.Bytes.WithOrdinals getOrdinalsValuesSource(String field) { + return new ValuesSource.Bytes.WithOrdinals() { + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + return getBytesValuesSource(field).bytesValues(context); + } + + @Override + public SortedSetDocValues ordinalsValues(LeafReaderContext context) throws IOException { + return context.reader().getSortedSetDocValues(field); + } + + @Override + public SortedSetDocValues globalOrdinalsValues(LeafReaderContext context) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean supportsGlobalOrdinalsMapping() { + return false; + } + + @Override + public LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context) { + throw new UnsupportedOperationException(); + } + }; + } + + static ValuesSource.Bytes getBytesValuesSource(String field) { + return new ValuesSource.Bytes() { + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { + final SortedSetDocValues dv = context.reader().getSortedSetDocValues(field); + return new SortedBinaryDocValues() { + @Override + public boolean advanceExact(int doc) throws IOException { + return dv.advanceExact(doc); + } + + @Override + public int docValueCount() { + return dv.docValueCount(); + } + + @Override + public BytesRef nextValue() throws IOException { + return dv.lookupOrd(dv.nextOrd()); + } + }; + } + }; + } + + /** + * Creates a {@link BigArrays} that tracks releases but doesn't throw circuit breaking exceptions. + */ + private BigArrays bigArrays() { + return new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + } + + public static void assertDriverContext(DriverContext driverContext) { + assertTrue(driverContext.isFinished()); + assertThat(driverContext.getSnapshot().releasables(), empty()); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/AggregatorFunctionTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/AggregatorFunctionTestCase.java new file mode 100644 index 0000000000000..e2f1c606a4c25 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/AggregatorFunctionTestCase.java @@ -0,0 +1,209 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AggregationOperator; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.ForkingOperatorTestCase; +import org.elasticsearch.compute.operator.NullInsertingSourceOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.PositionMergingSourceOperator; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.DoubleStream; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +import static java.util.stream.IntStream.range; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public abstract class AggregatorFunctionTestCase extends ForkingOperatorTestCase { + protected abstract AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels); + + protected final int aggregatorIntermediateBlockCount() { + try (var agg = aggregatorFunction(nonBreakingBigArrays(), List.of()).aggregator()) { + return agg.intermediateBlockCount(); + } + } + + protected abstract String expectedDescriptionOfAggregator(); + + protected abstract void assertSimpleOutput(List input, Block result); + + @Override + protected Operator.OperatorFactory simpleWithMode(BigArrays bigArrays, AggregatorMode mode) { + List channels = mode.isInputPartial() ? range(0, aggregatorIntermediateBlockCount()).boxed().toList() : List.of(0); + return new AggregationOperator.AggregationOperatorFactory( + List.of(aggregatorFunction(bigArrays, channels).aggregatorFactory(mode)), + mode + ); + } + + @Override + protected final String expectedDescriptionOfSimple() { + return "AggregationOperator[mode = SINGLE, aggs = " + expectedDescriptionOfAggregator() + "]"; + } + + @Override + protected final String expectedToStringOfSimple() { + String type = getClass().getSimpleName().replace("Tests", ""); + return "AggregationOperator[aggregators=[Aggregator[aggregatorFunction=" + type + "[channels=[0]], mode=SINGLE]]]"; + } + + @Override + protected final void assertSimpleOutput(List input, List results) { + assertThat(results, hasSize(1)); + assertThat(results.get(0).getBlockCount(), equalTo(1)); + assertThat(results.get(0).getPositionCount(), equalTo(1)); + + Block result = results.get(0).getBlock(0); + assertSimpleOutput(input.stream().map(p -> p.getBlock(0)).toList(), result); + } + + @Override + protected final ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big array so never breaks", false); + return null; + } + + public final void testIgnoresNulls() { + int end = between(1_000, 100_000); + List results = new ArrayList<>(); + List input = CannedSourceOperator.collectPages(simpleInput(end)); + DriverContext driverContext = new DriverContext(); + + try ( + Driver d = new Driver( + driverContext, + new NullInsertingSourceOperator(new CannedSourceOperator(input.iterator())), + List.of(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertSimpleOutput(input, results); + } + + public final void testMultivalued() { + int end = between(1_000, 100_000); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages(new PositionMergingSourceOperator(simpleInput(end))); + assertSimpleOutput(input, drive(simple(BigArrays.NON_RECYCLING_INSTANCE).get(driverContext), input.iterator())); + } + + public final void testMultivaluedWithNulls() { + int end = between(1_000, 100_000); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages( + new NullInsertingSourceOperator(new PositionMergingSourceOperator(simpleInput(end))) + ); + assertSimpleOutput(input, drive(simple(BigArrays.NON_RECYCLING_INSTANCE).get(driverContext), input.iterator())); + } + + public final void testEmptyInput() { + DriverContext driverContext = new DriverContext(); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), List.of().iterator()); + + assertThat(results, hasSize(1)); + assertOutputFromEmpty(results.get(0).getBlock(0)); + } + + public final void testEmptyInputInitialFinal() { + DriverContext driverContext = new DriverContext(); + List results = drive( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ), + List.of().iterator() + ); + + assertThat(results, hasSize(1)); + assertOutputFromEmpty(results.get(0).getBlock(0)); + } + + public final void testEmptyInputInitialIntermediateFinal() { + DriverContext driverContext = new DriverContext(); + List results = drive( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INTERMEDIATE).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ), + List.of().iterator() + ); + + assertThat(results, hasSize(1)); + assertOutputFromEmpty(results.get(0).getBlock(0)); + } + + /** + * Asserts that the output from an empty input is a {@link Block} containing + * only {@code null}. Override for {@code count} style aggregations that + * return other sorts of results. + */ + protected void assertOutputFromEmpty(Block b) { + assertThat(b.elementType(), equalTo(ElementType.NULL)); + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(b.areAllValuesNull(), equalTo(true)); + assertThat(b.isNull(0), equalTo(true)); + assertThat(b.getValueCount(0), equalTo(0)); + } + + protected static IntStream allValueOffsets(Block input) { + return IntStream.range(0, input.getPositionCount()).flatMap(p -> { + int start = input.getFirstValueIndex(p); + int end = start + input.getValueCount(p); + return IntStream.range(start, end); + }); + } + + protected static Stream allBytesRefs(Block input) { + BytesRefBlock b = (BytesRefBlock) input; + return allValueOffsets(b).mapToObj(i -> b.getBytesRef(i, new BytesRef())); + } + + protected static Stream allBooleans(Block input) { + BooleanBlock b = (BooleanBlock) input; + return allValueOffsets(b).mapToObj(i -> b.getBoolean(i)); + } + + protected static DoubleStream allDoubles(Block input) { + DoubleBlock b = (DoubleBlock) input; + return allValueOffsets(b).mapToDouble(i -> b.getDouble(i)); + } + + protected static IntStream allInts(Block input) { + IntBlock b = (IntBlock) input; + return allValueOffsets(b).map(i -> b.getInt(i)); + } + + protected static LongStream allLongs(Block input) { + LongBlock b = (LongBlock) input; + return allValueOffsets(b).mapToLong(i -> b.getLong(i)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ArrayStateTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ArrayStateTests.java new file mode 100644 index 0000000000000..1d8df3caf7b76 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ArrayStateTests.java @@ -0,0 +1,192 @@ +/* + * 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.compute.aggregation; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BlockTestUtils; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class ArrayStateTests extends ESTestCase { + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + + for (boolean inOrder : new boolean[] { true, false }) { + params.add(new Object[] { ElementType.INT, 1000, inOrder }); + params.add(new Object[] { ElementType.LONG, 1000, inOrder }); + params.add(new Object[] { ElementType.DOUBLE, 1000, inOrder }); + } + return params; + } + + private final ElementType elementType; + private final int valueCount; + private final boolean inOrder; + + public ArrayStateTests(ElementType elementType, int valueCount, boolean inOrder) { + this.elementType = elementType; + this.valueCount = valueCount; + this.inOrder = inOrder; + } + + public void testSetNoTracking() { + List values = randomList(valueCount, valueCount, this::randomValue); + + AbstractArrayState state = newState(); + setAll(state, values, 0); + for (int i = 0; i < values.size(); i++) { + assertTrue(state.hasValue(i)); + assertThat(get(state, i), equalTo(values.get(i))); + } + } + + public void testSetWithoutTrackingThenSetWithTracking() { + List values = randomList(valueCount, valueCount, this::nullableRandomValue); + + AbstractArrayState state = newState(); + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + setAll(state, values, 0); + for (int i = 0; i < values.size(); i++) { + if (values.get(i) == null) { + assertFalse(state.hasValue(i)); + } else { + assertTrue(state.hasValue(i)); + assertThat(get(state, i), equalTo(values.get(i))); + } + } + } + + public void testSetWithTracking() { + List withoutNulls = randomList(valueCount, valueCount, this::randomValue); + List withNulls = randomList(valueCount, valueCount, this::nullableRandomValue); + + AbstractArrayState state = newState(); + setAll(state, withoutNulls, 0); + state.enableGroupIdTracking(new SeenGroupIds.Range(0, withoutNulls.size())); + setAll(state, withNulls, withoutNulls.size()); + + for (int i = 0; i < withoutNulls.size(); i++) { + assertTrue(state.hasValue(i)); + assertThat(get(state, i), equalTo(withoutNulls.get(i))); + } + for (int i = 0; i < withNulls.size(); i++) { + if (withNulls.get(i) == null) { + assertFalse(state.hasValue(i + withoutNulls.size())); + } else { + assertTrue(state.hasValue(i + withoutNulls.size())); + assertThat(get(state, i + withoutNulls.size()), equalTo(withNulls.get(i))); + } + } + } + + public void testSetNotNullableThenOverwriteNullable() { + List first = randomList(valueCount, valueCount, this::randomValue); + List second = randomList(valueCount, valueCount, this::nullableRandomValue); + + AbstractArrayState state = newState(); + setAll(state, first, 0); + state.enableGroupIdTracking(new SeenGroupIds.Range(0, valueCount)); + setAll(state, second, 0); + + for (int i = 0; i < valueCount; i++) { + assertTrue(state.hasValue(i)); + Object expected = second.get(i); + expected = expected == null ? first.get(i) : expected; + assertThat(get(state, i), equalTo(expected)); + } + } + + public void testSetNullableThenOverwriteNullable() { + List first = randomList(valueCount, valueCount, this::nullableRandomValue); + List second = randomList(valueCount, valueCount, this::nullableRandomValue); + + AbstractArrayState state = newState(); + state.enableGroupIdTracking(new SeenGroupIds.Empty()); + setAll(state, first, 0); + setAll(state, second, 0); + + for (int i = 0; i < valueCount; i++) { + Object expected = second.get(i); + expected = expected == null ? first.get(i) : expected; + if (expected == null) { + assertFalse(state.hasValue(i)); + } else { + assertTrue(state.hasValue(i)); + assertThat(get(state, i), equalTo(expected)); + } + } + } + + private record ValueAndIndex(int index, Object value) {} + + private void setAll(AbstractArrayState state, List values, int offset) { + if (inOrder) { + for (int i = 0; i < values.size(); i++) { + if (values.get(i) != null) { + set(state, i + offset, values.get(i)); + } + } + return; + } + List shuffled = new ArrayList<>(values.size()); + for (int i = 0; i < values.size(); i++) { + shuffled.add(new ValueAndIndex(i, values.get(i))); + } + Randomness.shuffle(shuffled); + for (ValueAndIndex v : shuffled) { + if (v.value != null) { + set(state, v.index + offset, v.value); + } + } + } + + private AbstractArrayState newState() { + return switch (elementType) { + case INT -> new IntArrayState(BigArrays.NON_RECYCLING_INSTANCE, 1); + case LONG -> new LongArrayState(BigArrays.NON_RECYCLING_INSTANCE, 1); + case DOUBLE -> new DoubleArrayState(BigArrays.NON_RECYCLING_INSTANCE, 1); + default -> throw new IllegalArgumentException(); + }; + } + + private void set(AbstractArrayState state, int groupdId, Object value) { + switch (elementType) { + case INT -> ((IntArrayState) state).set(groupdId, (Integer) value); + case LONG -> ((LongArrayState) state).set(groupdId, (Long) value); + case DOUBLE -> ((DoubleArrayState) state).set(groupdId, (Double) value); + default -> throw new IllegalArgumentException(); + } + } + + private Object get(AbstractArrayState state, int index) { + return switch (elementType) { + case INT -> ((IntArrayState) state).get(index); + case LONG -> ((LongArrayState) state).get(index); + case DOUBLE -> ((DoubleArrayState) state).get(index); + default -> throw new IllegalArgumentException(); + }; + } + + private Object randomValue() { + return BlockTestUtils.randomValue(elementType); + } + + private Object nullableRandomValue() { + return randomBoolean() ? null : randomValue(); + } + +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountAggregatorFunctionTests.java new file mode 100644 index 0000000000000..11241020a6709 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountAggregatorFunctionTests.java @@ -0,0 +1,49 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class CountAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLong())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return CountAggregatorFunction.supplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long count = input.stream().flatMapToLong(b -> allLongs(b)).count(); + assertThat(((LongBlock) result).getLong(0), equalTo(count)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionTests.java new file mode 100644 index 0000000000000..74cd88feed3f4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanAggregatorFunctionTests.java @@ -0,0 +1,50 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.SequenceBooleanBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctBooleanAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceBooleanBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> randomBoolean()).toList()); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctBooleanAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of booleans"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long expected = input.stream().flatMap(b -> allBooleans(b)).distinct().count(); + long count = ((LongBlock) result).getLong(0); + assertThat(count, equalTo(expected)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..eab1b9cb2d8de --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBooleanGroupingAggregatorFunctionTests.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongBooleanTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctBooleanGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctBooleanAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of booleans"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongBooleanTupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), randomBoolean())) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long distinct = input.stream().flatMap(p -> allBooleans(p, group)).distinct().count(); + long count = ((LongBlock) result).getLong(position); + assertThat(count, equalTo(distinct)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionTests.java new file mode 100644 index 0000000000000..69ccc0a04c0f9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefAggregatorFunctionTests.java @@ -0,0 +1,58 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.BytesRefBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctBytesRefAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, Math.min(Integer.MAX_VALUE, Integer.MAX_VALUE / size)); + return new BytesRefBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> new BytesRef(String.valueOf(between(-max, max)))).toList() + ); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctBytesRefAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of bytes"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long expected = input.stream().flatMap(b -> allBytesRefs(b)).distinct().count(); + long count = ((LongBlock) result).getLong(0); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(expected, expected * 0.1)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..919d06af430fd --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctBytesRefGroupingAggregatorFunctionTests.java @@ -0,0 +1,60 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongBytesRefTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctBytesRefGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctBytesRefAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of bytes"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongBytesRefTupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), new BytesRef(String.valueOf(between(1, 10000))))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long distinct = input.stream().flatMap(p -> allBytesRefs(p, group)).distinct().count(); + long count = ((LongBlock) result).getLong(position); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(distinct, distinct * 0.1)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..c0678441cdc74 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleAggregatorFunctionTests.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.test.ESTestCase; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceDoubleBlockSourceOperator(LongStream.range(0, size).mapToDouble(l -> ESTestCase.randomDouble())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of doubles"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long expected = input.stream().flatMapToDouble(b -> allDoubles(b)).distinct().count(); + + long count = ((LongBlock) result).getLong(0); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(expected, expected * .1)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..5a928f12d33b7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,59 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of doubles"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), randomDoubleBetween(0, 100, true))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long distinct = input.stream().flatMapToDouble(p -> allDoubles(p, group)).distinct().count(); + long count = ((LongBlock) result).getLong(position); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(distinct, distinct * 0.1)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..e559dc4effccb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntAggregatorFunctionTests.java @@ -0,0 +1,78 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, Math.min(Integer.MAX_VALUE, Integer.MAX_VALUE / size)); + return new SequenceIntBlockSourceOperator(LongStream.range(0, size).mapToInt(l -> between(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctIntAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of ints"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long expected = input.stream().flatMapToInt(b -> allInts(b)).distinct().count(); + + long count = ((LongBlock) result).getLong(0); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(expected, expected * 0.1)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } + + public void testRejectsDouble() { + DriverContext driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(Iterators.single(new Page(new DoubleArrayVector(new double[] { 1.0 }, 1).asBlock()))), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> fail("shouldn't have made it this far")), + () -> {} + ) + ) { + expectThrows(Exception.class, () -> runDriver(d)); // ### find a more specific exception type + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..f2a46e9f4c3af --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,57 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctIntAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongIntBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), between(0, 10000)))); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long distinct = input.stream().flatMapToInt(p -> allInts(p, group)).distinct().count(); + long count = ((LongBlock) result).getLong(position); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(distinct, distinct * 0.1)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..57b90fb844f54 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongAggregatorFunctionTests.java @@ -0,0 +1,79 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctLongAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of longs"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long expected = input.stream().flatMapToLong(b -> allLongs(b)).distinct().count(); + long count = ((LongBlock) result).getLong(0); + + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(expected, expected * 0.1)); + } + + @Override + protected void assertOutputFromEmpty(Block b) { + assertThat(b.getPositionCount(), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(b, 0, 1), equalTo(List.of(List.of(0L)))); + } + + public void testRejectsDouble() { + DriverContext driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(Iterators.single(new Page(new DoubleArrayVector(new double[] { 1.0 }, 1).asBlock()))), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> fail("shouldn't have made it this far")), + () -> {} + ) + ) { + expectThrows(Exception.class, () -> runDriver(d)); // ### find a more specific exception type + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..a5959471b8e15 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,58 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class CountDistinctLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new CountDistinctLongAggregatorFunctionSupplier(bigArrays, inputChannels, 40000); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count_distinct of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new TupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), randomLongBetween(0, 100_000))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long expected = input.stream().flatMapToLong(p -> allLongs(p, group)).distinct().count(); + long count = ((LongBlock) result).getLong(position); + // HLL is an approximation algorithm and precision depends on the number of values computed and the precision_threshold param + // https://www.elastic.co/guide/en/elasticsearch/reference/current/search-aggregations-metrics-cardinality-aggregation.html + // For a number of values close to 10k and precision_threshold=1000, precision should be less than 10% + assertThat((double) count, closeTo(expected, expected * 0.1)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..54a35fcc19cb2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java @@ -0,0 +1,59 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class CountGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return CountAggregatorFunction.supplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "count"; + } + + @Override + protected SourceOperator simpleInput(int size) { + if (randomBoolean()) { + return new TupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong())) + ); + } + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomDouble())) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long count = input.stream().flatMapToInt(p -> allValueOffsets(p, group)).count(); + assertThat(((LongBlock) result).getLong(position), equalTo(count)); + } + + @Override + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(false)); + assertThat(b.getValueCount(position), equalTo(1)); + assertThat(((LongBlock) b).getLong(b.getFirstValueIndex(position)), equalTo(0L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java new file mode 100644 index 0000000000000..ac3edc4c61a88 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java @@ -0,0 +1,571 @@ +/* + * 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.compute.aggregation; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.ForkingOperatorTestCase; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.NullInsertingSourceOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.PositionMergingSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Releasables; + +import java.util.ArrayList; +import java.util.List; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.DoubleStream; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +import static java.util.stream.IntStream.range; +import static org.elasticsearch.compute.data.BlockTestUtils.append; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public abstract class GroupingAggregatorFunctionTestCase extends ForkingOperatorTestCase { + protected abstract AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels); + + protected final int aggregatorIntermediateBlockCount() { + try (var agg = aggregatorFunction(nonBreakingBigArrays(), List.of()).aggregator()) { + return agg.intermediateBlockCount(); + } + } + + protected abstract String expectedDescriptionOfAggregator(); + + protected abstract void assertSimpleGroup(List input, Block result, int position, Long group); + + @Override + protected final Operator.OperatorFactory simpleWithMode(BigArrays bigArrays, AggregatorMode mode) { + List channels = mode.isInputPartial() ? range(1, 1 + aggregatorIntermediateBlockCount()).boxed().toList() : List.of(1); + int emitChunkSize = between(100, 200); + + AggregatorFunctionSupplier supplier = aggregatorFunction(bigArrays, channels); + if (randomBoolean()) { + supplier = chunkGroups(emitChunkSize, supplier); + } + return new HashAggregationOperator.HashAggregationOperatorFactory( + List.of(new HashAggregationOperator.GroupSpec(0, ElementType.LONG)), + List.of(supplier.groupingAggregatorFactory(mode)), + randomPageSize(), + bigArrays + ); + } + + @Override + protected final String expectedDescriptionOfSimple() { + return "HashAggregationOperator[mode = , aggs = " + expectedDescriptionOfAggregator() + "]"; + } + + @Override + protected final String expectedToStringOfSimple() { + String hash = "blockHash=LongBlockHash{channel=0, entries=0, seenNull=false}"; + String type = getClass().getSimpleName().replace("Tests", ""); + return "HashAggregationOperator[" + + hash + + ", aggregators=[GroupingAggregator[aggregatorFunction=" + + type + + "[channels=[1]], mode=SINGLE]]]"; + } + + private SeenGroups seenGroups(List input) { + boolean seenNullGroup = false; + SortedSet seenGroups = new TreeSet<>(); + for (Page in : input) { + LongBlock groups = in.getBlock(0); + for (int p = 0; p < in.getPositionCount(); p++) { + if (groups.isNull(p)) { + seenNullGroup = true; + continue; + } + int start = groups.getFirstValueIndex(p); + int end = start + groups.getValueCount(p); + for (int g = start; g < end; g++) { + seenGroups.add(groups.getLong(g)); + } + } + } + return new SeenGroups(seenGroups, seenNullGroup); + } + + private record SeenGroups(SortedSet nonNull, boolean seenNull) { + int size() { + return nonNull.size() + (seenNull ? 1 : 0); + } + } + + protected long randomGroupId(int pageSize) { + int maxGroupId = pageSize < 10 && randomBoolean() ? 4 : 100; + return randomIntBetween(0, maxGroupId); + } + + @Override + protected final void assertSimpleOutput(List input, List results) { + SeenGroups seenGroups = seenGroups(input); + + assertThat(results, hasSize(1)); + assertThat(results.get(0).getBlockCount(), equalTo(2)); + assertThat(results.get(0).getPositionCount(), equalTo(seenGroups.size())); + + LongBlock groups = results.get(0).getBlock(0); + Block result = results.get(0).getBlock(1); + for (int i = 0; i < seenGroups.size(); i++) { + Long group = groups.isNull(i) ? null : groups.getLong(i); + assertSimpleGroup(input, result, i, group); + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + return ByteSizeValue.ofBytes(between(1, 32)); + } + + public final void testNullGroupsAndValues() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(new NullInsertingSourceOperator(simpleInput(end))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testNullGroups() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(nullGroups(simpleInput(end))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + private SourceOperator nullGroups(SourceOperator source) { + return new NullInsertingSourceOperator(source) { + @Override + protected void appendNull(ElementType elementType, Block.Builder builder, int blockId) { + if (blockId == 0) { + super.appendNull(elementType, builder, blockId); + } else { + // Append a small random value to make sure we don't overflow on things like sums + append(builder, switch (elementType) { + case BOOLEAN -> randomBoolean(); + case BYTES_REF -> new BytesRef(randomAlphaOfLength(3)); + case DOUBLE -> randomDouble(); + case INT -> 1; + case LONG -> 1L; + default -> throw new UnsupportedOperationException(); + }); + } + } + }; + } + + public final void testNullValues() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(nullValues(simpleInput(end))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testNullValuesInitialIntermediateFinal() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(nullValues(simpleInput(end))); + List results = drive( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INTERMEDIATE).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ), + input.iterator() + ); + assertSimpleOutput(input, results); + } + + private SourceOperator nullValues(SourceOperator source) { + return new NullInsertingSourceOperator(source) { + @Override + protected void appendNull(ElementType elementType, Block.Builder builder, int blockId) { + if (blockId == 0) { + ((LongBlock.Builder) builder).appendLong(between(0, 4)); + } else { + super.appendNull(elementType, builder, blockId); + } + } + }; + } + + public final void testMultivalued() { + DriverContext driverContext = new DriverContext(); + int end = between(1_000, 100_000); + List input = CannedSourceOperator.collectPages(mergeValues(simpleInput(end))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testMulitvaluedNullGroupsAndValues() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(new NullInsertingSourceOperator(mergeValues(simpleInput(end)))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testMulitvaluedNullGroup() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(nullGroups(mergeValues(simpleInput(end)))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testMulitvaluedNullValues() { + DriverContext driverContext = new DriverContext(); + int end = between(50, 60); + List input = CannedSourceOperator.collectPages(nullValues(mergeValues(simpleInput(end)))); + List results = drive(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext), input.iterator()); + assertSimpleOutput(input, results); + } + + public final void testNullOnly() { + DriverContext driverContext = new DriverContext(); + assertNullOnly(List.of(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext))); + } + + public final void testNullOnlyInputInitialFinal() { + DriverContext driverContext = new DriverContext(); + assertNullOnly( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ) + ); + } + + public final void testNullOnlyInputInitialIntermediateFinal() { + DriverContext driverContext = new DriverContext(); + assertNullOnly( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INTERMEDIATE).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ) + ); + } + + /** + * Run the aggregation passing only null values. + */ + private void assertNullOnly(List operators) { + LongBlock.Builder groupBuilder = LongBlock.newBlockBuilder(1); + if (randomBoolean()) { + groupBuilder.appendLong(1); + } else { + groupBuilder.appendNull(); + } + List source = List.of(new Page(groupBuilder.build(), Block.constantNullBlock(1))); + List results = drive(operators, source.iterator()); + + assertThat(results, hasSize(1)); + Block resultBlock = results.get(0).getBlock(1); + assertOutputFromNullOnly(resultBlock, 0); + } + + public final void testNullSome() { + DriverContext driverContext = new DriverContext(); + assertNullSome(List.of(simple(nonBreakingBigArrays().withCircuitBreaking()).get(driverContext))); + } + + public final void testNullSomeInitialFinal() { + DriverContext driverContext = new DriverContext(); + assertNullSome( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ) + ); + } + + public final void testNullSomeInitialIntermediateFinal() { + DriverContext driverContext = new DriverContext(); + assertNullSome( + List.of( + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.INTERMEDIATE).get(driverContext), + simpleWithMode(nonBreakingBigArrays().withCircuitBreaking(), AggregatorMode.FINAL).get(driverContext) + ) + ); + } + + /** + * Run the agg on some data where one group is always null. + */ + private void assertNullSome(List operators) { + List inputData = CannedSourceOperator.collectPages(simpleInput(1000)); + SeenGroups seenGroups = seenGroups(inputData); + + long nullGroup = randomFrom(seenGroups.nonNull); + List source = new ArrayList<>(inputData.size()); + for (Page page : inputData) { + LongVector groups = page.getBlock(0).asVector(); + Block values = page.getBlock(1); + Block.Builder copiedValues = values.elementType().newBlockBuilder(page.getPositionCount()); + for (int p = 0; p < page.getPositionCount(); p++) { + if (groups.getLong(p) == nullGroup) { + copiedValues.appendNull(); + } else { + copiedValues.copyFrom(values, p, p + 1); + } + } + source.add(new Page(groups.asBlock(), copiedValues.build())); + } + + List results = drive(operators, source.iterator()); + + assertThat(results, hasSize(1)); + LongVector groups = results.get(0).getBlock(0).asVector(); + Block resultBlock = results.get(0).getBlock(1); + boolean foundNullPosition = false; + for (int p = 0; p < groups.getPositionCount(); p++) { + if (groups.getLong(p) == nullGroup) { + foundNullPosition = true; + assertOutputFromNullOnly(resultBlock, p); + } + } + assertTrue("didn't find the null position. bad position range?", foundNullPosition); + } + + /** + * Asserts that the output from a group that contains only null values is + * a {@link Block} containing only {@code null}. Override for + * {@code count} style aggregations that return other sorts of results. + */ + protected void assertOutputFromNullOnly(Block b, int position) { + assertThat(b.isNull(position), equalTo(true)); + assertThat(b.getValueCount(position), equalTo(0)); + } + + private SourceOperator mergeValues(SourceOperator orig) { + return new PositionMergingSourceOperator(orig) { + @Override + protected Block merge(int blockIndex, Block block) { + // Merge positions for all blocks but the first. For the first just take the first position. + if (blockIndex != 0) { + return super.merge(blockIndex, block); + } + Block.Builder builder = block.elementType().newBlockBuilder(block.getPositionCount() / 2); + for (int p = 0; p + 1 < block.getPositionCount(); p += 2) { + builder.copyFrom(block, p, p + 1); + } + if (block.getPositionCount() % 2 == 1) { + builder.copyFrom(block, block.getPositionCount() - 1, block.getPositionCount()); + } + return builder.build(); + } + }; + } + + protected static IntStream allValueOffsets(Page page, Long group) { + LongBlock groupBlock = page.getBlock(0); + Block valueBlock = page.getBlock(1); + return IntStream.range(0, page.getPositionCount()).flatMap(p -> { + if (valueBlock.isNull(p)) { + return IntStream.of(); + } + if (group == null) { + if (false == groupBlock.isNull(p)) { + return IntStream.of(); + } + } else { + int groupStart = groupBlock.getFirstValueIndex(p); + int groupEnd = groupStart + groupBlock.getValueCount(p); + boolean matched = false; + for (int i = groupStart; i < groupEnd; i++) { + if (groupBlock.getLong(i) == group) { + matched = true; + break; + } + } + if (matched == false) { + return IntStream.of(); + } + } + int start = valueBlock.getFirstValueIndex(p); + int end = start + valueBlock.getValueCount(p); + return IntStream.range(start, end); + }); + } + + protected static Stream allBytesRefs(Page page, Long group) { + BytesRefBlock b = page.getBlock(1); + return allValueOffsets(page, group).mapToObj(i -> b.getBytesRef(i, new BytesRef())); + } + + protected static Stream allBooleans(Page page, Long group) { + BooleanBlock b = page.getBlock(1); + return allValueOffsets(page, group).mapToObj(i -> b.getBoolean(i)); + } + + protected static DoubleStream allDoubles(Page page, Long group) { + DoubleBlock b = page.getBlock(1); + return allValueOffsets(page, group).mapToDouble(i -> b.getDouble(i)); + } + + protected static IntStream allInts(Page page, Long group) { + IntBlock b = page.getBlock(1); + return allValueOffsets(page, group).map(i -> b.getInt(i)); + } + + protected static LongStream allLongs(Page page, Long group) { + LongBlock b = page.getBlock(1); + return allValueOffsets(page, group).mapToLong(i -> b.getLong(i)); + } + + /** + * Forcibly chunk groups on the way into the aggregator to make sure it can handle chunked + * groups. This is needed because our chunking logic for groups doesn't bother chunking + * in non-combinatorial explosion cases. We figure if the could fit into memory then the + * groups should too. But for testing we'd sometimes like to force chunking just so we + * run the aggregation with funny chunked inputs. + */ + private AggregatorFunctionSupplier chunkGroups(int emitChunkSize, AggregatorFunctionSupplier supplier) { + return new AggregatorFunctionSupplier() { + @Override + public AggregatorFunction aggregator() { + return supplier.aggregator(); + } + + @Override + public GroupingAggregatorFunction groupingAggregator() { + return new GroupingAggregatorFunction() { + GroupingAggregatorFunction delegate = supplier.groupingAggregator(); + BitArray seenGroupIds = new BitArray(0, nonBreakingBigArrays()); + + @Override + public AddInput prepareProcessPage(SeenGroupIds ignoredSeenGroupIds, Page page) { + return new AddInput() { + AddInput delegateAddInput = delegate.prepareProcessPage(bigArrays -> { + BitArray seen = new BitArray(0, bigArrays); + seen.or(seenGroupIds); + return seen; + }, page); + + @Override + public void add(int positionOffset, LongBlock groupIds) { + for (int offset = 0; offset < groupIds.getPositionCount(); offset += emitChunkSize) { + LongBlock.Builder builder = LongBlock.newBlockBuilder(emitChunkSize); + int endP = Math.min(groupIds.getPositionCount(), offset + emitChunkSize); + for (int p = offset; p < endP; p++) { + int start = groupIds.getFirstValueIndex(p); + int count = groupIds.getValueCount(p); + switch (count) { + case 0 -> builder.appendNull(); + case 1 -> { + long group = groupIds.getLong(start); + seenGroupIds.set(group); + builder.appendLong(group); + } + default -> { + int end = start + count; + for (int i = start; i < end; i++) { + long group = groupIds.getLong(i); + seenGroupIds.set(group); + builder.appendLong(group); + } + } + } + } + delegateAddInput.add(positionOffset + offset, builder.build()); + } + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + long[] chunk = new long[emitChunkSize]; + for (int offset = 0; offset < groupIds.getPositionCount(); offset += emitChunkSize) { + int count = 0; + for (int i = offset; i < Math.min(groupIds.getPositionCount(), offset + emitChunkSize); i++) { + long group = groupIds.getLong(i); + seenGroupIds.set(group); + chunk[count++] = group; + } + delegateAddInput.add(positionOffset + offset, new LongArrayVector(chunk, count)); + } + } + }; + } + + @Override + public void addIntermediateInput(int positionOffset, LongVector groupIds, Page page) { + long[] chunk = new long[emitChunkSize]; + for (int offset = 0; offset < groupIds.getPositionCount(); offset += emitChunkSize) { + int count = 0; + for (int i = offset; i < Math.min(groupIds.getPositionCount(), offset + emitChunkSize); i++) { + chunk[count++] = groupIds.getLong(i); + } + delegate.addIntermediateInput(positionOffset + offset, new LongArrayVector(chunk, count), page); + } + } + + @Override + public void addIntermediateRowInput(int groupId, GroupingAggregatorFunction input, int position) { + delegate.addIntermediateRowInput(groupId, input, position); + } + + @Override + public void evaluateIntermediate(Block[] blocks, int offset, IntVector selected) { + delegate.evaluateIntermediate(blocks, offset, selected); + } + + @Override + public void evaluateFinal(Block[] blocks, int offset, IntVector selected) { + delegate.evaluateFinal(blocks, offset, selected); + } + + @Override + public int intermediateBlockCount() { + return delegate.intermediateBlockCount(); + } + + @Override + public void close() { + Releasables.close(delegate::close, seenGroupIds); + } + + @Override + public String toString() { + return delegate.toString(); + } + }; + } + + @Override + public String describe() { + return supplier.describe(); + } + }; + } + +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..b67220b4909b7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleAggregatorFunctionTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.test.ESTestCase; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceDoubleBlockSourceOperator(LongStream.range(0, size).mapToDouble(l -> ESTestCase.randomDouble())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of doubles"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + double max = input.stream().flatMapToDouble(b -> allDoubles(b)).max().getAsDouble(); + assertThat(((DoubleBlock) result).getDouble(0), equalTo(max)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..3750aec95f3a7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,53 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalDouble; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, end).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomDouble())) + ); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of doubles"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalDouble max = input.stream().flatMapToDouble(p -> allDoubles(p, group)).max(); + if (max.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((DoubleBlock) result).getDouble(position), equalTo(max.getAsDouble())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..72cfa06222b50 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntAggregatorFunctionTests.java @@ -0,0 +1,42 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceIntBlockSourceOperator(IntStream.range(0, size).map(l -> randomInt())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of ints"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + int max = input.stream().flatMapToInt(b -> allInts(b)).max().getAsInt(); + assertThat(((IntBlock) result).getInt(0), equalTo(max)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..9ffee498eeba2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,50 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalInt; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongIntBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomInt()))); + } + + @Override + public void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalInt max = input.stream().flatMapToInt(p -> allInts(p, group)).max(); + if (max.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((IntBlock) result).getInt(position), equalTo(max.getAsInt())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..4e84f2e672b97 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongAggregatorFunctionTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of longs"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + long max = input.stream().flatMapToLong(b -> allLongs(b)).max().getAsLong(); + assertThat(((LongBlock) result).getLong(0), equalTo(max)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..e284f2a6103d1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,50 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalLong; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MaxLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MaxLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "max of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new TupleBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong()))); + } + + @Override + public void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalLong max = input.stream().flatMapToLong(p -> allLongs(p, group)).max(); + if (max.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((LongBlock) result).getLong(position), equalTo(max.getAsLong())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..74bda421a545e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleAggregatorFunctionTests.java @@ -0,0 +1,45 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + List values = Arrays.asList(1.2, 1.25, 2.0, 2.0, 4.3, 6.0, 9.0); + Randomness.shuffle(values); + return new SequenceDoubleBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of doubles"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + assertThat(((DoubleBlock) result).getDouble(0), equalTo(0.8)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..6751486453f30 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,81 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.DoubleStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + double[][] samples = new double[][] { + { 1.2, 1.25, 2.0, 2.0, 4.3, 6.0, 9.0 }, + { 0.1, 1.5, 2.0, 3.0, 4.0, 7.5, 100.0 }, + { 0.2, 1.75, 2.0, 2.5 }, + { 0.5, 3.0, 3.0, 3.0, 4.3 }, + { 0.25, 1.5, 3.0 } }; + List> values = new ArrayList<>(); + for (int i = 0; i < samples.length; i++) { + List list = Arrays.stream(samples[i]).boxed().collect(Collectors.toList()); + Randomness.shuffle(list); + for (double v : list) { + values.add(Tuple.tuple((long) i, v)); + } + } + return new LongDoubleTupleBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of doubles"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + assertThat( + ((DoubleBlock) result).getDouble(position), + equalTo(medianAbsoluteDeviation(input.stream().flatMapToDouble(p -> allDoubles(p, group)))) + ); + } + + static double medianAbsoluteDeviation(DoubleStream s) { + double[] data = s.toArray(); + double median = median(Arrays.stream(data)); + return median(Arrays.stream(data).map(d -> Math.abs(median - d))); + } + + static double median(DoubleStream s) { + // The input data is small enough that tdigest will find the actual median. + double[] data = s.sorted().toArray(); + if (data.length == 0) { + return 0; + } + int c = data.length / 2; + return data.length % 2 == 0 ? (data[c - 1] + data[c]) / 2 : data[c]; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..20506cc5c8f93 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntAggregatorFunctionTests.java @@ -0,0 +1,45 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + List values = Arrays.asList(12, 125, 20, 20, 43, 60, 90); + Randomness.shuffle(values); + return new SequenceIntBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of ints"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + assertThat(((DoubleBlock) result).getDouble(0), equalTo(23.0)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..20f62c67a16cc --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,65 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.elasticsearch.compute.aggregation.MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests.medianAbsoluteDeviation; +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + int[][] samples = new int[][] { + { 12, 125, 20, 20, 43, 60, 90 }, + { 1, 15, 20, 30, 40, 75, 1000 }, + { 2, 175, 20, 25 }, + { 5, 30, 30, 30, 43 }, + { 7, 15, 30 } }; + List> values = new ArrayList<>(); + for (int i = 0; i < samples.length; i++) { + List list = Arrays.stream(samples[i]).boxed().collect(Collectors.toList()); + Randomness.shuffle(list); + for (int v : list) { + values.add(Tuple.tuple((long) i, v)); + } + } + return new LongIntBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of ints"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + assertThat( + ((DoubleBlock) result).getDouble(position), + equalTo(medianAbsoluteDeviation(input.stream().flatMapToInt(p -> allInts(p, group)).asDoubleStream())) + ); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..d80415f83daa2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongAggregatorFunctionTests.java @@ -0,0 +1,45 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + List values = Arrays.asList(12L, 125L, 20L, 20L, 43L, 60L, 90L); + Randomness.shuffle(values); + return new SequenceLongBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of longs"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + assertThat(((DoubleBlock) result).getDouble(0), equalTo(23.0)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..c3cebad8e0e0b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,65 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.elasticsearch.compute.aggregation.MedianAbsoluteDeviationDoubleGroupingAggregatorFunctionTests.medianAbsoluteDeviation; +import static org.hamcrest.Matchers.equalTo; + +public class MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + long[][] samples = new long[][] { + { 12, 125, 20, 20, 43, 60, 90 }, + { 1, 15, 20, 30, 40, 75, 1000 }, + { 2, 175, 20, 25 }, + { 5, 30, 30, 30, 43 }, + { 7, 15, 30 } }; + List> values = new ArrayList<>(); + for (int i = 0; i < samples.length; i++) { + List list = Arrays.stream(samples[i]).boxed().collect(Collectors.toList()); + Randomness.shuffle(list); + for (long v : list) { + values.add(Tuple.tuple((long) i, v)); + } + } + return new TupleBlockSourceOperator(values); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "median_absolute_deviation of longs"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + assertThat( + ((DoubleBlock) result).getDouble(position), + equalTo(medianAbsoluteDeviation(input.stream().flatMapToLong(p -> allLongs(p, group)).asDoubleStream())) + ); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..622302d549fd0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleAggregatorFunctionTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.test.ESTestCase; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceDoubleBlockSourceOperator(LongStream.range(0, size).mapToDouble(l -> ESTestCase.randomDouble())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of doubles"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + double min = input.stream().flatMapToDouble(b -> allDoubles(b)).min().getAsDouble(); + assertThat(((DoubleBlock) result).getDouble(0), equalTo(min)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..12c63e354547a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,52 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalDouble; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int end) { + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, end).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomDouble())) + ); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of doubles"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalDouble min = input.stream().flatMapToDouble(p -> allDoubles(p, group)).min(); + if (min.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((DoubleBlock) result).getDouble(position), equalTo(min.getAsDouble())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..2dc0e893875ab --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntAggregatorFunctionTests.java @@ -0,0 +1,42 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceIntBlockSourceOperator(IntStream.range(0, size).map(l -> randomInt())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of ints"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + int max = input.stream().flatMapToInt(b -> allInts(b)).min().getAsInt(); + assertThat(((IntBlock) result).getInt(0), equalTo(max)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..4ffbe9b1396d3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,50 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalInt; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new LongIntBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomInt()))); + } + + @Override + public void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalInt min = input.stream().flatMapToInt(p -> allInts(p, group)).min(); + if (min.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((IntBlock) result).getInt(position), equalTo(min.getAsInt())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..25a420237893e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongAggregatorFunctionTests.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of longs"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + long min = input.stream().flatMapToLong(b -> allLongs(b)).min().getAsLong(); + assertThat(((LongBlock) result).getLong(0), equalTo(min)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..311e7e41ed9ac --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,50 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.OptionalLong; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MinLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new MinLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "min of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new TupleBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong()))); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + OptionalLong min = input.stream().flatMapToLong(p -> allLongs(p, group)).min(); + if (min.isEmpty()) { + assertThat(result.isNull(position), equalTo(true)); + return; + } + assertThat(result.isNull(position), equalTo(false)); + assertThat(((LongBlock) result).getLong(position), equalTo(min.getAsLong())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..96e61d4782022 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleAggregatorFunctionTests.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + + private double percentile; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of doubles"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceDoubleBlockSourceOperator(LongStream.range(0, size).mapToDouble(l -> ESTestCase.randomDouble())); + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToDouble(b -> allDoubles(b)).forEach(td::add); + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(0); + assertThat(value, closeTo(expected, expected * 0.1)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..c0d6595e088eb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,63 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + private double percentile; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of doubles"; + } + + @Override + protected SourceOperator simpleInput(int end) { + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, end).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomDouble())) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToDouble(p -> allDoubles(p, group)).forEach(td::add); + if (td.size() > 0) { + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(position); + assertThat(value, closeTo(expected, expected * 0.1)); + } else { + assertTrue(result.isNull(position)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..c34a01e608d1a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntAggregatorFunctionTests.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + + private double percentile; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileIntAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, (int) Math.min(Integer.MAX_VALUE, Long.MAX_VALUE / size)); + return new SequenceIntBlockSourceOperator(LongStream.range(0, size).mapToInt(l -> between(0, max))); + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToInt(b -> allInts(b)).forEach(td::add); + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(0); + assertThat(value, closeTo(expected, expected * 0.1)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..a018fba96e897 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,64 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + private double percentile; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileIntAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, (int) Math.min(Integer.MAX_VALUE, Long.MAX_VALUE / size)); + return new LongIntBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), between(-1, max))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToInt(p -> allInts(p, group)).forEach(td::add); + if (td.size() > 0) { + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(position); + assertThat(value, closeTo(expected, expected * 0.1)); + } else { + assertTrue(result.isNull(position)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..cf0b18840d91e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongAggregatorFunctionTests.java @@ -0,0 +1,56 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + + private double percentile; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileLongAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, 1_000_000); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(0, max))); + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToLong(p -> allLongs(p)).forEach(td::add); + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(0); + assertThat(value, closeTo(expected, expected * 0.1)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..609526532b72e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,64 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.aggregations.metrics.TDigestState; +import org.junit.Before; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class PercentileLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + + private double percentile = 0; + + @Before + public void initParameters() { + percentile = randomFrom(0, 1, 5, 10, 25, 50, 75, 90, 95, 99, 100); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new PercentileLongAggregatorFunctionSupplier(bigArrays, inputChannels, percentile); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "percentile of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size / 5); + return new TupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLongBetween(-0, max))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + TDigestState td = TDigestState.create(QuantileStates.DEFAULT_COMPRESSION); + input.stream().flatMapToLong(p -> allLongs(p, group)).forEach(td::add); + if (td.size() > 0) { + double expected = td.quantile(percentile / 100); + double value = ((DoubleBlock) result).getDouble(position); + assertThat(value, closeTo(expected, expected * 0.1)); + } else { + assertTrue(result.isNull(position)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionTests.java new file mode 100644 index 0000000000000..909b582bec732 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleAggregatorFunctionTests.java @@ -0,0 +1,155 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceDoubleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.DoubleStream; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; + +public class SumDoubleAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceDoubleBlockSourceOperator(LongStream.range(0, size).mapToDouble(l -> ESTestCase.randomDouble())); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of doubles"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + double sum = input.stream().flatMapToDouble(b -> allDoubles(b)).sum(); + assertThat(((DoubleBlock) result).getDouble(0), closeTo(sum, .0001)); + } + + public void testOverflowSucceeds() { + DriverContext driverContext = new DriverContext(); + List results = new ArrayList<>(); + try ( + Driver d = new Driver( + driverContext, + new SequenceDoubleBlockSourceOperator(DoubleStream.of(Double.MAX_VALUE - 1, 2)), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertThat(results.get(0).getBlock(0).getDouble(0), equalTo(Double.MAX_VALUE + 1)); + assertDriverContext(driverContext); + } + + public void testSummationAccuracy() { + DriverContext driverContext = new DriverContext(); + List results = new ArrayList<>(); + try ( + Driver d = new Driver( + driverContext, + new SequenceDoubleBlockSourceOperator( + DoubleStream.of(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7) + ), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertEquals(15.3, results.get(0).getBlock(0).getDouble(0), Double.MIN_NORMAL); + assertDriverContext(driverContext); + + // Summing up an array which contains NaN and infinities and expect a result same as naive summation + results.clear(); + int n = randomIntBetween(5, 10); + double[] values = new double[n]; + double sum = 0; + for (int i = 0; i < n; i++) { + values[i] = frequently() + ? randomFrom(Double.NaN, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY) + : randomDoubleBetween(Double.MIN_VALUE, Double.MAX_VALUE, true); + sum += values[i]; + } + driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new SequenceDoubleBlockSourceOperator(DoubleStream.of(values)), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertEquals(sum, results.get(0).getBlock(0).getDouble(0), 1e-10); + assertDriverContext(driverContext); + + // Summing up some big double values and expect infinity result + results.clear(); + n = randomIntBetween(5, 10); + double[] largeValues = new double[n]; + for (int i = 0; i < n; i++) { + largeValues[i] = Double.MAX_VALUE; + } + driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new SequenceDoubleBlockSourceOperator(DoubleStream.of(largeValues)), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertEquals(Double.POSITIVE_INFINITY, results.get(0).getBlock(0).getDouble(0), 0d); + assertDriverContext(driverContext); + + results.clear(); + for (int i = 0; i < n; i++) { + largeValues[i] = -Double.MAX_VALUE; + } + driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new SequenceDoubleBlockSourceOperator(DoubleStream.of(largeValues)), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertEquals(Double.NEGATIVE_INFINITY, results.get(0).getBlock(0).getDouble(0), 0d); + assertDriverContext(driverContext); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..03a7269b84690 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumDoubleGroupingAggregatorFunctionTests.java @@ -0,0 +1,49 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.closeTo; + +public class SumDoubleGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int end) { + return new LongDoubleTupleBlockSourceOperator( + LongStream.range(0, end).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomDouble())) + ); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of doubles"; + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + CompensatedSum sum = new CompensatedSum(); + input.stream().flatMapToDouble(p -> allDoubles(p, group)).forEach(sum::add); + // Won't precisely match in distributed case but will be close + assertThat(((DoubleBlock) result).getDouble(position), closeTo(sum.value(), 0.01)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionTests.java new file mode 100644 index 0000000000000..d9e073ace9b6e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntAggregatorFunctionTests.java @@ -0,0 +1,66 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class SumIntAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, (int) Math.min(Integer.MAX_VALUE, Long.MAX_VALUE / size)); + return new SequenceIntBlockSourceOperator(LongStream.range(0, size).mapToInt(l -> between(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of ints"; + } + + @Override + protected void assertSimpleOutput(List input, Block result) { + long sum = input.stream().flatMapToInt(b -> allInts(b)).asLongStream().sum(); + assertThat(((LongBlock) result).getLong(0), equalTo(sum)); + } + + public void testRejectsDouble() { + DriverContext driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(Iterators.single(new Page(new DoubleArrayVector(new double[] { 1.0 }, 1).asBlock()))), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> fail("shouldn't have made it this far")), + () -> {} + ) + ) { + expectThrows(Exception.class, () -> runDriver(d)); // ### find a more specific exception type + } + assertDriverContext(driverContext); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..71666024c819d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumIntGroupingAggregatorFunctionTests.java @@ -0,0 +1,47 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.LongIntBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class SumIntGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of ints"; + } + + @Override + protected SourceOperator simpleInput(int size) { + int max = between(1, (int) Math.min(Integer.MAX_VALUE, Long.MAX_VALUE / size)); + return new LongIntBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), between(-max, max))) + ); + } + + @Override + protected void assertSimpleGroup(List input, Block result, int position, Long group) { + long sum = input.stream().flatMapToInt(p -> allInts(p, group)).asLongStream().sum(); + assertThat(((LongBlock) result).getLong(position), equalTo(sum)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionTests.java new file mode 100644 index 0000000000000..25e3d62ae9ed8 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongAggregatorFunctionTests.java @@ -0,0 +1,81 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class SumLongAggregatorFunctionTests extends AggregatorFunctionTestCase { + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(-max, max))); + } + + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of longs"; + } + + @Override + public void assertSimpleOutput(List input, Block result) { + long sum = input.stream().flatMapToLong(b -> allLongs(b)).sum(); + assertThat(((LongBlock) result).getLong(0), equalTo(sum)); + } + + public void testOverflowFails() { + DriverContext driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new SequenceLongBlockSourceOperator(LongStream.of(Long.MAX_VALUE - 1, 2)), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> fail("shouldn't have made it this far")), + () -> {} + ) + ) { + Exception e = expectThrows(ArithmeticException.class, () -> runDriver(d)); + assertThat(e.getMessage(), equalTo("long overflow")); + } + } + + public void testRejectsDouble() { + DriverContext driverContext = new DriverContext(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(Iterators.single(new Page(new DoubleArrayVector(new double[] { 1.0 }, 1).asBlock()))), + List.of(simple(nonBreakingBigArrays()).get(driverContext)), + new PageConsumerOperator(page -> fail("shouldn't have made it this far")), + () -> {} + ) + ) { + expectThrows(Exception.class, () -> runDriver(d)); // ### find a more specific exception type + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java new file mode 100644 index 0000000000000..e0dc918b515d6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java @@ -0,0 +1,47 @@ +/* + * 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.compute.aggregation; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class SumLongGroupingAggregatorFunctionTests extends GroupingAggregatorFunctionTestCase { + @Override + protected AggregatorFunctionSupplier aggregatorFunction(BigArrays bigArrays, List inputChannels) { + return new SumLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected String expectedDescriptionOfAggregator() { + return "sum of longs"; + } + + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size / 5); + return new TupleBlockSourceOperator( + LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLongBetween(-max, max))) + ); + } + + @Override + public void assertSimpleGroup(List input, Block result, int position, Long group) { + long sum = input.stream().flatMapToLong(p -> allLongs(p, group)).sum(); + assertThat(((LongBlock) result).getLong(position), equalTo(sum)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java new file mode 100644 index 0000000000000..c585108a89fd0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashRandomizedTests.java @@ -0,0 +1,237 @@ +/* + * 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.compute.aggregation.blockhash; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.MultivalueDedupeTests; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.ListMatcher; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; + +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +//@TestLogging(value = "org.elasticsearch.compute:TRACE", reason = "debug") +public class BlockHashRandomizedTests extends ESTestCase { + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + + for (boolean forcePackedHash : new boolean[] { false, true }) { + for (int groups : new int[] { 1, 2, 3, 4, 5, 10 }) { + for (int maxValuesPerPosition : new int[] { 1, 3 }) { + for (int dups : new int[] { 0, 2 }) { + for (List allowedTypes : List.of( + /* + * Run with only `LONG` elements because we have some + * optimizations that hit if you only have those. + */ + List.of(ElementType.LONG), + /* + * Run with only `LONG` and `BYTES_REF` elements because + * we have some optimizations that hit if you only have + * those. + */ + List.of(ElementType.LONG, ElementType.BYTES_REF), + MultivalueDedupeTests.supportedTypes() + )) { + params.add(new Object[] { forcePackedHash, groups, maxValuesPerPosition, dups, allowedTypes }); + } + } + } + } + } + return params; + } + + private final boolean forcePackedHash; + private final int groups; + private final int maxValuesPerPosition; + private final int dups; + private final List allowedTypes; + + public BlockHashRandomizedTests( + @Name("forcePackedHash") boolean forcePackedHash, + @Name("groups") int groups, + @Name("maxValuesPerPosition") int maxValuesPerPosition, + @Name("dups") int dups, + @Name("allowedTypes") List allowedTypes + ) { + this.forcePackedHash = forcePackedHash; + this.groups = groups; + this.maxValuesPerPosition = maxValuesPerPosition; + this.dups = dups; + this.allowedTypes = allowedTypes; + } + + public void test() { + List types = randomList(groups, groups, () -> randomFrom(allowedTypes)); + BasicBlockTests.RandomBlock[] randomBlocks = new BasicBlockTests.RandomBlock[types.size()]; + Block[] blocks = new Block[types.size()]; + int pageCount = between(1, 10); + int positionCount = 100; + int emitBatchSize = 100; + try (BlockHash blockHash = newBlockHash(emitBatchSize, types)) { + /* + * Only the long/long, long/bytes_ref, and bytes_ref/long implementations don't collect nulls. + */ + Oracle oracle = new Oracle( + forcePackedHash + || false == (types.equals(List.of(ElementType.LONG, ElementType.LONG)) + || types.equals(List.of(ElementType.LONG, ElementType.BYTES_REF)) + || types.equals(List.of(ElementType.BYTES_REF, ElementType.LONG))) + ); + + for (int p = 0; p < pageCount; p++) { + for (int g = 0; g < blocks.length; g++) { + randomBlocks[g] = BasicBlockTests.randomBlock( + types.get(g), + positionCount, + randomBoolean(), + 1, + maxValuesPerPosition, + 0, + dups + ); + blocks[g] = randomBlocks[g].block(); + } + oracle.add(randomBlocks); + int[] batchCount = new int[1]; + // PackedValuesBlockHash always chunks but the normal single value ones don't + boolean usingSingle = forcePackedHash == false && types.size() == 1; + BlockHashTests.hash(false, blockHash, ordsAndKeys -> { + if (usingSingle == false) { + assertThat(ordsAndKeys.ords().getTotalValueCount(), lessThanOrEqualTo(emitBatchSize)); + } + batchCount[0]++; + }, blocks); + if (usingSingle) { + assertThat(batchCount[0], equalTo(1)); + } + } + + Block[] keyBlocks = blockHash.getKeys(); + Set> keys = new TreeSet<>(new KeyComparator()); + for (int p = 0; p < keyBlocks[0].getPositionCount(); p++) { + List key = new ArrayList<>(keyBlocks.length); + for (Block keyBlock : keyBlocks) { + if (keyBlock.isNull(p)) { + key.add(null); + } else { + key.add(BasicBlockTests.valuesAtPositions(keyBlock, p, p + 1).get(0).get(0)); + assertThat(keyBlock.getValueCount(p), equalTo(1)); + } + } + boolean contained = keys.add(key); + assertTrue(contained); + } + + if (false == keys.equals(oracle.keys)) { + List> keyList = new ArrayList<>(); + keyList.addAll(keys); + ListMatcher keyMatcher = matchesList(); + for (List k : oracle.keys) { + keyMatcher = keyMatcher.item(k); + } + assertMap(keyList, keyMatcher); + } + } + } + + private BlockHash newBlockHash(int emitBatchSize, List types) { + List specs = new ArrayList<>(types.size()); + for (int c = 0; c < types.size(); c++) { + specs.add(new HashAggregationOperator.GroupSpec(c, types.get(c))); + } + MockBigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + return forcePackedHash + ? new PackedValuesBlockHash(specs, bigArrays, emitBatchSize) + : BlockHash.build(specs, bigArrays, emitBatchSize); + } + + private static class KeyComparator implements Comparator> { + @Override + public int compare(List lhs, List rhs) { + for (int i = 0; i < lhs.size(); i++) { + @SuppressWarnings("unchecked") + Comparable l = (Comparable) lhs.get(i); + Object r = rhs.get(i); + if (l == null) { + if (r == null) { + continue; + } else { + return 1; + } + } + if (r == null) { + return -1; + } + int cmp = l.compareTo(r); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + } + + private static class Oracle { + private final NavigableSet> keys = new TreeSet<>(new KeyComparator()); + + private final boolean collectsNull; + + private Oracle(boolean collectsNull) { + this.collectsNull = collectsNull; + } + + void add(BasicBlockTests.RandomBlock[] randomBlocks) { + for (int p = 0; p < randomBlocks[0].block().getPositionCount(); p++) { + add(randomBlocks, p, List.of()); + } + } + + void add(BasicBlockTests.RandomBlock[] randomBlocks, int p, List key) { + if (key.size() == randomBlocks.length) { + keys.add(key); + return; + } + BasicBlockTests.RandomBlock block = randomBlocks[key.size()]; + List values = block.values().get(p); + if (values == null) { + if (collectsNull) { + List newKey = new ArrayList<>(key); + newKey.add(null); + add(randomBlocks, p, newKey); + } + return; + } + for (Object v : values) { + List newKey = new ArrayList<>(key); + newKey.add(v); + add(randomBlocks, p, newKey); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java new file mode 100644 index 0000000000000..7c56691a3ae41 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/BlockHashTests.java @@ -0,0 +1,1152 @@ +/* + * 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.compute.aggregation.blockhash; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.inject.name.Named; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.arrayWithSize; +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.startsWith; + +public class BlockHashTests extends ESTestCase { + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + params.add(new Object[] { false }); + params.add(new Object[] { true }); + return params; + } + + private final boolean forcePackedHash; + + public BlockHashTests(@Named("forcePackedHash") boolean forcePackedHash) { + this.forcePackedHash = forcePackedHash; + } + + public void testIntHash() { + int[] values = new int[] { 1, 2, 3, 1, 2, 3, 1, 2, 3 }; + IntBlock block = new IntArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:INT], entries=3, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 0L, 1L, 2L, 0L, 1L, 2L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } else { + assertThat(ordsAndKeys.description, equalTo("IntBlockHash{channel=0, entries=3, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 2L, 3L, 1L, 2L, 3L, 1L, 2L, 3L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 4))); + } + assertKeys(ordsAndKeys.keys, 1, 2, 3); + } + + public void testIntHashWithNulls() { + IntBlock.Builder builder = IntBlock.newBlockBuilder(4); + builder.appendInt(0); + builder.appendNull(); + builder.appendInt(2); + builder.appendNull(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:INT], entries=3, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 1L); + assertKeys(ordsAndKeys.keys, 0, null, 2); + } else { + assertThat(ordsAndKeys.description, equalTo("IntBlockHash{channel=0, entries=2, seenNull=true}")); + assertOrds(ordsAndKeys.ords, 1L, 0L, 2L, 0L); + assertKeys(ordsAndKeys.keys, null, 0, 2); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testIntHashWithMultiValuedFields() { + var builder = IntBlock.newBlockBuilder(8); + builder.appendInt(1); + builder.beginPositionEntry(); + builder.appendInt(1); + builder.appendInt(2); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendInt(3); + builder.appendInt(1); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendInt(3); + builder.appendInt(3); + builder.endPositionEntry(); + builder.appendNull(); + builder.beginPositionEntry(); + builder.appendInt(3); + builder.appendInt(2); + builder.appendInt(1); + builder.endPositionEntry(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:INT], entries=4, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0 }, + new long[] { 0, 1 }, + new long[] { 2, 0 }, + new long[] { 2 }, + new long[] { 3 }, + new long[] { 2, 1, 0 } + ); + assertKeys(ordsAndKeys.keys, 1, 2, 3, null); + } else { + assertThat(ordsAndKeys.description, equalTo("IntBlockHash{channel=0, entries=3, seenNull=true}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 1 }, + new long[] { 1, 2 }, + new long[] { 3, 1 }, + new long[] { 3 }, + new long[] { 0 }, + new long[] { 3, 2, 1 } + ); + assertKeys(ordsAndKeys.keys, null, 1, 2, 3); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + public void testLongHash() { + long[] values = new long[] { 2, 1, 4, 2, 4, 1, 3, 4 }; + LongBlock block = new LongArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG], entries=4, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 0L, 2L, 1L, 3L, 2L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } else { + assertThat(ordsAndKeys.description, equalTo("LongBlockHash{channel=0, entries=4, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 2L, 3L, 1L, 3L, 2L, 4L, 3L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 5))); + } + assertKeys(ordsAndKeys.keys, 2L, 1L, 4L, 3L); + } + + public void testLongHashWithNulls() { + LongBlock.Builder builder = LongBlock.newBlockBuilder(4); + builder.appendLong(0); + builder.appendNull(); + builder.appendLong(2); + builder.appendNull(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG], entries=3, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 1L); + assertKeys(ordsAndKeys.keys, 0L, null, 2L); + } else { + assertThat(ordsAndKeys.description, equalTo("LongBlockHash{channel=0, entries=2, seenNull=true}")); + assertOrds(ordsAndKeys.ords, 1L, 0L, 2L, 0L); + assertKeys(ordsAndKeys.keys, null, 0L, 2L); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testLongHashWithMultiValuedFields() { + var builder = LongBlock.newBlockBuilder(8); + builder.appendLong(1); + builder.beginPositionEntry(); + builder.appendLong(1); + builder.appendLong(2); + builder.appendLong(3); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendLong(1); + builder.appendLong(1); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendLong(3); + builder.endPositionEntry(); + builder.appendNull(); + builder.beginPositionEntry(); + builder.appendLong(3); + builder.appendLong(2); + builder.appendLong(1); + builder.endPositionEntry(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG], entries=4, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0 }, + new long[] { 0, 1, 2 }, + new long[] { 0 }, + new long[] { 2 }, + new long[] { 3 }, + new long[] { 2, 1, 0 } + ); + assertKeys(ordsAndKeys.keys, 1L, 2L, 3L, null); + } else { + assertThat(ordsAndKeys.description, equalTo("LongBlockHash{channel=0, entries=3, seenNull=true}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 1 }, + new long[] { 1, 2, 3 }, + new long[] { 1 }, + new long[] { 3 }, + new long[] { 0 }, + new long[] { 3, 2, 1 } + ); + assertKeys(ordsAndKeys.keys, null, 1L, 2L, 3L); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + public void testDoubleHash() { + double[] values = new double[] { 2.0, 1.0, 4.0, 2.0, 4.0, 1.0, 3.0, 4.0 }; + DoubleBlock block = new DoubleArrayVector(values, values.length).asBlock(); + OrdsAndKeys ordsAndKeys = hash(block); + + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:DOUBLE], entries=4, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 0L, 2L, 1L, 3L, 2L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } else { + assertThat(ordsAndKeys.description, equalTo("DoubleBlockHash{channel=0, entries=4, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 2L, 3L, 1L, 3L, 2L, 4L, 3L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 5))); + } + assertKeys(ordsAndKeys.keys, 2.0, 1.0, 4.0, 3.0); + } + + public void testDoubleHashWithNulls() { + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(4); + builder.appendDouble(0); + builder.appendNull(); + builder.appendDouble(2); + builder.appendNull(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:DOUBLE], entries=3, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 1L); + assertKeys(ordsAndKeys.keys, 0.0, null, 2.0); + } else { + assertThat(ordsAndKeys.description, equalTo("DoubleBlockHash{channel=0, entries=2, seenNull=true}")); + assertOrds(ordsAndKeys.ords, 1L, 0L, 2L, 0L); + assertKeys(ordsAndKeys.keys, null, 0.0, 2.0); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testDoubleHashWithMultiValuedFields() { + var builder = DoubleBlock.newBlockBuilder(8); + builder.appendDouble(1); + builder.beginPositionEntry(); + builder.appendDouble(2); + builder.appendDouble(3); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendDouble(3); + builder.appendDouble(2); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendDouble(1); + builder.endPositionEntry(); + builder.appendNull(); + builder.beginPositionEntry(); + builder.appendDouble(1); + builder.appendDouble(1); + builder.appendDouble(2); + builder.endPositionEntry(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:DOUBLE], entries=4, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0 }, + new long[] { 1, 2 }, + new long[] { 2, 1 }, + new long[] { 0 }, + new long[] { 3 }, + new long[] { 0, 1 } + ); + assertKeys(ordsAndKeys.keys, 1.0, 2.0, 3.0, null); + } else { + assertThat(ordsAndKeys.description, equalTo("DoubleBlockHash{channel=0, entries=3, seenNull=true}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 1 }, + new long[] { 2, 3 }, + new long[] { 3, 2 }, + new long[] { 1 }, + new long[] { 0 }, + new long[] { 1, 2 } + ); + assertKeys(ordsAndKeys.keys, null, 1.0, 2.0, 3.0); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + public void testBasicBytesRefHash() { + var builder = BytesRefBlock.newBlockBuilder(8); + builder.appendBytesRef(new BytesRef("item-2")); + builder.appendBytesRef(new BytesRef("item-1")); + builder.appendBytesRef(new BytesRef("item-4")); + builder.appendBytesRef(new BytesRef("item-2")); + builder.appendBytesRef(new BytesRef("item-4")); + builder.appendBytesRef(new BytesRef("item-1")); + builder.appendBytesRef(new BytesRef("item-3")); + builder.appendBytesRef(new BytesRef("item-4")); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BYTES_REF], entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 0L, 2L, 1L, 3L, 2L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } else { + assertThat(ordsAndKeys.description, startsWith("BytesRefBlockHash{channel=0, entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 2L, 3L, 1L, 3L, 2L, 4L, 3L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 5))); + } + assertKeys(ordsAndKeys.keys, "item-2", "item-1", "item-4", "item-3"); + } + + public void testBytesRefHashWithNulls() { + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(4); + builder.appendBytesRef(new BytesRef("cat")); + builder.appendNull(); + builder.appendBytesRef(new BytesRef("dog")); + builder.appendNull(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BYTES_REF], entries=3, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 1L); + assertKeys(ordsAndKeys.keys, "cat", null, "dog"); + } else { + assertThat(ordsAndKeys.description, startsWith("BytesRefBlockHash{channel=0, entries=2, size=")); + assertThat(ordsAndKeys.description, endsWith("b, seenNull=true}")); + assertOrds(ordsAndKeys.ords, 1L, 0L, 2L, 0L); + assertKeys(ordsAndKeys.keys, null, "cat", "dog"); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testBytesRefHashWithMultiValuedFields() { + var builder = BytesRefBlock.newBlockBuilder(8); + builder.appendBytesRef(new BytesRef("foo")); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("foo")); + builder.appendBytesRef(new BytesRef("bar")); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("bar")); + builder.appendBytesRef(new BytesRef("bort")); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("bort")); + builder.appendBytesRef(new BytesRef("bar")); + builder.endPositionEntry(); + builder.appendNull(); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("bort")); + builder.appendBytesRef(new BytesRef("bort")); + builder.appendBytesRef(new BytesRef("bar")); + builder.endPositionEntry(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BYTES_REF], entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0 }, + new long[] { 0, 1 }, + new long[] { 1, 2 }, + new long[] { 2, 1 }, + new long[] { 3 }, + new long[] { 2, 1 } + ); + assertKeys(ordsAndKeys.keys, "foo", "bar", "bort", null); + } else { + assertThat(ordsAndKeys.description, startsWith("BytesRefBlockHash{channel=0, entries=3, size=")); + assertThat(ordsAndKeys.description, endsWith("b, seenNull=true}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 1 }, + new long[] { 1, 2 }, + new long[] { 2, 3 }, + new long[] { 3, 2 }, + new long[] { 0 }, + new long[] { 3, 2 } + ); + assertKeys(ordsAndKeys.keys, null, "foo", "bar", "bort"); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + public void testBooleanHashFalseFirst() { + boolean[] values = new boolean[] { false, true, true, true, true }; + BooleanBlock block = new BooleanArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=2, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 1L, 1L, 1L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 2))); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=true, seenTrue=true, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 2L, 2L, 2L, 2L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 3))); + } + assertKeys(ordsAndKeys.keys, false, true); + } + + public void testBooleanHashTrueFirst() { + boolean[] values = new boolean[] { true, false, false, true, true }; + BooleanBlock block = new BooleanArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=2, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 1L, 0L, 0L); + assertKeys(ordsAndKeys.keys, true, false); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 2))); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=true, seenTrue=true, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 2L, 1L, 1L, 2L, 2L); + assertKeys(ordsAndKeys.keys, false, true); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(1, 3))); + } + } + + public void testBooleanHashTrueOnly() { + boolean[] values = new boolean[] { true, true, true, true }; + BooleanBlock block = new BooleanArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=1, size=")); + assertOrds(ordsAndKeys.ords, 0L, 0L, 0L, 0L); + assertKeys(ordsAndKeys.keys, true); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.newVectorBuilder(1).appendInt(0).build())); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=false, seenTrue=true, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 2L, 2L, 2L, 2L); + assertKeys(ordsAndKeys.keys, true); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.newVectorBuilder(1).appendInt(2).build())); + } + } + + public void testBooleanHashFalseOnly() { + boolean[] values = new boolean[] { false, false, false, false }; + BooleanBlock block = new BooleanArrayVector(values, values.length).asBlock(); + + OrdsAndKeys ordsAndKeys = hash(block); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=1, size=")); + assertOrds(ordsAndKeys.ords, 0L, 0L, 0L, 0L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.newVectorBuilder(1).appendInt(0).build())); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=true, seenTrue=false, seenNull=false}")); + assertOrds(ordsAndKeys.ords, 1L, 1L, 1L, 1L); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.newVectorBuilder(1).appendInt(1).build())); + } + assertKeys(ordsAndKeys.keys, false); + } + + public void testBooleanHashWithNulls() { + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(4); + builder.appendBoolean(false); + builder.appendNull(); + builder.appendBoolean(true); + builder.appendNull(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=3, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 1L); + assertKeys(ordsAndKeys.keys, false, null, true); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=true, seenTrue=true, seenNull=true}")); + assertOrds(ordsAndKeys.ords, 1L, 0L, 2L, 0L); + assertKeys(ordsAndKeys.keys, null, false, true); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testBooleanHashWithMultiValuedFields() { + var builder = BooleanBlock.newBlockBuilder(8); + builder.appendBoolean(false); + builder.beginPositionEntry(); + builder.appendBoolean(false); + builder.appendBoolean(true); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendBoolean(true); + builder.appendBoolean(false); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendBoolean(true); + builder.endPositionEntry(); + builder.appendNull(); + builder.beginPositionEntry(); + builder.appendBoolean(true); + builder.appendBoolean(true); + builder.appendBoolean(false); + builder.endPositionEntry(); + + OrdsAndKeys ordsAndKeys = hash(builder.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:BOOLEAN], entries=3, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0 }, + new long[] { 0, 1 }, + new long[] { 0, 1 }, // Order is not preserved + new long[] { 1 }, + new long[] { 2 }, + new long[] { 0, 1 } + ); + assertKeys(ordsAndKeys.keys, false, true, null); + } else { + assertThat(ordsAndKeys.description, equalTo("BooleanBlockHash{channel=0, seenFalse=true, seenTrue=true, seenNull=true}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 1 }, + new long[] { 1, 2 }, + new long[] { 1, 2 }, // Order is not preserved + new long[] { 2 }, + new long[] { 0 }, + new long[] { 1, 2 } + ); + assertKeys(ordsAndKeys.keys, null, false, true); + } + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 3))); + } + + public void testLongLongHash() { + long[] values1 = new long[] { 0, 1, 0, 1, 0, 1 }; + LongBlock block1 = new LongArrayVector(values1, values1.length).asBlock(); + long[] values2 = new long[] { 0, 0, 0, 1, 1, 1 }; + LongBlock block2 = new LongArrayVector(values2, values2.length).asBlock(); + Object[][] expectedKeys = { new Object[] { 0L, 0L }, new Object[] { 1L, 0L }, new Object[] { 1L, 1L }, new Object[] { 0L, 1L } }; + + OrdsAndKeys ordsAndKeys = hash(block1, block2); + assertThat( + ordsAndKeys.description, + forcePackedHash + ? startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:LONG], entries=4, size=") + : equalTo("LongLongBlockHash{channels=[0,1], entries=4}") + ); + assertOrds(ordsAndKeys.ords, 0L, 1L, 0L, 2L, 3L, 2L); + assertKeys(ordsAndKeys.keys, expectedKeys); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + private void append(LongBlock.Builder b1, LongBlock.Builder b2, long[] v1, long[] v2) { + if (v1 == null) { + b1.appendNull(); + } else if (v1.length == 1) { + b1.appendLong(v1[0]); + } else { + b1.beginPositionEntry(); + for (long v : v1) { + b1.appendLong(v); + } + b1.endPositionEntry(); + } + if (v2 == null) { + b2.appendNull(); + } else if (v2.length == 1) { + b2.appendLong(v2[0]); + } else { + b2.beginPositionEntry(); + for (long v : v2) { + b2.appendLong(v); + } + b2.endPositionEntry(); + } + } + + public void testLongLongHashWithMultiValuedFields() { + var b1 = LongBlock.newBlockBuilder(8); + var b2 = LongBlock.newBlockBuilder(8); + append(b1, b2, new long[] { 1, 2 }, new long[] { 10, 20 }); + append(b1, b2, new long[] { 1, 2 }, new long[] { 10 }); + append(b1, b2, new long[] { 1 }, new long[] { 10, 20 }); + append(b1, b2, new long[] { 1 }, new long[] { 10 }); + append(b1, b2, null, new long[] { 10 }); + append(b1, b2, new long[] { 1 }, null); + append(b1, b2, new long[] { 1, 1, 1 }, new long[] { 10, 10, 10 }); + append(b1, b2, new long[] { 1, 1, 2, 2 }, new long[] { 10, 20, 20 }); + append(b1, b2, new long[] { 1, 2, 3 }, new long[] { 30, 30, 10 }); + + OrdsAndKeys ordsAndKeys = hash(b1.build(), b2.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:LONG], entries=10, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0, 1, 2, 3 }, + new long[] { 0, 2 }, + new long[] { 0, 1 }, + new long[] { 0 }, + new long[] { 4 }, + new long[] { 5 }, + new long[] { 0 }, + new long[] { 0, 1, 2, 3 }, + new long[] { 6, 0, 7, 2, 8, 9 } + ); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, 10L }, + new Object[] { 1L, 20L }, + new Object[] { 2L, 10L }, + new Object[] { 2L, 20L }, + new Object[] { null, 10L }, + new Object[] { 1L, null }, + new Object[] { 1L, 30L }, + new Object[] { 2L, 30L }, + new Object[] { 3L, 30L }, + new Object[] { 3L, 10L }, } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 10))); + } else { + assertThat(ordsAndKeys.description, equalTo("LongLongBlockHash{channels=[0,1], entries=8}")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0, 1, 2, 3 }, + new long[] { 0, 2 }, + new long[] { 0, 1 }, + new long[] { 0 }, + null, + null, + new long[] { 0 }, + new long[] { 0, 1, 2, 3 }, + new long[] { 4, 0, 5, 2, 6, 7 } + ); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, 10L }, + new Object[] { 1L, 20L }, + new Object[] { 2L, 10L }, + new Object[] { 2L, 20L }, + new Object[] { 1L, 30L }, + new Object[] { 2L, 30L }, + new Object[] { 3L, 30L }, + new Object[] { 3L, 10L }, } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 8))); + } + } + + public void testLongLongHashHugeCombinatorialExplosion() { + long[] v1 = LongStream.range(0, 5000).toArray(); + long[] v2 = LongStream.range(100, 200).toArray(); + + var b1 = LongBlock.newBlockBuilder(v1.length); + var b2 = LongBlock.newBlockBuilder(v2.length); + append(b1, b2, v1, v2); + + int[] expectedEntries = new int[1]; + int pageSize = between(1000, 16 * 1024); + hash(ordsAndKeys -> { + int start = expectedEntries[0]; + expectedEntries[0] = Math.min(expectedEntries[0] + pageSize, v1.length * v2.length); + assertThat( + ordsAndKeys.description, + forcePackedHash + ? startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:LONG], entries=" + expectedEntries[0] + ", size=") + : equalTo("LongLongBlockHash{channels=[0,1], entries=" + expectedEntries[0] + "}") + ); + assertOrds(ordsAndKeys.ords, LongStream.range(start, expectedEntries[0]).toArray()); + assertKeys( + ordsAndKeys.keys, + IntStream.range(0, expectedEntries[0]) + .mapToObj(i -> new Object[] { v1[i / v2.length], v2[i % v2.length] }) + .toArray(l -> new Object[l][]) + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, expectedEntries[0]))); + }, pageSize, b1.build(), b2.build()); + + assertThat("misconfigured test", expectedEntries[0], greaterThan(0)); + } + + public void testIntLongHash() { + int[] values1 = new int[] { 0, 1, 0, 1, 0, 1 }; + IntBlock block1 = new IntArrayVector(values1, values1.length).asBlock(); + long[] values2 = new long[] { 0, 0, 0, 1, 1, 1 }; + LongBlock block2 = new LongArrayVector(values2, values2.length).asBlock(); + Object[][] expectedKeys = { new Object[] { 0, 0L }, new Object[] { 1, 0L }, new Object[] { 1, 1L }, new Object[] { 0, 1L } }; + + OrdsAndKeys ordsAndKeys = hash(block1, block2); + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:INT, 1:LONG], entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 0L, 2L, 3L, 2L); + assertKeys(ordsAndKeys.keys, expectedKeys); + } + + public void testLongDoubleHash() { + long[] values1 = new long[] { 0, 1, 0, 1, 0, 1 }; + LongBlock block1 = new LongArrayVector(values1, values1.length).asBlock(); + double[] values2 = new double[] { 0, 0, 0, 1, 1, 1 }; + DoubleBlock block2 = new DoubleArrayVector(values2, values2.length).asBlock(); + Object[][] expectedKeys = { new Object[] { 0L, 0d }, new Object[] { 1L, 0d }, new Object[] { 1L, 1d }, new Object[] { 0L, 1d } }; + OrdsAndKeys ordsAndKeys = hash(block1, block2); + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:DOUBLE], entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 0L, 2L, 3L, 2L); + assertKeys(ordsAndKeys.keys, expectedKeys); + } + + public void testIntBooleanHash() { + int[] values1 = new int[] { 0, 1, 0, 1, 0, 1 }; + IntBlock block1 = new IntArrayVector(values1, values1.length).asBlock(); + boolean[] values2 = new boolean[] { false, false, false, true, true, true }; + BooleanBlock block2 = new BooleanArrayVector(values2, values2.length).asBlock(); + Object[][] expectedKeys = { + new Object[] { 0, false }, + new Object[] { 1, false }, + new Object[] { 1, true }, + new Object[] { 0, true } }; + + OrdsAndKeys ordsAndKeys = hash(block1, block2); + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:INT, 1:BOOLEAN], entries=4, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 0L, 2L, 3L, 2L); + assertKeys(ordsAndKeys.keys, expectedKeys); + } + + public void testLongLongHashWithNull() { + LongBlock.Builder b1 = LongBlock.newBlockBuilder(2); + LongBlock.Builder b2 = LongBlock.newBlockBuilder(2); + b1.appendLong(1); + b2.appendLong(0); + b1.appendNull(); + b2.appendNull(); + b1.appendLong(0); + b2.appendLong(1); + b1.appendLong(0); + b2.appendNull(); + b1.appendNull(); + b2.appendLong(0); + + OrdsAndKeys ordsAndKeys = hash(b1.build(), b2.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:LONG], entries=5, size=")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 3L, 4L); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, 0L }, + new Object[] { null, null }, + new Object[] { 0L, 1L }, + new Object[] { 0L, null }, + new Object[] { null, 0L }, } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 5))); + } else { + assertThat(ordsAndKeys.description, equalTo("LongLongBlockHash{channels=[0,1], entries=2}")); + assertOrds(ordsAndKeys.ords, 0L, null, 1L, null, null); + assertKeys(ordsAndKeys.keys, new Object[][] { new Object[] { 1L, 0L }, new Object[] { 0L, 1L } }); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 2))); + } + } + + public void testLongBytesRefHash() { + long[] values1 = new long[] { 0, 1, 0, 1, 0, 1 }; + LongBlock block1 = new LongArrayVector(values1, values1.length).asBlock(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(8); + builder.appendBytesRef(new BytesRef("cat")); + builder.appendBytesRef(new BytesRef("cat")); + builder.appendBytesRef(new BytesRef("cat")); + builder.appendBytesRef(new BytesRef("dog")); + builder.appendBytesRef(new BytesRef("dog")); + builder.appendBytesRef(new BytesRef("dog")); + BytesRefBlock block2 = builder.build(); + Object[][] expectedKeys = { + new Object[] { 0L, "cat" }, + new Object[] { 1L, "cat" }, + new Object[] { 1L, "dog" }, + new Object[] { 0L, "dog" } }; + + OrdsAndKeys ordsAndKeys = hash(block1, block2); + assertThat( + ordsAndKeys.description, + startsWith( + forcePackedHash + ? "PackedValuesBlockHash{groups=[0:LONG, 1:BYTES_REF], entries=4, size=" + : "BytesRefLongBlockHash{keys=[BytesRefKey[channel=1], LongKey[channel=0]], entries=4, size=" + ) + ); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 0L, 2L, 3L, 2L); + assertKeys(ordsAndKeys.keys, expectedKeys); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 4))); + } + + public void testLongBytesRefHashWithNull() { + LongBlock.Builder b1 = LongBlock.newBlockBuilder(2); + BytesRefBlock.Builder b2 = BytesRefBlock.newBlockBuilder(2); + b1.appendLong(1); + b2.appendBytesRef(new BytesRef("cat")); + b1.appendNull(); + b2.appendNull(); + b1.appendLong(0); + b2.appendBytesRef(new BytesRef("dog")); + b1.appendLong(0); + b2.appendNull(); + b1.appendNull(); + b2.appendBytesRef(new BytesRef("vanish")); + + OrdsAndKeys ordsAndKeys = hash(b1.build(), b2.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:BYTES_REF], entries=5, size=")); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, 1L, 2L, 3L, 4L); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, "cat" }, + new Object[] { null, null }, + new Object[] { 0L, "dog" }, + new Object[] { 1L, null }, + new Object[] { null, "vanish" } } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 5))); + } else { + assertThat( + ordsAndKeys.description, + startsWith("BytesRefLongBlockHash{keys=[BytesRefKey[channel=1], LongKey[channel=0]], entries=2, size=") + ); + assertThat(ordsAndKeys.description, endsWith("b}")); + assertOrds(ordsAndKeys.ords, 0L, null, 1L, null, null); + assertKeys(ordsAndKeys.keys, new Object[][] { new Object[] { 1L, "cat" }, new Object[] { 0L, "dog" } }); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 2))); + } + } + + private void append(LongBlock.Builder b1, BytesRefBlock.Builder b2, long[] v1, String[] v2) { + if (v1 == null) { + b1.appendNull(); + } else if (v1.length == 1) { + b1.appendLong(v1[0]); + } else { + b1.beginPositionEntry(); + for (long v : v1) { + b1.appendLong(v); + } + b1.endPositionEntry(); + } + if (v2 == null) { + b2.appendNull(); + } else if (v2.length == 1) { + b2.appendBytesRef(new BytesRef(v2[0])); + } else { + b2.beginPositionEntry(); + for (String v : v2) { + b2.appendBytesRef(new BytesRef(v)); + } + b2.endPositionEntry(); + } + } + + public void testLongBytesRefHashWithMultiValuedFields() { + var b1 = LongBlock.newBlockBuilder(8); + var b2 = BytesRefBlock.newBlockBuilder(8); + append(b1, b2, new long[] { 1, 2 }, new String[] { "a", "b" }); + append(b1, b2, new long[] { 1, 2 }, new String[] { "a" }); + append(b1, b2, new long[] { 1 }, new String[] { "a", "b" }); + append(b1, b2, new long[] { 1 }, new String[] { "a" }); + append(b1, b2, null, new String[] { "a" }); + append(b1, b2, new long[] { 1 }, null); + append(b1, b2, new long[] { 1, 1, 1 }, new String[] { "a", "a", "a" }); + append(b1, b2, new long[] { 1, 1, 2, 2 }, new String[] { "a", "b", "b" }); + append(b1, b2, new long[] { 1, 2, 3 }, new String[] { "c", "c", "a" }); + + OrdsAndKeys ordsAndKeys = hash(b1.build(), b2.build()); + if (forcePackedHash) { + assertThat(ordsAndKeys.description, startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:BYTES_REF], entries=10, size=")); + assertOrds( + ordsAndKeys.ords, + new long[] { 0, 1, 2, 3 }, + new long[] { 0, 2 }, + new long[] { 0, 1 }, + new long[] { 0 }, + new long[] { 4 }, + new long[] { 5 }, + new long[] { 0 }, + new long[] { 0, 1, 2, 3 }, + new long[] { 6, 0, 7, 2, 8, 9 } + ); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, "a" }, + new Object[] { 1L, "b" }, + new Object[] { 2L, "a" }, + new Object[] { 2L, "b" }, + new Object[] { null, "a" }, + new Object[] { 1L, null }, + new Object[] { 1L, "c" }, + new Object[] { 2L, "c" }, + new Object[] { 3L, "c" }, + new Object[] { 3L, "a" }, } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 10))); + } else { + assertThat( + ordsAndKeys.description, + equalTo("BytesRefLongBlockHash{keys=[BytesRefKey[channel=1], LongKey[channel=0]], entries=8, size=491b}") + ); + assertOrds( + ordsAndKeys.ords, + new long[] { 0, 1, 2, 3 }, + new long[] { 0, 1 }, + new long[] { 0, 2 }, + new long[] { 0 }, + null, + null, + new long[] { 0 }, + new long[] { 0, 1, 2, 3 }, + new long[] { 4, 5, 6, 0, 1, 7 } + ); + assertKeys( + ordsAndKeys.keys, + new Object[][] { + new Object[] { 1L, "a" }, + new Object[] { 2L, "a" }, + new Object[] { 1L, "b" }, + new Object[] { 2L, "b" }, + new Object[] { 1L, "c" }, + new Object[] { 2L, "c" }, + new Object[] { 3L, "c" }, + new Object[] { 3L, "a" }, } + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, 8))); + } + } + + public void testBytesRefLongHashHugeCombinatorialExplosion() { + long[] v1 = LongStream.range(0, 3000).toArray(); + String[] v2 = LongStream.range(100, 200).mapToObj(l -> "a" + l).toArray(String[]::new); + + var b1 = LongBlock.newBlockBuilder(v1.length); + var b2 = BytesRefBlock.newBlockBuilder(v2.length); + append(b1, b2, v1, v2); + + int[] expectedEntries = new int[1]; + int pageSize = between(1000, 16 * 1024); + hash(ordsAndKeys -> { + int start = expectedEntries[0]; + expectedEntries[0] = Math.min(expectedEntries[0] + pageSize, v1.length * v2.length); + assertThat( + ordsAndKeys.description, + forcePackedHash + ? startsWith("PackedValuesBlockHash{groups=[0:LONG, 1:BYTES_REF], entries=" + expectedEntries[0] + ", size=") + : startsWith( + "BytesRefLongBlockHash{keys=[BytesRefKey[channel=1], LongKey[channel=0]], entries=" + expectedEntries[0] + ", size=" + ) + ); + assertOrds(ordsAndKeys.ords, LongStream.range(start, expectedEntries[0]).toArray()); + assertKeys( + ordsAndKeys.keys, + IntStream.range(0, expectedEntries[0]) + .mapToObj( + i -> forcePackedHash + ? new Object[] { v1[i / v2.length], v2[i % v2.length] } + : new Object[] { v1[i % v1.length], v2[i / v1.length] } + ) + .toArray(l -> new Object[l][]) + ); + assertThat(ordsAndKeys.nonEmpty, equalTo(IntVector.range(0, expectedEntries[0]))); + }, pageSize, b1.build(), b2.build()); + + assertThat("misconfigured test", expectedEntries[0], greaterThan(0)); + } + + record OrdsAndKeys(String description, int positionOffset, LongBlock ords, Block[] keys, IntVector nonEmpty) {} + + /** + * Hash some values into a single block of group ids. If the hash produces + * more than one block of group ids this will fail. + */ + private OrdsAndKeys hash(Block... values) { + OrdsAndKeys[] result = new OrdsAndKeys[1]; + hash(ordsAndKeys -> { + if (result[0] != null) { + throw new IllegalStateException("hash produced more than one block"); + } + result[0] = ordsAndKeys; + }, 16 * 1024, values); + return result[0]; + } + + private void hash(Consumer callback, int emitBatchSize, Block... values) { + List specs = new ArrayList<>(values.length); + for (int c = 0; c < values.length; c++) { + specs.add(new HashAggregationOperator.GroupSpec(c, values[c].elementType())); + } + MockBigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + try ( + BlockHash blockHash = forcePackedHash + ? new PackedValuesBlockHash(specs, bigArrays, emitBatchSize) + : BlockHash.build(specs, bigArrays, emitBatchSize) + ) { + hash(true, blockHash, callback, values); + } + } + + static void hash(boolean collectKeys, BlockHash blockHash, Consumer callback, Block... values) { + blockHash.add(new Page(values), new GroupingAggregatorFunction.AddInput() { + @Override + public void add(int positionOffset, LongBlock groupIds) { + OrdsAndKeys result = new OrdsAndKeys( + blockHash.toString(), + positionOffset, + groupIds, + collectKeys ? blockHash.getKeys() : null, + blockHash.nonEmpty() + ); + + Set allowedOrds = new HashSet<>(); + for (int p = 0; p < result.nonEmpty.getPositionCount(); p++) { + allowedOrds.add(Long.valueOf(result.nonEmpty.getInt(p))); + } + for (int p = 0; p < result.ords.getPositionCount(); p++) { + if (result.ords.isNull(p)) { + continue; + } + int start = result.ords.getFirstValueIndex(p); + int end = start + result.ords.getValueCount(p); + for (int i = start; i < end; i++) { + long ord = result.ords.getLong(i); + if (false == allowedOrds.contains(ord)) { + fail("ord is not allowed " + ord); + } + } + } + callback.accept(result); + } + + @Override + public void add(int positionOffset, LongVector groupIds) { + add(positionOffset, groupIds.asBlock()); + } + }); + } + + private void assertOrds(LongBlock ordsBlock, Long... expectedOrds) { + assertOrds(ordsBlock, Arrays.stream(expectedOrds).map(l -> l == null ? null : new long[] { l }).toArray(long[][]::new)); + } + + private void assertOrds(LongBlock ordsBlock, long[]... expectedOrds) { + assertEquals(expectedOrds.length, ordsBlock.getPositionCount()); + for (int p = 0; p < expectedOrds.length; p++) { + int start = ordsBlock.getFirstValueIndex(p); + int count = ordsBlock.getValueCount(p); + if (expectedOrds[p] == null) { + if (false == ordsBlock.isNull(p)) { + StringBuilder error = new StringBuilder(); + error.append(p); + error.append(": expected null but was ["); + for (int i = 0; i < count; i++) { + if (i != 0) { + error.append(", "); + } + error.append(ordsBlock.getLong(start + i)); + } + fail(error.append("]").toString()); + } + continue; + } + assertFalse(p + ": expected not null", ordsBlock.isNull(p)); + long[] actual = new long[count]; + for (int i = 0; i < count; i++) { + actual[i] = ordsBlock.getLong(start + i); + } + assertThat("position " + p, actual, equalTo(expectedOrds[p])); + } + } + + private void assertKeys(Block[] actualKeys, Object... expectedKeys) { + Object[][] flipped = new Object[expectedKeys.length][]; + for (int r = 0; r < flipped.length; r++) { + flipped[r] = new Object[] { expectedKeys[r] }; + } + assertKeys(actualKeys, flipped); + } + + private void assertKeys(Block[] actualKeys, Object[][] expectedKeys) { + for (int r = 0; r < expectedKeys.length; r++) { + assertThat(actualKeys, arrayWithSize(expectedKeys[r].length)); + } + for (int c = 0; c < actualKeys.length; c++) { + assertThat("block " + c, actualKeys[c].getPositionCount(), equalTo(expectedKeys.length)); + } + for (int r = 0; r < expectedKeys.length; r++) { + for (int c = 0; c < actualKeys.length; c++) { + if (expectedKeys[r][c] == null) { + assertThat("expected null", actualKeys[c].isNull(r), equalTo(true)); + return; + } + assertThat(actualKeys[c].isNull(r), equalTo(false)); + if (expectedKeys[r][c] instanceof Integer v) { + assertThat(((IntBlock) actualKeys[c]).getInt(r), equalTo(v)); + } else if (expectedKeys[r][c] instanceof Long v) { + assertThat(((LongBlock) actualKeys[c]).getLong(r), equalTo(v)); + } else if (expectedKeys[r][c] instanceof Double v) { + assertThat(((DoubleBlock) actualKeys[c]).getDouble(r), equalTo(v)); + } else if (expectedKeys[r][c] instanceof String v) { + assertThat(((BytesRefBlock) actualKeys[c]).getBytesRef(r, new BytesRef()), equalTo(new BytesRef(v))); + } else if (expectedKeys[r][c] instanceof Boolean v) { + assertThat(((BooleanBlock) actualKeys[c]).getBoolean(r), equalTo(v)); + } else { + throw new IllegalArgumentException("unsupported type " + expectedKeys[r][c].getClass()); + } + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java new file mode 100644 index 0000000000000..b3c0624496bde --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java @@ -0,0 +1,859 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class BasicBlockTests extends ESTestCase { + + public void testEmpty() { + assertThat( + new IntArrayBlock(new int[] {}, 0, new int[] {}, new BitSet(), randomFrom(Block.MvOrdering.values())).getPositionCount(), + is(0) + ); + assertThat(IntBlock.newBlockBuilder(0).build().getPositionCount(), is(0)); + assertThat(new IntArrayVector(new int[] {}, 0).getPositionCount(), is(0)); + assertThat(IntVector.newVectorBuilder(0).build().getPositionCount(), is(0)); + + assertThat( + new LongArrayBlock(new long[] {}, 0, new int[] {}, new BitSet(), randomFrom(Block.MvOrdering.values())).getPositionCount(), + is(0) + ); + assertThat(LongBlock.newBlockBuilder(0).build().getPositionCount(), is(0)); + assertThat(new LongArrayVector(new long[] {}, 0).getPositionCount(), is(0)); + assertThat(LongVector.newVectorBuilder(0).build().getPositionCount(), is(0)); + + assertThat( + new DoubleArrayBlock(new double[] {}, 0, new int[] {}, new BitSet(), randomFrom(Block.MvOrdering.values())).getPositionCount(), + is(0) + ); + assertThat(DoubleBlock.newBlockBuilder(0).build().getPositionCount(), is(0)); + assertThat(new DoubleArrayVector(new double[] {}, 0).getPositionCount(), is(0)); + assertThat(DoubleVector.newVectorBuilder(0).build().getPositionCount(), is(0)); + + var emptyArray = new BytesRefArray(0, BigArrays.NON_RECYCLING_INSTANCE); + assertThat( + new BytesRefArrayBlock(emptyArray, 0, new int[] {}, new BitSet(), randomFrom(Block.MvOrdering.values())).getPositionCount(), + is(0) + ); + assertThat(BytesRefBlock.newBlockBuilder(0).build().getPositionCount(), is(0)); + assertThat(new BytesRefArrayVector(emptyArray, 0).getPositionCount(), is(0)); + assertThat(BytesRefVector.newVectorBuilder(0).build().getPositionCount(), is(0)); + + assertThat( + new BooleanArrayBlock(new boolean[] {}, 0, new int[] {}, new BitSet(), randomFrom(Block.MvOrdering.values())) + .getPositionCount(), + is(0) + ); + assertThat(BooleanBlock.newBlockBuilder(0).build().getPositionCount(), is(0)); + assertThat(new BooleanArrayVector(new boolean[] {}, 0).getPositionCount(), is(0)); + assertThat(BooleanVector.newVectorBuilder(0).build().getPositionCount(), is(0)); + } + + public void testSmallSingleValueDenseGrowthInt() { + for (int initialSize : List.of(0, 1, 2, 3, 4, 5)) { + var blockBuilder = IntBlock.newBlockBuilder(initialSize); + IntStream.range(0, 10).forEach(blockBuilder::appendInt); + assertSingleValueDenseBlock(blockBuilder.build()); + } + } + + public void testSmallSingleValueDenseGrowthLong() { + for (int initialSize : List.of(0, 1, 2, 3, 4, 5)) { + var blockBuilder = LongBlock.newBlockBuilder(initialSize); + IntStream.range(0, 10).forEach(blockBuilder::appendLong); + assertSingleValueDenseBlock(blockBuilder.build()); + } + } + + public void testSmallSingleValueDenseGrowthDouble() { + for (int initialSize : List.of(0, 1, 2, 3, 4, 5)) { + var blockBuilder = DoubleBlock.newBlockBuilder(initialSize); + IntStream.range(0, 10).forEach(blockBuilder::appendDouble); + assertSingleValueDenseBlock(blockBuilder.build()); + } + } + + public void testSmallSingleValueDenseGrowthBytesRef() { + final BytesRef NULL_VALUE = new BytesRef(); + for (int initialSize : List.of(0, 1, 2, 3, 4, 5)) { + var blockBuilder = BytesRefBlock.newBlockBuilder(initialSize); + IntStream.range(0, 10).mapToObj(i -> NULL_VALUE).forEach(blockBuilder::appendBytesRef); + assertSingleValueDenseBlock(blockBuilder.build()); + } + } + + public void testSmallSingleValueDenseGrowthBoolean() { + for (int initialSize : List.of(0, 1, 2, 3, 4, 5)) { + var blockBuilder = BooleanBlock.newBlockBuilder(initialSize); + IntStream.range(0, 10).forEach(i -> blockBuilder.appendBoolean(i % 3 == 0)); + assertSingleValueDenseBlock(blockBuilder.build()); + } + } + + static void assertSingleValueDenseBlock(Block initialBlock) { + final int positionCount = initialBlock.getPositionCount(); + int depth = randomIntBetween(1, 5); + for (int d = 0; d < depth; d++) { + Block block = initialBlock; + assertThat(block.getTotalValueCount(), is(positionCount)); + assertThat(block.getPositionCount(), is(positionCount)); + for (int j = 0; j < 10; j++) { + int pos = randomPosition(positionCount); + assertThat(block.getFirstValueIndex(pos), is(pos)); + assertThat(block.getValueCount(pos), is(1)); + assertThat(block.isNull(pos), is(false)); + } + assertThat(block.asVector().getPositionCount(), is(positionCount)); + assertThat(block.asVector().asBlock().getTotalValueCount(), is(positionCount)); + assertThat(block.asVector().asBlock().getPositionCount(), is(positionCount)); + assertThat(block.nullValuesCount(), is(0)); + assertThat(block.mayHaveNulls(), is(false)); + assertThat(block.areAllValuesNull(), is(false)); + assertThat(block.mayHaveMultivaluedFields(), is(false)); + + initialBlock = block.asVector().asBlock(); + } + } + + public void testIntBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + IntBlock block; + if (randomBoolean()) { + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + IntBlock.Builder blockBuilder = IntBlock.newBlockBuilder(builderEstimateSize); + IntStream.range(0, positionCount).forEach(blockBuilder::appendInt); + block = blockBuilder.build(); + } else { + block = new IntArrayVector(IntStream.range(0, positionCount).toArray(), positionCount).asBlock(); + } + + assertThat(block.getPositionCount(), equalTo(positionCount)); + assertThat(block.getInt(0), equalTo(0)); + assertThat(block.getInt(positionCount - 1), equalTo(positionCount - 1)); + int pos = block.getInt(randomPosition(positionCount)); + assertThat(pos, is(block.getInt(pos))); + assertSingleValueDenseBlock(block); + + if (positionCount > 1) { + assertNullValues( + positionCount, + size -> IntBlock.newBlockBuilder(size), + (bb, value) -> bb.appendInt(value), + position -> position, + IntBlock.Builder::build, + (randomNonNullPosition, b) -> { + assertThat((int) randomNonNullPosition, is(b.getInt(randomNonNullPosition.intValue()))); + } + ); + } + + IntBlock.Builder blockBuilder = IntBlock.newBlockBuilder(1); + IntBlock copy = blockBuilder.copyFrom(block, 0, block.getPositionCount()).build(); + assertThat(copy, equalTo(block)); + + IntVector.Builder vectorBuilder = IntVector.newVectorBuilder( + randomBoolean() ? randomIntBetween(1, positionCount) : positionCount + ); + IntStream.range(0, positionCount).forEach(vectorBuilder::appendInt); + IntVector vector = vectorBuilder.build(); + assertSingleValueDenseBlock(vector.asBlock()); + } + } + + public void testConstantIntBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + int value = randomInt(); + IntBlock block; + if (randomBoolean()) { + block = IntBlock.newConstantBlockWith(value, positionCount); + } else { + block = new ConstantIntVector(value, positionCount).asBlock(); + } + assertThat(positionCount, is(block.getPositionCount())); + assertThat(value, is(block.getInt(0))); + assertThat(value, is(block.getInt(positionCount - 1))); + assertThat(value, is(block.getInt(randomPosition(positionCount)))); + assertThat(block.isNull(randomPosition(positionCount)), is(false)); + assertSingleValueDenseBlock(block); + } + } + + public void testLongBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + LongBlock block; + if (randomBoolean()) { + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + LongBlock.Builder blockBuilder = LongBlock.newBlockBuilder(builderEstimateSize); + LongStream.range(0, positionCount).forEach(blockBuilder::appendLong); + block = blockBuilder.build(); + } else { + block = new LongArrayVector(LongStream.range(0, positionCount).toArray(), positionCount).asBlock(); + } + + assertThat(positionCount, is(block.getPositionCount())); + assertThat(0L, is(block.getLong(0))); + assertThat((long) positionCount - 1, is(block.getLong(positionCount - 1))); + int pos = (int) block.getLong(randomPosition(positionCount)); + assertThat((long) pos, is(block.getLong(pos))); + assertSingleValueDenseBlock(block); + + if (positionCount > 1) { + assertNullValues( + positionCount, + size -> LongBlock.newBlockBuilder(size), + (bb, value) -> bb.appendLong(value), + position -> (long) position, + LongBlock.Builder::build, + (randomNonNullPosition, b) -> { + assertThat((long) randomNonNullPosition, is(b.getLong(randomNonNullPosition.intValue()))); + } + ); + } + + LongBlock.Builder blockBuilder = LongBlock.newBlockBuilder(1); + LongBlock copy = blockBuilder.copyFrom(block, 0, block.getPositionCount()).build(); + assertThat(copy, equalTo(block)); + + LongVector.Builder vectorBuilder = LongVector.newVectorBuilder( + randomBoolean() ? randomIntBetween(1, positionCount) : positionCount + ); + LongStream.range(0, positionCount).forEach(vectorBuilder::appendLong); + LongVector vector = vectorBuilder.build(); + assertSingleValueDenseBlock(vector.asBlock()); + } + } + + public void testConstantLongBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + long value = randomLong(); + LongBlock block; + if (randomBoolean()) { + block = LongBlock.newConstantBlockWith(value, positionCount); + } else { + block = new ConstantLongVector(value, positionCount).asBlock(); + } + assertThat(positionCount, is(block.getPositionCount())); + assertThat(value, is(block.getLong(0))); + assertThat(value, is(block.getLong(positionCount - 1))); + assertThat(value, is(block.getLong(randomPosition(positionCount)))); + assertThat(block.isNull(randomPosition(positionCount)), is(false)); + assertSingleValueDenseBlock(block); + } + } + + public void testDoubleBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + DoubleBlock block; + if (randomBoolean()) { + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = DoubleBlock.newBlockBuilder(builderEstimateSize); + LongStream.range(0, positionCount).asDoubleStream().forEach(blockBuilder::appendDouble); + block = blockBuilder.build(); + } else { + block = new DoubleArrayVector(LongStream.range(0, positionCount).asDoubleStream().toArray(), positionCount).asBlock(); + } + + assertThat(positionCount, is(block.getPositionCount())); + assertThat(0d, is(block.getDouble(0))); + assertThat((double) positionCount - 1, is(block.getDouble(positionCount - 1))); + int pos = (int) block.getDouble(randomPosition(positionCount)); + assertThat((double) pos, is(block.getDouble(pos))); + assertSingleValueDenseBlock(block); + + if (positionCount > 1) { + assertNullValues( + positionCount, + size -> DoubleBlock.newBlockBuilder(size), + (bb, value) -> bb.appendDouble(value), + position -> (double) position, + DoubleBlock.Builder::build, + (randomNonNullPosition, b) -> { + assertThat((double) randomNonNullPosition, is(b.getDouble(randomNonNullPosition.intValue()))); + } + ); + } + + DoubleBlock.Builder blockBuilder = DoubleBlock.newBlockBuilder(1); + DoubleBlock copy = blockBuilder.copyFrom(block, 0, block.getPositionCount()).build(); + assertThat(copy, equalTo(block)); + + DoubleVector.Builder vectorBuilder = DoubleVector.newVectorBuilder( + randomBoolean() ? randomIntBetween(1, positionCount) : positionCount + ); + IntStream.range(0, positionCount).mapToDouble(ii -> 1.0 / ii).forEach(vectorBuilder::appendDouble); + DoubleVector vector = vectorBuilder.build(); + assertSingleValueDenseBlock(vector.asBlock()); + } + } + + public void testConstantDoubleBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + double value = randomDouble(); + DoubleBlock block; + if (randomBoolean()) { + block = DoubleBlock.newConstantBlockWith(value, positionCount); + } else { + block = new ConstantDoubleVector(value, positionCount).asBlock(); + } + assertThat(positionCount, is(block.getPositionCount())); + assertThat(value, is(block.getDouble(0))); + assertThat(value, is(block.getDouble(positionCount - 1))); + assertThat(value, is(block.getDouble(randomPosition(positionCount)))); + assertSingleValueDenseBlock(block); + } + } + + public void testBytesRefBlock() { + int positionCount = randomIntBetween(1, 16 * 1024); + BytesRef[] values = new BytesRef[positionCount]; + for (int i = 0; i < positionCount; i++) { + BytesRef bytesRef = new BytesRef(randomByteArrayOfLength(between(1, 20))); + if (bytesRef.length > 0 && randomBoolean()) { + bytesRef.offset = randomIntBetween(0, bytesRef.length - 1); + bytesRef.length = randomIntBetween(0, bytesRef.length - bytesRef.offset); + } + values[i] = bytesRef; + } + + BytesRefBlock block; + if (randomBoolean()) { + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = BytesRefBlock.newBlockBuilder(builderEstimateSize); + Arrays.stream(values).map(obj -> randomBoolean() ? obj : BytesRef.deepCopyOf(obj)).forEach(blockBuilder::appendBytesRef); + block = blockBuilder.build(); + } else { + BytesRefArray array = new BytesRefArray(0, BigArrays.NON_RECYCLING_INSTANCE); + Arrays.stream(values).forEach(array::append); + block = new BytesRefArrayVector(array, positionCount).asBlock(); + } + + assertThat(positionCount, is(block.getPositionCount())); + BytesRef bytes = new BytesRef(); + for (int i = 0; i < positionCount; i++) { + int pos = randomIntBetween(0, positionCount - 1); + bytes = block.getBytesRef(pos, bytes); + assertThat(bytes, equalTo(values[pos])); + } + assertSingleValueDenseBlock(block); + + if (positionCount > 1) { + assertNullValues( + positionCount, + size -> BytesRefBlock.newBlockBuilder(size), + (bb, value) -> bb.appendBytesRef(value), + position -> values[position], + BytesRefBlock.Builder::build, + (randomNonNullPosition, b) -> assertThat( + values[randomNonNullPosition], + is(b.getBytesRef(randomNonNullPosition, new BytesRef())) + ) + ); + } + + BytesRefBlock.Builder blockBuilder = BytesRefBlock.newBlockBuilder(1); + BytesRefBlock copy = blockBuilder.copyFrom(block, 0, block.getPositionCount()).build(); + assertThat(copy, equalTo(block)); + + BytesRefVector.Builder vectorBuilder = BytesRefVector.newVectorBuilder( + randomBoolean() ? randomIntBetween(1, positionCount) : positionCount + ); + IntStream.range(0, positionCount).mapToObj(ii -> new BytesRef(randomAlphaOfLength(5))).forEach(vectorBuilder::appendBytesRef); + BytesRefVector vector = vectorBuilder.build(); + assertSingleValueDenseBlock(vector.asBlock()); + } + + public void testBytesRefBlockBuilderWithNulls() { + int positionCount = randomIntBetween(0, 16 * 1024); + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = BytesRefBlock.newBlockBuilder(builderEstimateSize); + BytesRef[] values = new BytesRef[positionCount]; + for (int i = 0; i < positionCount; i++) { + if (randomBoolean()) { + // Add random sparseness + blockBuilder.appendNull(); + values[i] = null; + } else { + BytesRef bytesRef = new BytesRef(randomByteArrayOfLength(between(1, 20))); + if (bytesRef.length > 0 && randomBoolean()) { + bytesRef.offset = randomIntBetween(0, bytesRef.length - 1); + bytesRef.length = randomIntBetween(0, bytesRef.length - bytesRef.offset); + } + values[i] = bytesRef; + if (randomBoolean()) { + bytesRef = BytesRef.deepCopyOf(bytesRef); + } + blockBuilder.appendBytesRef(bytesRef); + } + } + BytesRefBlock block = blockBuilder.build(); + assertThat(positionCount, is(block.getPositionCount())); + BytesRef bytes = new BytesRef(); + for (int i = 0; i < positionCount; i++) { + int pos = randomIntBetween(0, positionCount - 1); + bytes = block.getBytesRef(pos, bytes); + if (values[pos] == null) { + assertThat(block.isNull(pos), equalTo(true)); + assertThat(bytes, equalTo(new BytesRef())); + } else { + assertThat(bytes, equalTo(values[pos])); + assertThat(block.getBytesRef(pos, bytes), equalTo(values[pos])); + } + } + } + + public void testConstantBytesRefBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + BytesRef value = new BytesRef(randomByteArrayOfLength(between(1, 20))); + BytesRefBlock block; + if (randomBoolean()) { + block = BytesRefBlock.newConstantBlockWith(value, positionCount); + } else { + block = new ConstantBytesRefVector(value, positionCount).asBlock(); + } + assertThat(block.getPositionCount(), is(positionCount)); + + BytesRef bytes = new BytesRef(); + bytes = block.getBytesRef(0, bytes); + assertThat(bytes, is(value)); + bytes = block.getBytesRef(positionCount - 1, bytes); + assertThat(bytes, is(value)); + bytes = block.getBytesRef(randomPosition(positionCount), bytes); + assertThat(bytes, is(value)); + assertSingleValueDenseBlock(block); + } + } + + public void testBooleanBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + BooleanBlock block; + if (randomBoolean()) { + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = BooleanBlock.newBlockBuilder(builderEstimateSize); + IntStream.range(0, positionCount).forEach(p -> blockBuilder.appendBoolean(p % 10 == 0)); + block = blockBuilder.build(); + } else { + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + values[p] = p % 10 == 0; + } + block = new BooleanArrayVector(values, positionCount).asBlock(); + } + + assertThat(block.getPositionCount(), is(positionCount)); + assertThat(block.getBoolean(0), is(true)); + assertThat(block.getBoolean(positionCount - 1), is((positionCount - 1) % 10 == 0)); + assertSingleValueDenseBlock(block); + + if (positionCount > 1) { + assertNullValues( + positionCount, + BooleanBlock::newBlockBuilder, + (bb, value) -> bb.appendBoolean(value), + position -> position % 10 == 0, + BooleanBlock.Builder::build, + (randomNonNullPosition, b) -> { + assertThat(b.getBoolean(randomNonNullPosition.intValue()), is(randomNonNullPosition % 10 == 0)); + } + ); + } + + BooleanBlock.Builder blockBuilder = BooleanBlock.newBlockBuilder(1); + BooleanBlock copy = blockBuilder.copyFrom(block, 0, block.getPositionCount()).build(); + assertThat(copy, equalTo(block)); + + BooleanVector.Builder vectorBuilder = BooleanVector.newVectorBuilder( + randomBoolean() ? randomIntBetween(1, positionCount) : positionCount + ); + IntStream.range(0, positionCount).mapToObj(ii -> randomBoolean()).forEach(vectorBuilder::appendBoolean); + BooleanVector vector = vectorBuilder.build(); + assertSingleValueDenseBlock(vector.asBlock()); + } + } + + public void testConstantBooleanBlock() { + for (int i = 0; i < 1000; i++) { + int positionCount = randomIntBetween(1, 16 * 1024); + boolean value = randomBoolean(); + BooleanBlock block; + if (randomBoolean()) { + block = BooleanBlock.newConstantBlockWith(value, positionCount); + } else { + block = new ConstantBooleanVector(value, positionCount).asBlock(); + } + assertThat(positionCount, is(block.getPositionCount())); + assertThat(block.getBoolean(0), is(value)); + assertThat(block.getBoolean(positionCount - 1), is(value)); + assertThat(block.getBoolean(randomPosition(positionCount)), is(value)); + assertSingleValueDenseBlock(block); + } + } + + public void testSingleValueSparseInt() { + int positionCount = randomIntBetween(2, 16 * 1024); + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = IntBlock.newBlockBuilder(builderEstimateSize); + + int actualValueCount = 0; + int[] values = new int[positionCount]; + for (int i = 0; i < positionCount; i++) { + if (randomBoolean()) { + values[i] = randomInt(); + blockBuilder.appendInt(values[i]); + actualValueCount++; + } else { + blockBuilder.appendNull(); + } + } + IntBlock block = blockBuilder.build(); + + assertThat(block.getPositionCount(), is(positionCount)); + assertThat(block.getTotalValueCount(), is(actualValueCount)); + int nullCount = 0; + for (int i = 0; i < positionCount; i++) { + if (block.isNull(i)) { + nullCount++; + // assertThat(block.getInt(i), is(0)); // Q: do we wanna allow access to the default value + } else { + assertThat(block.getInt(i), is(values[i])); + } + } + assertThat(block.nullValuesCount(), is(nullCount)); + assertThat(block.asVector(), nullCount > 0 ? is(nullValue()) : is(notNullValue())); + } + + public void testSingleValueSparseLong() { + int positionCount = randomIntBetween(2, 16 * 1024); + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = LongBlock.newBlockBuilder(builderEstimateSize); + + int actualValueCount = 0; + long[] values = new long[positionCount]; + for (int i = 0; i < positionCount; i++) { + if (randomBoolean()) { + values[i] = randomLong(); + blockBuilder.appendLong(values[i]); + actualValueCount++; + } else { + blockBuilder.appendNull(); + } + } + LongBlock block = blockBuilder.build(); + + assertThat(block.getPositionCount(), is(positionCount)); + assertThat(block.getTotalValueCount(), is(actualValueCount)); + int nullCount = 0; + for (int i = 0; i < positionCount; i++) { + if (block.isNull(i)) { + nullCount++; + } else { + assertThat(block.getLong(i), is(values[i])); + } + } + assertThat(block.nullValuesCount(), is(nullCount)); + assertThat(block.asVector(), nullCount > 0 ? is(nullValue()) : is(notNullValue())); + } + + public void testSingleValueSparseDouble() { + int positionCount = randomIntBetween(2, 16 * 1024); + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = DoubleBlock.newBlockBuilder(builderEstimateSize); + + int actualValueCount = 0; + double[] values = new double[positionCount]; + for (int i = 0; i < positionCount; i++) { + if (randomBoolean()) { + values[i] = randomDouble(); + blockBuilder.appendDouble(values[i]); + actualValueCount++; + } else { + blockBuilder.appendNull(); + } + } + DoubleBlock block = blockBuilder.build(); + + assertThat(block.getPositionCount(), is(positionCount)); + assertThat(block.getTotalValueCount(), is(actualValueCount)); + int nullCount = 0; + for (int i = 0; i < positionCount; i++) { + if (block.isNull(i)) { + nullCount++; + } else { + assertThat(block.getDouble(i), is(values[i])); + } + } + assertThat(block.nullValuesCount(), is(nullCount)); + assertThat(block.asVector(), nullCount > 0 ? is(nullValue()) : is(notNullValue())); + } + + public void testSingleValueSparseBoolean() { + int positionCount = randomIntBetween(2, 16 * 1024); + final int builderEstimateSize = randomBoolean() ? randomIntBetween(1, positionCount) : positionCount; + var blockBuilder = BooleanBlock.newBlockBuilder(builderEstimateSize); + + boolean[] values = new boolean[positionCount]; + int actualValueCount = 0; + for (int i = 0; i < positionCount; i++) { + if (randomBoolean()) { + values[i] = randomBoolean(); + blockBuilder.appendBoolean(values[i]); + actualValueCount++; + } else { + blockBuilder.appendNull(); + } + } + BooleanBlock block = blockBuilder.build(); + + assertThat(block.getPositionCount(), is(positionCount)); + assertThat(block.getTotalValueCount(), is(actualValueCount)); + int nullCount = 0; + for (int i = 0; i < positionCount; i++) { + if (block.isNull(i)) { + nullCount++; + } else { + assertThat(block.getBoolean(i), is(values[i])); + } + } + assertThat(block.nullValuesCount(), is(nullCount)); + assertThat(block.asVector(), nullCount > 0 ? is(nullValue()) : is(notNullValue())); + } + + public void testToStringSmall() { + final int estimatedSize = randomIntBetween(1024, 4096); + + var boolBlock = BooleanBlock.newBlockBuilder(estimatedSize).appendBoolean(true).appendBoolean(false).build(); + var boolVector = BooleanVector.newVectorBuilder(estimatedSize).appendBoolean(true).appendBoolean(false).build(); + for (Object obj : List.of(boolVector, boolBlock, boolBlock.asVector())) { + String s = obj.toString(); + assertThat(s, containsString("[true, false]")); + assertThat(s, containsString("positions=2")); + } + + var intBlock = IntBlock.newBlockBuilder(estimatedSize).appendInt(1).appendInt(2).build(); + var intVector = IntVector.newVectorBuilder(estimatedSize).appendInt(1).appendInt(2).build(); + for (Object obj : List.of(intVector, intBlock, intBlock.asVector())) { + String s = obj.toString(); + assertThat(s, containsString("[1, 2]")); + assertThat(s, containsString("positions=2")); + } + for (IntBlock block : List.of(intBlock, intVector.asBlock())) { + assertThat(block.filter(0).toString(), containsString("FilterIntVector[positions=1, values=[1]]")); + assertThat(block.filter(1).toString(), containsString("FilterIntVector[positions=1, values=[2]]")); + assertThat(block.filter(0, 1).toString(), containsString("FilterIntVector[positions=2, values=[1, 2]]")); + assertThat(block.filter().toString(), containsString("FilterIntVector[positions=0, values=[]]")); + } + for (IntVector vector : List.of(intVector, intBlock.asVector())) { + assertThat(vector.filter(0).toString(), containsString("FilterIntVector[positions=1, values=[1]]")); + assertThat(vector.filter(1).toString(), containsString("FilterIntVector[positions=1, values=[2]]")); + assertThat(vector.filter(0, 1).toString(), containsString("FilterIntVector[positions=2, values=[1, 2]]")); + assertThat(vector.filter().toString(), containsString("FilterIntVector[positions=0, values=[]]")); + } + + var longBlock = LongBlock.newBlockBuilder(estimatedSize).appendLong(10L).appendLong(20L).build(); + var longVector = LongVector.newVectorBuilder(estimatedSize).appendLong(10L).appendLong(20L).build(); + for (Object obj : List.of(longVector, longBlock, longBlock.asVector())) { + String s = obj.toString(); + assertThat(s, containsString("[10, 20]")); + assertThat(s, containsString("positions=2")); + } + + var doubleBlock = DoubleBlock.newBlockBuilder(estimatedSize).appendDouble(3.3).appendDouble(4.4).build(); + var doubleVector = DoubleVector.newVectorBuilder(estimatedSize).appendDouble(3.3).appendDouble(4.4).build(); + for (Object obj : List.of(doubleVector, doubleBlock, doubleBlock.asVector())) { + String s = obj.toString(); + assertThat(s, containsString("[3.3, 4.4]")); + assertThat(s, containsString("positions=2")); + } + + assert new BytesRef("1a").toString().equals("[31 61]") && new BytesRef("2b").toString().equals("[32 62]"); + var bytesRefBlock = BytesRefBlock.newBlockBuilder(estimatedSize) + .appendBytesRef(new BytesRef("1a")) + .appendBytesRef(new BytesRef("2b")) + .build(); + var bytesRefVector = BytesRefVector.newVectorBuilder(estimatedSize) + .appendBytesRef(new BytesRef("1a")) + .appendBytesRef(new BytesRef("2b")) + .build(); + for (Object obj : List.of(bytesRefVector, bytesRefVector, bytesRefBlock.asVector())) { + String s = obj.toString(); + assertThat(s, containsString("positions=2")); + } + } + + public static List> valuesAtPositions(Block block, int from, int to) { + List> result = new ArrayList<>(to - from); + for (int p = from; p < to; p++) { + if (block.isNull(p)) { + result.add(null); + continue; + } + int count = block.getValueCount(p); + List positionValues = new ArrayList<>(count); + int i = block.getFirstValueIndex(p); + for (int v = 0; v < count; v++) { + positionValues.add(switch (block.elementType()) { + case INT -> ((IntBlock) block).getInt(i++); + case LONG -> ((LongBlock) block).getLong(i++); + case DOUBLE -> ((DoubleBlock) block).getDouble(i++); + case BYTES_REF -> ((BytesRefBlock) block).getBytesRef(i++, new BytesRef()); + case BOOLEAN -> ((BooleanBlock) block).getBoolean(i++); + default -> throw new IllegalArgumentException("unsupported element type [" + block.elementType() + "]"); + }); + } + result.add(positionValues); + } + return result; + } + + public record RandomBlock(List> values, Block block) { + int valueCount() { + return values.stream().mapToInt(l -> l == null ? 0 : l.size()).sum(); + } + } + + public static RandomBlock randomBlock( + ElementType elementType, + int positionCount, + boolean nullAllowed, + int minValuesPerPosition, + int maxValuesPerPosition, + int minDupsPerPosition, + int maxDupsPerPosition + ) { + List> values = new ArrayList<>(); + var builder = elementType.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = between(minValuesPerPosition, maxValuesPerPosition); + if (valueCount == 0 || nullAllowed && randomBoolean()) { + values.add(null); + builder.appendNull(); + continue; + } + int dupCount = between(minDupsPerPosition, maxDupsPerPosition); + if (valueCount != 1 || dupCount != 0) { + builder.beginPositionEntry(); + } + List valuesAtPosition = new ArrayList<>(); + values.add(valuesAtPosition); + for (int v = 0; v < valueCount; v++) { + switch (elementType) { + case INT -> { + int i = randomInt(); + valuesAtPosition.add(i); + ((IntBlock.Builder) builder).appendInt(i); + } + case LONG -> { + long l = randomLong(); + valuesAtPosition.add(l); + ((LongBlock.Builder) builder).appendLong(l); + } + case DOUBLE -> { + double d = randomDouble(); + valuesAtPosition.add(d); + ((DoubleBlock.Builder) builder).appendDouble(d); + } + case BYTES_REF -> { + BytesRef b = new BytesRef(randomRealisticUnicodeOfLength(4)); + valuesAtPosition.add(b); + ((BytesRefBlock.Builder) builder).appendBytesRef(b); + } + case BOOLEAN -> { + boolean b = randomBoolean(); + valuesAtPosition.add(b); + ((BooleanBlock.Builder) builder).appendBoolean(b); + } + default -> throw new IllegalArgumentException("unsupported element type [" + elementType + "]"); + } + } + for (int i = 0; i < dupCount; i++) { + BlockTestUtils.append(builder, randomFrom(valuesAtPosition)); + } + if (valueCount != 1 || dupCount != 0) { + builder.endPositionEntry(); + } + } + return new RandomBlock(values, builder.build()); + } + + interface BlockBuilderFactory { + B create(int estimatedSize); + } + + interface BlockProducer { + B build(BB blockBuilder); + } + + interface ValueAppender { + void appendValue(BB blockBuilder, T value); + } + + interface ValueSupplier { + T getValue(int position); + } + + private static void assertNullValues( + int positionCount, + BlockBuilderFactory blockBuilderFactory, + ValueAppender valueAppender, + ValueSupplier valueSupplier, + BlockProducer blockProducer, + BiConsumer asserter + ) { + assertThat("test needs at least two positions", positionCount, greaterThan(1)); + int randomNullPosition = randomIntBetween(0, positionCount - 1); + int randomNonNullPosition = randomValueOtherThan(randomNullPosition, () -> randomIntBetween(0, positionCount - 1)); + BitSet nullsMask = new BitSet(positionCount); + nullsMask.set(randomNullPosition); + + var blockBuilder = blockBuilderFactory.create(positionCount); + IntStream.range(0, positionCount).forEach(position -> { + if (nullsMask.get(position)) { + blockBuilder.appendNull(); + } else { + valueAppender.appendValue(blockBuilder, valueSupplier.getValue(position)); + } + }); + var block = blockProducer.build(blockBuilder); + + assertThat(block.getPositionCount(), equalTo(positionCount)); + assertThat(block.getTotalValueCount(), equalTo(positionCount - 1)); + asserter.accept(randomNonNullPosition, block); + assertTrue(block.isNull(randomNullPosition)); + assertFalse(block.isNull(randomNonNullPosition)); + } + + static int randomPosition(int positionCount) { + return positionCount == 1 ? 0 : randomIntBetween(0, positionCount - 1); + } +} 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 new file mode 100644 index 0000000000000..d79e99cb28225 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicPageTests.java @@ -0,0 +1,199 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.EqualsHashCodeTestUtils; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.is; + +public class BasicPageTests extends SerializationTestCase { + + static final Class NPE = NullPointerException.class; + static final Class IAE = IllegalArgumentException.class; + static final Class AE = AssertionError.class; + + public void testExceptions() { + expectThrows(NPE, () -> new Page((Block[]) null)); + + expectThrows(IAE, () -> new Page()); + expectThrows(IAE, () -> new Page(new Block[] {})); + + // Temporarily disable, until the intermediate state of grouping aggs is resolved. + // Intermediate state consists of a Page with two blocks: one of size N with the groups, the + // other has a single entry containing the serialized binary state. + // expectThrows(AE, () -> new Page(new Block[] { new IntArrayBlock(new int[] { 1, 2 }, 2), new ConstantIntBlock(1, 1) })); + } + + public void testEqualityAndHashCodeSmallInput() { + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(0, new Block[] {}), + page -> new Page(0, new Block[] {}), + page -> new Page(1, IntBlock.newConstantBlockWith(1, 1)) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(new int[] {}, 0).asBlock()), + page -> new Page(new IntArrayVector(new int[] {}, 0).asBlock()), + page -> new Page(new IntArrayVector(new int[] { 1 }, 1).asBlock()) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(new int[] { 1 }, 0).asBlock()), + page -> new Page(new IntArrayVector(new int[] { 1 }, 0).asBlock()), + page -> new Page(new IntArrayVector(new int[] { 1 }, 1).asBlock()) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(new int[] { 1, 1, 1 }, 3).asBlock()), + page -> new Page(IntBlock.newConstantBlockWith(1, 3)), + page -> new Page(IntBlock.newConstantBlockWith(1, 2)) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(IntStream.range(0, 10).toArray(), 10).asBlock()), + page -> new Page(new IntArrayVector(IntStream.range(0, 10).toArray(), 10).asBlock()), + page -> new Page(new IntArrayVector(IntStream.range(0, 10).toArray(), 9).asBlock()) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(IntStream.range(0, 100).toArray(), 100).asBlock()), + page -> new Page(new IntArrayVector(IntStream.range(0, 100).toArray(), 100).asBlock()), + page -> new Page(new LongArrayVector(LongStream.range(0, 100).toArray(), 100).asBlock()) + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + new Page(new IntArrayVector(new int[] { 1 }, 1).asBlock()), + page -> new Page(1, page.getBlock(0)), + page -> new Page(new IntArrayVector(new int[] { 1 }, 1).asBlock(), new IntArrayVector(new int[] { 1 }, 1).asBlock()) + ); + } + + public void testEqualityAndHashCode() throws IOException { + final EqualsHashCodeTestUtils.CopyFunction copyPageFunction = page -> { + Block[] blocks = new Block[page.getBlockCount()]; + for (int blockIndex = 0; blockIndex < blocks.length; blockIndex++) { + blocks[blockIndex] = page.getBlock(blockIndex); + } + return new Page(page.getPositionCount(), blocks); + }; + + final EqualsHashCodeTestUtils.MutateFunction mutatePageFunction = page -> { + assert page.getPositionCount() > 0; + Block[] blocks = new Block[page.getBlockCount()]; + int positions = randomInt(page.getPositionCount() - 1); + for (int blockIndex = 0; blockIndex < blocks.length; blockIndex++) { + Block block = page.getBlock(blockIndex); + blocks[blockIndex] = block.elementType().newBlockBuilder(positions).copyFrom(block, 0, page.getPositionCount() - 1).build(); + } + return new Page(blocks); + }; + + int positions = randomIntBetween(1, 512); + int blockCount = randomIntBetween(1, 256); + Block[] blocks = new Block[blockCount]; + for (int blockIndex = 0; blockIndex < blockCount; blockIndex++) { + blocks[blockIndex] = switch (randomInt(6)) { + case 0 -> new IntArrayVector(randomInts(positions).toArray(), positions).asBlock(); + case 1 -> new LongArrayVector(randomLongs(positions).toArray(), positions).asBlock(); + case 2 -> new DoubleArrayVector(randomDoubles(positions).toArray(), positions).asBlock(); + case 3 -> IntBlock.newConstantBlockWith(randomInt(), positions); + case 4 -> LongBlock.newConstantBlockWith(randomLong(), positions); + case 5 -> DoubleBlock.newConstantBlockWith(randomDouble(), positions); + case 6 -> BytesRefBlock.newConstantBlockWith(new BytesRef(Integer.toHexString(randomInt())), positions); + default -> throw new AssertionError(); + }; + } + Page page = new Page(positions, blocks); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(page, copyPageFunction, mutatePageFunction); + + EqualsHashCodeTestUtils.checkEqualsAndHashCode(page, unused -> serializeDeserializePage(page)); + } + + public void testBasic() { + int positions = randomInt(1024); + Page page = new Page(new IntArrayVector(IntStream.range(0, positions).toArray(), positions).asBlock()); + assertThat(1, is(page.getBlockCount())); + assertThat(positions, is(page.getPositionCount())); + IntBlock block = page.getBlock(0); + IntStream.range(0, positions).forEach(i -> assertThat(i, is(block.getInt(i)))); + } + + public void testAppend() { + Page page1 = new Page(new IntArrayVector(IntStream.range(0, 10).toArray(), 10).asBlock()); + Page page2 = page1.appendBlock(new LongArrayVector(LongStream.range(0, 10).toArray(), 10).asBlock()); + assertThat(1, is(page1.getBlockCount())); + assertThat(2, is(page2.getBlockCount())); + IntBlock block1 = page2.getBlock(0); + IntStream.range(0, 10).forEach(i -> assertThat(i, is(block1.getInt(i)))); + LongBlock block2 = page2.getBlock(1); + IntStream.range(0, 10).forEach(i -> assertThat((long) i, is(block2.getLong(i)))); + } + + public void testPageSerializationSimple() throws IOException { + try (var bytesRefArray = bytesRefArrayOf("0a", "1b", "2c", "3d", "4e", "5f", "6g", "7h", "8i", "9j")) { + final BytesStreamOutput out = new BytesStreamOutput(); + Page origPage = new Page( + new IntArrayVector(IntStream.range(0, 10).toArray(), 10).asBlock(), + new LongArrayVector(LongStream.range(10, 20).toArray(), 10).asBlock(), + new DoubleArrayVector(LongStream.range(30, 40).mapToDouble(i -> i).toArray(), 10).asBlock(), + new BytesRefArrayVector(bytesRefArray, 10).asBlock(), + IntBlock.newConstantBlockWith(randomInt(), 10), + LongBlock.newConstantBlockWith(randomInt(), 10), + DoubleBlock.newConstantBlockWith(randomInt(), 10), + BytesRefBlock.newConstantBlockWith(new BytesRef(Integer.toHexString(randomInt())), 10), + new IntArrayVector(IntStream.range(0, 20).toArray(), 20).filter(5, 6, 7, 8, 9, 10, 11, 12, 13, 14).asBlock() + ); + Page deserPage = serializeDeserializePage(origPage); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origPage, unused -> deserPage); + + for (int i = 0; i < origPage.getBlockCount(); i++) { + Vector vector = origPage.getBlock(i).asVector(); + if (vector != null) { + assertEquals(vector.isConstant(), deserPage.getBlock(i).asVector().isConstant()); + } + } + } + } + + public void testSerializationListPages() throws IOException { + final int positions = randomIntBetween(1, 64); + List origPages = List.of( + new Page(new IntArrayVector(randomInts(positions).toArray(), positions).asBlock()), + new Page( + new LongArrayVector(randomLongs(positions).toArray(), positions).asBlock(), + DoubleBlock.newConstantBlockWith(randomInt(), positions) + ), + new Page(BytesRefBlock.newConstantBlockWith(new BytesRef("Hello World"), positions)) + ); + final BytesStreamOutput out = new BytesStreamOutput(); + out.writeList(origPages); + StreamInput in = new NamedWriteableAwareStreamInput(ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), registry); + + List deserPages = in.readList(new Page.PageReader()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origPages, unused -> deserPages); + } + + final BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + + BytesRefArray bytesRefArrayOf(String... values) { + var array = new BytesRefArray(values.length, bigArrays); + Arrays.stream(values).map(BytesRef::new).forEach(array::append); + return array; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BigArrayVectorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BigArrayVectorTests.java new file mode 100644 index 0000000000000..66f62a2052689 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BigArrayVectorTests.java @@ -0,0 +1,149 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.common.util.IntArray; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.EqualsHashCodeTestUtils; + +import java.io.IOException; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class BigArrayVectorTests extends SerializationTestCase { + + final MockBigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + + public void testBoolean() throws IOException { + int positionCount = randomIntBetween(1, 16 * 1024); + Boolean[] values = IntStream.range(0, positionCount).mapToObj(i -> randomBoolean()).toArray(Boolean[]::new); + BitArray array = new BitArray(positionCount, bigArrays); + IntStream.range(0, positionCount).filter(i -> values[i]).forEach(array::set); + try (var vector = new BooleanBigArrayVector(array, positionCount)) { + assertThat(vector.elementType(), is(ElementType.BOOLEAN)); + assertThat(positionCount, is(vector.getPositionCount())); + IntStream.range(0, positionCount).forEach(i -> assertThat(vector.getBoolean(i), is(values[i]))); + assertThat(vector.isConstant(), is(false)); + BooleanVector filtered = vector.filter(IntStream.range(0, positionCount).toArray()); + IntStream.range(0, positionCount).forEach(i -> assertThat(filtered.getBoolean(i), is(values[i]))); + assertThat(filtered.isConstant(), is(false)); + BooleanBlock block = vector.asBlock(); + assertThat(block, is(notNullValue())); + IntStream.range(0, positionCount).forEach(i -> { + assertThat(block.getBoolean(i), is(values[i])); + assertThat(block.isNull(i), is(false)); + assertThat(block.getValueCount(i), is(1)); + assertThat(block.getFirstValueIndex(i), is(i)); + assertThat(block.filter(i).getBoolean(0), is(values[i])); + }); + BasicBlockTests.assertSingleValueDenseBlock(vector.asBlock()); + assertSerialization(block); + assertThat(vector.toString(), containsString("BooleanBigArrayVector[positions=" + positionCount)); + } + } + + public void testInt() throws IOException { + int positionCount = randomIntBetween(1, 16 * 1024); + int[] values = IntStream.range(0, positionCount).map(i -> randomInt()).toArray(); + IntArray array = bigArrays.newIntArray(positionCount); + IntStream.range(0, positionCount).forEach(i -> array.set(i, values[i])); + try (var vector = new IntBigArrayVector(array, positionCount)) { + assertThat(vector.elementType(), is(ElementType.INT)); + assertThat(positionCount, is(vector.getPositionCount())); + IntStream.range(0, positionCount).forEach(i -> assertThat(vector.getInt(i), is(values[i]))); + assertThat(vector.isConstant(), is(false)); + IntVector filtered = vector.filter(IntStream.range(0, positionCount).toArray()); + IntStream.range(0, positionCount).forEach(i -> assertThat(filtered.getInt(i), is(values[i]))); + assertThat(filtered.isConstant(), is(false)); + IntBlock block = vector.asBlock(); + assertThat(block, is(notNullValue())); + IntStream.range(0, positionCount).forEach(i -> { + assertThat(block.getInt(i), is(values[i])); + assertThat(block.isNull(i), is(false)); + assertThat(block.getValueCount(i), is(1)); + assertThat(block.getFirstValueIndex(i), is(i)); + assertThat(block.filter(i).getInt(0), is(values[i])); + }); + BasicBlockTests.assertSingleValueDenseBlock(vector.asBlock()); + assertSerialization(block); + assertThat(vector.toString(), containsString("IntBigArrayVector[positions=" + positionCount)); + } + } + + public void testLong() throws IOException { + int positionCount = randomIntBetween(1, 16 * 1024); + long[] values = IntStream.range(0, positionCount).mapToLong(i -> randomLong()).toArray(); + LongArray array = bigArrays.newLongArray(positionCount); + IntStream.range(0, positionCount).forEach(i -> array.set(i, values[i])); + try (var vector = new LongBigArrayVector(array, positionCount)) { + assertThat(vector.elementType(), is(ElementType.LONG)); + assertThat(positionCount, is(vector.getPositionCount())); + IntStream.range(0, positionCount).forEach(i -> assertThat(vector.getLong(i), is(values[i]))); + assertThat(vector.isConstant(), is(false)); + LongVector filtered = vector.filter(IntStream.range(0, positionCount).toArray()); + IntStream.range(0, positionCount).forEach(i -> assertThat(filtered.getLong(i), is(values[i]))); + assertThat(filtered.isConstant(), is(false)); + LongBlock block = vector.asBlock(); + assertThat(block, is(notNullValue())); + IntStream.range(0, positionCount).forEach(i -> { + assertThat(block.getLong(i), is(values[i])); + assertThat(block.isNull(i), is(false)); + assertThat(block.getValueCount(i), is(1)); + assertThat(block.getFirstValueIndex(i), is(i)); + assertThat(block.filter(i).getLong(0), is(values[i])); + }); + BasicBlockTests.assertSingleValueDenseBlock(vector.asBlock()); + assertSerialization(block); + assertThat(vector.toString(), containsString("LongBigArrayVector[positions=" + positionCount)); + } + } + + public void testDouble() throws IOException { + int positionCount = randomIntBetween(1, 16 * 1024); + double[] values = IntStream.range(0, positionCount).mapToDouble(i -> randomDouble()).toArray(); + DoubleArray array = bigArrays.newDoubleArray(positionCount); + IntStream.range(0, positionCount).forEach(i -> array.set(i, values[i])); + try (var vector = new DoubleBigArrayVector(array, positionCount)) { + assertThat(vector.elementType(), is(ElementType.DOUBLE)); + assertThat(positionCount, is(vector.getPositionCount())); + IntStream.range(0, positionCount).forEach(i -> assertThat(vector.getDouble(i), is(values[i]))); + assertThat(vector.isConstant(), is(false)); + DoubleVector filtered = vector.filter(IntStream.range(0, positionCount).toArray()); + IntStream.range(0, positionCount).forEach(i -> assertThat(filtered.getDouble(i), is(values[i]))); + assertThat(filtered.isConstant(), is(false)); + DoubleBlock block = vector.asBlock(); + assertThat(block, is(notNullValue())); + IntStream.range(0, positionCount).forEach(i -> { + assertThat(block.getDouble(i), is(values[i])); + assertThat(block.isNull(i), is(false)); + assertThat(block.getValueCount(i), is(1)); + assertThat(block.getFirstValueIndex(i), is(i)); + assertThat(block.filter(i).getDouble(0), is(values[i])); + }); + BasicBlockTests.assertSingleValueDenseBlock(vector.asBlock()); + assertSerialization(block); + assertThat(vector.toString(), containsString("DoubleBigArrayVector[positions=" + positionCount)); + } + } + + void assertSerialization(Block origBlock) throws IOException { + Block deserBlock = serializeDeserializeBlock(origBlock); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock, unused -> deserBlock); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock.asVector(), unused -> deserBlock.asVector()); + assertThat(deserBlock.asVector(), is(origBlock.asVector())); + assertThat(deserBlock.asVector().isConstant(), is(origBlock.asVector().isConstant())); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderAppendBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderAppendBlockTests.java new file mode 100644 index 0000000000000..1feabec374170 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderAppendBlockTests.java @@ -0,0 +1,132 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class BlockBuilderAppendBlockTests extends ESTestCase { + + public void testBasic() { + IntBlock src = new IntBlockBuilder(10).appendInt(1) + .appendNull() + .beginPositionEntry() + .appendInt(4) + .appendInt(6) + .endPositionEntry() + .appendInt(10) + .appendInt(20) + .appendInt(30) + .appendNull() + .beginPositionEntry() + .appendInt(1) + .endPositionEntry() + .build(); + // copy position by position + { + IntBlock.Builder dst = IntBlock.newBlockBuilder(randomIntBetween(1, 20)); + for (int i = 0; i < src.getPositionCount(); i++) { + dst.appendAllValuesToCurrentPosition(src.filter(i)); + } + assertThat(dst.build(), equalTo(src)); + } + // copy all block + { + IntBlock.Builder dst = IntBlock.newBlockBuilder(randomIntBetween(1, 20)); + IntBlock block = dst.appendAllValuesToCurrentPosition(src).build(); + assertThat(block.getPositionCount(), equalTo(1)); + assertThat(BlockUtils.toJavaObject(block, 0), equalTo(List.of(1, 4, 6, 10, 20, 30, 1))); + } + { + Block dst = randomlyDivideAndMerge(src); + assertThat(dst.getPositionCount(), equalTo(1)); + assertThat(BlockUtils.toJavaObject(dst, 0), equalTo(List.of(1, 4, 6, 10, 20, 30, 1))); + } + } + + public void testRandomNullBlock() { + IntBlock.Builder src = IntBlock.newBlockBuilder(10); + src.appendAllValuesToCurrentPosition(new ConstantNullBlock(between(1, 100))); + src.appendInt(101); + src.appendAllValuesToCurrentPosition(new ConstantNullBlock(between(1, 100))); + IntBlock block = src.build(); + assertThat(block.getPositionCount(), equalTo(3)); + assertTrue(block.isNull(0)); + assertThat(block.getInt(1), equalTo(101)); + assertTrue(block.isNull(2)); + Block flatten = randomlyDivideAndMerge(block); + assertThat(flatten.getPositionCount(), equalTo(1)); + assertThat(BlockUtils.toJavaObject(flatten, 0), equalTo(101)); + } + + public void testRandom() { + ElementType elementType = randomFrom(ElementType.INT, ElementType.BYTES_REF, ElementType.BOOLEAN); + Block block = BasicBlockTests.randomBlock( + elementType, + randomIntBetween(1, 1024), + randomBoolean(), + 0, + between(1, 16), + 0, + between(0, 16) + ).block(); + randomlyDivideAndMerge(block); + } + + private Block randomlyDivideAndMerge(Block block) { + while (block.getPositionCount() > 1 || randomBoolean()) { + int positionCount = block.getPositionCount(); + int offset = 0; + Block.Builder builder = block.elementType().newBlockBuilder(randomIntBetween(1, 100)); + List expected = new ArrayList<>(); + while (offset < positionCount) { + int length = randomIntBetween(1, positionCount - offset); + int[] positions = new int[length]; + for (int i = 0; i < length; i++) { + positions[i] = offset + i; + } + offset += length; + Block sub = block.filter(positions); + expected.add(extractAndFlattenBlockValues(sub)); + builder.appendAllValuesToCurrentPosition(sub); + } + block = builder.build(); + assertThat(block.getPositionCount(), equalTo(expected.size())); + for (int i = 0; i < block.getPositionCount(); i++) { + assertThat(BlockUtils.toJavaObject(block, i), equalTo(expected.get(i))); + } + } + return block; + } + + static Object extractAndFlattenBlockValues(Block block) { + List values = new ArrayList<>(); + for (int i = 0; i < block.getPositionCount(); i++) { + Object v = BlockUtils.toJavaObject(block, i); + if (v == null) { + continue; + } + if (v instanceof List l) { + values.addAll(l); + } else { + values.add(v); + } + } + if (values.isEmpty()) { + return null; + } else if (values.size() == 1) { + return values.get(0); + } else { + return values; + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java new file mode 100644 index 0000000000000..529c1afeaaf44 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java @@ -0,0 +1,107 @@ +/* + * 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.compute.data; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.elasticsearch.compute.data.BlockValueAsserter.assertBlockValues; + +public class BlockBuilderCopyFromTests extends ESTestCase { + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + for (ElementType elementType : ElementType.values()) { + if (elementType == ElementType.UNKNOWN || elementType == ElementType.NULL || elementType == ElementType.DOC) { + continue; + } + for (boolean nullAllowed : new boolean[] { false, true }) { + for (int[] valuesPerPosition : new int[][] { new int[] { 1, 1 }, new int[] { 1, 10 } }) { // TODO 0 + params.add(new Object[] { elementType, nullAllowed, valuesPerPosition[0], valuesPerPosition[1] }); + } + } + } + return params; + } + + private final ElementType elementType; + private final boolean nullAllowed; + private final int minValuesPerPosition; + private final int maxValuesPerPosition; + + public BlockBuilderCopyFromTests( + @Name("elementType") ElementType elementType, + @Name("nullAllowed") boolean nullAllowed, + @Name("minValuesPerPosition") int minValuesPerPosition, + @Name("maxValuesPerPosition") int maxValuesPerPosition + ) { + this.elementType = elementType; + this.nullAllowed = nullAllowed; + this.minValuesPerPosition = minValuesPerPosition; + this.maxValuesPerPosition = maxValuesPerPosition; + } + + public void testSmall() { + assertSmall(randomBlock()); + } + + public void testEvens() { + assertEvens(randomBlock()); + } + + public void testSmallFiltered() { + assertSmall(randomFilteredBlock()); + } + + public void testEvensFiltered() { + assertEvens(randomFilteredBlock()); + } + + public void testSmallAllNull() { + assertSmall(Block.constantNullBlock(10)); + } + + public void testEvensAllNull() { + assertEvens(Block.constantNullBlock(10)); + } + + private void assertSmall(Block block) { + int smallSize = Math.min(block.getPositionCount(), 10); + Block.Builder builder = elementType.newBlockBuilder(smallSize); + builder.copyFrom(block, 0, smallSize); + assertBlockValues(builder.build(), BasicBlockTests.valuesAtPositions(block, 0, smallSize)); + } + + private void assertEvens(Block block) { + Block.Builder builder = elementType.newBlockBuilder(block.getPositionCount() / 2); + List> expected = new ArrayList<>(); + for (int i = 0; i < block.getPositionCount(); i += 2) { + builder.copyFrom(block, i, i + 1); + expected.add(BasicBlockTests.valuesAtPositions(block, i, i + 1).get(0)); + } + assertBlockValues(builder.build(), expected); + } + + private Block randomBlock() { + int positionCount = randomIntBetween(1, 16 * 1024); + return BasicBlockTests.randomBlock(elementType, positionCount, nullAllowed, minValuesPerPosition, maxValuesPerPosition, 0, 0) + .block(); + } + + private Block randomFilteredBlock() { + int keepers = between(0, 4); + Block orig = randomBlock(); + return orig.filter(IntStream.range(0, orig.getPositionCount()).filter(i -> i % 5 == keepers).toArray()); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java new file mode 100644 index 0000000000000..de552d242afa2 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java @@ -0,0 +1,68 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.List; + +import static org.hamcrest.Matchers.is; + +public class BlockBuilderTests extends ESTestCase { + + public void testAllNullsInt() { + for (int numEntries : List.of(1, randomIntBetween(1, 100))) { + testAllNullsImpl(IntBlock.newBlockBuilder(0), numEntries); + testAllNullsImpl(IntBlock.newBlockBuilder(100), numEntries); + testAllNullsImpl(IntBlock.newBlockBuilder(1000), numEntries); + testAllNullsImpl(IntBlock.newBlockBuilder(randomIntBetween(0, 100)), numEntries); + } + } + + public void testAllNullsLong() { + for (int numEntries : List.of(1, randomIntBetween(1, 100))) { + testAllNullsImpl(LongBlock.newBlockBuilder(0), numEntries); + testAllNullsImpl(LongBlock.newBlockBuilder(100), numEntries); + testAllNullsImpl(LongBlock.newBlockBuilder(1000), numEntries); + testAllNullsImpl(LongBlock.newBlockBuilder(randomIntBetween(0, 100)), numEntries); + } + } + + public void testAllNullsDouble() { + for (int numEntries : List.of(1, randomIntBetween(1, 100))) { + testAllNullsImpl(DoubleBlock.newBlockBuilder(0), numEntries); + testAllNullsImpl(DoubleBlock.newBlockBuilder(100), numEntries); + testAllNullsImpl(DoubleBlock.newBlockBuilder(1000), numEntries); + testAllNullsImpl(DoubleBlock.newBlockBuilder(randomIntBetween(0, 100)), numEntries); + } + } + + public void testAllNullsBytesRef() { + for (int numEntries : List.of(1, randomIntBetween(1, 100))) { + testAllNullsImpl(BytesRefBlock.newBlockBuilder(0), numEntries); + testAllNullsImpl(BytesRefBlock.newBlockBuilder(100), numEntries); + testAllNullsImpl(BytesRefBlock.newBlockBuilder(1000), numEntries); + testAllNullsImpl(BytesRefBlock.newBlockBuilder(randomIntBetween(0, 100)), numEntries); + } + } + + private void testAllNullsImpl(Block.Builder builder, int numEntries) { + for (int i = 0; i < numEntries; i++) { + builder.appendNull(); + } + Block block = builder.build(); + assertThat(block.getPositionCount(), is(numEntries)); + assertThat(block.isNull(0), is(true)); + assertThat(block.isNull(numEntries - 1), is(true)); + assertThat(block.isNull(randomPosition(numEntries)), is(true)); + } + + static int randomPosition(int positionCount) { + return positionCount == 1 ? 0 : randomIntBetween(0, positionCount - 1); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockMultiValuedTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockMultiValuedTests.java new file mode 100644 index 0000000000000..9a362ad4e3ca3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockMultiValuedTests.java @@ -0,0 +1,164 @@ +/* + * 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.compute.data; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.equalTo; + +public class BlockMultiValuedTests extends ESTestCase { + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + for (ElementType elementType : ElementType.values()) { + if (elementType == ElementType.UNKNOWN || elementType == ElementType.NULL || elementType == ElementType.DOC) { + continue; + } + for (boolean nullAllowed : new boolean[] { false, true }) { + params.add(new Object[] { elementType, nullAllowed }); + } + } + return params; + } + + private final ElementType elementType; + private final boolean nullAllowed; + + public BlockMultiValuedTests(@Name("elementType") ElementType elementType, @Name("nullAllowed") boolean nullAllowed) { + this.elementType = elementType; + this.nullAllowed = nullAllowed; + } + + public void testMultiValued() { + int positionCount = randomIntBetween(1, 16 * 1024); + var b = BasicBlockTests.randomBlock(elementType, positionCount, nullAllowed, 0, 10, 0, 0); + + assertThat(b.block().getPositionCount(), equalTo(positionCount)); + assertThat(b.block().getTotalValueCount(), equalTo(b.valueCount())); + for (int p = 0; p < positionCount; p++) { + BlockTestUtils.assertPositionValues(b.block(), p, equalTo(b.values().get(p))); + } + + assertThat(b.block().mayHaveMultivaluedFields(), equalTo(b.values().stream().anyMatch(l -> l != null && l.size() > 1))); + } + + public void testExpand() { + int positionCount = randomIntBetween(1, 16 * 1024); + var b = BasicBlockTests.randomBlock(elementType, positionCount, nullAllowed, 0, 100, 0, 0); + assertExpanded(b.block()); + } + + public void testFilteredNoop() { + assertFiltered(true, false); + } + + public void testFilteredReordered() { + assertFiltered(true, true); + } + + public void testFilteredSubset() { + assertFiltered(false, false); + } + + public void testFilteredJumbledSubset() { + assertFiltered(false, true); + } + + public void testFilteredNoopThenExpanded() { + assertFilteredThenExpanded(true, false); + } + + public void testFilteredReorderedThenExpanded() { + assertFilteredThenExpanded(true, true); + } + + public void testFilteredSubsetThenExpanded() { + assertFilteredThenExpanded(false, false); + } + + public void testFilteredJumbledSubsetThenExpanded() { + assertFilteredThenExpanded(false, true); + } + + private void assertFiltered(boolean all, boolean shuffled) { + int positionCount = randomIntBetween(1, 16 * 1024); + var b = BasicBlockTests.randomBlock(elementType, positionCount, nullAllowed, 0, 10, 0, 0); + int[] positions = randomFilterPositions(b.block(), all, shuffled); + Block filtered = b.block().filter(positions); + + assertThat(filtered.getPositionCount(), equalTo(positions.length)); + + int expectedValueCount = 0; + for (int p : positions) { + List values = b.values().get(p); + if (values != null) { + expectedValueCount += values.size(); + } + } + assertThat(filtered.getTotalValueCount(), equalTo(expectedValueCount)); + for (int r = 0; r < positions.length; r++) { + if (b.values().get(positions[r]) == null) { + assertThat(filtered.getValueCount(r), equalTo(0)); + assertThat(filtered.isNull(r), equalTo(true)); + } else { + assertThat(filtered.getValueCount(r), equalTo(b.values().get(positions[r]).size())); + assertThat(BasicBlockTests.valuesAtPositions(filtered, r, r + 1).get(0), equalTo(b.values().get(positions[r]))); + } + } + + assertThat(b.block().mayHaveMultivaluedFields(), equalTo(b.values().stream().anyMatch(l -> l != null && l.size() > 1))); + } + + private int[] randomFilterPositions(Block orig, boolean all, boolean shuffled) { + int[] positions = IntStream.range(0, orig.getPositionCount()).toArray(); + if (shuffled) { + Randomness.shuffle(Arrays.asList(positions)); + } + if (all) { + return positions; + } + return IntStream.range(0, between(1, orig.getPositionCount())).map(i -> positions[i]).toArray(); + } + + private void assertExpanded(Block orig) { + Block expanded = orig.expand(); + assertThat(expanded.getPositionCount(), equalTo(orig.getTotalValueCount() + orig.nullValuesCount())); + assertThat(expanded.getTotalValueCount(), equalTo(orig.getTotalValueCount())); + + int np = 0; + for (int op = 0; op < orig.getPositionCount(); op++) { + if (orig.isNull(op)) { + assertThat(expanded.isNull(np), equalTo(true)); + assertThat(expanded.getValueCount(np++), equalTo(0)); + continue; + } + List oValues = BasicBlockTests.valuesAtPositions(orig, op, op + 1).get(0); + for (Object ov : oValues) { + assertThat(expanded.isNull(np), equalTo(false)); + assertThat(expanded.getValueCount(np), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(expanded, np, ++np).get(0), equalTo(List.of(ov))); + } + } + } + + private void assertFilteredThenExpanded(boolean all, boolean shuffled) { + int positionCount = randomIntBetween(1, 16 * 1024); + var b = BasicBlockTests.randomBlock(elementType, positionCount, nullAllowed, 0, 10, 0, 0); + int[] positions = randomFilterPositions(b.block(), all, shuffled); + assertExpanded(b.block().filter(positions)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockSerializationTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockSerializationTests.java new file mode 100644 index 0000000000000..275479f4c714e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockSerializationTests.java @@ -0,0 +1,124 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunction; +import org.elasticsearch.test.EqualsHashCodeTestUtils; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.is; + +public class BlockSerializationTests extends SerializationTestCase { + + public void testConstantIntBlock() throws IOException { + assertConstantBlockImpl(IntBlock.newConstantBlockWith(randomInt(), randomIntBetween(1, 8192))); + } + + public void testConstantLongBlockLong() throws IOException { + assertConstantBlockImpl(LongBlock.newConstantBlockWith(randomLong(), randomIntBetween(1, 8192))); + } + + public void testConstantDoubleBlock() throws IOException { + assertConstantBlockImpl(DoubleBlock.newConstantBlockWith(randomDouble(), randomIntBetween(1, 8192))); + } + + public void testConstantBytesRefBlock() throws IOException { + Block block = BytesRefBlock.newConstantBlockWith(new BytesRef(((Integer) randomInt()).toString()), randomIntBetween(1, 8192)); + assertConstantBlockImpl(block); + } + + private void assertConstantBlockImpl(Block origBlock) throws IOException { + assertThat(origBlock.asVector().isConstant(), is(true)); + Block deserBlock = serializeDeserializeBlock(origBlock); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock, unused -> deserBlock); + assertThat(deserBlock.asVector().isConstant(), is(true)); + } + + public void testEmptyIntBlock() { + assertEmptyBlock(IntBlock.newBlockBuilder(0).build()); + } + + public void testEmptyLongBlock() { + assertEmptyBlock(LongBlock.newBlockBuilder(0).build()); + } + + public void testEmptyDoubleBlock() { + assertEmptyBlock(DoubleBlock.newBlockBuilder(0).build()); + } + + public void testEmptyBytesRefBlock() { + assertEmptyBlock(BytesRefBlock.newBlockBuilder(0).build()); + } + + private void assertEmptyBlock(Block origBlock) { + assertThat(origBlock.getPositionCount(), is(0)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock, block -> serializeDeserializeBlock(block)); + } + + public void testFilterIntBlock() throws IOException { + assertFilterBlock(IntBlock.newBlockBuilder(0).appendInt(1).appendInt(2).build().filter(1)); + } + + public void testFilterLongBlock() throws IOException { + assertFilterBlock(LongBlock.newBlockBuilder(0).appendLong(1).appendLong(2).build().filter(1)); + } + + public void testFilterDoubleBlock() throws IOException { + assertFilterBlock(DoubleBlock.newBlockBuilder(0).appendDouble(1).appendDouble(2).build().filter(1)); + } + + public void testFilterBytesRefBlock() throws IOException { + BytesRefBlock block = BytesRefBlock.newBlockBuilder(0) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .build() + .filter(1); + assertFilterBlock(block); + } + + private void assertFilterBlock(Block origBlock) throws IOException { + assertThat(origBlock.getPositionCount(), is(1)); + Block deserBlock = serializeDeserializeBlock(origBlock); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock, unused -> deserBlock); + assertThat(deserBlock.getPositionCount(), is(1)); + } + + public void testConstantNullBlock() throws IOException { + Block origBlock = new ConstantNullBlock(randomIntBetween(1, 8192)); + Block deserBlock = serializeDeserializeBlock(origBlock); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origBlock, unused -> deserBlock); + } + + // TODO: more types, grouping, etc... + public void testAggregatorStateBlock() throws IOException { + Page page = new Page(new LongArrayVector(new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 }, 10).asBlock()); + var bigArrays = BigArrays.NON_RECYCLING_INSTANCE; + var params = new Object[] {}; + var function = SumLongAggregatorFunction.create(List.of(0)); + function.addRawInput(page); + Block[] blocks = new Block[function.intermediateBlockCount()]; + function.evaluateIntermediate(blocks, 0); + + Block[] deserBlocks = Arrays.stream(blocks).map(this::uncheckedSerializeDeserializeBlock).toArray(Block[]::new); + IntStream.range(0, blocks.length).forEach(i -> EqualsHashCodeTestUtils.checkEqualsAndHashCode(blocks[i], unused -> deserBlocks[i])); + + var inputChannels = IntStream.range(0, SumLongAggregatorFunction.intermediateStateDesc().size()).boxed().toList(); + var finalAggregator = SumLongAggregatorFunction.create(inputChannels); + finalAggregator.addIntermediateInput(new Page(deserBlocks)); + Block[] finalBlocks = new Block[1]; + finalAggregator.evaluateFinal(finalBlocks, 0); + var finalBlock = (LongBlock) finalBlocks[0]; + assertThat(finalBlock.getLong(0), is(55L)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockTestUtils.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockTestUtils.java new file mode 100644 index 0000000000000..a98467ad43617 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockTestUtils.java @@ -0,0 +1,100 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.hamcrest.Matcher; + +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.elasticsearch.test.ESTestCase.between; +import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; +import static org.elasticsearch.test.ESTestCase.randomBoolean; +import static org.elasticsearch.test.ESTestCase.randomDouble; +import static org.elasticsearch.test.ESTestCase.randomInt; +import static org.elasticsearch.test.ESTestCase.randomLong; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +public class BlockTestUtils { + /** + * Generate a random value of the appropriate type to fit into blocks of {@code e}. + */ + public static Object randomValue(ElementType e) { + return switch (e) { + case INT -> randomInt(); + case LONG -> randomLong(); + case DOUBLE -> randomDouble(); + case BYTES_REF -> new BytesRef(randomAlphaOfLength(5)); + case BOOLEAN -> randomBoolean(); + case DOC -> new BlockUtils.Doc(randomInt(), randomInt(), between(0, Integer.MAX_VALUE)); + case NULL -> null; + case UNKNOWN -> throw new IllegalArgumentException("can't make random values for [" + e + "]"); + }; + } + + /** + * Append {@code value} to {@code builder} or throw an + * {@link IllegalArgumentException} if the types don't line up. + */ + public static void append(Block.Builder builder, Object value) { + if (value == null) { + builder.appendNull(); + } else if (builder instanceof IntBlock.Builder b && value instanceof Integer v) { + b.appendInt(v); + } else if (builder instanceof LongBlock.Builder b && value instanceof Long v) { + b.appendLong(v); + } else if (builder instanceof DoubleBlock.Builder b && value instanceof Double v) { + b.appendDouble(v); + } else if (builder instanceof BytesRefBlock.Builder b && value instanceof BytesRef v) { + b.appendBytesRef(v); + } else if (builder instanceof BooleanBlock.Builder b && value instanceof Boolean v) { + b.appendBoolean(v); + } else if (builder instanceof DocBlock.Builder b && value instanceof BlockUtils.Doc v) { + b.appendShard(v.shard()).appendSegment(v.segment()).appendDoc(v.doc()); + } else { + throw new IllegalArgumentException("Can't append [" + value + "/" + value.getClass() + "] to [" + builder + "]"); + } + } + + public static void readInto(List> values, Page page) { + if (values.isEmpty()) { + while (values.size() < page.getBlockCount()) { + values.add(new ArrayList<>()); + } + } else { + if (values.size() != page.getBlockCount()) { + throw new IllegalArgumentException("Can't load values from pages with different numbers of blocks"); + } + } + for (int i = 0; i < page.getBlockCount(); i++) { + readInto(values.get(i), page.getBlock(i)); + } + } + + public static void readInto(List values, Block block) { + for (int p = 0; p < block.getPositionCount(); p++) { + values.add(toJavaObject(block, p)); + } + } + + /** + * Assert that the values at a particular position match the provided {@link Matcher}. + */ + @SuppressWarnings("unchecked") + public static void assertPositionValues(Block b, int p, Matcher valuesMatcher) { + List value = BasicBlockTests.valuesAtPositions(b, p, p + 1).get(0); + assertThat((T) value, valuesMatcher); + if (value == null) { + assertThat(b.getValueCount(p), equalTo(0)); + assertThat(b.isNull(p), equalTo(true)); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java new file mode 100644 index 0000000000000..6ad66c54b7568 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java @@ -0,0 +1,84 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class BlockValueAsserter { + + static void assertBlockValues(Block block, List> expectedBlockValues) { + assertThat(block.getPositionCount(), is(equalTo(expectedBlockValues.size()))); + for (int pos = 0; pos < expectedBlockValues.size(); pos++) { + List expectedRowValues = expectedBlockValues.get(pos); + if (expectedRowValues == null || expectedRowValues.isEmpty()) { // TODO empty is not the same as null + assertThat(block.isNull(pos), is(equalTo(true))); + assertThat(block.getValueCount(pos), is(equalTo(0))); + } else { + assertThat(block.isNull(pos), is(equalTo(false))); + final int valueCount = block.getValueCount(pos); + assertThat(expectedRowValues.size(), is(equalTo(valueCount))); + final int firstValueIndex = block.getFirstValueIndex(pos); + switch (block.elementType()) { + case INT -> assertIntRowValues((IntBlock) block, firstValueIndex, valueCount, expectedRowValues); + case LONG -> assertLongRowValues((LongBlock) block, firstValueIndex, valueCount, expectedRowValues); + case DOUBLE -> assertDoubleRowValues((DoubleBlock) block, firstValueIndex, valueCount, expectedRowValues); + case BYTES_REF -> assertBytesRefRowValues((BytesRefBlock) block, firstValueIndex, valueCount, expectedRowValues); + case BOOLEAN -> assertBooleanRowValues((BooleanBlock) block, firstValueIndex, valueCount, expectedRowValues); + default -> throw new IllegalArgumentException("Unsupported element type [" + block.elementType() + "]"); + } + } + } + } + + private static void assertIntRowValues(IntBlock block, int firstValueIndex, int valueCount, List expectedRowValues) { + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + int expectedValue = ((Number) expectedRowValues.get(valueIndex)).intValue(); + assertThat(block.getInt(firstValueIndex + valueIndex), is(equalTo(expectedValue))); + } + } + + private static void assertLongRowValues(LongBlock block, int firstValueIndex, int valueCount, List expectedRowValues) { + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + long expectedValue = ((Number) expectedRowValues.get(valueIndex)).longValue(); + assertThat(block.getLong(firstValueIndex + valueIndex), is(equalTo(expectedValue))); + } + } + + private static void assertDoubleRowValues(DoubleBlock block, int firstValueIndex, int valueCount, List expectedRowValues) { + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + double expectedValue = ((Number) expectedRowValues.get(valueIndex)).doubleValue(); + assertThat(block.getDouble(firstValueIndex + valueIndex), is(equalTo(expectedValue))); + } + } + + private static void assertBytesRefRowValues(BytesRefBlock block, int firstValueIndex, int valueCount, List expectedRowValues) { + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + Object value = expectedRowValues.get(valueIndex); + BytesRef expectedValue; + if (value instanceof BytesRef b) { + expectedValue = b; + } else { + expectedValue = new BytesRef(expectedRowValues.get(valueIndex).toString()); + } + assertThat(block.getBytesRef(firstValueIndex + valueIndex, new BytesRef()), is(equalTo(expectedValue))); + } + } + + private static void assertBooleanRowValues(BooleanBlock block, int firstValueIndex, int valueCount, List expectedRowValues) { + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + boolean expectedValue = (Boolean) expectedRowValues.get(valueIndex); + assertThat(block.getBoolean(firstValueIndex + valueIndex), is(equalTo(expectedValue))); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BooleanBlockEqualityTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BooleanBlockEqualityTests.java new file mode 100644 index 0000000000000..ae0d56d8612ce --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BooleanBlockEqualityTests.java @@ -0,0 +1,252 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.BitSet; +import java.util.List; + +public class BooleanBlockEqualityTests extends ESTestCase { + + public void testEmptyVector() { + // all these "empty" vectors should be equivalent + List vectors = List.of( + new BooleanArrayVector(new boolean[] {}, 0), + new BooleanArrayVector(new boolean[] { randomBoolean() }, 0), + BooleanBlock.newConstantBlockWith(randomBoolean(), 0).asVector(), + BooleanBlock.newConstantBlockWith(randomBoolean(), 0).filter().asVector(), + BooleanBlock.newBlockBuilder(0).build().asVector(), + BooleanBlock.newBlockBuilder(0).appendBoolean(randomBoolean()).build().asVector().filter() + ); + assertAllEquals(vectors); + } + + public void testEmptyBlock() { + // all these "empty" vectors should be equivalent + List blocks = List.of( + new BooleanArrayBlock( + new boolean[] {}, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + new BooleanArrayBlock( + new boolean[] { randomBoolean() }, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + BooleanBlock.newConstantBlockWith(randomBoolean(), 0), + BooleanBlock.newBlockBuilder(0).build(), + BooleanBlock.newBlockBuilder(0).appendBoolean(randomBoolean()).build().filter(), + BooleanBlock.newBlockBuilder(0).appendNull().build().filter() + ); + assertAllEquals(blocks); + } + + public void testVectorEquality() { + // all these vectors should be equivalent + List vectors = List.of( + new BooleanArrayVector(new boolean[] { true, false, true }, 3), + new BooleanArrayVector(new boolean[] { true, false, true }, 3).asBlock().asVector(), + new BooleanArrayVector(new boolean[] { true, false, true, false }, 3), + new BooleanArrayVector(new boolean[] { true, false, true }, 3).filter(0, 1, 2), + new BooleanArrayVector(new boolean[] { true, false, true, false }, 4).filter(0, 1, 2), + new BooleanArrayVector(new boolean[] { false, true, false, true }, 4).filter(1, 2, 3), + new BooleanArrayVector(new boolean[] { true, true, false, true }, 4).filter(0, 2, 3), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(false).appendBoolean(true).build().asVector(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(false).appendBoolean(true).build().asVector().filter(0, 1, 2), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendBoolean(true) + .appendBoolean(false) + .appendBoolean(true) + .build() + .filter(0, 2, 3) + .asVector(), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendBoolean(true) + .appendBoolean(false) + .appendBoolean(true) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(vectors); + + // all these constant-like vectors should be equivalent + List moreVectors = List.of( + new BooleanArrayVector(new boolean[] { true, true, true }, 3), + new BooleanArrayVector(new boolean[] { true, true, true }, 3).asBlock().asVector(), + new BooleanArrayVector(new boolean[] { true, true, true, true }, 3), + new BooleanArrayVector(new boolean[] { true, true, true }, 3).filter(0, 1, 2), + new BooleanArrayVector(new boolean[] { true, true, true, false }, 4).filter(0, 1, 2), + new BooleanArrayVector(new boolean[] { false, true, true, true }, 4).filter(1, 2, 3), + new BooleanArrayVector(new boolean[] { true, false, true, true }, 4).filter(0, 2, 3), + BooleanBlock.newConstantBlockWith(true, 3).asVector(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(true).appendBoolean(true).build().asVector(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(true).appendBoolean(true).build().asVector().filter(0, 1, 2), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendBoolean(false) + .appendBoolean(true) + .appendBoolean(true) + .build() + .filter(0, 2, 3) + .asVector(), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendBoolean(false) + .appendBoolean(true) + .appendBoolean(true) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(moreVectors); + } + + public void testBlockEquality() { + // all these blocks should be equivalent + List blocks = List.of( + new BooleanArrayVector(new boolean[] { true, false, true }, 3).asBlock(), + new BooleanArrayBlock( + new boolean[] { true, false, true }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BooleanArrayBlock( + new boolean[] { true, false, true, false }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b1000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BooleanArrayVector(new boolean[] { true, false, true }, 3).filter(0, 1, 2).asBlock(), + new BooleanArrayVector(new boolean[] { true, false, true, false }, 3).filter(0, 1, 2).asBlock(), + new BooleanArrayVector(new boolean[] { true, false, true, false }, 4).filter(0, 1, 2).asBlock(), + new BooleanArrayVector(new boolean[] { true, false, false, true }, 4).filter(0, 1, 3).asBlock(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(false).appendBoolean(true).build(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(false).appendBoolean(true).build().filter(0, 1, 2), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendBoolean(true) + .appendBoolean(false) + .appendBoolean(true) + .build() + .filter(0, 2, 3), + BooleanBlock.newBlockBuilder(3) + .appendBoolean(true) + .appendNull() + .appendBoolean(false) + .appendBoolean(true) + .build() + .filter(0, 2, 3) + ); + assertAllEquals(blocks); + + // all these constant-like blocks should be equivalent + List moreBlocks = List.of( + new BooleanArrayVector(new boolean[] { true, true }, 2).asBlock(), + new BooleanArrayBlock( + new boolean[] { true, true }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BooleanArrayBlock( + new boolean[] { true, true, false }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b100 }), + randomFrom(Block.MvOrdering.values()) + ), + new BooleanArrayVector(new boolean[] { true, true }, 2).filter(0, 1).asBlock(), + new BooleanArrayVector(new boolean[] { true, true, false }, 2).filter(0, 1).asBlock(), + new BooleanArrayVector(new boolean[] { true, true, false }, 3).filter(0, 1).asBlock(), + new BooleanArrayVector(new boolean[] { true, false, true }, 3).filter(0, 2).asBlock(), + BooleanBlock.newConstantBlockWith(true, 2), + BooleanBlock.newBlockBuilder(2).appendBoolean(true).appendBoolean(true).build(), + BooleanBlock.newBlockBuilder(2).appendBoolean(true).appendBoolean(true).build().filter(0, 1), + BooleanBlock.newBlockBuilder(2).appendBoolean(true).appendBoolean(true).appendBoolean(true).build().filter(0, 2), + BooleanBlock.newBlockBuilder(2).appendBoolean(true).appendNull().appendBoolean(true).build().filter(0, 2) + ); + assertAllEquals(moreBlocks); + } + + public void testVectorInequality() { + // all these vectors should NOT be equivalent + List notEqualVectors = List.of( + new BooleanArrayVector(new boolean[] { true }, 1), + new BooleanArrayVector(new boolean[] { false }, 1), + new BooleanArrayVector(new boolean[] { true, false }, 2), + new BooleanArrayVector(new boolean[] { true, false, true }, 3), + new BooleanArrayVector(new boolean[] { false, true, false }, 3), + BooleanBlock.newConstantBlockWith(true, 2).asVector(), + BooleanBlock.newBlockBuilder(2).appendBoolean(false).appendBoolean(true).build().asVector(), + BooleanBlock.newBlockBuilder(3).appendBoolean(false).appendBoolean(false).appendBoolean(true).build().asVector(), + BooleanBlock.newBlockBuilder(1) + .appendBoolean(false) + .appendBoolean(false) + .appendBoolean(false) + .appendBoolean(true) + .build() + .asVector() + ); + assertAllNotEquals(notEqualVectors); + } + + public void testBlockInequality() { + // all these blocks should NOT be equivalent + List notEqualBlocks = List.of( + new BooleanArrayVector(new boolean[] { false }, 1).asBlock(), + new BooleanArrayVector(new boolean[] { true }, 1).asBlock(), + new BooleanArrayVector(new boolean[] { false, true }, 2).asBlock(), + new BooleanArrayVector(new boolean[] { false, true, false }, 3).asBlock(), + new BooleanArrayVector(new boolean[] { false, false, true }, 3).asBlock(), + BooleanBlock.newConstantBlockWith(true, 2), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).appendBoolean(false).appendBoolean(false).build(), + BooleanBlock.newBlockBuilder(1).appendBoolean(true).appendBoolean(false).appendBoolean(true).appendBoolean(false).build(), + BooleanBlock.newBlockBuilder(1).appendBoolean(true).appendNull().build(), + BooleanBlock.newBlockBuilder(1).appendBoolean(true).appendNull().appendBoolean(false).build(), + BooleanBlock.newBlockBuilder(1).appendBoolean(true).appendBoolean(false).build(), + BooleanBlock.newBlockBuilder(3).appendBoolean(true).beginPositionEntry().appendBoolean(false).appendBoolean(false).build() + ); + assertAllNotEquals(notEqualBlocks); + } + + static void assertAllEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + assertEquals(obj1, obj2); + // equal objects MUST generate the same hash code + assertEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } + + static void assertAllNotEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + if (obj1 == obj2) { + continue; // skip self + } + assertNotEquals(obj1, obj2); + // unequal objects SHOULD generate the different hash code + assertNotEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BytesRefBlockEqualityTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BytesRefBlockEqualityTests.java new file mode 100644 index 0000000000000..0eb9beec2e7f9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BytesRefBlockEqualityTests.java @@ -0,0 +1,412 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; +import java.util.stream.IntStream; + +public class BytesRefBlockEqualityTests extends ESTestCase { + + final BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + + public void testEmptyVector() { + // all these "empty" vectors should be equivalent + try (var bytesRefArray1 = new BytesRefArray(0, bigArrays); var bytesRefArray2 = new BytesRefArray(1, bigArrays)) { + List vectors = List.of( + new BytesRefArrayVector(bytesRefArray1, 0), + new BytesRefArrayVector(bytesRefArray2, 0), + BytesRefBlock.newConstantBlockWith(new BytesRef(), 0).asVector(), + BytesRefBlock.newConstantBlockWith(new BytesRef(), 0).filter().asVector(), + BytesRefBlock.newBlockBuilder(0).build().asVector(), + BytesRefBlock.newBlockBuilder(0).appendBytesRef(new BytesRef()).build().asVector().filter() + ); + assertAllEquals(vectors); + } + } + + public void testEmptyBlock() { + // all these "empty" vectors should be equivalent + try (var bytesRefArray1 = new BytesRefArray(0, bigArrays); var bytesRefArray2 = new BytesRefArray(1, bigArrays)) { + List blocks = List.of( + new BytesRefArrayBlock( + bytesRefArray1, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + new BytesRefArrayBlock( + bytesRefArray2, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + BytesRefBlock.newConstantBlockWith(new BytesRef(), 0), + BytesRefBlock.newBlockBuilder(0).build(), + BytesRefBlock.newBlockBuilder(0).appendBytesRef(new BytesRef()).build().filter(), + BytesRefBlock.newBlockBuilder(0).appendNull().build().filter() + ); + assertAllEquals(blocks); + } + } + + public void testVectorEquality() { + // all these vectors should be equivalent + try (var bytesRefArray1 = arrayOf("1", "2", "3"); var bytesRefArray2 = arrayOf("1", "2", "3", "4")) { + List vectors = List.of( + new BytesRefArrayVector(bytesRefArray1, 3), + new BytesRefArrayVector(bytesRefArray1, 3).asBlock().asVector(), + new BytesRefArrayVector(bytesRefArray2, 3), + new BytesRefArrayVector(bytesRefArray1, 3).filter(0, 1, 2), + new BytesRefArrayVector(bytesRefArray2, 4).filter(0, 1, 2), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .asVector(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .asVector() + .filter(0, 1, 2), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .filter(0, 2, 3) + .asVector(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(vectors); + } + + // all these constant-like vectors should be equivalent + try (var bytesRefArray1 = arrayOf("1", "1", "1"); var bytesRefArray2 = arrayOf("1", "1", "1", "4")) { + List moreVectors = List.of( + new BytesRefArrayVector(bytesRefArray1, 3), + new BytesRefArrayVector(bytesRefArray1, 3).asBlock().asVector(), + new BytesRefArrayVector(bytesRefArray2, 3), + new BytesRefArrayVector(bytesRefArray1, 3).filter(0, 1, 2), + new BytesRefArrayVector(bytesRefArray2, 4).filter(0, 1, 2), + BytesRefBlock.newConstantBlockWith(new BytesRef("1"), 3).asVector(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .build() + .asVector(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .build() + .asVector() + .filter(0, 1, 2), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .build() + .filter(0, 2, 3) + .asVector(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("1")) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(moreVectors); + } + } + + public void testBlockEquality() { + // all these blocks should be equivalent + try (var bytesRefArray1 = arrayOf("1", "2", "3"); var bytesRefArray2 = arrayOf("1", "2", "3", "4")) { + List blocks = List.of( + new BytesRefArrayVector(bytesRefArray1, 3).asBlock(), + new BytesRefArrayBlock( + bytesRefArray1, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BytesRefArrayBlock( + bytesRefArray2, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b1000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BytesRefArrayVector(bytesRefArray1, 3).filter(0, 1, 2).asBlock(), + new BytesRefArrayVector(bytesRefArray2, 3).filter(0, 1, 2).asBlock(), + new BytesRefArrayVector(bytesRefArray2, 4).filter(0, 1, 2).asBlock(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build(), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .filter(0, 1, 2), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .filter(0, 2, 3), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendNull() + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .build() + .filter(0, 2, 3) + ); + assertAllEquals(blocks); + } + + // all these constant-like blocks should be equivalent + try (var bytesRefArray1 = arrayOf("9", "9"); var bytesRefArray2 = arrayOf("9", "9", "4")) { + List moreBlocks = List.of( + new BytesRefArrayVector(bytesRefArray1, 2).asBlock(), + new BytesRefArrayBlock( + bytesRefArray1, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new BytesRefArrayBlock( + bytesRefArray2, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b100 }), + randomFrom(Block.MvOrdering.values()) + ), + new BytesRefArrayVector(bytesRefArray1, 2).filter(0, 1).asBlock(), + new BytesRefArrayVector(bytesRefArray2, 2).filter(0, 1).asBlock(), + new BytesRefArrayVector(bytesRefArray2, 3).filter(0, 1).asBlock(), + BytesRefBlock.newConstantBlockWith(new BytesRef("9"), 2), + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("9")).appendBytesRef(new BytesRef("9")).build(), + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("9")).appendBytesRef(new BytesRef("9")).build().filter(0, 1), + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("9")) + .appendBytesRef(new BytesRef("4")) + .appendBytesRef(new BytesRef("9")) + .build() + .filter(0, 2), + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("9")) + .appendNull() + .appendBytesRef(new BytesRef("9")) + .build() + .filter(0, 2) + ); + assertAllEquals(moreBlocks); + } + } + + public void testVectorInequality() { + // all these vectors should NOT be equivalent + try ( + var bytesRefArray1 = arrayOf("1"); + var bytesRefArray2 = arrayOf("9"); + var bytesRefArray3 = arrayOf("1", "2"); + var bytesRefArray4 = arrayOf("1", "2", "3"); + var bytesRefArray5 = arrayOf("1", "2", "4") + ) { + List notEqualVectors = List.of( + new BytesRefArrayVector(bytesRefArray1, 1), + new BytesRefArrayVector(bytesRefArray2, 1), + new BytesRefArrayVector(bytesRefArray3, 2), + new BytesRefArrayVector(bytesRefArray4, 3), + new BytesRefArrayVector(bytesRefArray5, 3), + BytesRefBlock.newConstantBlockWith(new BytesRef("9"), 2).asVector(), + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .build() + .asVector() + .filter(1), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("5")) + .build() + .asVector(), + BytesRefBlock.newBlockBuilder(1) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .appendBytesRef(new BytesRef("4")) + .build() + .asVector() + ); + assertAllNotEquals(notEqualVectors); + } + } + + public void testBlockInequality() { + // all these blocks should NOT be equivalent + try ( + var bytesRefArray1 = arrayOf("1"); + var bytesRefArray2 = arrayOf("9"); + var bytesRefArray3 = arrayOf("1", "2"); + var bytesRefArray4 = arrayOf("1", "2", "3"); + var bytesRefArray5 = arrayOf("1", "2", "4") + ) { + List notEqualBlocks = List.of( + new BytesRefArrayVector(bytesRefArray1, 1).asBlock(), + new BytesRefArrayVector(bytesRefArray2, 1).asBlock(), + new BytesRefArrayVector(bytesRefArray3, 2).asBlock(), + new BytesRefArrayVector(bytesRefArray4, 3).asBlock(), + new BytesRefArrayVector(bytesRefArray5, 3).asBlock(), + BytesRefBlock.newConstantBlockWith(new BytesRef("9"), 2), + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("1")).appendBytesRef(new BytesRef("2")).build().filter(1), + BytesRefBlock.newBlockBuilder(3) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("5")) + .build(), + BytesRefBlock.newBlockBuilder(1) + .appendBytesRef(new BytesRef("1")) + .appendBytesRef(new BytesRef("2")) + .appendBytesRef(new BytesRef("3")) + .appendBytesRef(new BytesRef("4")) + .build(), + BytesRefBlock.newBlockBuilder(1).appendBytesRef(new BytesRef("1")).appendNull().build(), + BytesRefBlock.newBlockBuilder(1).appendBytesRef(new BytesRef("1")).appendNull().appendBytesRef(new BytesRef("3")).build(), + BytesRefBlock.newBlockBuilder(1).appendBytesRef(new BytesRef("1")).appendBytesRef(new BytesRef("3")).build() + ); + assertAllNotEquals(notEqualBlocks); + } + } + + public void testSimpleBlockWithSingleNull() { + List blocks = List.of( + BytesRefBlock.newBlockBuilder(3).appendBytesRef(new BytesRef("1")).appendNull().appendBytesRef(new BytesRef("3")).build(), + BytesRefBlock.newBlockBuilder(3).appendBytesRef(new BytesRef("1")).appendNull().appendBytesRef(new BytesRef("3")).build() + ); + assertEquals(3, blocks.get(0).getPositionCount()); + assertTrue(blocks.get(0).isNull(1)); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyNulls() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = BytesRefBlock.newBlockBuilder(grow ? 0 : positions); + IntStream.range(0, positions).forEach(i -> builder.appendNull()); + BytesRefBlock block1 = builder.build(); + BytesRefBlock block2 = builder.build(); + assertEquals(positions, block1.getPositionCount()); + assertTrue(block1.mayHaveNulls()); + assertTrue(block1.isNull(0)); + + List blocks = List.of(block1, block2); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithSingleMultiValue() { + List blocks = List.of( + BytesRefBlock.newBlockBuilder(1) + .beginPositionEntry() + .appendBytesRef(new BytesRef("1a")) + .appendBytesRef(new BytesRef("2b")) + .build(), + BytesRefBlock.newBlockBuilder(1) + .beginPositionEntry() + .appendBytesRef(new BytesRef("1a")) + .appendBytesRef(new BytesRef("2b")) + .build() + ); + assertEquals(1, blocks.get(0).getPositionCount()); + assertEquals(2, blocks.get(0).getValueCount(0)); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyMultiValues() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = BytesRefBlock.newBlockBuilder(grow ? 0 : positions); + for (int pos = 0; pos < positions; pos++) { + builder.beginPositionEntry(); + int values = randomIntBetween(1, 16); + IntStream.range(0, values).forEach(i -> builder.appendBytesRef(new BytesRef(Integer.toHexString(randomInt())))); + } + BytesRefBlock block1 = builder.build(); + BytesRefBlock block2 = builder.build(); + BytesRefBlock block3 = builder.build(); + + assertEquals(positions, block1.getPositionCount()); + assertAllEquals(List.of(block1, block2, block3)); + } + + BytesRefArray arrayOf(String... values) { + var array = new BytesRefArray(values.length, bigArrays); + Arrays.stream(values).map(BytesRef::new).forEach(array::append); + return array; + } + + static void assertAllEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + assertEquals(obj1, obj2); + assertEquals(obj2, obj1); + // equal objects must generate the same hash code + assertEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } + + static void assertAllNotEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + if (obj1 == obj2) { + continue; // skip self + } + assertNotEquals(obj1, obj2); + assertNotEquals(obj2, obj1); + // unequal objects SHOULD generate the different hash code + assertNotEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java new file mode 100644 index 0000000000000..85e578fcbd38f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java @@ -0,0 +1,126 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class DocVectorTests extends ESTestCase { + public void testNonDecreasingSetTrue() { + int length = between(1, 100); + DocVector docs = new DocVector(IntVector.range(0, length), IntVector.range(0, length), IntVector.range(0, length), true); + assertTrue(docs.singleSegmentNonDecreasing()); + } + + public void testNonDecreasingSetFalse() { + DocVector docs = new DocVector(IntVector.range(0, 2), IntVector.range(0, 2), new IntArrayVector(new int[] { 1, 0 }, 2), false); + assertFalse(docs.singleSegmentNonDecreasing()); + } + + public void testNonDecreasingNonConstantShard() { + DocVector docs = new DocVector(IntVector.range(0, 2), IntBlock.newConstantBlockWith(0, 2).asVector(), IntVector.range(0, 2), null); + assertFalse(docs.singleSegmentNonDecreasing()); + } + + public void testNonDecreasingNonConstantSegment() { + DocVector docs = new DocVector(IntBlock.newConstantBlockWith(0, 2).asVector(), IntVector.range(0, 2), IntVector.range(0, 2), null); + assertFalse(docs.singleSegmentNonDecreasing()); + } + + public void testNonDecreasingDescendingDocs() { + DocVector docs = new DocVector( + IntBlock.newConstantBlockWith(0, 2).asVector(), + IntBlock.newConstantBlockWith(0, 2).asVector(), + new IntArrayVector(new int[] { 1, 0 }, 2), + null + ); + assertFalse(docs.singleSegmentNonDecreasing()); + } + + public void testShardSegmentDocMap() { + assertShardSegmentDocMap( + new int[][] { + new int[] { 1, 0, 0 }, + new int[] { 1, 1, 1 }, + new int[] { 1, 1, 0 }, + new int[] { 0, 0, 2 }, + new int[] { 0, 1, 1 }, + new int[] { 0, 1, 0 }, + new int[] { 0, 2, 1 }, + new int[] { 0, 2, 0 }, + new int[] { 0, 2, 2 }, + new int[] { 0, 2, 3 }, }, + new int[][] { + new int[] { 0, 0, 2 }, + new int[] { 0, 1, 0 }, + new int[] { 0, 1, 1 }, + new int[] { 0, 2, 0 }, + new int[] { 0, 2, 1 }, + new int[] { 0, 2, 2 }, + new int[] { 0, 2, 3 }, + new int[] { 1, 0, 0 }, + new int[] { 1, 1, 0 }, + new int[] { 1, 1, 1 }, } + ); + } + + public void testRandomShardSegmentDocMap() { + int[][] tracker = new int[5][]; + for (int shard = 0; shard < 5; shard++) { + tracker[shard] = new int[] { 0, 0, 0, 0, 0 }; + } + List data = new ArrayList<>(); + for (int r = 0; r < 10000; r++) { + int shard = between(0, 4); + int segment = between(0, 4); + data.add(new int[] { shard, segment, tracker[shard][segment]++ }); + } + Randomness.shuffle(data); + + List sorted = new ArrayList<>(data); + Collections.sort(sorted, Comparator.comparing((int[] r) -> r[0]).thenComparing(r -> r[1]).thenComparing(r -> r[2])); + assertShardSegmentDocMap(data.toArray(int[][]::new), sorted.toArray(int[][]::new)); + } + + private void assertShardSegmentDocMap(int[][] data, int[][] expected) { + DocBlock.Builder builder = DocBlock.newBlockBuilder(data.length); + for (int r = 0; r < data.length; r++) { + builder.appendShard(data[r][0]); + builder.appendSegment(data[r][1]); + builder.appendDoc(data[r][2]); + } + DocVector docVector = builder.build().asVector(); + int[] forwards = docVector.shardSegmentDocMapForwards(); + + int[][] result = new int[docVector.getPositionCount()][]; + for (int p = 0; p < result.length; p++) { + result[p] = new int[] { + docVector.shards().getInt(forwards[p]), + docVector.segments().getInt(forwards[p]), + docVector.docs().getInt(forwards[p]) }; + } + assertThat(result, equalTo(expected)); + + int[] backwards = docVector.shardSegmentDocMapBackwards(); + for (int p = 0; p < result.length; p++) { + result[p] = new int[] { + docVector.shards().getInt(backwards[forwards[p]]), + docVector.segments().getInt(backwards[forwards[p]]), + docVector.docs().getInt(backwards[forwards[p]]) }; + } + + assertThat(result, equalTo(data)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DoubleBlockEqualityTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DoubleBlockEqualityTests.java new file mode 100644 index 0000000000000..2abbcc0b989f1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DoubleBlockEqualityTests.java @@ -0,0 +1,289 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.BitSet; +import java.util.List; +import java.util.stream.IntStream; + +public class DoubleBlockEqualityTests extends ESTestCase { + + public void testEmptyVector() { + // all these "empty" vectors should be equivalent + List vectors = List.of( + new DoubleArrayVector(new double[] {}, 0), + new DoubleArrayVector(new double[] { 0 }, 0), + DoubleBlock.newConstantBlockWith(0, 0).asVector(), + DoubleBlock.newConstantBlockWith(0, 0).filter().asVector(), + DoubleBlock.newBlockBuilder(0).build().asVector(), + DoubleBlock.newBlockBuilder(0).appendDouble(1).build().asVector().filter() + ); + assertAllEquals(vectors); + } + + public void testEmptyBlock() { + // all these "empty" vectors should be equivalent + List blocks = List.of( + new DoubleArrayBlock( + new double[] {}, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + new DoubleArrayBlock( + new double[] { 0 }, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + DoubleBlock.newConstantBlockWith(0, 0), + DoubleBlock.newBlockBuilder(0).build(), + DoubleBlock.newBlockBuilder(0).appendDouble(1).build().filter(), + DoubleBlock.newBlockBuilder(0).appendNull().build().filter() + ); + assertAllEquals(blocks); + } + + public void testVectorEquality() { + // all these vectors should be equivalent + List vectors = List.of( + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3), + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3).asBlock().asVector(), + new DoubleArrayVector(new double[] { 1, 2, 3, 4 }, 3), + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3).filter(0, 1, 2), + new DoubleArrayVector(new double[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2), + new DoubleArrayVector(new double[] { 0, 1, 2, 3 }, 4).filter(1, 2, 3), + new DoubleArrayVector(new double[] { 1, 4, 2, 3 }, 4).filter(0, 2, 3), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(3).build().asVector(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(3).build().asVector().filter(0, 1, 2), + DoubleBlock.newBlockBuilder(3) + .appendDouble(1) + .appendDouble(4) + .appendDouble(2) + .appendDouble(3) + .build() + .filter(0, 2, 3) + .asVector(), + DoubleBlock.newBlockBuilder(3) + .appendDouble(1) + .appendDouble(4) + .appendDouble(2) + .appendDouble(3) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(vectors); + + // all these constant-like vectors should be equivalent + List moreVectors = List.of( + new DoubleArrayVector(new double[] { 1, 1, 1 }, 3), + new DoubleArrayVector(new double[] { 1, 1, 1 }, 3).asBlock().asVector(), + new DoubleArrayVector(new double[] { 1, 1, 1, 1 }, 3), + new DoubleArrayVector(new double[] { 1, 1, 1 }, 3).filter(0, 1, 2), + new DoubleArrayVector(new double[] { 1, 1, 1, 4 }, 4).filter(0, 1, 2), + new DoubleArrayVector(new double[] { 3, 1, 1, 1 }, 4).filter(1, 2, 3), + new DoubleArrayVector(new double[] { 1, 4, 1, 1 }, 4).filter(0, 2, 3), + DoubleBlock.newConstantBlockWith(1, 3).asVector(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(1).appendDouble(1).build().asVector(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(1).appendDouble(1).build().asVector().filter(0, 1, 2), + DoubleBlock.newBlockBuilder(3) + .appendDouble(1) + .appendDouble(4) + .appendDouble(1) + .appendDouble(1) + .build() + .filter(0, 2, 3) + .asVector(), + DoubleBlock.newBlockBuilder(3) + .appendDouble(1) + .appendDouble(4) + .appendDouble(1) + .appendDouble(1) + .build() + .asVector() + .filter(0, 2, 3) + ); + assertAllEquals(moreVectors); + } + + public void testBlockEquality() { + // all these blocks should be equivalent + List blocks = List.of( + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3).asBlock(), + new DoubleArrayBlock( + new double[] { 1, 2, 3 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new DoubleArrayBlock( + new double[] { 1, 2, 3, 4 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b1000 }), + randomFrom(Block.MvOrdering.values()) + ), + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3).filter(0, 1, 2).asBlock(), + new DoubleArrayVector(new double[] { 1, 2, 3, 4 }, 3).filter(0, 1, 2).asBlock(), + new DoubleArrayVector(new double[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2).asBlock(), + new DoubleArrayVector(new double[] { 1, 2, 4, 3 }, 4).filter(0, 1, 3).asBlock(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(3).build(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(3).build().filter(0, 1, 2), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(4).appendDouble(2).appendDouble(3).build().filter(0, 2, 3), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendNull().appendDouble(2).appendDouble(3).build().filter(0, 2, 3) + ); + assertAllEquals(blocks); + + // all these constant-like blocks should be equivalent + List moreBlocks = List.of( + new DoubleArrayVector(new double[] { 9, 9 }, 2).asBlock(), + new DoubleArrayBlock( + new double[] { 9, 9 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new DoubleArrayBlock( + new double[] { 9, 9, 4 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b100 }), + randomFrom(Block.MvOrdering.values()) + ), + new DoubleArrayVector(new double[] { 9, 9 }, 2).filter(0, 1).asBlock(), + new DoubleArrayVector(new double[] { 9, 9, 4 }, 2).filter(0, 1).asBlock(), + new DoubleArrayVector(new double[] { 9, 9, 4 }, 3).filter(0, 1).asBlock(), + new DoubleArrayVector(new double[] { 9, 4, 9 }, 3).filter(0, 2).asBlock(), + DoubleBlock.newConstantBlockWith(9, 2), + DoubleBlock.newBlockBuilder(2).appendDouble(9).appendDouble(9).build(), + DoubleBlock.newBlockBuilder(2).appendDouble(9).appendDouble(9).build().filter(0, 1), + DoubleBlock.newBlockBuilder(2).appendDouble(9).appendDouble(4).appendDouble(9).build().filter(0, 2), + DoubleBlock.newBlockBuilder(2).appendDouble(9).appendNull().appendDouble(9).build().filter(0, 2) + ); + assertAllEquals(moreBlocks); + } + + public void testVectorInequality() { + // all these vectors should NOT be equivalent + List notEqualVectors = List.of( + new DoubleArrayVector(new double[] { 1 }, 1), + new DoubleArrayVector(new double[] { 9 }, 1), + new DoubleArrayVector(new double[] { 1, 2 }, 2), + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3), + new DoubleArrayVector(new double[] { 1, 2, 4 }, 3), + DoubleBlock.newConstantBlockWith(9, 2).asVector(), + DoubleBlock.newBlockBuilder(2).appendDouble(1).appendDouble(2).build().asVector().filter(1), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(5).build().asVector(), + DoubleBlock.newBlockBuilder(1).appendDouble(1).appendDouble(2).appendDouble(3).appendDouble(4).build().asVector() + ); + assertAllNotEquals(notEqualVectors); + } + + public void testBlockInequality() { + // all these blocks should NOT be equivalent + List notEqualBlocks = List.of( + new DoubleArrayVector(new double[] { 1 }, 1).asBlock(), + new DoubleArrayVector(new double[] { 9 }, 1).asBlock(), + new DoubleArrayVector(new double[] { 1, 2 }, 2).asBlock(), + new DoubleArrayVector(new double[] { 1, 2, 3 }, 3).asBlock(), + new DoubleArrayVector(new double[] { 1, 2, 4 }, 3).asBlock(), + DoubleBlock.newConstantBlockWith(9, 2), + DoubleBlock.newBlockBuilder(2).appendDouble(1).appendDouble(2).build().filter(1), + DoubleBlock.newBlockBuilder(3).appendDouble(1).appendDouble(2).appendDouble(5).build(), + DoubleBlock.newBlockBuilder(1).appendDouble(1).appendDouble(2).appendDouble(3).appendDouble(4).build(), + DoubleBlock.newBlockBuilder(1).appendDouble(1).appendNull().build(), + DoubleBlock.newBlockBuilder(1).appendDouble(1).appendNull().appendDouble(3).build(), + DoubleBlock.newBlockBuilder(1).appendDouble(1).appendDouble(3).build(), + DoubleBlock.newBlockBuilder(3).appendDouble(1).beginPositionEntry().appendDouble(2).appendDouble(3).build() + ); + assertAllNotEquals(notEqualBlocks); + } + + public void testSimpleBlockWithSingleNull() { + List blocks = List.of( + DoubleBlock.newBlockBuilder(3).appendDouble(1.1).appendNull().appendDouble(3.1).build(), + DoubleBlock.newBlockBuilder(3).appendDouble(1.1).appendNull().appendDouble(3.1).build() + ); + assertEquals(3, blocks.get(0).getPositionCount()); + assertTrue(blocks.get(0).isNull(1)); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyNulls() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = DoubleBlock.newBlockBuilder(grow ? 0 : positions); + IntStream.range(0, positions).forEach(i -> builder.appendNull()); + DoubleBlock block1 = builder.build(); + DoubleBlock block2 = builder.build(); + assertEquals(positions, block1.getPositionCount()); + assertTrue(block1.mayHaveNulls()); + assertTrue(block1.isNull(0)); + + List blocks = List.of(block1, block2); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithSingleMultiValue() { + List blocks = List.of( + DoubleBlock.newBlockBuilder(1).beginPositionEntry().appendDouble(1.1).appendDouble(2.2).build(), + DoubleBlock.newBlockBuilder(1).beginPositionEntry().appendDouble(1.1).appendDouble(2.2).build() + ); + assert blocks.get(0).getPositionCount() == 1 && blocks.get(0).getValueCount(0) == 2; + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyMultiValues() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = DoubleBlock.newBlockBuilder(grow ? 0 : positions); + for (int pos = 0; pos < positions; pos++) { + builder.beginPositionEntry(); + int values = randomIntBetween(1, 16); + IntStream.range(0, values).forEach(i -> builder.appendDouble(randomDouble())); + } + DoubleBlock block1 = builder.build(); + DoubleBlock block2 = builder.build(); + DoubleBlock block3 = builder.build(); + + assertEquals(positions, block1.getPositionCount()); + assertAllEquals(List.of(block1, block2, block3)); + } + + static void assertAllEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + assertEquals(obj1, obj2); + assertEquals(obj2, obj1); + // equal objects must generate the same hash code + assertEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } + + static void assertAllNotEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + if (obj1 == obj2) { + continue; // skip self + } + assertNotEquals(obj1, obj2); + assertNotEquals(obj2, obj1); + // unequal objects SHOULD generate the different hash code + assertNotEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/FilteredBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/FilteredBlockTests.java new file mode 100644 index 0000000000000..d93a0bdff3d84 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/FilteredBlockTests.java @@ -0,0 +1,266 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.BitSet; +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.containsString; + +public class FilteredBlockTests extends ESTestCase { + + public void testFilterAllPositions() { + var positionCount = 100; + var vector = new IntArrayVector(IntStream.range(0, positionCount).toArray(), positionCount); + var filteredVector = vector.filter(); + + assertEquals(0, filteredVector.getPositionCount()); + expectThrows(ArrayIndexOutOfBoundsException.class, () -> filteredVector.getInt(0)); + + var filteredBlock = vector.asBlock().filter(); + assertEquals(0, filteredBlock.getPositionCount()); + expectThrows(ArrayIndexOutOfBoundsException.class, () -> filteredBlock.getInt(0)); + } + + public void testKeepAllPositions() { + var positionCount = 100; + var vector = new IntArrayVector(IntStream.range(0, positionCount).toArray(), positionCount); + var positions = IntStream.range(0, positionCount).toArray(); + + var filteredVector = vector.filter(positions); + assertEquals(positionCount, filteredVector.getPositionCount()); + var anyPosition = randomPosition(positionCount); + assertEquals(anyPosition, filteredVector.getInt(anyPosition)); + + var filteredBlock = vector.filter(positions).asBlock(); + assertEquals(positionCount, filteredBlock.getPositionCount()); + assertEquals(anyPosition, filteredBlock.getInt(anyPosition)); + } + + public void testKeepSomePositions() { + var positionCount = 100; + var vector = new IntArrayVector(IntStream.range(0, positionCount).toArray(), positionCount); + var positions = IntStream.range(0, positionCount).filter(i -> i % 2 == 0).toArray(); + + var filteredVector = vector.filter(positions); + assertEquals(positionCount / 2, filteredVector.getPositionCount()); + var anyPosition = randomIntBetween(0, (positionCount / 2) - 1); + assertEquals(anyPosition * 2, filteredVector.getInt(anyPosition)); + assertEquals(anyPosition * 2, filteredVector.asBlock().getInt(anyPosition)); + + var filteredBlock = vector.asBlock().filter(positions); + assertEquals(positionCount / 2, filteredBlock.getPositionCount()); + assertEquals(anyPosition * 2, filteredBlock.getInt(anyPosition)); + } + + public void testFilterOnFilter() { // TODO: tired of this sv / mv block here. do more below + var positionCount = 100; + var vector = new IntArrayVector(IntStream.range(0, positionCount).toArray(), positionCount); + + var filteredVector = vector.filter(IntStream.range(0, positionCount).filter(i1 -> i1 % 2 == 0).toArray()); + var filteredTwice = filteredVector.filter(IntStream.range(0, positionCount / 2).filter(i -> i % 2 == 0).toArray()); + + assertEquals(positionCount / 4, filteredTwice.getPositionCount()); + var anyPosition = randomIntBetween(0, positionCount / 4 - 1); + assertEquals(anyPosition * 4, filteredTwice.getInt(anyPosition)); + } + + public void testFilterOnNull() { + IntBlock block; + if (randomBoolean()) { + var nulls = new BitSet(); + nulls.set(1); + block = new IntArrayBlock(new int[] { 10, 0, 30, 40 }, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + } else { + var blockBuilder = IntBlock.newBlockBuilder(4); + blockBuilder.appendInt(10); + blockBuilder.appendNull(); + blockBuilder.appendInt(30); + blockBuilder.appendInt(40); + block = blockBuilder.build(); + } + + var filtered = block.filter(1, 2, 3); + + assertTrue(filtered.isNull(0)); + assertTrue(filtered.mayHaveNulls()); + assertFalse(filtered.areAllValuesNull()); + assertEquals(1, filtered.nullValuesCount()); + assertEquals(2, filtered.getTotalValueCount()); + assertFalse(filtered.isNull(1)); + assertEquals(30, filtered.getInt(filtered.getFirstValueIndex(1))); + } + + public void testFilterOnAllNullsBlock() { + Block block; + if (randomBoolean()) { + var nulls = new BitSet(); + nulls.set(0, 4); + block = new IntArrayBlock(new int[] { 0, 0, 0, 0 }, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + } else { + var blockBuilder = IntBlock.newBlockBuilder(4); + blockBuilder.appendNull(); + blockBuilder.appendNull(); + blockBuilder.appendNull(); + blockBuilder.appendNull(); + block = blockBuilder.build(); + } + + var filtered = block.filter(1, 2, 3); + + assertTrue(filtered.isNull(0)); + assertTrue(filtered.mayHaveNulls()); + assertTrue(filtered.areAllValuesNull()); + assertEquals(3, filtered.nullValuesCount()); + assertEquals(0, filtered.getTotalValueCount()); + } + + public void testFilterOnNoNullsBlock() { + IntBlock block; + if (randomBoolean()) { + block = new IntArrayVector(new int[] { 10, 20, 30, 40 }, 4).asBlock(); + } else { + var blockBuilder = IntBlock.newBlockBuilder(4); + blockBuilder.appendInt(10); + blockBuilder.appendInt(20); + blockBuilder.appendInt(30); + blockBuilder.appendInt(40); + block = blockBuilder.build(); + } + var filtered = block.filter(1, 2, 3); + + assertFalse(filtered.isNull(0)); + assertFalse(filtered.mayHaveNulls()); + assertFalse(filtered.areAllValuesNull()); + assertEquals(0, filtered.nullValuesCount()); + assertEquals(3, filtered.getTotalValueCount()); + + assertEquals(20, filtered.asVector().getInt(0)); + assertEquals(30, filtered.asVector().getInt(1)); + assertEquals(40, filtered.asVector().getInt(2)); + } + + public void testFilterToStringSimple() { + BitSet nulls = BitSet.valueOf(new byte[] { 0x08 }); // any non-empty bitset, that does not affect the filter, should suffice + + var boolVector = new BooleanArrayVector(new boolean[] { true, false, false, true }, 4); + var boolBlock = new BooleanArrayBlock( + new boolean[] { true, false, false, true }, + 4, + null, + nulls, + randomFrom(Block.MvOrdering.values()) + ); + for (Object obj : List.of(boolVector.filter(0, 2), boolVector.asBlock().filter(0, 2), boolBlock.filter(0, 2))) { + String s = obj.toString(); + assertThat(s, containsString("[true, false]")); + assertThat(s, containsString("positions=2")); + } + + var intVector = new IntArrayVector(new int[] { 10, 20, 30, 40 }, 4); + var intBlock = new IntArrayBlock(new int[] { 10, 20, 30, 40 }, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + for (Object obj : List.of(intVector.filter(0, 2), intVector.asBlock().filter(0, 2), intBlock.filter(0, 2))) { + String s = obj.toString(); + assertThat(s, containsString("[10, 30]")); + assertThat(s, containsString("positions=2")); + } + + var longVector = new LongArrayVector(new long[] { 100L, 200L, 300L, 400L }, 4); + var longBlock = new LongArrayBlock(new long[] { 100L, 200L, 300L, 400L }, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + for (Object obj : List.of(longVector.filter(0, 2), longVector.asBlock().filter(0, 2), longBlock.filter(0, 2))) { + String s = obj.toString(); + assertThat(s, containsString("[100, 300]")); + assertThat(s, containsString("positions=2")); + } + + var doubleVector = new DoubleArrayVector(new double[] { 1.1, 2.2, 3.3, 4.4 }, 4); + var doubleBlock = new DoubleArrayBlock(new double[] { 1.1, 2.2, 3.3, 4.4 }, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + for (Object obj : List.of(doubleVector.filter(0, 2), doubleVector.asBlock().filter(0, 2), doubleBlock.filter(0, 2))) { + String s = obj.toString(); + assertThat(s, containsString("[1.1, 3.3]")); + assertThat(s, containsString("positions=2")); + } + + assert new BytesRef("1a").toString().equals("[31 61]") && new BytesRef("3c").toString().equals("[33 63]"); + try (var bytesRefArray = arrayOf("1a", "2b", "3c", "4d")) { + var bytesRefVector = new BytesRefArrayVector(bytesRefArray, 4); + var bytesRefBlock = new BytesRefArrayBlock(bytesRefArray, 4, null, nulls, randomFrom(Block.MvOrdering.values())); + for (Object obj : List.of(bytesRefVector.filter(0, 2), bytesRefVector.asBlock().filter(0, 2), bytesRefBlock.filter(0, 2))) { + String s = obj.toString(); + assertThat(s, containsString("[[31 61], [33 63]]")); + assertThat(s, containsString("positions=2")); + } + } + } + + public void testFilterToStringMultiValue() { + var bb = BooleanBlock.newBlockBuilder(6); + bb.beginPositionEntry().appendBoolean(true).appendBoolean(true).endPositionEntry(); + bb.beginPositionEntry().appendBoolean(false).appendBoolean(false).endPositionEntry(); + bb.beginPositionEntry().appendBoolean(false).appendBoolean(false).endPositionEntry(); + Block filter = bb.build().filter(0, 1); + assertThat(filter.toString(), containsString("[[true, true], [false, false]]")); + assertThat(filter.toString(), containsString("positions=2")); + + var ib = IntBlock.newBlockBuilder(6); + ib.beginPositionEntry().appendInt(0).appendInt(10).endPositionEntry(); + ib.beginPositionEntry().appendInt(20).appendInt(50).endPositionEntry(); + ib.beginPositionEntry().appendInt(90).appendInt(1000).endPositionEntry(); + filter = ib.build().filter(0, 1); + assertThat(filter.toString(), containsString("[[0, 10], [20, 50]]")); + assertThat(filter.toString(), containsString("positions=2")); + + var lb = LongBlock.newBlockBuilder(6); + lb.beginPositionEntry().appendLong(0).appendLong(10).endPositionEntry(); + lb.beginPositionEntry().appendLong(20).appendLong(50).endPositionEntry(); + lb.beginPositionEntry().appendLong(90).appendLong(1000).endPositionEntry(); + filter = lb.build().filter(0, 1); + assertThat(filter.toString(), containsString("[[0, 10], [20, 50]]")); + assertThat(filter.toString(), containsString("positions=2")); + + var db = DoubleBlock.newBlockBuilder(6); + db.beginPositionEntry().appendDouble(0).appendDouble(10).endPositionEntry(); + db.beginPositionEntry().appendDouble(0.002).appendDouble(10e8).endPositionEntry(); + db.beginPositionEntry().appendDouble(90).appendDouble(1000).endPositionEntry(); + filter = db.build().filter(0, 1); + assertThat(filter.toString(), containsString("[[0.0, 10.0], [0.002, 1.0E9]]")); + assertThat(filter.toString(), containsString("positions=2")); + + assert new BytesRef("1a").toString().equals("[31 61]") && new BytesRef("3c").toString().equals("[33 63]"); + assert new BytesRef("cat").toString().equals("[63 61 74]") && new BytesRef("dog").toString().equals("[64 6f 67]"); + var bytesBlock = BytesRefBlock.newBlockBuilder(6); + bytesBlock.beginPositionEntry().appendBytesRef(new BytesRef("1a")).appendBytesRef(new BytesRef("3c")).endPositionEntry(); + bytesBlock.beginPositionEntry().appendBytesRef(new BytesRef("cat")).appendBytesRef(new BytesRef("dog")).endPositionEntry(); + bytesBlock.beginPositionEntry().appendBytesRef(new BytesRef("pig")).appendBytesRef(new BytesRef("chicken")).endPositionEntry(); + filter = bytesBlock.build().filter(0, 1); + assertThat(filter.toString(), containsString("[[[31 61], [33 63]], [[63 61 74], [64 6f 67]]")); + assertThat(filter.toString(), containsString("positions=2")); + } + + static int randomPosition(int positionCount) { + return positionCount == 1 ? 0 : randomIntBetween(0, positionCount - 1); + } + + BytesRefArray arrayOf(String... values) { + var array = new BytesRefArray(values.length, bigArrays); + Arrays.stream(values).map(BytesRef::new).forEach(array::append); + return array; + } + + final BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/IntBlockEqualityTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/IntBlockEqualityTests.java new file mode 100644 index 0000000000000..c4e19106d4368 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/IntBlockEqualityTests.java @@ -0,0 +1,251 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.BitSet; +import java.util.List; +import java.util.stream.IntStream; + +public class IntBlockEqualityTests extends ESTestCase { + + public void testEmptyVector() { + // all these "empty" vectors should be equivalent + List vectors = List.of( + new IntArrayVector(new int[] {}, 0), + new IntArrayVector(new int[] { 0 }, 0), + IntBlock.newConstantBlockWith(0, 0).asVector(), + IntBlock.newConstantBlockWith(0, 0).filter().asVector(), + IntBlock.newBlockBuilder(0).build().asVector(), + IntBlock.newBlockBuilder(0).appendInt(1).build().asVector().filter() + ); + assertAllEquals(vectors); + } + + public void testEmptyBlock() { + // all these "empty" vectors should be equivalent + List blocks = List.of( + new IntArrayBlock(new int[] {}, 0, new int[] {}, BitSet.valueOf(new byte[] { 0b00 }), randomFrom(Block.MvOrdering.values())), + new IntArrayBlock(new int[] { 0 }, 0, new int[] {}, BitSet.valueOf(new byte[] { 0b00 }), randomFrom(Block.MvOrdering.values())), + IntBlock.newConstantBlockWith(0, 0), + IntBlock.newBlockBuilder(0).build(), + IntBlock.newBlockBuilder(0).appendInt(1).build().filter(), + IntBlock.newBlockBuilder(0).appendNull().build().filter() + ); + assertAllEquals(blocks); + } + + public void testVectorEquality() { + // all these vectors should be equivalent + List vectors = List.of( + new IntArrayVector(new int[] { 1, 2, 3 }, 3), + new IntArrayVector(new int[] { 1, 2, 3 }, 3).asBlock().asVector(), + new IntArrayVector(new int[] { 1, 2, 3, 4 }, 3), + new IntArrayVector(new int[] { 1, 2, 3 }, 3).filter(0, 1, 2), + new IntArrayVector(new int[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2), + new IntArrayVector(new int[] { 0, 1, 2, 3 }, 4).filter(1, 2, 3), + new IntArrayVector(new int[] { 1, 4, 2, 3 }, 4).filter(0, 2, 3), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(3).build().asVector(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(3).build().asVector().filter(0, 1, 2), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(4).appendInt(2).appendInt(3).build().filter(0, 2, 3).asVector(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(4).appendInt(2).appendInt(3).build().asVector().filter(0, 2, 3) + ); + assertAllEquals(vectors); + + // all these constant-like vectors should be equivalent + List moreVectors = List.of( + new IntArrayVector(new int[] { 1, 1, 1 }, 3), + new IntArrayVector(new int[] { 1, 1, 1 }, 3).asBlock().asVector(), + new IntArrayVector(new int[] { 1, 1, 1, 1 }, 3), + new IntArrayVector(new int[] { 1, 1, 1 }, 3).filter(0, 1, 2), + new IntArrayVector(new int[] { 1, 1, 1, 4 }, 4).filter(0, 1, 2), + new IntArrayVector(new int[] { 3, 1, 1, 1 }, 4).filter(1, 2, 3), + new IntArrayVector(new int[] { 1, 4, 1, 1 }, 4).filter(0, 2, 3), + IntBlock.newConstantBlockWith(1, 3).asVector(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(1).appendInt(1).build().asVector(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(1).appendInt(1).build().asVector().filter(0, 1, 2), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(4).appendInt(1).appendInt(1).build().filter(0, 2, 3).asVector(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(4).appendInt(1).appendInt(1).build().asVector().filter(0, 2, 3) + ); + assertAllEquals(moreVectors); + } + + public void testBlockEquality() { + // all these blocks should be equivalent + List blocks = List.of( + new IntArrayVector(new int[] { 1, 2, 3 }, 3).asBlock(), + new IntArrayBlock( + new int[] { 1, 2, 3 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new IntArrayBlock( + new int[] { 1, 2, 3, 4 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b1000 }), + randomFrom(Block.MvOrdering.values()) + ), + new IntArrayVector(new int[] { 1, 2, 3 }, 3).filter(0, 1, 2).asBlock(), + new IntArrayVector(new int[] { 1, 2, 3, 4 }, 3).filter(0, 1, 2).asBlock(), + new IntArrayVector(new int[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2).asBlock(), + new IntArrayVector(new int[] { 1, 2, 4, 3 }, 4).filter(0, 1, 3).asBlock(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(3).build(), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(3).build().filter(0, 1, 2), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(4).appendInt(2).appendInt(3).build().filter(0, 2, 3), + IntBlock.newBlockBuilder(3).appendInt(1).appendNull().appendInt(2).appendInt(3).build().filter(0, 2, 3) + ); + assertAllEquals(blocks); + + // all these constant-like blocks should be equivalent + List moreBlocks = List.of( + new IntArrayVector(new int[] { 9, 9 }, 2).asBlock(), + new IntArrayBlock( + new int[] { 9, 9 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new IntArrayBlock( + new int[] { 9, 9, 4 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b100 }), + randomFrom(Block.MvOrdering.values()) + ), + new IntArrayVector(new int[] { 9, 9 }, 2).filter(0, 1).asBlock(), + new IntArrayVector(new int[] { 9, 9, 4 }, 2).filter(0, 1).asBlock(), + new IntArrayVector(new int[] { 9, 9, 4 }, 3).filter(0, 1).asBlock(), + new IntArrayVector(new int[] { 9, 4, 9 }, 3).filter(0, 2).asBlock(), + IntBlock.newConstantBlockWith(9, 2), + IntBlock.newBlockBuilder(2).appendInt(9).appendInt(9).build(), + IntBlock.newBlockBuilder(2).appendInt(9).appendInt(9).build().filter(0, 1), + IntBlock.newBlockBuilder(2).appendInt(9).appendInt(4).appendInt(9).build().filter(0, 2), + IntBlock.newBlockBuilder(2).appendInt(9).appendNull().appendInt(9).build().filter(0, 2) + ); + assertAllEquals(moreBlocks); + } + + public void testVectorInequality() { + // all these vectors should NOT be equivalent + List notEqualVectors = List.of( + new IntArrayVector(new int[] { 1 }, 1), + new IntArrayVector(new int[] { 9 }, 1), + new IntArrayVector(new int[] { 1, 2 }, 2), + new IntArrayVector(new int[] { 1, 2, 3 }, 3), + new IntArrayVector(new int[] { 1, 2, 4 }, 3), + IntBlock.newConstantBlockWith(9, 2).asVector(), + IntBlock.newBlockBuilder(2).appendInt(1).appendInt(2).build().asVector().filter(1), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(5).build().asVector(), + IntBlock.newBlockBuilder(1).appendInt(1).appendInt(2).appendInt(3).appendInt(4).build().asVector() + ); + assertAllNotEquals(notEqualVectors); + } + + public void testBlockInequality() { + // all these blocks should NOT be equivalent + List notEqualBlocks = List.of( + new IntArrayVector(new int[] { 1 }, 1).asBlock(), + new IntArrayVector(new int[] { 9 }, 1).asBlock(), + new IntArrayVector(new int[] { 1, 2 }, 2).asBlock(), + new IntArrayVector(new int[] { 1, 2, 3 }, 3).asBlock(), + new IntArrayVector(new int[] { 1, 2, 4 }, 3).asBlock(), + IntBlock.newConstantBlockWith(9, 2), + IntBlock.newBlockBuilder(2).appendInt(1).appendInt(2).build().filter(1), + IntBlock.newBlockBuilder(3).appendInt(1).appendInt(2).appendInt(5).build(), + IntBlock.newBlockBuilder(1).appendInt(1).appendInt(2).appendInt(3).appendInt(4).build(), + IntBlock.newBlockBuilder(1).appendInt(1).appendNull().build(), + IntBlock.newBlockBuilder(1).appendInt(1).appendNull().appendInt(3).build(), + IntBlock.newBlockBuilder(1).appendInt(1).appendInt(3).build(), + IntBlock.newBlockBuilder(3).appendInt(1).beginPositionEntry().appendInt(2).appendInt(3).build() + ); + assertAllNotEquals(notEqualBlocks); + } + + public void testSimpleBlockWithSingleNull() { + List blocks = List.of( + IntBlock.newBlockBuilder(1).appendInt(1).appendNull().appendInt(3).build(), + IntBlock.newBlockBuilder(1).appendInt(1).appendNull().appendInt(3).build() + ); + assertEquals(3, blocks.get(0).getPositionCount()); + assertTrue(blocks.get(0).isNull(1)); + assertTrue(blocks.get(0).asVector() == null); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyNulls() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = IntBlock.newBlockBuilder(grow ? 0 : positions); + IntStream.range(0, positions).forEach(i -> builder.appendNull()); + IntBlock block1 = builder.build(); + IntBlock block2 = builder.build(); + assertEquals(positions, block1.getPositionCount()); + assertTrue(block1.mayHaveNulls()); + assertTrue(block1.isNull(0)); + + List blocks = List.of(block1, block2); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithSingleMultiValue() { + List blocks = List.of( + IntBlock.newBlockBuilder(1).beginPositionEntry().appendInt(1).appendInt(2).build(), + IntBlock.newBlockBuilder(1).beginPositionEntry().appendInt(1).appendInt(2).build() + ); + assertEquals(1, blocks.get(0).getPositionCount()); + assertEquals(2, blocks.get(0).getValueCount(0)); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyMultiValues() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = IntBlock.newBlockBuilder(grow ? 0 : positions); + for (int pos = 0; pos < positions; pos++) { + builder.beginPositionEntry(); + int values = randomIntBetween(1, 16); + IntStream.range(0, values).forEach(i -> builder.appendInt(randomInt())); + } + IntBlock block1 = builder.build(); + IntBlock block2 = builder.build(); + IntBlock block3 = builder.build(); + + assertEquals(positions, block1.getPositionCount()); + assertAllEquals(List.of(block1, block2, block3)); + } + + static void assertAllEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + assertEquals(obj1, obj2); + assertEquals(obj2, obj1); + // equal objects MUST generate the same hash code + assertEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } + + static void assertAllNotEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + if (obj1 == obj2) { + continue; // skip self + } + assertNotEquals(obj1, obj2); + assertNotEquals(obj2, obj1); + // unequal objects SHOULD generate the different hash code + assertNotEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/LongBlockEqualityTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/LongBlockEqualityTests.java new file mode 100644 index 0000000000000..3d08b2a96d635 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/LongBlockEqualityTests.java @@ -0,0 +1,257 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.ESTestCase; + +import java.util.BitSet; +import java.util.List; +import java.util.stream.IntStream; + +public class LongBlockEqualityTests extends ESTestCase { + + public void testEmptyVector() { + // all these "empty" vectors should be equivalent + List vectors = List.of( + new LongArrayVector(new long[] {}, 0), + new LongArrayVector(new long[] { 0 }, 0), + LongBlock.newConstantBlockWith(0, 0).asVector(), + LongBlock.newConstantBlockWith(0, 0).filter().asVector(), + LongBlock.newBlockBuilder(0).build().asVector(), + LongBlock.newBlockBuilder(0).appendLong(1).build().asVector().filter() + ); + assertAllEquals(vectors); + } + + public void testEmptyBlock() { + // all these "empty" vectors should be equivalent + List blocks = List.of( + new LongArrayBlock(new long[] {}, 0, new int[] {}, BitSet.valueOf(new byte[] { 0b00 }), randomFrom(Block.MvOrdering.values())), + new LongArrayBlock( + new long[] { 0 }, + 0, + new int[] {}, + BitSet.valueOf(new byte[] { 0b00 }), + randomFrom(Block.MvOrdering.values()) + ), + LongBlock.newConstantBlockWith(0, 0), + LongBlock.newBlockBuilder(0).build(), + LongBlock.newBlockBuilder(0).appendLong(1).build().filter(), + LongBlock.newBlockBuilder(0).appendNull().build().filter() + ); + assertAllEquals(blocks); + } + + public void testVectorEquality() { + // all these vectors should be equivalent + List vectors = List.of( + new LongArrayVector(new long[] { 1, 2, 3 }, 3), + new LongArrayVector(new long[] { 1, 2, 3 }, 3).asBlock().asVector(), + new LongArrayVector(new long[] { 1, 2, 3, 4 }, 3), + new LongArrayVector(new long[] { 1, 2, 3 }, 3).filter(0, 1, 2), + new LongArrayVector(new long[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2), + new LongArrayVector(new long[] { 0, 1, 2, 3 }, 4).filter(1, 2, 3), + new LongArrayVector(new long[] { 1, 4, 2, 3 }, 4).filter(0, 2, 3), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(3).build().asVector(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(3).build().asVector().filter(0, 1, 2), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(4).appendLong(2).appendLong(3).build().filter(0, 2, 3).asVector(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(4).appendLong(2).appendLong(3).build().asVector().filter(0, 2, 3) + ); + assertAllEquals(vectors); + + // all these constant-like vectors should be equivalent + List moreVectors = List.of( + new LongArrayVector(new long[] { 1, 1, 1 }, 3), + new LongArrayVector(new long[] { 1, 1, 1 }, 3).asBlock().asVector(), + new LongArrayVector(new long[] { 1, 1, 1, 1 }, 3), + new LongArrayVector(new long[] { 1, 1, 1 }, 3).filter(0, 1, 2), + new LongArrayVector(new long[] { 1, 1, 1, 4 }, 4).filter(0, 1, 2), + new LongArrayVector(new long[] { 3, 1, 1, 1 }, 4).filter(1, 2, 3), + new LongArrayVector(new long[] { 1, 4, 1, 1 }, 4).filter(0, 2, 3), + LongBlock.newConstantBlockWith(1, 3).asVector(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(1).appendLong(1).build().asVector(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(1).appendLong(1).build().asVector().filter(0, 1, 2), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(4).appendLong(1).appendLong(1).build().filter(0, 2, 3).asVector(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(4).appendLong(1).appendLong(1).build().asVector().filter(0, 2, 3) + ); + assertAllEquals(moreVectors); + } + + public void testBlockEquality() { + // all these blocks should be equivalent + List blocks = List.of( + new LongArrayVector(new long[] { 1, 2, 3 }, 3).asBlock(), + new LongArrayBlock( + new long[] { 1, 2, 3 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new LongArrayBlock( + new long[] { 1, 2, 3, 4 }, + 3, + new int[] { 0, 1, 2, 3 }, + BitSet.valueOf(new byte[] { 0b1000 }), + randomFrom(Block.MvOrdering.values()) + ), + new LongArrayVector(new long[] { 1, 2, 3 }, 3).filter(0, 1, 2).asBlock(), + new LongArrayVector(new long[] { 1, 2, 3, 4 }, 3).filter(0, 1, 2).asBlock(), + new LongArrayVector(new long[] { 1, 2, 3, 4 }, 4).filter(0, 1, 2).asBlock(), + new LongArrayVector(new long[] { 1, 2, 4, 3 }, 4).filter(0, 1, 3).asBlock(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(3).build(), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(3).build().filter(0, 1, 2), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(4).appendLong(2).appendLong(3).build().filter(0, 2, 3), + LongBlock.newBlockBuilder(3).appendLong(1).appendNull().appendLong(2).appendLong(3).build().filter(0, 2, 3) + ); + assertAllEquals(blocks); + + // all these constant-like blocks should be equivalent + List moreBlocks = List.of( + new LongArrayVector(new long[] { 9, 9 }, 2).asBlock(), + new LongArrayBlock( + new long[] { 9, 9 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b000 }), + randomFrom(Block.MvOrdering.values()) + ), + new LongArrayBlock( + new long[] { 9, 9, 4 }, + 2, + new int[] { 0, 1, 2 }, + BitSet.valueOf(new byte[] { 0b100 }), + randomFrom(Block.MvOrdering.values()) + ), + new LongArrayVector(new long[] { 9, 9 }, 2).filter(0, 1).asBlock(), + new LongArrayVector(new long[] { 9, 9, 4 }, 2).filter(0, 1).asBlock(), + new LongArrayVector(new long[] { 9, 9, 4 }, 3).filter(0, 1).asBlock(), + new LongArrayVector(new long[] { 9, 4, 9 }, 3).filter(0, 2).asBlock(), + LongBlock.newConstantBlockWith(9, 2), + LongBlock.newBlockBuilder(2).appendLong(9).appendLong(9).build(), + LongBlock.newBlockBuilder(2).appendLong(9).appendLong(9).build().filter(0, 1), + LongBlock.newBlockBuilder(2).appendLong(9).appendLong(4).appendLong(9).build().filter(0, 2), + LongBlock.newBlockBuilder(2).appendLong(9).appendNull().appendLong(9).build().filter(0, 2) + ); + assertAllEquals(moreBlocks); + } + + public void testVectorInequality() { + // all these vectors should NOT be equivalent + List notEqualVectors = List.of( + new LongArrayVector(new long[] { 1 }, 1), + new LongArrayVector(new long[] { 9 }, 1), + new LongArrayVector(new long[] { 1, 2 }, 2), + new LongArrayVector(new long[] { 1, 2, 3 }, 3), + new LongArrayVector(new long[] { 1, 2, 4 }, 3), + LongBlock.newConstantBlockWith(9, 2).asVector(), + LongBlock.newBlockBuilder(2).appendLong(1).appendLong(2).build().asVector().filter(1), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(5).build().asVector(), + LongBlock.newBlockBuilder(1).appendLong(1).appendLong(2).appendLong(3).appendLong(4).build().asVector() + ); + assertAllNotEquals(notEqualVectors); + } + + public void testBlockInequality() { + // all these blocks should NOT be equivalent + List notEqualBlocks = List.of( + new LongArrayVector(new long[] { 1 }, 1).asBlock(), + new LongArrayVector(new long[] { 9 }, 1).asBlock(), + new LongArrayVector(new long[] { 1, 2 }, 2).asBlock(), + new LongArrayVector(new long[] { 1, 2, 3 }, 3).asBlock(), + new LongArrayVector(new long[] { 1, 2, 4 }, 3).asBlock(), + LongBlock.newConstantBlockWith(9, 2), + LongBlock.newBlockBuilder(2).appendLong(1).appendLong(2).build().filter(1), + LongBlock.newBlockBuilder(3).appendLong(1).appendLong(2).appendLong(5).build(), + LongBlock.newBlockBuilder(1).appendLong(1).appendLong(2).appendLong(3).appendLong(4).build(), + LongBlock.newBlockBuilder(1).appendLong(1).appendNull().build(), + LongBlock.newBlockBuilder(1).appendLong(1).appendNull().appendLong(3).build(), + LongBlock.newBlockBuilder(1).appendLong(1).appendLong(3).build(), + LongBlock.newBlockBuilder(3).appendLong(1).beginPositionEntry().appendLong(2).appendLong(3).build() + ); + assertAllNotEquals(notEqualBlocks); + } + + public void testSimpleBlockWithSingleNull() { + List blocks = List.of( + LongBlock.newBlockBuilder(1).appendLong(1).appendNull().appendLong(3).build(), + LongBlock.newBlockBuilder(1).appendLong(1).appendNull().appendLong(3).build() + ); + assertEquals(3, blocks.get(0).getPositionCount()); + assertTrue(blocks.get(0).isNull(1)); + assertTrue(blocks.get(0).asVector() == null); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyNulls() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = LongBlock.newBlockBuilder(grow ? 0 : positions); + IntStream.range(0, positions).forEach(i -> builder.appendNull()); + LongBlock block1 = builder.build(); + LongBlock block2 = builder.build(); + assertEquals(positions, block1.getPositionCount()); + assertTrue(block1.mayHaveNulls()); + assertTrue(block1.isNull(0)); + + List blocks = List.of(block1, block2); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithSingleMultiValue() { + List blocks = List.of( + LongBlock.newBlockBuilder(1).beginPositionEntry().appendLong(1).appendLong(2).build(), + LongBlock.newBlockBuilder(1).beginPositionEntry().appendLong(1).appendLong(2).build() + ); + assertEquals(1, blocks.get(0).getPositionCount()); + assertEquals(2, blocks.get(0).getValueCount(0)); + assertAllEquals(blocks); + } + + public void testSimpleBlockWithManyMultiValues() { + int positions = randomIntBetween(1, 256); + boolean grow = randomBoolean(); + var builder = LongBlock.newBlockBuilder(grow ? 0 : positions); + for (int pos = 0; pos < positions; pos++) { + builder.beginPositionEntry(); + int values = randomIntBetween(1, 16); + IntStream.range(0, values).forEach(i -> builder.appendLong(randomLong())); + } + LongBlock block1 = builder.build(); + LongBlock block2 = builder.build(); + LongBlock block3 = builder.build(); + + assertEquals(positions, block1.getPositionCount()); + assertAllEquals(List.of(block1, block2, block3)); + } + + static void assertAllEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + assertEquals(obj1, obj2); + assertEquals(obj2, obj1); + // equal objects must generate the same hash code + assertEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } + + static void assertAllNotEquals(List objs) { + for (Object obj1 : objs) { + for (Object obj2 : objs) { + if (obj1 == obj2) { + continue; // skip self + } + assertNotEquals(obj1, obj2); + assertNotEquals(obj2, obj1); + // unequal objects SHOULD generate the different hash code + assertNotEquals(obj1.hashCode(), obj2.hashCode()); + } + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/MultiValueBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/MultiValueBlockTests.java new file mode 100644 index 0000000000000..9e809ac511a79 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/MultiValueBlockTests.java @@ -0,0 +1,236 @@ +/* + * 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.compute.data; + +import org.elasticsearch.test.EqualsHashCodeTestUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class MultiValueBlockTests extends SerializationTestCase { + + public void testIntBlockTrivial1() { + var blockBuilder = IntBlock.newBlockBuilder(4); + blockBuilder.appendInt(10); + blockBuilder.beginPositionEntry(); + blockBuilder.appendInt(21); + blockBuilder.appendInt(22); + blockBuilder.appendInt(23); + IntBlock block = blockBuilder.build(); + + // expect two positions + assertThat(block.getPositionCount(), is(2)); + + // expect four values + assertThat(block.getTotalValueCount(), is(4)); + + // assert first position + assertThat(block.getValueCount(0), is(1)); + assertThat(block.getFirstValueIndex(0), is(0)); + assertThat(block.getInt(block.getFirstValueIndex(0)), is(10)); + + // assert second position + assertThat(block.getValueCount(1), is(3)); + assertThat(block.getFirstValueIndex(1), is(1)); + int expectedValue = 21; + for (int i = 0; i < block.getValueCount(1); i++) { + assertThat(block.getInt(block.getFirstValueIndex(1) + i), is(expectedValue)); + expectedValue++; + } + + // cannot get a Vector view + assertNull(block.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(block, b -> serializeDeserializeBlock(b)); + } + + public void testIntBlockTrivial() { + var blockBuilder = IntBlock.newBlockBuilder(10); + blockBuilder.appendInt(1); + blockBuilder.beginPositionEntry(); + blockBuilder.appendInt(21); + blockBuilder.appendInt(22); + blockBuilder.appendInt(23); + blockBuilder.endPositionEntry(); + blockBuilder.beginPositionEntry(); + blockBuilder.appendInt(31); + blockBuilder.appendInt(32); + blockBuilder.endPositionEntry(); + blockBuilder.beginPositionEntry(); + blockBuilder.appendInt(41); + blockBuilder.endPositionEntry(); + IntBlock block = blockBuilder.build(); + + assertThat(block.getPositionCount(), is(4)); + assertThat(block.getFirstValueIndex(0), is(0)); + assertThat(block.getValueCount(0), is(1)); + assertThat(block.getInt(block.getFirstValueIndex(0)), is(1)); + assertNull(block.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(block, b -> serializeDeserializeBlock(b)); + } + + public void testEmpty() { + for (int initialSize : new int[] { 0, 10, 100, randomInt(512) }) { + IntBlock intBlock = IntBlock.newBlockBuilder(initialSize).build(); + assertThat(intBlock.getPositionCount(), is(0)); + assertThat(intBlock.asVector(), is(notNullValue())); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + LongBlock longBlock = LongBlock.newBlockBuilder(initialSize).build(); + assertThat(longBlock.getPositionCount(), is(0)); + assertThat(longBlock.asVector(), is(notNullValue())); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(longBlock, block -> serializeDeserializeBlock(block)); + + DoubleBlock doubleBlock = DoubleBlock.newBlockBuilder(initialSize).build(); + assertThat(doubleBlock.getPositionCount(), is(0)); + assertThat(doubleBlock.asVector(), is(notNullValue())); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(doubleBlock, block -> serializeDeserializeBlock(block)); + + BytesRefBlock bytesRefBlock = BytesRefBlock.newBlockBuilder(initialSize).build(); + assertThat(bytesRefBlock.getPositionCount(), is(0)); + assertThat(bytesRefBlock.asVector(), is(notNullValue())); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(bytesRefBlock, block -> serializeDeserializeBlock(block)); + } + } + + public void testNullOnly() throws IOException { + for (int initialSize : new int[] { 0, 10, 100, randomInt(512) }) { + IntBlock intBlock = IntBlock.newBlockBuilder(initialSize).appendNull().build(); + assertThat(intBlock.getPositionCount(), is(1)); + assertThat(intBlock.getValueCount(0), is(0)); + assertNull(intBlock.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + LongBlock longBlock = LongBlock.newBlockBuilder(initialSize).appendNull().build(); + assertThat(longBlock.getPositionCount(), is(1)); + assertThat(longBlock.getValueCount(0), is(0)); + assertNull(longBlock.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(longBlock, block -> serializeDeserializeBlock(block)); + + DoubleBlock doubleBlock = DoubleBlock.newBlockBuilder(initialSize).appendNull().build(); + assertThat(doubleBlock.getPositionCount(), is(1)); + assertThat(doubleBlock.getValueCount(0), is(0)); + assertNull(doubleBlock.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(doubleBlock, block -> serializeDeserializeBlock(block)); + + BytesRefBlock bytesRefBlock = BytesRefBlock.newBlockBuilder(initialSize).appendNull().build(); + assertThat(bytesRefBlock.getPositionCount(), is(1)); + assertThat(bytesRefBlock.getValueCount(0), is(0)); + assertNull(bytesRefBlock.asVector()); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(bytesRefBlock, block -> serializeDeserializeBlock(block)); + } + } + + public void testNullsFollowedByValues() { + List> blockValues = List.of( + List.of(), + List.of(), + List.of(), + List.of(), + List.of(), + List.of(), + List.of(), + List.of(), + List.of(), + List.of(1), + List.of(2) + ); + + Block intBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.INT); + assertThat(intBlock.elementType(), is(equalTo(ElementType.INT))); + BlockValueAsserter.assertBlockValues(intBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block longBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.LONG); + assertThat(longBlock.elementType(), is(equalTo(ElementType.LONG))); + BlockValueAsserter.assertBlockValues(longBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block doubleBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.DOUBLE); + assertThat(doubleBlock.elementType(), is(equalTo(ElementType.DOUBLE))); + BlockValueAsserter.assertBlockValues(doubleBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block bytesRefBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.BYTES_REF); + assertThat(bytesRefBlock.elementType(), is(equalTo(ElementType.BYTES_REF))); + BlockValueAsserter.assertBlockValues(bytesRefBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + } + + public void testMultiValuesAndNullsSmall() { + List> blockValues = List.of( + List.of(100), + List.of(), + List.of(20, 21, 22), + List.of(), + List.of(), + List.of(50), + List.of(61, 62, 63) + ); + + Block intBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.INT); + assertThat(intBlock.elementType(), is(equalTo(ElementType.INT))); + BlockValueAsserter.assertBlockValues(intBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block longBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.LONG); + assertThat(longBlock.elementType(), is(equalTo(ElementType.LONG))); + BlockValueAsserter.assertBlockValues(longBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block doubleBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.DOUBLE); + assertThat(doubleBlock.elementType(), is(equalTo(ElementType.DOUBLE))); + BlockValueAsserter.assertBlockValues(doubleBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block bytesRefBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.BYTES_REF); + assertThat(bytesRefBlock.elementType(), is(equalTo(ElementType.BYTES_REF))); + BlockValueAsserter.assertBlockValues(bytesRefBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + } + + public void testMultiValuesAndNulls() { + List> blockValues = new ArrayList<>(); + int positions = randomInt(512); + for (int i = 0; i < positions; i++) { + boolean isNull = randomBoolean(); + if (isNull) { + blockValues.add(List.of()); // empty / null + } else { + int rowValueCount = randomInt(16); + List row = new ArrayList<>(); + randomInts(rowValueCount).forEach(row::add); + blockValues.add(row); + } + } + + Block intBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.INT); + assertThat(intBlock.elementType(), is(equalTo(ElementType.INT))); + BlockValueAsserter.assertBlockValues(intBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block longBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.LONG); + assertThat(longBlock.elementType(), is(equalTo(ElementType.LONG))); + BlockValueAsserter.assertBlockValues(longBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block doubleBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.DOUBLE); + assertThat(doubleBlock.elementType(), is(equalTo(ElementType.DOUBLE))); + BlockValueAsserter.assertBlockValues(doubleBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + + Block bytesRefBlock = TestBlockBuilder.blockFromValues(blockValues, ElementType.BYTES_REF); + assertThat(bytesRefBlock.elementType(), is(equalTo(ElementType.BYTES_REF))); + BlockValueAsserter.assertBlockValues(bytesRefBlock, blockValues); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(intBlock, block -> serializeDeserializeBlock(block)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SerializationTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SerializationTestCase.java new file mode 100644 index 0000000000000..62b754d76fe49 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SerializationTestCase.java @@ -0,0 +1,49 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.io.UncheckedIOException; + +abstract class SerializationTestCase extends ESTestCase { + + final NamedWriteableRegistry registry = new NamedWriteableRegistry(Block.getNamedWriteables()); + + Page serializeDeserializePage(Page origPage) throws IOException { + try (BytesStreamOutput out = new BytesStreamOutput()) { + origPage.writeTo(out); + StreamInput in = new NamedWriteableAwareStreamInput(ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), registry); + return new Page(in); + } + } + + @SuppressWarnings("unchecked") + T serializeDeserializeBlock(T origBlock) throws IOException { + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.writeNamedWriteable(origBlock); + StreamInput in = new NamedWriteableAwareStreamInput(ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), registry); + return (T) in.readNamedWriteable(Block.class); + } + } + + T uncheckedSerializeDeserializeBlock(T origBlock) { + try { + return serializeDeserializeBlock(origBlock); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/TestBlockBuilder.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/TestBlockBuilder.java new file mode 100644 index 0000000000000..0af2c631f5a4a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/TestBlockBuilder.java @@ -0,0 +1,296 @@ +/* + * 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.compute.data; + +import org.apache.lucene.util.BytesRef; + +import java.util.List; + +/** + * A generic block builder that builds blocks from boxed data. Allows to build similarly shaped and + * valued blocks of different types. + */ +public abstract class TestBlockBuilder implements Block.Builder { + + public abstract TestBlockBuilder appendObject(Object object); + + @Override + public abstract TestBlockBuilder appendNull(); + + @Override + public abstract TestBlockBuilder beginPositionEntry(); + + @Override + public abstract TestBlockBuilder endPositionEntry(); + + static Block blockFromValues(List> blockValues, ElementType elementType) { + TestBlockBuilder builder = builderOf(elementType); + for (List rowValues : blockValues) { + if (rowValues.isEmpty()) { + builder.appendNull(); + } else { + builder.beginPositionEntry(); + for (Object rowValue : rowValues) { + builder.appendObject(rowValue); + } + builder.endPositionEntry(); + } + } + return builder.build(); + } + + static TestBlockBuilder builderOf(ElementType type) { + return switch (type) { + case INT -> new TestIntBlockBuilder(0); + case LONG -> new TestLongBlockBuilder(0); + case DOUBLE -> new TestDoubleBlockBuilder(0); + case BYTES_REF -> new TestBytesRefBlockBuilder(0); + default -> throw new AssertionError(type); + }; + } + + static TestBlockBuilder ofInt(int estimatedSize) { + return new TestIntBlockBuilder(estimatedSize); + } + + static TestBlockBuilder ofLong(int estimatedSize) { + return new TestLongBlockBuilder(estimatedSize); + } + + static TestBlockBuilder ofDouble(int estimatedSize) { + return new TestDoubleBlockBuilder(estimatedSize); + } + + static TestBlockBuilder ofBytesRef(int estimatedSize) { + return new TestBytesRefBlockBuilder(estimatedSize); + } + + private static class TestIntBlockBuilder extends TestBlockBuilder { + + private final IntBlock.Builder builder; + + TestIntBlockBuilder(int estimatedSize) { + builder = IntBlock.newBlockBuilder(estimatedSize); + } + + @Override + public TestBlockBuilder appendObject(Object object) { + builder.appendInt(((Number) object).intValue()); + return this; + } + + @Override + public TestBlockBuilder appendNull() { + builder.appendNull(); + return this; + } + + @Override + public TestBlockBuilder beginPositionEntry() { + builder.beginPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder endPositionEntry() { + builder.endPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + builder.copyFrom(block, beginInclusive, endExclusive); + return this; + } + + @Override + public TestBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + builder.mvOrdering(mvOrdering); + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + builder.appendAllValuesToCurrentPosition(block); + return this; + } + + @Override + public IntBlock build() { + return builder.build(); + } + } + + private static class TestLongBlockBuilder extends TestBlockBuilder { + + private final LongBlock.Builder builder; + + TestLongBlockBuilder(int estimatedSize) { + builder = LongBlock.newBlockBuilder(estimatedSize); + } + + @Override + public TestBlockBuilder appendObject(Object object) { + builder.appendLong(((Number) object).longValue()); + return this; + } + + @Override + public TestBlockBuilder appendNull() { + builder.appendNull(); + return this; + } + + @Override + public TestBlockBuilder beginPositionEntry() { + builder.beginPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder endPositionEntry() { + builder.endPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + builder.copyFrom(block, beginInclusive, endExclusive); + return this; + } + + @Override + public TestBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + builder.mvOrdering(mvOrdering); + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + builder.appendAllValuesToCurrentPosition(block); + return this; + } + + @Override + public LongBlock build() { + return builder.build(); + } + } + + private static class TestDoubleBlockBuilder extends TestBlockBuilder { + + private final DoubleBlock.Builder builder; + + TestDoubleBlockBuilder(int estimatedSize) { + builder = DoubleBlock.newBlockBuilder(estimatedSize); + } + + @Override + public TestBlockBuilder appendObject(Object object) { + builder.appendDouble(((Number) object).doubleValue()); + return this; + } + + @Override + public TestBlockBuilder appendNull() { + builder.appendNull(); + return this; + } + + @Override + public TestBlockBuilder beginPositionEntry() { + builder.beginPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder endPositionEntry() { + builder.endPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + builder.copyFrom(block, beginInclusive, endExclusive); + return this; + } + + @Override + public TestBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + builder.mvOrdering(mvOrdering); + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + builder.appendAllValuesToCurrentPosition(block); + return this; + } + + @Override + public DoubleBlock build() { + return builder.build(); + } + } + + private static class TestBytesRefBlockBuilder extends TestBlockBuilder { + + private final BytesRefBlock.Builder builder; + + TestBytesRefBlockBuilder(int estimatedSize) { + builder = BytesRefBlock.newBlockBuilder(estimatedSize); + } + + @Override + public TestBlockBuilder appendObject(Object object) { + builder.appendBytesRef(new BytesRef(((Integer) object).toString())); + return this; + } + + @Override + public TestBlockBuilder appendNull() { + builder.appendNull(); + return this; + } + + @Override + public TestBlockBuilder beginPositionEntry() { + builder.beginPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder endPositionEntry() { + builder.endPositionEntry(); + return this; + } + + @Override + public TestBlockBuilder copyFrom(Block block, int beginInclusive, int endExclusive) { + builder.copyFrom(block, beginInclusive, endExclusive); + return this; + } + + @Override + public TestBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + builder.mvOrdering(mvOrdering); + return this; + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + builder.appendAllValuesToCurrentPosition(block); + return this; + } + + @Override + public BytesRefBlock build() { + return builder.build(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorStatusTests.java new file mode 100644 index 0000000000000..149ea1c216e52 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorStatusTests.java @@ -0,0 +1,88 @@ +/* + * 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.compute.lucene; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class LuceneSourceOperatorStatusTests extends AbstractWireSerializingTestCase { + public static LuceneSourceOperator.Status simple() { + return new LuceneSourceOperator.Status(0, 1, 5, 123, 99990); + } + + public static String simpleToJson() { + return """ + {"current_leaf":0,"total_leaves":1,"leaf_position":123,"leaf_size":99990,"pages_emitted":5}"""; + } + + public void testToXContent() { + assertThat(Strings.toString(simple()), equalTo(simpleToJson())); + } + + @Override + protected Writeable.Reader instanceReader() { + return LuceneSourceOperator.Status::new; + } + + @Override + public LuceneSourceOperator.Status createTestInstance() { + return new LuceneSourceOperator.Status( + randomNonNegativeInt(), + randomNonNegativeInt(), + randomNonNegativeInt(), + randomNonNegativeInt(), + randomNonNegativeInt() + ); + } + + @Override + protected LuceneSourceOperator.Status mutateInstance(LuceneSourceOperator.Status instance) { + return switch (between(0, 4)) { + case 0 -> new LuceneSourceOperator.Status( + randomValueOtherThan(instance.currentLeaf(), ESTestCase::randomNonNegativeInt), + instance.totalLeaves(), + instance.pagesEmitted(), + instance.leafPosition(), + instance.leafSize() + ); + case 1 -> new LuceneSourceOperator.Status( + instance.currentLeaf(), + randomValueOtherThan(instance.totalLeaves(), ESTestCase::randomNonNegativeInt), + instance.pagesEmitted(), + instance.leafPosition(), + instance.leafSize() + ); + case 2 -> new LuceneSourceOperator.Status( + instance.currentLeaf(), + instance.totalLeaves(), + randomValueOtherThan(instance.pagesEmitted(), ESTestCase::randomNonNegativeInt), + instance.leafPosition(), + instance.leafSize() + ); + case 3 -> new LuceneSourceOperator.Status( + instance.currentLeaf(), + instance.totalLeaves(), + instance.pagesEmitted(), + randomValueOtherThan(instance.leafPosition(), ESTestCase::randomNonNegativeInt), + instance.leafSize() + ); + case 4 -> new LuceneSourceOperator.Status( + instance.currentLeaf(), + instance.totalLeaves(), + instance.pagesEmitted(), + instance.leafPosition(), + randomValueOtherThan(instance.leafSize(), ESTestCase::randomNonNegativeInt) + ); + default -> throw new UnsupportedOperationException(); + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java new file mode 100644 index 0000000000000..35ac2f588a3ee --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java @@ -0,0 +1,187 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AnyOperatorTestCase; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OperatorTestCase; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.fielddata.FieldDataContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NestedLookup; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.query.support.NestedScope; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortBuilder; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class LuceneSourceOperatorTests extends AnyOperatorTestCase { + private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.INTEGER); + private Directory directory = newDirectory(); + private IndexReader reader; + + @After + public void closeIndex() throws IOException { + IOUtils.close(reader, directory); + } + + @Override + protected LuceneSourceOperator.LuceneSourceOperatorFactory simple(BigArrays bigArrays) { + return simple(bigArrays, DataPartitioning.SHARD, 10_000, 100); + } + + private LuceneSourceOperator.LuceneSourceOperatorFactory simple( + BigArrays bigArrays, + DataPartitioning dataPartitioning, + int size, + int limit + ) { + int commitEvery = Math.max(1, size / 10); + try ( + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + for (int d = 0; d < size; d++) { + List doc = new ArrayList<>(); + doc.add(new SortedNumericDocValuesField("s", d)); + writer.addDocument(doc); + if (d % commitEvery == 0) { + writer.commit(); + } + } + reader = writer.getReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + SearchContext ctx = mock(SearchContext.class); + SearchExecutionContext ectx = mock(SearchExecutionContext.class); + when(ctx.getSearchExecutionContext()).thenReturn(ectx); + when(ectx.getFieldType(anyString())).thenAnswer(inv -> { + String name = inv.getArgument(0); + return switch (name) { + case "s" -> S_FIELD; + default -> throw new IllegalArgumentException("don't support [" + name + "]"); + }; + }); + when(ectx.getForField(any(), any())).thenAnswer(inv -> { + MappedFieldType ft = inv.getArgument(0); + IndexFieldData.Builder builder = ft.fielddataBuilder(FieldDataContext.noRuntimeFields("test")); + return builder.build(new IndexFieldDataCache.None(), bigArrays.breakerService()); + }); + when(ectx.nestedScope()).thenReturn(new NestedScope()); + when(ectx.nestedLookup()).thenReturn(NestedLookup.EMPTY); + when(ectx.getIndexReader()).thenReturn(reader); + Function queryFunction = c -> new MatchAllDocsQuery(); + int taskConcurrency = 0; + int maxPageSize = between(10, Math.max(10, size)); + List> sorts = List.of(new FieldSortBuilder("s")); + return new LuceneSourceOperator.LuceneSourceOperatorFactory( + List.of(ctx), + queryFunction, + dataPartitioning, + taskConcurrency, + maxPageSize, + limit + ); + } + + @Override + protected String expectedToStringOfSimple() { + assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this + return "LuceneSourceOperator[shardId=0, maxPageSize=**random**]"; + } + + @Override + protected String expectedDescriptionOfSimple() { + assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this + return """ + LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = **random**, limit = 100, sorts = [{"s":{"order":"asc"}}]]"""; + } + + // TODO tests for the other data partitioning configurations + + public void testShardDataPartitioning() { + int size = between(1_000, 20_000); + int limit = between(10, size); + testSimple(size, limit); + } + + public void testEmpty() { + testSimple(0, between(10, 10_000)); + } + + private void testSimple(int size, int limit) { + DriverContext ctx = new DriverContext(); + LuceneSourceOperator.LuceneSourceOperatorFactory factory = simple(nonBreakingBigArrays(), DataPartitioning.SHARD, size, limit); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.LONG, + S_FIELD + ); + + List results = new ArrayList<>(); + OperatorTestCase.runDriver( + new Driver(ctx, factory.get(ctx), List.of(readS.get(ctx)), new PageConsumerOperator(page -> results.add(page)), () -> {}) + ); + OperatorTestCase.assertDriverContext(ctx); + + for (Page page : results) { + assertThat(page.getPositionCount(), lessThanOrEqualTo(factory.maxPageSize)); + } + + for (Page page : results) { + LongBlock sBlock = page.getBlock(1); + for (int p = 0; p < page.getPositionCount(); p++) { + assertThat(sBlock.getLong(sBlock.getFirstValueIndex(p)), both(greaterThanOrEqualTo(0L)).and(lessThan((long) size))); + } + } + int maxPages = Math.min(size, limit); + int minPages = (int) Math.ceil(maxPages / factory.maxPageSize); + assertThat(results, hasSize(both(greaterThanOrEqualTo(minPages)).and(lessThanOrEqualTo(maxPages)))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java new file mode 100644 index 0000000000000..7c732ec121ac3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java @@ -0,0 +1,191 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AnyOperatorTestCase; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OperatorTestCase; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.fielddata.FieldDataContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NestedLookup; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.query.support.NestedScope; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortBuilder; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class LuceneTopNSourceOperatorTests extends AnyOperatorTestCase { + private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.INTEGER); + private Directory directory = newDirectory(); + private IndexReader reader; + + @After + public void closeIndex() throws IOException { + IOUtils.close(reader, directory); + } + + @Override + protected LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory simple(BigArrays bigArrays) { + return simple(bigArrays, DataPartitioning.SHARD, 10_000, 100); + } + + private LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory simple( + BigArrays bigArrays, + DataPartitioning dataPartitioning, + int size, + int limit + ) { + int commitEvery = Math.max(1, size / 10); + try ( + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + for (int d = 0; d < size; d++) { + List doc = new ArrayList<>(); + doc.add(new SortedNumericDocValuesField("s", d)); + writer.addDocument(doc); + if (d % commitEvery == 0) { + writer.commit(); + } + } + reader = writer.getReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + SearchContext ctx = mock(SearchContext.class); + SearchExecutionContext ectx = mock(SearchExecutionContext.class); + when(ctx.getSearchExecutionContext()).thenReturn(ectx); + when(ectx.getFieldType(anyString())).thenAnswer(inv -> { + String name = inv.getArgument(0); + return switch (name) { + case "s" -> S_FIELD; + default -> throw new IllegalArgumentException("don't support [" + name + "]"); + }; + }); + when(ectx.getForField(any(), any())).thenAnswer(inv -> { + MappedFieldType ft = inv.getArgument(0); + IndexFieldData.Builder builder = ft.fielddataBuilder(FieldDataContext.noRuntimeFields("test")); + return builder.build(new IndexFieldDataCache.None(), bigArrays.breakerService()); + }); + when(ectx.nestedScope()).thenReturn(new NestedScope()); + when(ectx.nestedLookup()).thenReturn(NestedLookup.EMPTY); + when(ectx.getIndexReader()).thenReturn(reader); + Function queryFunction = c -> new MatchAllDocsQuery(); + int taskConcurrency = 0; + int maxPageSize = between(10, Math.max(10, size)); + List> sorts = List.of(new FieldSortBuilder("s")); + return new LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory( + List.of(ctx), + queryFunction, + dataPartitioning, + taskConcurrency, + maxPageSize, + limit, + sorts + ); + } + + @Override + protected String expectedToStringOfSimple() { + assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this + return "LuceneTopNSourceOperator[shardId=0, maxPageSize=**random**]"; + } + + @Override + protected String expectedDescriptionOfSimple() { + assumeFalse("can't support variable maxPageSize", true); // TODO allow testing this + return """ + LuceneTopNSourceOperator[dataPartitioning = SHARD, maxPageSize = **random**, limit = 100, sorts = [{"s":{"order":"asc"}}]]"""; + } + + // TODO tests for the other data partitioning configurations + + public void testShardDataPartitioning() { + int size = between(1_000, 20_000); + int limit = between(10, size); + testSimple(size, limit); + } + + public void testEmpty() { + testSimple(0, between(10, 10_000)); + } + + private void testSimple(int size, int limit) { + DriverContext ctx = new DriverContext(); + LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory factory = simple( + nonBreakingBigArrays(), + DataPartitioning.SHARD, + size, + limit + ); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.LONG, + S_FIELD + ); + + List results = new ArrayList<>(); + OperatorTestCase.runDriver( + new Driver(ctx, factory.get(ctx), List.of(readS.get(ctx)), new PageConsumerOperator(page -> results.add(page)), () -> {}) + ); + OperatorTestCase.assertDriverContext(ctx); + + long expectedS = 0; + for (Page page : results) { + if (limit - expectedS < factory.maxPageSize) { + assertThat(page.getPositionCount(), equalTo((int) (limit - expectedS))); + } else { + assertThat(page.getPositionCount(), equalTo(factory.maxPageSize)); + } + LongBlock sBlock = page.getBlock(1); + for (int p = 0; p < page.getPositionCount(); p++) { + assertThat(sBlock.getLong(sBlock.getFirstValueIndex(p)), equalTo(expectedS++)); + } + } + int pages = (int) Math.ceil((float) Math.min(size, limit) / factory.maxPageSize); + assertThat(results, hasSize(pages)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java new file mode 100644 index 0000000000000..6f0317b509e3b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorStatusTests.java @@ -0,0 +1,70 @@ +/* + * 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.compute.lucene; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +import java.io.IOException; +import java.util.Map; +import java.util.TreeMap; + +import static org.hamcrest.Matchers.equalTo; + +public class ValuesSourceReaderOperatorStatusTests extends AbstractWireSerializingTestCase { + public static ValuesSourceReaderOperator.Status simple() { + return new ValuesSourceReaderOperator.Status(Map.of("ReaderType", 3), 123); + } + + public static String simpleToJson() { + return """ + {"readers_built":{"ReaderType":3},"pages_processed":123}"""; + } + + public void testToXContent() { + assertThat(Strings.toString(simple()), equalTo(simpleToJson())); + } + + @Override + protected Writeable.Reader instanceReader() { + return ValuesSourceReaderOperator.Status::new; + } + + @Override + public ValuesSourceReaderOperator.Status createTestInstance() { + return new ValuesSourceReaderOperator.Status(randomReadersBuilt(), between(0, Integer.MAX_VALUE)); + } + + private Map randomReadersBuilt() { + int size = between(0, 10); + Map result = new TreeMap<>(); + while (result.size() < size) { + result.put(randomAlphaOfLength(4), between(0, Integer.MAX_VALUE)); + } + return result; + } + + @Override + protected ValuesSourceReaderOperator.Status mutateInstance(ValuesSourceReaderOperator.Status instance) throws IOException { + switch (between(0, 1)) { + case 0: + return new ValuesSourceReaderOperator.Status( + randomValueOtherThan(instance.readersBuilt(), this::randomReadersBuilt), + instance.pagesProcessed() + ); + case 1: + return new ValuesSourceReaderOperator.Status( + instance.readersBuilt(), + randomValueOtherThan(instance.pagesProcessed(), () -> between(0, Integer.MAX_VALUE)) + ); + default: + throw new UnsupportedOperationException(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java new file mode 100644 index 0000000000000..b30f1a5c27eff --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java @@ -0,0 +1,410 @@ +/* + * 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.compute.lucene; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoubleDocValuesField; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.NumericUtils; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.CannedSourceOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OperatorTestCase; +import org.elasticsearch.compute.operator.PageConsumerOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.fielddata.FieldDataContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; +import org.elasticsearch.index.mapper.BooleanFieldMapper; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class ValuesSourceReaderOperatorTests extends OperatorTestCase { + private static final String[] PREFIX = new String[] { "a", "b", "c" }; + private static final boolean[][] BOOLEANS = new boolean[][] { + { true }, + { false, true }, + { false, true, true }, + { false, false, true, true } }; + + private Directory directory = newDirectory(); + private IndexReader reader; + + @After + public void closeIndex() throws IOException { + IOUtils.close(reader, directory); + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.LONG, + new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG) + ); + } + + static Operator.OperatorFactory factory(IndexReader reader, ValuesSourceType vsType, ElementType elementType, MappedFieldType ft) { + IndexFieldData fd = ft.fielddataBuilder(FieldDataContext.noRuntimeFields("test")) + .build(new IndexFieldDataCache.None(), new NoneCircuitBreakerService()); + FieldContext fc = new FieldContext(ft.name(), fd, ft); + ValuesSource vs = vsType.getField(fc, null); + return new ValuesSourceReaderOperator.ValuesSourceReaderOperatorFactory( + List.of(new ValueSourceInfo(vsType, vs, elementType, reader)), + 0, + ft.name() + ); + } + + @Override + protected SourceOperator simpleInput(int size) { + // The test wants more than one segment. We shoot for about 10. + int commitEvery = Math.max(1, size / 10); + try ( + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + for (int d = 0; d < size; d++) { + List doc = new ArrayList<>(); + doc.add(new SortedNumericDocValuesField("key", d)); + doc.add(new SortedNumericDocValuesField("long", d)); + doc.add( + new KeywordFieldMapper.KeywordField("kwd", new BytesRef(Integer.toString(d)), KeywordFieldMapper.Defaults.FIELD_TYPE) + ); + doc.add(new SortedNumericDocValuesField("bool", d % 2 == 0 ? 1 : 0)); + doc.add(new SortedNumericDocValuesField("double", NumericUtils.doubleToSortableLong(d / 123_456d))); + for (int v = 0; v <= d % 3; v++) { + doc.add( + new KeywordFieldMapper.KeywordField("mv_kwd", new BytesRef(PREFIX[v] + d), KeywordFieldMapper.Defaults.FIELD_TYPE) + ); + doc.add(new SortedNumericDocValuesField("mv_bool", v % 2 == 0 ? 1 : 0)); + doc.add(new SortedNumericDocValuesField("mv_key", 1_000 * d + v)); + doc.add(new SortedNumericDocValuesField("mv_long", -1_000 * d + v)); + doc.add(new SortedNumericDocValuesField("mv_double", NumericUtils.doubleToSortableLong(d / 123_456d + v))); + } + writer.addDocument(doc); + if (d % commitEvery == 0) { + writer.commit(); + } + } + reader = writer.getReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return new LuceneSourceOperator(reader, 0, new MatchAllDocsQuery(), OperatorTestCase.randomPageSize(), LuceneOperator.NO_LIMIT); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "ValuesSourceReaderOperator[field = long]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + long expectedSum = 0; + long current = 0; + + long sum = 0; + for (Page r : results) { + LongBlock b = r.getBlock(r.getBlockCount() - 1); + for (int p = 0; p < b.getPositionCount(); p++) { + expectedSum += current; + current++; + sum += b.getLong(p); + } + } + + assertThat(sum, equalTo(expectedSum)); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } + + public void testLoadAll() { + loadSimpleAndAssert(CannedSourceOperator.collectPages(simpleInput(between(1_000, 100 * 1024)))); + } + + public void testLoadAllInOnePage() { + loadSimpleAndAssert( + List.of(CannedSourceOperator.mergePages(CannedSourceOperator.collectPages(simpleInput(between(1_000, 100 * 1024))))) + ); + } + + public void testEmpty() { + loadSimpleAndAssert(CannedSourceOperator.collectPages(simpleInput(0))); + } + + public void testLoadAllInOnePageShuffled() { + Page source = CannedSourceOperator.mergePages(CannedSourceOperator.collectPages(simpleInput(between(1_000, 100 * 1024)))); + List shuffleList = new ArrayList<>(); + IntStream.range(0, source.getPositionCount()).forEach(i -> shuffleList.add(i)); + Randomness.shuffle(shuffleList); + int[] shuffleArray = shuffleList.stream().mapToInt(Integer::intValue).toArray(); + Block[] shuffledBlocks = new Block[source.getBlockCount()]; + for (int b = 0; b < shuffledBlocks.length; b++) { + shuffledBlocks[b] = source.getBlock(b).filter(shuffleArray); + } + source = new Page(shuffledBlocks); + loadSimpleAndAssert(List.of(source)); + } + + private void loadSimpleAndAssert(List input) { + DriverContext driverContext = new DriverContext(); + List results = new ArrayList<>(); + List operators = List.of( + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.INT, + new NumberFieldMapper.NumberFieldType("key", NumberFieldMapper.NumberType.INTEGER) + ).get(driverContext), + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.LONG, + new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG) + ).get(driverContext), + factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, new KeywordFieldMapper.KeywordFieldType("kwd")).get( + driverContext + ), + factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, new KeywordFieldMapper.KeywordFieldType("mv_kwd")).get( + driverContext + ), + factory(reader, CoreValuesSourceType.BOOLEAN, ElementType.BOOLEAN, new BooleanFieldMapper.BooleanFieldType("bool")).get( + driverContext + ), + factory(reader, CoreValuesSourceType.BOOLEAN, ElementType.BOOLEAN, new BooleanFieldMapper.BooleanFieldType("mv_bool")).get( + driverContext + ), + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.INT, + new NumberFieldMapper.NumberFieldType("mv_key", NumberFieldMapper.NumberType.INTEGER) + ).get(driverContext), + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.LONG, + new NumberFieldMapper.NumberFieldType("mv_long", NumberFieldMapper.NumberType.LONG) + ).get(driverContext), + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.DOUBLE, + new NumberFieldMapper.NumberFieldType("double", NumberFieldMapper.NumberType.DOUBLE) + ).get(driverContext), + factory( + reader, + CoreValuesSourceType.NUMERIC, + ElementType.DOUBLE, + new NumberFieldMapper.NumberFieldType("mv_double", NumberFieldMapper.NumberType.DOUBLE) + ).get(driverContext) + ); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(input.iterator()), + operators, + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertThat(results, hasSize(input.size())); + for (Page p : results) { + assertThat(p.getBlockCount(), equalTo(11)); + IntVector keys = p.getBlock(1).asVector(); + LongVector longs = p.getBlock(2).asVector(); + BytesRefVector keywords = p.getBlock(3).asVector(); + BytesRefBlock mvKeywords = p.getBlock(4); + BooleanVector bools = p.getBlock(5).asVector(); + BooleanBlock mvBools = p.getBlock(6); + IntBlock mvInts = p.getBlock(7); + LongBlock mvLongs = p.getBlock(8); + DoubleVector doubles = p.getBlock(9).asVector(); + DoubleBlock mvDoubles = p.getBlock(10); + + for (int i = 0; i < p.getPositionCount(); i++) { + int key = keys.getInt(i); + assertThat(longs.getLong(i), equalTo((long) key)); + assertThat(keywords.getBytesRef(i, new BytesRef()).utf8ToString(), equalTo(Integer.toString(key))); + + assertThat(mvKeywords.getValueCount(i), equalTo(key % 3 + 1)); + int offset = mvKeywords.getFirstValueIndex(i); + for (int v = 0; v <= key % 3; v++) { + assertThat(mvKeywords.getBytesRef(offset + v, new BytesRef()).utf8ToString(), equalTo(PREFIX[v] + key)); + } + if (key % 3 > 0) { + assertThat(mvKeywords.mvOrdering(), equalTo(Block.MvOrdering.ASCENDING)); + } + + assertThat(bools.getBoolean(i), equalTo(key % 2 == 0)); + assertThat(mvBools.getValueCount(i), equalTo(key % 3 + 1)); + offset = mvBools.getFirstValueIndex(i); + for (int v = 0; v <= key % 3; v++) { + assertThat(mvBools.getBoolean(offset + v), equalTo(BOOLEANS[key % 3][v])); + } + if (key % 3 > 0) { + assertThat(mvBools.mvOrdering(), equalTo(Block.MvOrdering.ASCENDING)); + } + + assertThat(mvInts.getValueCount(i), equalTo(key % 3 + 1)); + offset = mvInts.getFirstValueIndex(i); + for (int v = 0; v <= key % 3; v++) { + assertThat(mvInts.getInt(offset + v), equalTo(1_000 * key + v)); + } + if (key % 3 > 0) { + assertThat(mvInts.mvOrdering(), equalTo(Block.MvOrdering.ASCENDING)); + } + + assertThat(mvLongs.getValueCount(i), equalTo(key % 3 + 1)); + offset = mvLongs.getFirstValueIndex(i); + for (int v = 0; v <= key % 3; v++) { + assertThat(mvLongs.getLong(offset + v), equalTo(-1_000L * key + v)); + } + if (key % 3 > 0) { + assertThat(mvLongs.mvOrdering(), equalTo(Block.MvOrdering.ASCENDING)); + } + + assertThat(doubles.getDouble(i), equalTo(key / 123_456d)); + offset = mvDoubles.getFirstValueIndex(i); + for (int v = 0; v <= key % 3; v++) { + assertThat(mvDoubles.getDouble(offset + v), equalTo(key / 123_456d + v)); + } + if (key % 3 > 0) { + assertThat(mvDoubles.mvOrdering(), equalTo(Block.MvOrdering.ASCENDING)); + } + } + } + for (Operator op : operators) { + assertThat(((ValuesSourceReaderOperator) op).status().pagesProcessed(), equalTo(input.size())); + } + assertDriverContext(driverContext); + } + + public void testValuesSourceReaderOperatorWithNulls() throws IOException { + MappedFieldType intFt = new NumberFieldMapper.NumberFieldType("i", NumberFieldMapper.NumberType.INTEGER); + MappedFieldType longFt = new NumberFieldMapper.NumberFieldType("j", NumberFieldMapper.NumberType.LONG); + MappedFieldType doubleFt = new NumberFieldMapper.NumberFieldType("d", NumberFieldMapper.NumberType.DOUBLE); + MappedFieldType kwFt = new KeywordFieldMapper.KeywordFieldType("kw"); + + NumericDocValuesField intField = new NumericDocValuesField(intFt.name(), 0); + NumericDocValuesField longField = new NumericDocValuesField(longFt.name(), 0); + NumericDocValuesField doubleField = new DoubleDocValuesField(doubleFt.name(), 0); + final int numDocs = 100_000; + try (RandomIndexWriter w = new RandomIndexWriter(random(), directory)) { + Document doc = new Document(); + for (int i = 0; i < numDocs; i++) { + doc.clear(); + intField.setLongValue(i); + doc.add(intField); + if (i % 100 != 0) { // Do not set field for every 100 values + longField.setLongValue(i); + doc.add(longField); + doubleField.setDoubleValue(i); + doc.add(doubleField); + doc.add(new SortedDocValuesField(kwFt.name(), new BytesRef("kw=" + i))); + } + w.addDocument(doc); + } + w.commit(); + reader = w.getReader(); + } + + DriverContext driverContext = new DriverContext(); + try ( + Driver driver = new Driver( + driverContext, + new LuceneSourceOperator(reader, 0, new MatchAllDocsQuery(), randomPageSize(), LuceneOperator.NO_LIMIT), + List.of( + factory(reader, CoreValuesSourceType.NUMERIC, ElementType.INT, intFt).get(driverContext), + factory(reader, CoreValuesSourceType.NUMERIC, ElementType.LONG, longFt).get(driverContext), + factory(reader, CoreValuesSourceType.NUMERIC, ElementType.DOUBLE, doubleFt).get(driverContext), + factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, kwFt).get(driverContext) + ), + new PageConsumerOperator(page -> { + logger.debug("New page: {}", page); + IntBlock intValuesBlock = page.getBlock(1); + LongBlock longValuesBlock = page.getBlock(2); + DoubleBlock doubleValuesBlock = page.getBlock(3); + BytesRefBlock keywordValuesBlock = page.getBlock(4); + + for (int i = 0; i < page.getPositionCount(); i++) { + assertFalse(intValuesBlock.isNull(i)); + long j = intValuesBlock.getInt(i); + // Every 100 documents we set fields to null + boolean fieldIsEmpty = j % 100 == 0; + assertEquals(fieldIsEmpty, longValuesBlock.isNull(i)); + assertEquals(fieldIsEmpty, doubleValuesBlock.isNull(i)); + assertEquals(fieldIsEmpty, keywordValuesBlock.isNull(i)); + } + }), + () -> {} + ) + ) { + runDriver(driver); + } + assertDriverContext(driverContext); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractBlockSourceOperator.java new file mode 100644 index 0000000000000..e44ebf304b621 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractBlockSourceOperator.java @@ -0,0 +1,61 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.test.ESTestCase; + +/** + * An abstract source operator. Implementations of this operator produce pages with a random + * number of positions up to a maximum of the given maxPagePositions positions. + */ +public abstract class AbstractBlockSourceOperator extends SourceOperator { + + private final int maxPagePositions; + + private boolean finished; + + /** The position of the next element to output. */ + protected int currentPosition; + + protected AbstractBlockSourceOperator(int maxPagePositions) { + this.maxPagePositions = maxPagePositions; + } + + /** The number of remaining elements that this source operator will produce. */ + protected abstract int remaining(); + + /** Creates a page containing a block with {@code length} positions, from the given position offset. */ + protected abstract Page createPage(int positionOffset, int length); + + @Override + public final Page getOutput() { + if (finished) { + return null; + } + if (remaining() <= 0) { + finish(); + return null; + } + int length = Math.min(ESTestCase.randomInt(maxPagePositions), remaining()); + return createPage(currentPosition, length); + } + + @Override + public final void close() {} + + @Override + public final boolean isFinished() { + return finished; + } + + @Override + public final void finish() { + finished = true; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractPageMappingOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractPageMappingOperatorStatusTests.java new file mode 100644 index 0000000000000..c72e87bb96a81 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AbstractPageMappingOperatorStatusTests.java @@ -0,0 +1,45 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class AbstractPageMappingOperatorStatusTests extends AbstractWireSerializingTestCase { + public static AbstractPageMappingOperator.Status simple() { + return new AbstractPageMappingOperator.Status(123); + } + + public static String simpleToJson() { + return """ + {"pages_processed":123}"""; + } + + public void testToXContent() { + assertThat(Strings.toString(simple()), equalTo(simpleToJson())); + } + + @Override + protected Writeable.Reader instanceReader() { + return AbstractPageMappingOperator.Status::new; + } + + @Override + public AbstractPageMappingOperator.Status createTestInstance() { + return new AbstractPageMappingOperator.Status(randomNonNegativeInt()); + } + + @Override + protected AbstractPageMappingOperator.Status mutateInstance(AbstractPageMappingOperator.Status instance) { + return new AbstractPageMappingOperator.Status(randomValueOtherThan(instance.pagesProcessed(), ESTestCase::randomNonNegativeInt)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AggregationOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AggregationOperatorTests.java new file mode 100644 index 0000000000000..9eaa1e333f66e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AggregationOperatorTests.java @@ -0,0 +1,89 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunction; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunctionTests; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunction; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunctionTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class AggregationOperatorTests extends ForkingOperatorTestCase { + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> randomLongBetween(-max, max))); + } + + @Override + protected Operator.OperatorFactory simpleWithMode(BigArrays bigArrays, AggregatorMode mode) { + List sumChannels, maxChannels; + if (mode.isInputPartial()) { + int sumInterChannelCount = SumLongAggregatorFunction.intermediateStateDesc().size(); + int maxInterChannelCount = MaxLongAggregatorFunction.intermediateStateDesc().size(); + sumChannels = IntStream.range(0, sumInterChannelCount).boxed().toList(); + maxChannels = IntStream.range(sumInterChannelCount, sumInterChannelCount + maxInterChannelCount).boxed().toList(); + } else { + sumChannels = maxChannels = List.of(0); + } + + return new AggregationOperator.AggregationOperatorFactory( + List.of( + new SumLongAggregatorFunctionSupplier(bigArrays, sumChannels).aggregatorFactory(mode), + new MaxLongAggregatorFunctionSupplier(bigArrays, maxChannels).aggregatorFactory(mode) + ), + mode + ); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "AggregationOperator[mode = SINGLE, aggs = sum of longs, max of longs]"; + } + + @Override + protected String expectedToStringOfSimple() { + return "AggregationOperator[aggregators=[" + + "Aggregator[aggregatorFunction=SumLongAggregatorFunction[channels=[0]], mode=SINGLE], " + + "Aggregator[aggregatorFunction=MaxLongAggregatorFunction[channels=[0]], mode=SINGLE]]]"; + } + + @Override + protected void assertSimpleOutput(List input, List results) { + assertThat(results, hasSize(1)); + assertThat(results.get(0).getBlockCount(), equalTo(2)); + assertThat(results.get(0).getPositionCount(), equalTo(1)); + + SumLongAggregatorFunctionTests sum = new SumLongAggregatorFunctionTests(); + MaxLongAggregatorFunctionTests max = new MaxLongAggregatorFunctionTests(); + + Block sums = results.get(0).getBlock(0); + Block maxs = results.get(0).getBlock(1); + sum.assertSimpleOutput(input.stream().map(p -> p.getBlock(0)).toList(), sums); + max.assertSimpleOutput(input.stream().map(p -> p.getBlock(0)).toList(), maxs); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big array so never breaks", false); + return null; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AnyOperatorTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AnyOperatorTestCase.java new file mode 100644 index 0000000000000..e70160041047e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AnyOperatorTestCase.java @@ -0,0 +1,88 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.matchesPattern; + +/** + * Superclass for testing any {@link Operator}, including {@link SourceOperator}s. + */ +public abstract class AnyOperatorTestCase extends ESTestCase { + /** + * The operator configured a "simple" or basic way, used for smoke testing + * descriptions and {@link BigArrays} and scatter/gather. + */ + protected abstract Operator.OperatorFactory simple(BigArrays bigArrays); + + /** + * The description of the operator produced by {@link #simple}. + */ + protected abstract String expectedDescriptionOfSimple(); + + /** + * The {@link #toString} of the operator produced by {@link #simple}. + * This {@linkplain #toString} is used by the status reporting and + * generally useful debug information. + */ + protected abstract String expectedToStringOfSimple(); + + /** + * the description of an Operator should be "OperatorName(additional info)" + * eg. "LimitOperator(limit = 10)" + * Additional info are optional + */ + private static final String OPERATOR_DESCRIBE_PATTERN = "^\\w*\\[.*\\]$"; + + /** + * the name a grouping agg function should be "aggName of type" for typed aggregations, eg. "avg of ints" + * or "aggName" for type agnostic aggregations, eg. "count" + */ + private static final String GROUPING_AGG_FUNCTION_DESCRIBE_PATTERN = "^\\w*( of \\w*$)?"; + + /** + * Makes sure the description of {@link #simple} matches the {@link #expectedDescriptionOfSimple}. + */ + public final void testSimpleDescription() { + Operator.OperatorFactory factory = simple(nonBreakingBigArrays()); + String description = factory.describe(); + assertThat(description, equalTo(expectedDescriptionOfSimple())); + DriverContext driverContext = new DriverContext(); + try (Operator op = factory.get(driverContext)) { + if (op instanceof GroupingAggregatorFunction) { + assertThat(description, matchesPattern(GROUPING_AGG_FUNCTION_DESCRIBE_PATTERN)); + } else { + assertThat(description, matchesPattern(OPERATOR_DESCRIBE_PATTERN)); + } + } + } + + /** + * Makes sure the description of {@link #simple} matches the {@link #expectedDescriptionOfSimple}. + */ + public final void testSimpleToString() { + try (Operator operator = simple(nonBreakingBigArrays()).get(new DriverContext())) { + assertThat(operator.toString(), equalTo(expectedToStringOfSimple())); + } + } + + /** + * A {@link BigArrays} that won't throw {@link CircuitBreakingException}. + */ + protected final BigArrays nonBreakingBigArrays() { + return new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()).withCircuitBreaking(); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AsyncOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AsyncOperatorTests.java new file mode 100644 index 0000000000000..1dea25cc0f02d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/AsyncOperatorTests.java @@ -0,0 +1,206 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.Matchers.equalTo; + +public class AsyncOperatorTests extends ESTestCase { + + private TestThreadPool threadPool; + + @Before + public void setThreadPool() { + int numThreads = randomBoolean() ? 1 : between(2, 16); + threadPool = new TestThreadPool( + "test", + new FixedExecutorBuilder(Settings.EMPTY, "esql_test_executor", numThreads, 1024, "esql", EsExecutors.TaskTrackingConfig.DEFAULT) + ); + } + + @After + public void shutdownThreadPool() { + terminate(threadPool); + } + + public void testBasic() { + int positions = randomIntBetween(0, 10_000); + List ids = new ArrayList<>(positions); + Map dict = new HashMap<>(); + for (int i = 0; i < positions; i++) { + long id = randomLong(); + ids.add(id); + if (randomBoolean()) { + dict.computeIfAbsent(id, k -> randomAlphaOfLength(5)); + } + } + SourceOperator sourceOperator = new AbstractBlockSourceOperator(randomIntBetween(10, 1000)) { + @Override + protected int remaining() { + return ids.size() - currentPosition; + } + + @Override + protected Page createPage(int positionOffset, int length) { + LongVector.Builder builder = LongVector.newVectorBuilder(length); + for (int i = 0; i < length; i++) { + builder.appendLong(ids.get(currentPosition++)); + } + return new Page(builder.build().asBlock()); + } + }; + int maxConcurrentRequests = randomIntBetween(1, 10); + AsyncOperator asyncOperator = new AsyncOperator(maxConcurrentRequests) { + final LookupService lookupService = new LookupService(threadPool, dict, maxConcurrentRequests); + + @Override + protected void performAsync(Page inputPage, ActionListener listener) { + lookupService.lookupAsync(inputPage, listener); + } + + @Override + public void close() { + + } + }; + Iterator it = ids.iterator(); + SinkOperator outputOperator = new PageConsumerOperator(page -> { + assertThat(page.getBlockCount(), equalTo(2)); + LongBlock b1 = page.getBlock(0); + BytesRefBlock b2 = page.getBlock(1); + BytesRef scratch = new BytesRef(); + for (int i = 0; i < page.getPositionCount(); i++) { + assertTrue(it.hasNext()); + long key = b1.getLong(i); + assertThat(key, equalTo(it.next())); + String v = dict.get(key); + if (v == null) { + assertTrue(b2.isNull(i)); + } else { + assertThat(b2.getBytesRef(i, scratch), equalTo(new BytesRef(v))); + } + } + }); + PlainActionFuture future = new PlainActionFuture<>(); + Driver driver = new Driver( + new DriverContext(), + sourceOperator, + List.of(asyncOperator), + outputOperator, + () -> assertFalse(it.hasNext()) + ); + Driver.start(threadPool.executor("esql_test_executor"), driver, between(1, 10000), future); + future.actionGet(); + } + + public void testStatus() { + Map> handlers = new HashMap<>(); + AsyncOperator operator = new AsyncOperator(2) { + @Override + protected void performAsync(Page inputPage, ActionListener listener) { + handlers.put(inputPage, listener); + } + + @Override + public void close() { + + } + }; + assertTrue(operator.isBlocked().isDone()); + assertTrue(operator.needsInput()); + + Page page1 = new Page(Block.constantNullBlock(1)); + operator.addInput(page1); + assertFalse(operator.isBlocked().isDone()); + ListenableActionFuture blocked1 = operator.isBlocked(); + assertTrue(operator.needsInput()); + + Page page2 = new Page(Block.constantNullBlock(2)); + operator.addInput(page2); + assertFalse(operator.needsInput()); // reached the max outstanding requests + assertFalse(operator.isBlocked().isDone()); + assertThat(operator.isBlocked(), equalTo(blocked1)); + + Page page3 = new Page(Block.constantNullBlock(3)); + handlers.remove(page1).onResponse(page3); + assertFalse(operator.needsInput()); // still have 2 outstanding requests + assertTrue(operator.isBlocked().isDone()); + assertTrue(blocked1.isDone()); + + assertThat(operator.getOutput(), equalTo(page3)); + assertTrue(operator.needsInput()); + assertFalse(operator.isBlocked().isDone()); + + operator.close(); + } + + static class LookupService { + private final ThreadPool threadPool; + private final Map dict; + private final int maxConcurrentRequests; + private final AtomicInteger pendingRequests = new AtomicInteger(); + + LookupService(ThreadPool threadPool, Map dict, int maxConcurrentRequests) { + this.threadPool = threadPool; + this.dict = dict; + this.maxConcurrentRequests = maxConcurrentRequests; + } + + public void lookupAsync(Page input, ActionListener listener) { + int total = pendingRequests.incrementAndGet(); + assert total <= maxConcurrentRequests : "too many pending requests: total=" + total + ",limit=" + maxConcurrentRequests; + ActionRunnable command = new ActionRunnable<>(listener) { + @Override + protected void doRun() { + LongBlock ids = input.getBlock(0); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(ids.getPositionCount()); + for (int i = 0; i < ids.getPositionCount(); i++) { + String v = dict.get(ids.getLong(i)); + if (v != null) { + builder.appendBytesRef(new BytesRef(v)); + } else { + builder.appendNull(); + } + } + int current = pendingRequests.decrementAndGet(); + assert current >= 0 : "pending requests must be non-negative"; + Page result = input.appendBlock(builder.build()); + listener.onResponse(result); + } + }; + TimeValue delay = TimeValue.timeValueMillis(randomIntBetween(0, 50)); + threadPool.schedule(command, delay, "esql_test_executor"); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/BytesRefBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/BytesRefBlockSourceOperator.java new file mode 100644 index 0000000000000..8dddeac27fe52 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/BytesRefBlockSourceOperator.java @@ -0,0 +1,48 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; + +/** + * A source operator whose output is the given BytesRef values. This operator produces pages + * containing a single Block. The Block contains the BytesRef values from the given list, in order. + */ +public class BytesRefBlockSourceOperator extends AbstractBlockSourceOperator { + + static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final BytesRef[] values; + + public BytesRefBlockSourceOperator(List values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public BytesRefBlockSourceOperator(List values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toArray(new BytesRef[0]); + } + + @Override + protected Page createPage(int positionOffset, int length) { + BytesRefVector.Builder builder = BytesRefVector.newVectorBuilder(length); + for (int i = 0; i < length; i++) { + builder.appendBytesRef(values[positionOffset + i]); + } + currentPosition += length; + return new Page(builder.build().asBlock()); + } + + protected int remaining() { + return values.length - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/CannedSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/CannedSourceOperator.java new file mode 100644 index 0000000000000..d5b07a713b8b4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/CannedSourceOperator.java @@ -0,0 +1,81 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * {@link SourceOperator} that returns a sequence of pre-built {@link Page}s. + */ +public class CannedSourceOperator extends SourceOperator { + public static List collectPages(SourceOperator source) { + try { + List pages = new ArrayList<>(); + while (source.isFinished() == false) { + Page in = source.getOutput(); + if (in == null) { + continue; + } + pages.add(in); + } + return pages; + } finally { + source.close(); + } + } + + public static Page mergePages(List pages) { + int totalPositions = pages.stream().mapToInt(Page::getPositionCount).sum(); + Page first = pages.get(0); + Block.Builder[] builders = new Block.Builder[first.getBlockCount()]; + for (int b = 0; b < builders.length; b++) { + builders[b] = first.getBlock(b).elementType().newBlockBuilder(totalPositions); + } + for (Page p : pages) { + for (int b = 0; b < builders.length; b++) { + builders[b].copyFrom(p.getBlock(b), 0, p.getPositionCount()); + } + } + Block[] blocks = new Block[builders.length]; + for (int b = 0; b < blocks.length; b++) { + blocks[b] = builders[b].build(); + } + return new Page(blocks); + } + + private final Iterator page; + + public CannedSourceOperator(Iterator page) { + this.page = page; + } + + @Override + public void finish() { + while (page.hasNext()) { + page.next(); + } + } + + @Override + public boolean isFinished() { + return false == page.hasNext(); + } + + @Override + public Page getOutput() { + return page.next(); + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ColumnExtractOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ColumnExtractOperatorTests.java new file mode 100644 index 0000000000000..da67f9e6a68c0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ColumnExtractOperatorTests.java @@ -0,0 +1,82 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.BytesRefs; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +public class ColumnExtractOperatorTests extends OperatorTestCase { + + @Override + protected SourceOperator simpleInput(int end) { + List input = LongStream.range(0, end) + .mapToObj(l -> new BytesRef("word1_" + l + " word2_" + l + " word3_" + l)) + .collect(Collectors.toList()); + return new BytesRefBlockSourceOperator(input); + } + + record FirstWord(int channelA) implements ColumnExtractOperator.Evaluator { + @Override + public void computeRow(BytesRefBlock inputBlock, int index, Block.Builder[] target, BytesRef spare) { + BytesRef input = inputBlock.getBytesRef(index, spare); + ((BytesRefBlock.Builder) target[channelA]).appendBytesRef(BytesRefs.toBytesRef(input.utf8ToString().split(" ")[0])); + } + + @Override + public String toString() { + return "FirstWord"; + } + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + Supplier expEval = () -> new FirstWord(0); + return new ColumnExtractOperator.Factory(new ElementType[] { ElementType.BYTES_REF }, () -> page -> page.getBlock(0), expEval); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "ColumnExtractOperator[evaluator=FirstWord]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + BytesRef buffer = new BytesRef(); + int pos = 0; + for (var page : results) { + BytesRefBlock block1 = page.getBlock(1); + + for (int i = 0; i < page.getPositionCount(); i++) { + assertEquals(new BytesRef("word1_" + pos).utf8ToString(), block1.getBytesRef(i, buffer).utf8ToString()); + pos++; + } + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java new file mode 100644 index 0000000000000..715460c45b77e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java @@ -0,0 +1,276 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.Collections; +import java.util.HashSet; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collector; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class DriverContextTests extends ESTestCase { + + final BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new NoneCircuitBreakerService()); + + public void testEmptyFinished() { + DriverContext driverContext = new DriverContext(); + driverContext.finish(); + assertTrue(driverContext.isFinished()); + var snapshot = driverContext.getSnapshot(); + assertThat(snapshot.releasables(), empty()); + } + + public void testAddByIdentity() { + DriverContext driverContext = new DriverContext(); + ReleasablePoint point1 = new ReleasablePoint(1, 2); + ReleasablePoint point2 = new ReleasablePoint(1, 2); + assertThat(point1, equalTo(point2)); + driverContext.addReleasable(point1); + driverContext.addReleasable(point2); + driverContext.finish(); + assertTrue(driverContext.isFinished()); + var snapshot = driverContext.getSnapshot(); + assertThat(snapshot.releasables(), hasSize(2)); + assertThat(snapshot.releasables(), contains(point1, point2)); + } + + public void testAddFinish() { + DriverContext driverContext = new DriverContext(); + int count = randomInt(128); + Set releasables = IntStream.range(0, count).mapToObj(i -> randomReleasable()).collect(toIdentitySet()); + assertThat(releasables, hasSize(count)); + + releasables.forEach(driverContext::addReleasable); + driverContext.finish(); + var snapshot = driverContext.getSnapshot(); + assertThat(snapshot.releasables(), hasSize(count)); + assertThat(snapshot.releasables(), containsInAnyOrder(releasables.toArray())); + assertTrue(driverContext.isFinished()); + releasables.forEach(Releasable::close); + releasables.stream().filter(o -> CheckableReleasable.class.isAssignableFrom(o.getClass())).forEach(Releasable::close); + } + + public void testRemoveAbsent() { + DriverContext driverContext = new DriverContext(); + boolean removed = driverContext.removeReleasable(new NoOpReleasable()); + assertThat(removed, equalTo(false)); + driverContext.finish(); + assertTrue(driverContext.isFinished()); + var snapshot = driverContext.getSnapshot(); + assertThat(snapshot.releasables(), empty()); + } + + public void testAddRemoveFinish() { + DriverContext driverContext = new DriverContext(); + int count = randomInt(128); + Set releasables = IntStream.range(0, count).mapToObj(i -> randomReleasable()).collect(toIdentitySet()); + assertThat(releasables, hasSize(count)); + + releasables.forEach(driverContext::addReleasable); + releasables.forEach(driverContext::removeReleasable); + driverContext.finish(); + var snapshot = driverContext.getSnapshot(); + assertThat(snapshot.releasables(), empty()); + assertTrue(driverContext.isFinished()); + releasables.forEach(Releasable::close); + } + + public void testMultiThreaded() throws Exception { + ExecutorService executor = threadPool.executor("esql_test_executor"); + + int tasks = randomIntBetween(4, 32); + List testDrivers = IntStream.range(0, tasks) + .mapToObj(i -> new TestDriver(new AssertingDriverContext(), randomInt(128), bigArrays)) + .toList(); + List> futures = executor.invokeAll(testDrivers, 1, TimeUnit.MINUTES); + assertThat(futures, hasSize(tasks)); + for (var fut : futures) { + fut.get(); // ensures that all completed without an error + } + + int expectedTotal = testDrivers.stream().mapToInt(TestDriver::numReleasables).sum(); + List> finishedReleasables = testDrivers.stream() + .map(TestDriver::driverContext) + .map(DriverContext::getSnapshot) + .map(DriverContext.Snapshot::releasables) + .toList(); + assertThat(finishedReleasables.stream().mapToInt(Set::size).sum(), equalTo(expectedTotal)); + assertThat( + testDrivers.stream().map(TestDriver::driverContext).map(DriverContext::isFinished).anyMatch(b -> b == false), + equalTo(false) + ); + finishedReleasables.stream().flatMap(Set::stream).forEach(Releasable::close); + } + + static class AssertingDriverContext extends DriverContext { + volatile Thread thread; + + @Override + public boolean addReleasable(Releasable releasable) { + checkThread(); + return super.addReleasable(releasable); + } + + @Override + public boolean removeReleasable(Releasable releasable) { + checkThread(); + return super.removeReleasable(releasable); + } + + @Override + public Snapshot getSnapshot() { + // can be called by either the Driver thread or the runner thread, but typically the runner + return super.getSnapshot(); + } + + @Override + public boolean isFinished() { + // can be called by either the Driver thread or the runner thread + return super.isFinished(); + } + + public void finish() { + checkThread(); + super.finish(); + } + + void checkThread() { + if (thread == null) { + thread = Thread.currentThread(); + } + assertThat(thread, equalTo(Thread.currentThread())); + } + + } + + record TestDriver(DriverContext driverContext, int numReleasables, BigArrays bigArrays) implements Callable { + @Override + public Void call() { + int extraToAdd = randomInt(16); + Set releasables = IntStream.range(0, numReleasables + extraToAdd) + .mapToObj(i -> randomReleasable(bigArrays)) + .collect(toIdentitySet()); + assertThat(releasables, hasSize(numReleasables + extraToAdd)); + Set toRemove = randomNFromCollection(releasables, extraToAdd); + for (var r : releasables) { + driverContext.addReleasable(r); + if (toRemove.contains(r)) { + driverContext.removeReleasable(r); + r.close(); + } + } + assertThat(driverContext.workingSet, hasSize(numReleasables)); + driverContext.finish(); + return null; + } + } + + // Selects a number of random elements, n, from the given Set. + static Set randomNFromCollection(Set input, int n) { + final int size = input.size(); + if (n < 0 || n > size) { + throw new IllegalArgumentException(n + " is out of bounds for set of size:" + size); + } + if (n == size) { + return input; + } + Set result = Collections.newSetFromMap(new IdentityHashMap<>()); + Set selected = new HashSet<>(); + while (selected.size() < n) { + int idx = randomValueOtherThanMany(selected::contains, () -> randomInt(size - 1)); + selected.add(idx); + result.add(input.stream().skip(idx).findFirst().get()); + } + assertThat(result.size(), equalTo(n)); + assertTrue(input.containsAll(result)); + return result; + } + + Releasable randomReleasable() { + return randomReleasable(bigArrays); + } + + static Releasable randomReleasable(BigArrays bigArrays) { + return switch (randomInt(3)) { + case 0 -> new NoOpReleasable(); + case 1 -> new ReleasablePoint(1, 2); + case 2 -> new CheckableReleasable(); + case 3 -> bigArrays.newLongArray(32, false); + default -> throw new AssertionError(); + }; + } + + record ReleasablePoint(int x, int y) implements Releasable { + @Override + public void close() {} + } + + static class NoOpReleasable implements Releasable { + + @Override + public void close() { + // no-op + } + } + + static class CheckableReleasable implements Releasable { + + boolean closed; + + @Override + public void close() { + closed = true; + } + } + + static Collector> toIdentitySet() { + return Collectors.toCollection(() -> Collections.newSetFromMap(new IdentityHashMap<>())); + } + + private TestThreadPool threadPool; + + @Before + public void setThreadPool() { + int numThreads = randomBoolean() ? 1 : between(2, 16); + threadPool = new TestThreadPool( + "test", + new FixedExecutorBuilder(Settings.EMPTY, "esql_test_executor", numThreads, 1024, "esql", EsExecutors.TaskTrackingConfig.DEFAULT) + ); + } + + @After + public void shutdownThreadPool() { + terminate(threadPool); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java new file mode 100644 index 0000000000000..775c30223589b --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverStatusTests.java @@ -0,0 +1,114 @@ +/* + * 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.compute.operator; + +import com.carrotsearch.randomizedtesting.generators.RandomStrings; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.LuceneSourceOperatorStatusTests; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperatorStatusTests; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class DriverStatusTests extends AbstractWireSerializingTestCase { + public void testToXContent() { + DriverStatus status = new DriverStatus( + "ABC:123", + 123413243214L, + DriverStatus.Status.RUNNING, + List.of( + new DriverStatus.OperatorStatus("LuceneSource", LuceneSourceOperatorStatusTests.simple()), + new DriverStatus.OperatorStatus("ValuesSourceReader", ValuesSourceReaderOperatorStatusTests.simple()) + ) + ); + assertThat( + Strings.toString(status), + equalTo( + """ + {"sessionId":"ABC:123","last_updated":"1973-11-29T09:27:23.214Z","status":"running","active_operators":[""" + + """ + {"operator":"LuceneSource","status":""" + + LuceneSourceOperatorStatusTests.simpleToJson() + + "},{\"operator\":\"ValuesSourceReader\",\"status\":" + + ValuesSourceReaderOperatorStatusTests.simpleToJson() + + "}]}" + ) + ); + } + + @Override + protected Writeable.Reader instanceReader() { + return DriverStatus::new; + } + + @Override + protected DriverStatus createTestInstance() { + return new DriverStatus(randomSessionId(), randomLong(), randomStatus(), randomActiveOperators()); + } + + private String randomSessionId() { + return RandomStrings.randomAsciiLettersOfLengthBetween(random(), 1, 15); + } + + private DriverStatus.Status randomStatus() { + return randomFrom(DriverStatus.Status.values()); + } + + private List randomActiveOperators() { + return randomList(0, 5, this::randomOperatorStatus); + } + + private DriverStatus.OperatorStatus randomOperatorStatus() { + Supplier status = randomFrom( + new LuceneSourceOperatorStatusTests()::createTestInstance, + new ValuesSourceReaderOperatorStatusTests()::createTestInstance, + () -> null + ); + return new DriverStatus.OperatorStatus(randomAlphaOfLength(3), status.get()); + } + + @Override + protected DriverStatus mutateInstance(DriverStatus instance) throws IOException { + var sessionId = instance.sessionId(); + long lastUpdated = instance.lastUpdated(); + var status = instance.status(); + var operators = instance.activeOperators(); + switch (between(0, 3)) { + case 0: + sessionId = randomValueOtherThan(sessionId, this::randomSessionId); + break; + case 1: + lastUpdated = randomValueOtherThan(lastUpdated, ESTestCase::randomLong); + break; + case 2: + status = randomValueOtherThan(status, this::randomStatus); + break; + case 3: + operators = randomValueOtherThan(operators, this::randomActiveOperators); + break; + default: + throw new UnsupportedOperationException(); + } + return new DriverStatus(sessionId, lastUpdated, status, operators); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(List.of(LuceneSourceOperator.Status.ENTRY, ValuesSourceReaderOperator.Status.ENTRY)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java new file mode 100644 index 0000000000000..2143e77d3ffc6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +public class EvalOperatorTests extends OperatorTestCase { + @Override + protected SourceOperator simpleInput(int end) { + return new TupleBlockSourceOperator(LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + } + + record Addition(int lhs, int rhs) implements EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + LongVector lhsVector = page.getBlock(0).asVector(); + LongVector rhsVector = page.getBlock(1).asVector(); + LongVector.Builder result = LongVector.newVectorBuilder(page.getPositionCount()); + for (int p = 0; p < page.getPositionCount(); p++) { + result.appendLong(lhsVector.getLong(p) + rhsVector.getLong(p)); + } + return result.build().asBlock(); + } + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return new EvalOperator.EvalOperatorFactory(() -> new Addition(0, 1)); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "EvalOperator[evaluator=Addition[lhs=0, rhs=1]]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + final int positions = input.stream().map(page -> page.getBlock(0)).mapToInt(Block::getPositionCount).sum(); + final int expectedValue = positions; + final int resultChannel = 2; + for (var page : results) { + LongBlock lb = page.getBlock(resultChannel); + IntStream.range(0, lb.getPositionCount()).forEach(pos -> assertEquals(expectedValue, lb.getLong(pos))); + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java new file mode 100644 index 0000000000000..4724e09324fd5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java @@ -0,0 +1,87 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class FilterOperatorTests extends OperatorTestCase { + @Override + protected SourceOperator simpleInput(int end) { + return new TupleBlockSourceOperator(LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + } + + record SameLastDigit(int lhs, int rhs) implements EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + LongVector lhsVector = page.getBlock(0).asVector(); + LongVector rhsVector = page.getBlock(1).asVector(); + BooleanVector.Builder result = BooleanVector.newVectorBuilder(page.getPositionCount()); + for (int p = 0; p < page.getPositionCount(); p++) { + result.appendBoolean(lhsVector.getLong(p) % 10 == rhsVector.getLong(p) % 10); + } + return result.build().asBlock(); + } + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return new FilterOperator.FilterOperatorFactory(() -> new SameLastDigit(0, 1)); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "FilterOperator[evaluator=SameLastDigit[lhs=0, rhs=1]]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + int expectedCount = 0; + for (var page : input) { + LongVector lhs = page.getBlock(0).asVector(); + LongVector rhs = page.getBlock(1).asVector(); + for (int p = 0; p < page.getPositionCount(); p++) { + if (lhs.getLong(p) % 10 == rhs.getLong(p) % 10) { + expectedCount++; + } + } + } + int actualCount = 0; + for (var page : results) { + LongVector lhs = page.getBlock(0).asVector(); + LongVector rhs = page.getBlock(1).asVector(); + for (int p = 0; p < page.getPositionCount(); p++) { + assertThat(lhs.getLong(p) % 10, equalTo(rhs.getLong(p) % 10)); + actualCount++; + } + } + assertThat(actualCount, equalTo(expectedCount)); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ForkingOperatorTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ForkingOperatorTestCase.java new file mode 100644 index 0000000000000..f572e08fed424 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ForkingOperatorTestCase.java @@ -0,0 +1,353 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkOperator; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceOperator; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.startsWith; + +/** + * Test case for all operators that support parallel operation in the + * shape of "single", "initial", "intermediate", and "final" modes. + */ +public abstract class ForkingOperatorTestCase extends OperatorTestCase { + protected abstract Operator.OperatorFactory simpleWithMode(BigArrays bigArrays, AggregatorMode mode); + + @Override + protected final Operator.OperatorFactory simple(BigArrays bigArrays) { + return simpleWithMode(bigArrays, AggregatorMode.SINGLE); + } + + public final void testInitialFinal() { + BigArrays bigArrays = nonBreakingBigArrays(); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + List results = new ArrayList<>(); + + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(input.iterator()), + List.of( + simpleWithMode(bigArrays, AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(bigArrays, AggregatorMode.FINAL).get(driverContext) + ), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertSimpleOutput(input, results); + assertDriverContext(driverContext); + } + + public final void testManyInitialFinal() { + BigArrays bigArrays = nonBreakingBigArrays(); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + List partials = oneDriverPerPage(input, () -> List.of(simpleWithMode(bigArrays, AggregatorMode.INITIAL).get(driverContext))); + List results = new ArrayList<>(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(partials.iterator()), + List.of(simpleWithMode(bigArrays, AggregatorMode.FINAL).get(driverContext)), + new PageConsumerOperator(results::add), + () -> {} + ) + ) { + runDriver(d); + } + assertSimpleOutput(input, results); + assertDriverContext(driverContext); + } + + public final void testInitialIntermediateFinal() { + BigArrays bigArrays = nonBreakingBigArrays(); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + List results = new ArrayList<>(); + + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(input.iterator()), + List.of( + simpleWithMode(bigArrays, AggregatorMode.INITIAL).get(driverContext), + simpleWithMode(bigArrays, AggregatorMode.INTERMEDIATE).get(driverContext), + simpleWithMode(bigArrays, AggregatorMode.FINAL).get(driverContext) + ), + new PageConsumerOperator(page -> results.add(page)), + () -> {} + ) + ) { + runDriver(d); + } + assertSimpleOutput(input, results); + assertDriverContext(driverContext); + } + + public final void testManyInitialManyPartialFinal() { + BigArrays bigArrays = nonBreakingBigArrays(); + DriverContext driverContext = new DriverContext(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + + List partials = oneDriverPerPage(input, () -> List.of(simpleWithMode(bigArrays, AggregatorMode.INITIAL).get(driverContext))); + Collections.shuffle(partials, random()); + List intermediates = oneDriverPerPageList( + randomSplits(partials).iterator(), + () -> List.of(simpleWithMode(bigArrays, AggregatorMode.INTERMEDIATE).get(driverContext)) + ); + + List results = new ArrayList<>(); + try ( + Driver d = new Driver( + driverContext, + new CannedSourceOperator(intermediates.iterator()), + List.of(simpleWithMode(bigArrays, AggregatorMode.FINAL).get(driverContext)), + new PageConsumerOperator(results::add), + () -> {} + ) + ) { + runDriver(d); + } + assertSimpleOutput(input, results); + assertDriverContext(driverContext); + } + + // Similar to testManyInitialManyPartialFinal, but uses with the DriverRunner infrastructure + // to move the data through the pipeline. + public final void testManyInitialManyPartialFinalRunner() { + BigArrays bigArrays = nonBreakingBigArrays(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + List results = new ArrayList<>(); + + List drivers = createDriversForInput(bigArrays, input, results, false /* no throwing ops */); + var runner = new DriverRunner() { + @Override + protected void start(Driver driver, ActionListener listener) { + Driver.start(threadPool.executor("esql_test_executor"), driver, between(1, 10000), listener); + } + }; + PlainActionFuture future = new PlainActionFuture<>(); + runner.runToCompletion(drivers, future); + future.actionGet(TimeValue.timeValueMinutes(1)); + assertSimpleOutput(input, results); + drivers.stream().map(Driver::driverContext).forEach(OperatorTestCase::assertDriverContext); + } + + // Similar to testManyInitialManyPartialFinalRunner, but creates a pipeline that contains an + // operator that throws - fails. The primary motivation for this is to ensure that the driver + // runner behaves correctly and also releases all resources (bigArrays) appropriately. + public final void testManyInitialManyPartialFinalRunnerThrowing() { + BigArrays bigArrays = nonBreakingBigArrays(); + List input = CannedSourceOperator.collectPages(simpleInput(between(1_000, 100_000))); + List results = new ArrayList<>(); + + List drivers = createDriversForInput(bigArrays, input, results, true /* one throwing op */); + var runner = new DriverRunner() { + @Override + protected void start(Driver driver, ActionListener listener) { + Driver.start(threadPool.executor("esql_test_executor"), driver, between(1, 1000), listener); + } + }; + PlainActionFuture future = new PlainActionFuture<>(); + runner.runToCompletion(drivers, future); + BadException e = expectThrows(BadException.class, () -> future.actionGet(TimeValue.timeValueMinutes(1))); + assertThat(e.getMessage(), startsWith("bad exception from")); + drivers.stream().map(Driver::driverContext).forEach(OperatorTestCase::assertDriverContext); + } + + // Creates a set of drivers that splits the execution into two separate sets of pipelines. The + // first is a number of source drivers that consume the input (split across them), and output + // intermediate results. The second is a single operator that consumes intermediate input and + // produces the final results. The throwingOp param allows to construct a pipeline that will + // fail by throwing an exception in one of the operators. + List createDriversForInput(BigArrays bigArrays, List input, List results, boolean throwingOp) { + Collection> splitInput = randomSplits(input, randomIntBetween(2, 4)); + + ExchangeSinkHandler sinkExchanger = new ExchangeSinkHandler(randomIntBetween(2, 10), threadPool::relativeTimeInMillis); + ExchangeSourceHandler sourceExchanger = new ExchangeSourceHandler( + randomIntBetween(1, 4), + threadPool.executor("esql_test_executor") + ); + sourceExchanger.addRemoteSink(sinkExchanger::fetchPageAsync, 1); + + Iterator intermediateOperatorItr; + int itrSize = (splitInput.size() * 3) + 3; // 3 inter ops per initial source drivers, and 3 per final + if (throwingOp) { + intermediateOperatorItr = randomPassThroughOperatorListWithSingleThrowingOp(itrSize).iterator(); + } else { + intermediateOperatorItr = IntStream.range(0, itrSize).mapToObj(i -> new PassThroughOperator()).toList().iterator(); + } + + List drivers = new ArrayList<>(); + for (List pages : splitInput) { + DriverContext driver1Context = new DriverContext(); + drivers.add( + new Driver( + driver1Context, + new CannedSourceOperator(pages.iterator()), + List.of( + intermediateOperatorItr.next(), + simpleWithMode(bigArrays, AggregatorMode.INITIAL).get(driver1Context), + intermediateOperatorItr.next(), + simpleWithMode(bigArrays, AggregatorMode.INTERMEDIATE).get(driver1Context), + intermediateOperatorItr.next() + ), + new ExchangeSinkOperator(sinkExchanger.createExchangeSink(), Function.identity()), + () -> {} + ) + ); + } + DriverContext driver2Context = new DriverContext(); + drivers.add( + new Driver( + driver2Context, + new ExchangeSourceOperator(sourceExchanger.createExchangeSource()), + List.of( + intermediateOperatorItr.next(), + simpleWithMode(bigArrays, AggregatorMode.INTERMEDIATE).get(driver2Context), + intermediateOperatorItr.next(), + simpleWithMode(bigArrays, AggregatorMode.FINAL).get(driver2Context), + intermediateOperatorItr.next() + ), + new PageConsumerOperator(results::add), + () -> {} + ) + ); + assert intermediateOperatorItr.hasNext() == false; + return drivers; + } + + static class BadException extends RuntimeException { + BadException(String message) { + super(message); + } + } + + // Returns a random list of operators, where all but one are PassThrough and exactly one is a + // Throwing operator. + static List randomPassThroughOperatorListWithSingleThrowingOp(int size) { + assert size > 1; + List l = Stream.concat( + IntStream.range(0, size - 1).mapToObj(i -> new PassThroughOperator()), + Stream.of(randomThrowingOperator()) + ).collect(toList()); + Collections.shuffle(l, random()); + assert l.size() == size; + return l; + } + + static Operator randomThrowingOperator() { + return randomFrom( + Set.of( + new ThrowInNeedsInputOperator(), + new ThrowInAddInputOperator(), + new ThrowInGetOutputOperator(), + new ThrowInIsFinishedOperator(), + new ThrowInFinishOperator(), + new ThrowInCloseOperator() + ) + ); + } + + private static class ThrowInNeedsInputOperator extends PassThroughOperator { + @Override + public boolean needsInput() { + throw new BadException("bad exception from needsInput"); + } + } + + private static class ThrowInAddInputOperator extends PassThroughOperator { + @Override + public void addInput(Page page) { + throw new BadException("bad exception from addInput"); + } + } + + private static class ThrowInGetOutputOperator extends PassThroughOperator { + @Override + public Page getOutput() { + throw new BadException("bad exception from getOutput"); + } + } + + private static class ThrowInIsFinishedOperator extends PassThroughOperator { + @Override + public boolean isFinished() { + throw new BadException("bad exception from isFinished"); + } + } + + private static class ThrowInFinishOperator extends PassThroughOperator { + @Override + public void finish() { + throw new BadException("bad exception from finish"); + } + } + + private static class ThrowInCloseOperator extends PassThroughOperator { + @Override + public void close() { + throw new BadException("bad exception from close"); + } + } + + private static Collection> randomSplits(List in) { + return randomSplits(in, in.size()); + } + + private static Collection> randomSplits(List in, int maxGroups) { + return in.stream().collect(Collectors.groupingBy(s -> randomInt(in.size() - 1) % maxGroups)).values(); + } + + private TestThreadPool threadPool; + + @Before + public void setThreadPool() { + int numThreads = randomBoolean() ? 1 : between(2, 16); + threadPool = new TestThreadPool( + "test", + new FixedExecutorBuilder(Settings.EMPTY, "esql_test_executor", numThreads, 1024, "esql", EsExecutors.TaskTrackingConfig.DEFAULT) + ); + } + + @After + public void shutdownThreadPool() { + terminate(threadPool); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java new file mode 100644 index 0000000000000..954a1f179f259 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java @@ -0,0 +1,97 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunction; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxLongGroupingAggregatorFunctionTests; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunction; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumLongGroupingAggregatorFunctionTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.LongStream; + +import static java.util.stream.IntStream.range; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class HashAggregationOperatorTests extends ForkingOperatorTestCase { + @Override + protected SourceOperator simpleInput(int size) { + long max = randomLongBetween(1, Long.MAX_VALUE / size); + return new TupleBlockSourceOperator(LongStream.range(0, size).mapToObj(l -> Tuple.tuple(l % 5, randomLongBetween(-max, max)))); + } + + @Override + protected Operator.OperatorFactory simpleWithMode(BigArrays bigArrays, AggregatorMode mode) { + List sumChannels, maxChannels; + if (mode.isInputPartial()) { + int sumChannelCount = SumLongAggregatorFunction.intermediateStateDesc().size(); + int maxChannelCount = MaxLongAggregatorFunction.intermediateStateDesc().size(); + sumChannels = range(1, 1 + sumChannelCount).boxed().toList(); + maxChannels = range(1 + sumChannelCount, 1 + sumChannelCount + maxChannelCount).boxed().toList(); + } else { + sumChannels = maxChannels = List.of(1); + } + + return new HashAggregationOperator.HashAggregationOperatorFactory( + List.of(new HashAggregationOperator.GroupSpec(0, ElementType.LONG)), + List.of( + new SumLongAggregatorFunctionSupplier(bigArrays, sumChannels).groupingAggregatorFactory(mode), + new MaxLongAggregatorFunctionSupplier(bigArrays, maxChannels).groupingAggregatorFactory(mode) + ), + randomPageSize(), + bigArrays + ); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "HashAggregationOperator[mode = , aggs = sum of longs, max of longs]"; + } + + @Override + protected String expectedToStringOfSimple() { + return "HashAggregationOperator[blockHash=LongBlockHash{channel=0, entries=0, seenNull=false}, aggregators=[" + + "GroupingAggregator[aggregatorFunction=SumLongGroupingAggregatorFunction[channels=[1]], mode=SINGLE], " + + "GroupingAggregator[aggregatorFunction=MaxLongGroupingAggregatorFunction[channels=[1]], mode=SINGLE]]]"; + } + + @Override + protected void assertSimpleOutput(List input, List results) { + assertThat(results, hasSize(1)); + assertThat(results.get(0).getBlockCount(), equalTo(3)); + assertThat(results.get(0).getPositionCount(), equalTo(5)); + + SumLongGroupingAggregatorFunctionTests sum = new SumLongGroupingAggregatorFunctionTests(); + MaxLongGroupingAggregatorFunctionTests max = new MaxLongGroupingAggregatorFunctionTests(); + + LongBlock groups = results.get(0).getBlock(0); + Block sums = results.get(0).getBlock(1); + Block maxs = results.get(0).getBlock(2); + for (int i = 0; i < 5; i++) { + long group = groups.getLong(i); + sum.assertSimpleGroup(input, sums, i, group); + max.assertSimpleGroup(input, maxs, i, group); + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + return ByteSizeValue.ofBytes(between(1, 32)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitOperatorTests.java new file mode 100644 index 0000000000000..3f83d775f99c5 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitOperatorTests.java @@ -0,0 +1,82 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class LimitOperatorTests extends OperatorTestCase { + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return new LimitOperator.Factory(100); + } + + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceLongBlockSourceOperator(LongStream.range(0, size)); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "LimitOperator[limit = 100]"; + } + + @Override + protected String expectedToStringOfSimple() { + return "LimitOperator[limit = 100/100]"; + } + + @Override + protected void assertSimpleOutput(List input, List results) { + int inputPositionCount = input.stream().mapToInt(p -> p.getPositionCount()).sum(); + int outputPositionCount = results.stream().mapToInt(p -> p.getPositionCount()).sum(); + assertThat(outputPositionCount, equalTo(Math.min(100, inputPositionCount))); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeFalse("doesn't use big arrays", true); + return null; + } + + public void testStatus() { + LimitOperator op = (LimitOperator) simple(BigArrays.NON_RECYCLING_INSTANCE).get(new DriverContext()); + + LimitOperator.Status status = op.status(); + assertThat(status.limit(), equalTo(100)); + assertThat(status.limitRemaining(), equalTo(100)); + assertThat(status.pagesProcessed(), equalTo(0)); + + Page p = new Page(Block.constantNullBlock(10)); + op.addInput(p); + assertSame(p, op.getOutput()); + status = op.status(); + assertThat(status.limit(), equalTo(100)); + assertThat(status.limitRemaining(), equalTo(90)); + assertThat(status.pagesProcessed(), equalTo(1)); + } + + public void testNeedInput() { + LimitOperator op = (LimitOperator) simple(BigArrays.NON_RECYCLING_INSTANCE).get(new DriverContext()); + assertTrue(op.needsInput()); + Page p = new Page(Block.constantNullBlock(10)); + op.addInput(p); + assertFalse(op.needsInput()); + op.getOutput(); + assertTrue(op.needsInput()); + op.finish(); + assertFalse(op.needsInput()); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitStatusTests.java new file mode 100644 index 0000000000000..fd2b75f6bd819 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LimitStatusTests.java @@ -0,0 +1,54 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; + +public class LimitStatusTests extends AbstractWireSerializingTestCase { + public void testToXContent() { + assertThat(Strings.toString(new LimitOperator.Status(10, 1, 1)), equalTo(""" + {"limit":10,"limit_remaining":1,"pages_processed":1}""")); + } + + @Override + protected Writeable.Reader instanceReader() { + return LimitOperator.Status::new; + } + + @Override + protected LimitOperator.Status createTestInstance() { + return new LimitOperator.Status(between(0, Integer.MAX_VALUE), between(0, Integer.MAX_VALUE), between(0, Integer.MAX_VALUE)); + } + + @Override + protected LimitOperator.Status mutateInstance(LimitOperator.Status instance) throws IOException { + int limit = instance.limit(); + int limitRemaining = instance.limitRemaining(); + int pagesProcessed = instance.pagesProcessed(); + switch (between(0, 2)) { + case 0: + limit = randomValueOtherThan(limit, () -> between(0, Integer.MAX_VALUE)); + break; + case 1: + limitRemaining = randomValueOtherThan(limitRemaining, () -> between(0, Integer.MAX_VALUE)); + break; + case 2: + pagesProcessed = randomValueOtherThan(pagesProcessed, () -> between(0, Integer.MAX_VALUE)); + break; + default: + throw new IllegalArgumentException(); + } + return new LimitOperator.Status(limit, limitRemaining, pagesProcessed); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBooleanTupleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBooleanTupleBlockSourceOperator.java new file mode 100644 index 0000000000000..3de1e8f9bb80d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBooleanTupleBlockSourceOperator.java @@ -0,0 +1,71 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.Stream; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class LongBooleanTupleBlockSourceOperator extends AbstractBlockSourceOperator { + + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + + public LongBooleanTupleBlockSourceOperator(Stream> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongBooleanTupleBlockSourceOperator(Stream> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toList(); + } + + public LongBooleanTupleBlockSourceOperator(List> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongBooleanTupleBlockSourceOperator(List> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values; + } + + @Override + protected Page createPage(int positionOffset, int length) { + var blockBuilder1 = LongBlock.newBlockBuilder(length); + var blockBuilder2 = BooleanBlock.newBlockBuilder(length); + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + blockBuilder1.appendLong(item.v1()); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + blockBuilder2.appendBoolean(item.v2()); + } + } + currentPosition += length; + return new Page(blockBuilder1.build(), blockBuilder2.build()); + } + + @Override + protected int remaining() { + return values.size() - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBytesRefTupleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBytesRefTupleBlockSourceOperator.java new file mode 100644 index 0000000000000..66a7b9b342f28 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongBytesRefTupleBlockSourceOperator.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.Stream; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class LongBytesRefTupleBlockSourceOperator extends AbstractBlockSourceOperator { + + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + + public LongBytesRefTupleBlockSourceOperator(Stream> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongBytesRefTupleBlockSourceOperator(Stream> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toList(); + } + + public LongBytesRefTupleBlockSourceOperator(List> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongBytesRefTupleBlockSourceOperator(List> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values; + } + + @Override + protected Page createPage(int positionOffset, int length) { + var blockBuilder1 = LongBlock.newBlockBuilder(length); + var blockBuilder2 = BytesRefBlock.newBlockBuilder(length); + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + blockBuilder1.appendLong(item.v1()); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + blockBuilder2.appendBytesRef(item.v2()); + } + } + currentPosition += length; + return new Page(blockBuilder1.build(), blockBuilder2.build()); + } + + @Override + protected int remaining() { + return values.size() - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongDoubleTupleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongDoubleTupleBlockSourceOperator.java new file mode 100644 index 0000000000000..f283b31a8a5d6 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongDoubleTupleBlockSourceOperator.java @@ -0,0 +1,71 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.Stream; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class LongDoubleTupleBlockSourceOperator extends AbstractBlockSourceOperator { + + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + + public LongDoubleTupleBlockSourceOperator(Stream> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongDoubleTupleBlockSourceOperator(Stream> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toList(); + } + + public LongDoubleTupleBlockSourceOperator(List> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongDoubleTupleBlockSourceOperator(List> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values; + } + + @Override + protected Page createPage(int positionOffset, int length) { + var blockBuilder1 = LongBlock.newBlockBuilder(length); + var blockBuilder2 = DoubleBlock.newBlockBuilder(length); + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + blockBuilder1.appendLong(item.v1()); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + blockBuilder2.appendDouble(item.v2()); + } + } + currentPosition += length; + return new Page(blockBuilder1.build(), blockBuilder2.build()); + } + + @Override + protected int remaining() { + return values.size() - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongIntBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongIntBlockSourceOperator.java new file mode 100644 index 0000000000000..85ed36656675a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/LongIntBlockSourceOperator.java @@ -0,0 +1,71 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.Stream; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class LongIntBlockSourceOperator extends AbstractBlockSourceOperator { + + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + + public LongIntBlockSourceOperator(Stream> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongIntBlockSourceOperator(Stream> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toList(); + } + + public LongIntBlockSourceOperator(List> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public LongIntBlockSourceOperator(List> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values; + } + + @Override + protected Page createPage(int positionOffset, int length) { + var blockBuilder1 = LongBlock.newBlockBuilder(length); + var blockBuilder2 = IntBlock.newBlockBuilder(length); + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + blockBuilder1.appendLong(item.v1()); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + blockBuilder2.appendInt(item.v2()); + } + } + currentPosition += length; + return new Page(blockBuilder1.build(), blockBuilder2.build()); + } + + @Override + protected int remaining() { + return values.size() - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MappingSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MappingSourceOperator.java new file mode 100644 index 0000000000000..b7d02e8479ef7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MappingSourceOperator.java @@ -0,0 +1,44 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; + +public abstract class MappingSourceOperator extends SourceOperator { + private final SourceOperator delegate; + + public MappingSourceOperator(SourceOperator delegate) { + this.delegate = delegate; + } + + protected abstract Page map(Page page); + + @Override + public void finish() { + delegate.finish(); + } + + @Override + public boolean isFinished() { + return delegate.isFinished(); + } + + @Override + public Page getOutput() { + Page p = delegate.getOutput(); + if (p == null) { + return p; + } + return map(p); + } + + @Override + public void close() { + delegate.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MultivalueDedupeTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MultivalueDedupeTests.java new file mode 100644 index 0000000000000..b616b9f9bff7e --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MultivalueDedupeTests.java @@ -0,0 +1,388 @@ +/* + * 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.compute.operator; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockTestUtils; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matcher; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.function.LongFunction; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; + +public class MultivalueDedupeTests extends ESTestCase { + public static List supportedTypes() { + List supported = new ArrayList<>(); + for (ElementType elementType : ElementType.values()) { + if (elementType == ElementType.UNKNOWN || elementType == ElementType.NULL || elementType == ElementType.DOC) { + continue; + } + supported.add(elementType); + } + return supported; + } + + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + for (ElementType elementType : supportedTypes()) { + if (elementType == ElementType.UNKNOWN || elementType == ElementType.NULL || elementType == ElementType.DOC) { + continue; + } + for (boolean nullAllowed : new boolean[] { false, true }) { + for (int max : new int[] { 10, 100, 1000 }) { + params.add(new Object[] { elementType, 1000, nullAllowed, 1, max, 0, 0 }); + params.add(new Object[] { elementType, 1000, nullAllowed, 1, max, 0, 100 }); + } + } + } + return params; + } + + private final ElementType elementType; + private final int positionCount; + private final boolean nullAllowed; + private final int minValuesPerPosition; + private final int maxValuesPerPosition; + private final int minDupsPerPosition; + private final int maxDupsPerPosition; + + public MultivalueDedupeTests( + ElementType elementType, + int positionCount, + boolean nullAllowed, + int minValuesPerPosition, + int maxValuesPerPosition, + int minDupsPerPosition, + int maxDupsPerPosition + ) { + this.elementType = elementType; + this.positionCount = positionCount; + this.nullAllowed = nullAllowed; + this.minValuesPerPosition = minValuesPerPosition; + this.maxValuesPerPosition = maxValuesPerPosition; + this.minDupsPerPosition = minDupsPerPosition; + this.maxDupsPerPosition = maxDupsPerPosition; + } + + public void testDedupeAdaptive() { + BasicBlockTests.RandomBlock b = randomBlock(); + assertDeduped(b, MultivalueDedupe.dedupeToBlockAdaptive(b.block())); + } + + public void testDedupeViaCopyAndSort() { + BasicBlockTests.RandomBlock b = randomBlock(); + assertDeduped(b, MultivalueDedupe.dedupeToBlockUsingCopyAndSort(b.block())); + } + + public void testDedupeViaCopyMissing() { + BasicBlockTests.RandomBlock b = randomBlock(); + assertDeduped(b, MultivalueDedupe.dedupeToBlockUsingCopyMissing(b.block())); + } + + private BasicBlockTests.RandomBlock randomBlock() { + return BasicBlockTests.randomBlock( + elementType, + positionCount, + nullAllowed, + minValuesPerPosition, + maxValuesPerPosition, + minDupsPerPosition, + maxDupsPerPosition + ); + } + + private void assertDeduped(BasicBlockTests.RandomBlock b, Block deduped) { + for (int p = 0; p < b.block().getPositionCount(); p++) { + List v = b.values().get(p); + Matcher matcher = v == null + ? nullValue() + : containsInAnyOrder(v.stream().collect(Collectors.toSet()).stream().sorted().toArray()); + BlockTestUtils.assertPositionValues(deduped, p, matcher); + } + } + + public void testHash() { + BasicBlockTests.RandomBlock b = randomBlock(); + + switch (b.block().elementType()) { + case BOOLEAN -> assertBooleanHash(Set.of(), b); + case BYTES_REF -> assertBytesRefHash(Set.of(), b); + case INT -> assertIntHash(Set.of(), b); + case LONG -> assertLongHash(Set.of(), b); + case DOUBLE -> assertDoubleHash(Set.of(), b); + default -> throw new IllegalArgumentException(); + } + } + + public void testHashWithPreviousValues() { + BasicBlockTests.RandomBlock b = randomBlock(); + + switch (b.block().elementType()) { + case BOOLEAN -> { + Set previousValues = switch (between(0, 2)) { + case 0 -> Set.of(false); + case 1 -> Set.of(true); + case 2 -> Set.of(false, true); + default -> throw new IllegalArgumentException(); + }; + assertBooleanHash(previousValues, b); + } + case BYTES_REF -> { + int prevSize = between(1, 10000); + Set previousValues = new HashSet<>(prevSize); + while (previousValues.size() < prevSize) { + previousValues.add(new BytesRef(randomAlphaOfLengthBetween(1, 20))); + } + assertBytesRefHash(previousValues, b); + } + case INT -> { + int prevSize = between(1, 10000); + Set previousValues = new HashSet<>(prevSize); + while (previousValues.size() < prevSize) { + previousValues.add(randomInt()); + } + assertIntHash(previousValues, b); + } + case LONG -> { + int prevSize = between(1, 10000); + Set previousValues = new HashSet<>(prevSize); + while (previousValues.size() < prevSize) { + previousValues.add(randomLong()); + } + assertLongHash(previousValues, b); + } + case DOUBLE -> { + int prevSize = between(1, 10000); + Set previousValues = new HashSet<>(prevSize); + while (previousValues.size() < prevSize) { + previousValues.add(randomDouble()); + } + assertDoubleHash(previousValues, b); + } + default -> throw new IllegalArgumentException(); + } + } + + public void testBatchEncodeAll() { + int initCapacity = Math.toIntExact(ByteSizeValue.ofKb(10).getBytes()); + BasicBlockTests.RandomBlock b = randomBlock(); + BatchEncoder encoder = MultivalueDedupe.batchEncoder(b.block(), initCapacity); + + int valueOffset = 0; + for (int p = 0, positionOffset = Integer.MAX_VALUE; p < b.block().getPositionCount(); p++, positionOffset++) { + while (positionOffset >= encoder.positionCount()) { + encoder.encodeNextBatch(); + positionOffset = 0; + valueOffset = 0; + } + assertThat(encoder.bytesCapacity(), greaterThanOrEqualTo(initCapacity)); + valueOffset = assertEncodedPosition(b, encoder, p, positionOffset, valueOffset); + } + } + + public void testBatchEncoderStartSmall() { + assumeFalse("Booleans don't grow in the same way", elementType == ElementType.BOOLEAN); + BasicBlockTests.RandomBlock b = randomBlock(); + BatchEncoder encoder = MultivalueDedupe.batchEncoder(b.block(), 0); + + /* + * We run can't fit the first non-null position into our 0 bytes. + * *unless we're doing booleans, those don't bother with expanding + * and go with a minimum block size of 2. + */ + int leadingNulls = 0; + while (leadingNulls < b.values().size() && b.values().get(leadingNulls) == null) { + leadingNulls++; + } + encoder.encodeNextBatch(); + assertThat(encoder.bytesLength(), equalTo(0)); + assertThat(encoder.positionCount(), equalTo(leadingNulls)); + + /* + * When we add against we scale the array up to handle at least one position. + * We may get more than one position because the scaling oversizes the destination + * and may end up with enough extra room to fit more trailing positions. + */ + encoder.encodeNextBatch(); + assertThat(encoder.bytesLength(), greaterThan(0)); + assertThat(encoder.positionCount(), greaterThanOrEqualTo(1)); + assertThat(encoder.firstPosition(), equalTo(leadingNulls)); + assertEncodedPosition(b, encoder, leadingNulls, 0, 0); + } + + private void assertBooleanHash(Set previousValues, BasicBlockTests.RandomBlock b) { + boolean[] everSeen = new boolean[3]; + if (previousValues.contains(false)) { + everSeen[1] = true; + } + if (previousValues.contains(true)) { + everSeen[2] = true; + } + LongBlock hashes = new MultivalueDedupeBoolean((BooleanBlock) b.block()).hash(everSeen); + List hashedValues = new ArrayList<>(); + if (everSeen[1]) { + hashedValues.add(false); + } + if (everSeen[2]) { + hashedValues.add(true); + } + assertHash(b, hashes, hashedValues.size(), previousValues, i -> hashedValues.get((int) i)); + } + + private void assertBytesRefHash(Set previousValues, BasicBlockTests.RandomBlock b) { + BytesRefHash hash = new BytesRefHash(1, BigArrays.NON_RECYCLING_INSTANCE); + previousValues.stream().forEach(hash::add); + MultivalueDedupe.HashResult hashes = new MultivalueDedupeBytesRef((BytesRefBlock) b.block()).hash(hash); + assertThat(hashes.sawNull(), equalTo(b.values().stream().anyMatch(v -> v == null))); + assertHash(b, hashes.ords(), hash.size(), previousValues, i -> hash.get(i, new BytesRef())); + } + + private void assertIntHash(Set previousValues, BasicBlockTests.RandomBlock b) { + LongHash hash = new LongHash(1, BigArrays.NON_RECYCLING_INSTANCE); + previousValues.stream().forEach(hash::add); + MultivalueDedupe.HashResult hashes = new MultivalueDedupeInt((IntBlock) b.block()).hash(hash); + assertThat(hashes.sawNull(), equalTo(b.values().stream().anyMatch(v -> v == null))); + assertHash(b, hashes.ords(), hash.size(), previousValues, i -> (int) hash.get(i)); + } + + private void assertLongHash(Set previousValues, BasicBlockTests.RandomBlock b) { + LongHash hash = new LongHash(1, BigArrays.NON_RECYCLING_INSTANCE); + previousValues.stream().forEach(hash::add); + MultivalueDedupe.HashResult hashes = new MultivalueDedupeLong((LongBlock) b.block()).hash(hash); + assertThat(hashes.sawNull(), equalTo(b.values().stream().anyMatch(v -> v == null))); + assertHash(b, hashes.ords(), hash.size(), previousValues, i -> hash.get(i)); + } + + private void assertDoubleHash(Set previousValues, BasicBlockTests.RandomBlock b) { + LongHash hash = new LongHash(1, BigArrays.NON_RECYCLING_INSTANCE); + previousValues.stream().forEach(d -> hash.add(Double.doubleToLongBits(d))); + MultivalueDedupe.HashResult hashes = new MultivalueDedupeDouble((DoubleBlock) b.block()).hash(hash); + assertThat(hashes.sawNull(), equalTo(b.values().stream().anyMatch(v -> v == null))); + assertHash(b, hashes.ords(), hash.size(), previousValues, i -> Double.longBitsToDouble(hash.get(i))); + } + + private void assertHash( + BasicBlockTests.RandomBlock b, + LongBlock hashes, + long hashSize, + Set previousValues, + LongFunction lookup + ) { + Set allValues = new HashSet<>(); + allValues.addAll(previousValues); + for (int p = 0; p < b.block().getPositionCount(); p++) { + assertThat(hashes.isNull(p), equalTo(false)); + int count = hashes.getValueCount(p); + int start = hashes.getFirstValueIndex(p); + List v = b.values().get(p); + if (v == null) { + assertThat(count, equalTo(1)); + assertThat(hashes.getLong(start), equalTo(0L)); + return; + } + List actualValues = new ArrayList<>(count); + int end = start + count; + for (int i = start; i < end; i++) { + actualValues.add(lookup.apply(hashes.getLong(i) - 1)); + } + assertThat(actualValues, containsInAnyOrder(v.stream().collect(Collectors.toSet()).stream().sorted().toArray())); + allValues.addAll(v); + } + + Set hashedValues = new HashSet<>((int) hashSize); + for (long i = 0; i < hashSize; i++) { + hashedValues.add(lookup.apply(i)); + } + assertThat(hashedValues, equalTo(allValues)); + } + + private int assertEncodedPosition(BasicBlockTests.RandomBlock b, BatchEncoder encoder, int position, int offset, int valueOffset) { + List expected = b.values().get(position); + if (expected == null) { + expected = new ArrayList<>(); + expected.add(null); + // BatchEncoder encodes null as a special empty value, but it counts as a value + } else { + NavigableSet set = new TreeSet<>(); + set.addAll(expected); + expected = new ArrayList<>(set); + } + + /* + * Decode all values at the positions into a block so we can compare them easily. + * This produces a block with a single value per position, but it's good enough + * for comparison. + */ + Block.Builder builder = elementType.newBlockBuilder(encoder.valueCount(offset)); + BytesRef[] toDecode = new BytesRef[encoder.valueCount(offset)]; + for (int i = 0; i < toDecode.length; i++) { + toDecode[i] = encoder.read(valueOffset++, new BytesRef()); + if (b.values().get(position) == null) { + // Nulls are encoded as 0 length values + assertThat(toDecode[i].length, equalTo(0)); + } else { + switch (elementType) { + case INT -> assertThat(toDecode[i].length, equalTo(Integer.BYTES)); + case LONG -> assertThat(toDecode[i].length, equalTo(Long.BYTES)); + case DOUBLE -> assertThat(toDecode[i].length, equalTo(Double.BYTES)); + case BOOLEAN -> assertThat(toDecode[i].length, equalTo(1)); + case BYTES_REF -> { + // Not a well defined length + } + default -> fail("unsupported type"); + } + } + } + BatchEncoder.decoder(elementType).decode(builder, i -> toDecode[i].length == 0, toDecode, toDecode.length); + for (int i = 0; i < toDecode.length; i++) { + assertThat(toDecode[i].length, equalTo(0)); + } + Block decoded = builder.build(); + assertThat(decoded.getPositionCount(), equalTo(toDecode.length)); + List actual = new ArrayList<>(); + BasicBlockTests.valuesAtPositions(decoded, 0, decoded.getPositionCount()) + .stream() + .forEach(l -> actual.add(l == null ? null : l.get(0))); + Collections.sort(actual, Comparator.comparing(o -> { + @SuppressWarnings("unchecked") // This is totally comparable, believe me + var c = (Comparable) o; + return c; + })); // Sort for easier visual comparison of errors + assertThat(actual, equalTo(expected)); + return valueOffset; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorStatusTests.java new file mode 100644 index 0000000000000..fe281bbf16131 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorStatusTests.java @@ -0,0 +1,58 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class MvExpandOperatorStatusTests extends AbstractWireSerializingTestCase { + public static MvExpandOperator.Status simple() { + return new MvExpandOperator.Status(10, 9); + } + + public static String simpleToJson() { + return """ + {"pages_processed":10,"noops":9}"""; + } + + public void testToXContent() { + assertThat(Strings.toString(simple()), equalTo(simpleToJson())); + } + + @Override + protected Writeable.Reader instanceReader() { + return MvExpandOperator.Status::new; + } + + @Override + public MvExpandOperator.Status createTestInstance() { + return new MvExpandOperator.Status(randomNonNegativeInt(), randomNonNegativeInt()); + } + + @Override + protected MvExpandOperator.Status mutateInstance(MvExpandOperator.Status instance) { + switch (between(0, 1)) { + case 0: + return new MvExpandOperator.Status( + randomValueOtherThan(instance.pagesProcessed(), ESTestCase::randomNonNegativeInt), + instance.noops() + ); + case 1: + return new MvExpandOperator.Status( + instance.pagesProcessed(), + randomValueOtherThan(instance.noops(), ESTestCase::randomNonNegativeInt) + ); + default: + throw new UnsupportedOperationException(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorTests.java new file mode 100644 index 0000000000000..80ac57ed539e7 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/MvExpandOperatorTests.java @@ -0,0 +1,134 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.BasicBlockTests; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; + +import static org.elasticsearch.compute.data.BasicBlockTests.randomBlock; +import static org.elasticsearch.compute.data.BasicBlockTests.valuesAtPositions; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class MvExpandOperatorTests extends OperatorTestCase { + @Override + protected SourceOperator simpleInput(int end) { + return new AbstractBlockSourceOperator(8 * 1024) { + private int idx; + + @Override + protected int remaining() { + return end - idx; + } + + @Override + protected Page createPage(int positionOffset, int length) { + idx += length; + return new Page( + randomBlock(ElementType.INT, length, true, 1, 10, 0, 0).block(), + randomBlock(ElementType.INT, length, false, 1, 10, 0, 0).block() + ); + } + }; + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return new MvExpandOperator.Factory(0); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "MvExpandOperator[channel=0]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + assertThat(results, hasSize(results.size())); + for (int i = 0; i < results.size(); i++) { + IntBlock origExpanded = input.get(i).getBlock(0); + IntBlock resultExpanded = results.get(i).getBlock(0); + int np = 0; + for (int op = 0; op < origExpanded.getPositionCount(); op++) { + if (origExpanded.isNull(op)) { + assertThat(resultExpanded.isNull(np), equalTo(true)); + assertThat(resultExpanded.getValueCount(np++), equalTo(0)); + continue; + } + List oValues = BasicBlockTests.valuesAtPositions(origExpanded, op, op + 1).get(0); + for (Object ov : oValues) { + assertThat(resultExpanded.isNull(np), equalTo(false)); + assertThat(resultExpanded.getValueCount(np), equalTo(1)); + assertThat(BasicBlockTests.valuesAtPositions(resultExpanded, np, ++np).get(0), equalTo(List.of(ov))); + } + } + + IntBlock origDuplicated = input.get(i).getBlock(1); + IntBlock resultDuplicated = results.get(i).getBlock(1); + np = 0; + for (int op = 0; op < origDuplicated.getPositionCount(); op++) { + int copies = origExpanded.isNull(op) ? 1 : origExpanded.getValueCount(op); + for (int c = 0; c < copies; c++) { + if (origDuplicated.isNull(op)) { + assertThat(resultDuplicated.isNull(np), equalTo(true)); + assertThat(resultDuplicated.getValueCount(np++), equalTo(0)); + continue; + } + assertThat(resultDuplicated.isNull(np), equalTo(false)); + assertThat(resultDuplicated.getValueCount(np), equalTo(origDuplicated.getValueCount(op))); + assertThat( + BasicBlockTests.valuesAtPositions(resultDuplicated, np, ++np).get(0), + equalTo(BasicBlockTests.valuesAtPositions(origDuplicated, op, op + 1).get(0)) + ); + } + } + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } + + public void testNoopStatus() { + MvExpandOperator op = new MvExpandOperator(0); + List result = drive( + op, + List.of(new Page(IntVector.newVectorBuilder(2).appendInt(1).appendInt(2).build().asBlock())).iterator() + ); + assertThat(result, hasSize(1)); + assertThat(valuesAtPositions(result.get(0).getBlock(0), 0, 2), equalTo(List.of(List.of(1), List.of(2)))); + MvExpandOperator.Status status = (MvExpandOperator.Status) op.status(); + assertThat(status.pagesProcessed(), equalTo(1)); + assertThat(status.noops(), equalTo(1)); + } + + public void testExpandStatus() { + MvExpandOperator op = new MvExpandOperator(0); + var builder = IntBlock.newBlockBuilder(2).beginPositionEntry().appendInt(1).appendInt(2).endPositionEntry(); + List result = drive(op, List.of(new Page(builder.build())).iterator()); + assertThat(result, hasSize(1)); + assertThat(valuesAtPositions(result.get(0).getBlock(0), 0, 2), equalTo(List.of(List.of(1), List.of(2)))); + MvExpandOperator.Status status = (MvExpandOperator.Status) op.status(); + assertThat(status.pagesProcessed(), equalTo(1)); + assertThat(status.noops(), equalTo(0)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/NullInsertingSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/NullInsertingSourceOperator.java new file mode 100644 index 0000000000000..2373abb14d697 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/NullInsertingSourceOperator.java @@ -0,0 +1,100 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; + +import java.util.Arrays; + +import static org.elasticsearch.test.ESTestCase.between; + +/** + * Inserts nulls into blocks + */ +public class NullInsertingSourceOperator extends MappingSourceOperator { + public NullInsertingSourceOperator(SourceOperator delegate) { + super(delegate); + } + + @Override + protected Page map(Page page) { + if (page == null) { + return null; + } + Block.Builder[] builders = new Block.Builder[page.getBlockCount()]; + for (int b = 0; b < builders.length; b++) { + builders[b] = page.getBlock(b).elementType().newBlockBuilder(page.getPositionCount()); + } + for (int position = 0; position < page.getPositionCount(); position++) { + for (int nulls = between(0, 3); nulls > 0; nulls--) { + for (int b = 0; b < builders.length; b++) { + appendNull(page.getBlock(b).elementType(), builders[b], b); + } + } + for (int b = 0; b < builders.length; b++) { + copyValues(page.getBlock(b), position, builders[b]); + } + } + return new Page(Arrays.stream(builders).map(Block.Builder::build).toArray(Block[]::new)); + } + + protected void appendNull(ElementType elementType, Block.Builder builder, int blockId) { + builder.appendNull(); + } + + private void copyValues(Block from, int position, Block.Builder into) { + if (from.isNull(position)) { + into.appendNull(); + return; + } + + int valueCount = from.getValueCount(position); + int firstValue = from.getFirstValueIndex(position); + if (valueCount == 1) { + copyValue(from, firstValue, into); + return; + } + into.beginPositionEntry(); + int end = firstValue + valueCount; + for (int valueIndex = firstValue; valueIndex < end; valueIndex++) { + copyValue(from, valueIndex, into); + } + into.endPositionEntry(); + } + + private void copyValue(Block from, int valueIndex, Block.Builder into) { + ElementType elementType = from.elementType(); + switch (elementType) { + case BOOLEAN: + ((BooleanBlock.Builder) into).appendBoolean(((BooleanBlock) from).getBoolean(valueIndex)); + break; + case BYTES_REF: + ((BytesRefBlock.Builder) into).appendBytesRef(((BytesRefBlock) from).getBytesRef(valueIndex, new BytesRef())); + break; + case LONG: + ((LongBlock.Builder) into).appendLong(((LongBlock) from).getLong(valueIndex)); + break; + case INT: + ((IntBlock.Builder) into).appendInt(((IntBlock) from).getInt(valueIndex)); + break; + case DOUBLE: + ((DoubleBlock.Builder) into).appendDouble(((DoubleBlock) from).getDouble(valueIndex)); + break; + default: + throw new IllegalArgumentException("unknown block type " + elementType); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/OperatorTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/OperatorTestCase.java new file mode 100644 index 0000000000000..f32ef67ab766c --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/OperatorTestCase.java @@ -0,0 +1,200 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArray; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.indices.CrankyCircuitBreakerService; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.AssumptionViolatedException; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; + +/** + * Base tests for {@link Operator}s that are not {@link SourceOperator} or {@link SinkOperator}. + */ +public abstract class OperatorTestCase extends AnyOperatorTestCase { + /** + * Valid input to be sent to {@link #simple}; + */ + protected abstract SourceOperator simpleInput(int size); + + /** + * Assert that output from {@link #simple} is correct for the + * given input. + */ + protected abstract void assertSimpleOutput(List input, List results); + + /** + * A {@link ByteSizeValue} that is so small any input to the operator + * will cause it to circuit break. If the operator can't break then + * throw an {@link AssumptionViolatedException}. + */ + protected abstract ByteSizeValue smallEnoughToCircuitBreak(); + + /** + * Test a small input set against {@link #simple}. Smaller input sets + * are more likely to discover accidental behavior for clumped inputs. + */ + public final void testSimpleSmallInput() { + assertSimple(nonBreakingBigArrays(), between(10, 100)); + } + + /** + * Test a larger input set against {@link #simple}. + */ + public final void testSimpleLargeInput() { + assertSimple(nonBreakingBigArrays(), between(1_000, 10_000)); + } + + /** + * Run {@link #simple} with a circuit breaker configured by + * {@link #smallEnoughToCircuitBreak} and assert that it breaks + * in a sane way. + */ + public final void testSimpleCircuitBreaking() { + BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, smallEnoughToCircuitBreak()); + Exception e = expectThrows(CircuitBreakingException.class, () -> assertSimple(bigArrays, between(1_000, 10_000))); + assertThat(e.getMessage(), equalTo(MockBigArrays.ERROR_MESSAGE)); + } + + /** + * Run {@link #simple} with the {@link CrankyCircuitBreakerService} + * which fails randomly. This will catch errors caused by not + * properly cleaning up things like {@link BigArray}s, particularly + * in ctors. + */ + public final void testSimpleWithCranky() { + CrankyCircuitBreakerService breaker = new CrankyCircuitBreakerService(); + BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, breaker).withCircuitBreaking(); + try { + assertSimple(bigArrays, between(1_000, 10_000)); + // Either we get lucky and cranky doesn't throw and the test completes or we don't and it throws + } catch (CircuitBreakingException e) { + assertThat(e.getMessage(), equalTo(CrankyCircuitBreakerService.ERROR_MESSAGE)); + } + } + + /** + * Run the {@code operators} once per page in the {@code input}. + */ + protected final List oneDriverPerPage(List input, Supplier> operators) { + return oneDriverPerPageList(input.stream().map(List::of).iterator(), operators); + } + + /** + * Run the {@code operators} once to entry in the {@code source}. + */ + protected final List oneDriverPerPageList(Iterator> source, Supplier> operators) { + List result = new ArrayList<>(); + while (source.hasNext()) { + List in = source.next(); + try ( + Driver d = new Driver( + new DriverContext(), + new CannedSourceOperator(in.iterator()), + operators.get(), + new PageConsumerOperator(result::add), + () -> {} + ) + ) { + runDriver(d); + } + } + return result; + } + + private void assertSimple(BigArrays bigArrays, int size) { + List input = CannedSourceOperator.collectPages(simpleInput(size)); + List results = drive(simple(bigArrays.withCircuitBreaking()).get(new DriverContext()), input.iterator()); + assertSimpleOutput(input, results); + } + + protected final List drive(Operator operator, Iterator input) { + return drive(List.of(operator), input); + } + + protected final List drive(List operators, Iterator input) { + List results = new ArrayList<>(); + try ( + Driver d = new Driver( + new DriverContext(), + new CannedSourceOperator(input), + operators, + new PageConsumerOperator(results::add), + () -> {} + ) + ) { + runDriver(d); + } + return results; + } + + public static void runDriver(Driver driver) { + runDriver(List.of(driver)); + } + + public static void runDriver(List drivers) { + drivers = new ArrayList<>(drivers); + int dummyDrivers = between(0, 10); + for (int i = 0; i < dummyDrivers; i++) { + drivers.add( + new Driver( + "dummy-session", + new DriverContext(), + () -> "dummy-driver", + new SequenceLongBlockSourceOperator(LongStream.range(0, between(1, 100)), between(1, 100)), + List.of(), + new PageConsumerOperator(page -> {}), + () -> {} + ) + ); + } + Randomness.shuffle(drivers); + int numThreads = between(1, 16); + ThreadPool threadPool = new TestThreadPool( + getTestClass().getSimpleName(), + new FixedExecutorBuilder(Settings.EMPTY, "esql", numThreads, 1024, "esql", EsExecutors.TaskTrackingConfig.DEFAULT) + ); + try { + DriverRunner.runToCompletion(threadPool, between(1, 10000), drivers); + } finally { + terminate(threadPool); + } + } + + public static void assertDriverContext(DriverContext driverContext) { + assertTrue(driverContext.isFinished()); + assertThat(driverContext.getSnapshot().releasables(), empty()); + } + + public static int randomPageSize() { + if (randomBoolean()) { + return between(1, 16); + } else { + return between(1, 16 * 1024); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PassThroughOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PassThroughOperator.java new file mode 100644 index 0000000000000..72c9c5f86f417 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PassThroughOperator.java @@ -0,0 +1,48 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Page; + +/** An operator that just passes pages through until it is finished. */ +public class PassThroughOperator implements Operator { + + boolean finished; + Page page; + + @Override + public boolean needsInput() { + return page == null && finished == false; + } + + @Override + public void addInput(Page page) { + assert this.page == null; + this.page = page; + } + + @Override + public Page getOutput() { + Page p = page; + page = null; + return p; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean isFinished() { + return finished && page == null; + } + + @Override + public void close() {} +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PositionMergingSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PositionMergingSourceOperator.java new file mode 100644 index 0000000000000..67cbe27dba5e4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/PositionMergingSourceOperator.java @@ -0,0 +1,79 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; + +public class PositionMergingSourceOperator extends MappingSourceOperator { + public PositionMergingSourceOperator(SourceOperator delegate) { + super(delegate); + } + + @Override + protected Page map(Page page) { + Block[] merged = new Block[page.getBlockCount()]; + for (int b = 0; b < page.getBlockCount(); b++) { + merged[b] = merge(b, page.getBlock(b)); + } + return new Page(merged); + } + + protected Block merge(int blockIndex, Block block) { + Block.Builder builder = block.elementType().newBlockBuilder(block.getPositionCount()); + for (int p = 0; p + 1 < block.getPositionCount(); p += 2) { + if (block.isNull(p) || block.isNull(p + 1)) { + builder.appendNull(); + continue; + } + + int firstCount = block.getValueCount(p); + int secondCount = block.getValueCount(p + 1); + if (firstCount + secondCount == 1) { + if (firstCount == 1) { + builder.copyFrom(block, p, p + 1); + } else { + builder.copyFrom(block, p + 1, p + 2); + } + continue; + } + + builder.beginPositionEntry(); + copyTo(builder, block, p, firstCount); + copyTo(builder, block, p + 1, secondCount); + builder.endPositionEntry(); + } + if (block.getPositionCount() % 2 == 1) { + builder.copyFrom(block, block.getPositionCount() - 1, block.getPositionCount()); + } + return builder.build(); + } + + private void copyTo(Block.Builder builder, Block in, int position, int valueCount) { + int start = in.getFirstValueIndex(position); + int end = start + valueCount; + BytesRef scratch = new BytesRef(); + for (int i = start; i < end; i++) { + switch (in.elementType()) { + case BOOLEAN -> ((BooleanBlock.Builder) builder).appendBoolean(((BooleanBlock) in).getBoolean(i)); + case BYTES_REF -> ((BytesRefBlock.Builder) builder).appendBytesRef(((BytesRefBlock) in).getBytesRef(i, scratch)); + case DOUBLE -> ((DoubleBlock.Builder) builder).appendDouble(((DoubleBlock) in).getDouble(i)); + case INT -> ((IntBlock.Builder) builder).appendInt(((IntBlock) in).getInt(i)); + case LONG -> ((LongBlock.Builder) builder).appendLong(((LongBlock) in).getLong(i)); + default -> throw new IllegalArgumentException("unsupported type [" + in.elementType() + "]"); + } + } + } + +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java new file mode 100644 index 0000000000000..691c6f6cdbf56 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java @@ -0,0 +1,109 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.BitSet; +import java.util.List; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class ProjectOperatorTests extends OperatorTestCase { + public void testProjectionOnEmptyPage() { + var page = new Page(0); + var projection = new ProjectOperator(randomMask(randomIntBetween(2, 10))); + projection.addInput(page); + assertEquals(page, projection.getOutput()); + } + + public void testProjection() { + var size = randomIntBetween(2, 5); + var blocks = new Block[size]; + for (int i = 0; i < blocks.length; i++) { + blocks[i] = new ConstantIntVector(i, size).asBlock(); + } + + var page = new Page(size, blocks); + var mask = randomMask(size); + + var projection = new ProjectOperator(mask); + projection.addInput(page); + var out = projection.getOutput(); + assertEquals(mask.cardinality(), out.getBlockCount()); + + int lastSetIndex = -1; + for (int i = 0; i < out.getBlockCount(); i++) { + var block = out.getBlock(i); + var shouldBeSetInMask = block.getInt(0); + assertTrue(mask.get(shouldBeSetInMask)); + lastSetIndex = mask.nextSetBit(lastSetIndex + 1); + assertEquals(shouldBeSetInMask, lastSetIndex); + } + } + + private BitSet randomMask(int size) { + var mask = new BitSet(size); + for (int i = 0; i < size; i++) { + mask.set(i, randomBoolean()); + } + return mask; + } + + @Override + protected SourceOperator simpleInput(int end) { + return new TupleBlockSourceOperator(LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + BitSet mask = new BitSet(); + mask.set(1, true); + return new ProjectOperator.ProjectOperatorFactory(mask); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "ProjectOperator[mask = {1}]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + long expected = input.stream().mapToInt(Page::getPositionCount).sum(); + int total = 0; + for (Page page : results) { + assertThat(page.getBlockCount(), equalTo(1)); + LongBlock remaining = page.getBlock(0); + total += page.getPositionCount(); + for (int i = 0; i < page.getPositionCount(); i++) { + assertThat(remaining.getLong(i), equalTo(expected)); + expected--; + } + } + assertThat(total, equalTo(input.stream().mapToInt(Page::getPositionCount).sum())); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowOperatorTests.java new file mode 100644 index 0000000000000..ac7bc2f7e4ad1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowOperatorTests.java @@ -0,0 +1,74 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class RowOperatorTests extends ESTestCase { + final DriverContext driverContext = new DriverContext(); + + public void testBoolean() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(List.of(false)); + assertThat(factory.describe(), equalTo("RowOperator[objects = false]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[false]]")); + BooleanBlock block = factory.get(driverContext).getOutput().getBlock(0); + assertThat(block.getBoolean(0), equalTo(false)); + } + + public void testInt() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(List.of(213)); + assertThat(factory.describe(), equalTo("RowOperator[objects = 213]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[213]]")); + IntBlock block = factory.get(driverContext).getOutput().getBlock(0); + assertThat(block.getInt(0), equalTo(213)); + } + + public void testLong() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(List.of(21321343214L)); + assertThat(factory.describe(), equalTo("RowOperator[objects = 21321343214]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[21321343214]]")); + LongBlock block = factory.get(driverContext).getOutput().getBlock(0); + assertThat(block.getLong(0), equalTo(21321343214L)); + } + + public void testDouble() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(List.of(2.0)); + assertThat(factory.describe(), equalTo("RowOperator[objects = 2.0]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[2.0]]")); + DoubleBlock block = factory.get(driverContext).getOutput().getBlock(0); + assertThat(block.getDouble(0), equalTo(2.0)); + } + + public void testString() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(List.of(new BytesRef("cat"))); + assertThat(factory.describe(), equalTo("RowOperator[objects = [63 61 74]]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[[63 61 74]]]")); + BytesRefBlock block = factory.get(driverContext).getOutput().getBlock(0); + assertThat(block.getBytesRef(0, new BytesRef()), equalTo(new BytesRef("cat"))); + } + + public void testNull() { + RowOperator.RowOperatorFactory factory = new RowOperator.RowOperatorFactory(Arrays.asList(new Object[] { null })); + assertThat(factory.describe(), equalTo("RowOperator[objects = null]")); + assertThat(factory.get(driverContext).toString(), equalTo("RowOperator[objects=[null]]")); + Block block = factory.get(driverContext).getOutput().getBlock(0); + assertTrue(block.isNull(0)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceBooleanBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceBooleanBlockSourceOperator.java new file mode 100644 index 0000000000000..b85d328271c6f --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceBooleanBlockSourceOperator.java @@ -0,0 +1,50 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; + +/** + * A source operator whose output is the given boolean values. This operator produces pages + * containing a single Block. The Block contains the boolean values from the given list, in order. + */ +public class SequenceBooleanBlockSourceOperator extends AbstractBlockSourceOperator { + + static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final boolean[] values; + + public SequenceBooleanBlockSourceOperator(List values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceBooleanBlockSourceOperator(List values, int maxPagePositions) { + super(maxPagePositions); + this.values = new boolean[values.size()]; + for (int i = 0; i < values.size(); i++) { + this.values[i] = values.get(i); + } + } + + @Override + protected Page createPage(int positionOffset, int length) { + BooleanVector.Builder builder = BooleanVector.newVectorBuilder(length); + for (int i = 0; i < length; i++) { + builder.appendBoolean(values[positionOffset + i]); + } + currentPosition += length; + return new Page(builder.build().asBlock()); + } + + protected int remaining() { + return values.length - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceDoubleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceDoubleBlockSourceOperator.java new file mode 100644 index 0000000000000..4c1590ae9b8ff --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceDoubleBlockSourceOperator.java @@ -0,0 +1,57 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.stream.DoubleStream; + +/** + * A source operator whose output is the given double values. This operator produces pages + * containing a single Block. The Block contains the double values from the given list, in order. + */ +public class SequenceDoubleBlockSourceOperator extends AbstractBlockSourceOperator { + + static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final double[] values; + + public SequenceDoubleBlockSourceOperator(DoubleStream values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceDoubleBlockSourceOperator(DoubleStream values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toArray(); + } + + public SequenceDoubleBlockSourceOperator(List values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceDoubleBlockSourceOperator(List values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.stream().mapToDouble(Double::doubleValue).toArray(); + } + + @Override + protected Page createPage(int positionOffset, int length) { + final double[] array = new double[length]; + for (int i = 0; i < length; i++) { + array[i] = values[positionOffset + i]; + } + currentPosition += length; + return new Page(new DoubleArrayVector(array, array.length).asBlock()); + } + + protected int remaining() { + return values.length - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceIntBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceIntBlockSourceOperator.java new file mode 100644 index 0000000000000..135fdd1dae436 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceIntBlockSourceOperator.java @@ -0,0 +1,57 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.stream.IntStream; + +/** + * A source operator whose output is the given integer values. This operator produces pages + * containing a single Block. The Block contains the integer values from the given list, in order. + */ +public class SequenceIntBlockSourceOperator extends AbstractBlockSourceOperator { + + static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final int[] values; + + public SequenceIntBlockSourceOperator(IntStream values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceIntBlockSourceOperator(IntStream values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toArray(); + } + + public SequenceIntBlockSourceOperator(List values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceIntBlockSourceOperator(List values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.stream().mapToInt(Integer::intValue).toArray(); + } + + @Override + protected Page createPage(int positionOffset, int length) { + IntVector.Builder builder = IntVector.newVectorBuilder(length); + for (int i = 0; i < length; i++) { + builder.appendInt(values[positionOffset + i]); + } + currentPosition += length; + return new Page(builder.build().asBlock()); + } + + protected int remaining() { + return values.length - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceLongBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceLongBlockSourceOperator.java new file mode 100644 index 0000000000000..8600237401ed0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/SequenceLongBlockSourceOperator.java @@ -0,0 +1,57 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.stream.LongStream; + +/** + * A source operator whose output is the given long values. This operator produces pages + * containing a single Block. The Block contains the long values from the given list, in order. + */ +public class SequenceLongBlockSourceOperator extends AbstractBlockSourceOperator { + + static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final long[] values; + + public SequenceLongBlockSourceOperator(LongStream values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceLongBlockSourceOperator(LongStream values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toArray(); + } + + public SequenceLongBlockSourceOperator(List values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public SequenceLongBlockSourceOperator(List values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.stream().mapToLong(Long::longValue).toArray(); + } + + @Override + protected Page createPage(int positionOffset, int length) { + final long[] array = new long[length]; + for (int i = 0; i < length; i++) { + array[i] = values[positionOffset + i]; + } + currentPosition += length; + return new Page(new LongArrayVector(array, array.length).asBlock()); + } + + protected int remaining() { + return values.length - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/StringExtractOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/StringExtractOperatorTests.java new file mode 100644 index 0000000000000..12bc5da607934 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/StringExtractOperatorTests.java @@ -0,0 +1,113 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.Page; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.hamcrest.Matchers.equalTo; + +public class StringExtractOperatorTests extends OperatorTestCase { + @Override + protected SourceOperator simpleInput(int end) { + List input = LongStream.range(0, end) + .mapToObj(l -> new BytesRef("word1_" + l + " word2_" + l + " word3_" + l)) + .collect(Collectors.toList()); + return new BytesRefBlockSourceOperator(input); + } + + record FirstWord(String fieldName) implements Function> { + @Override + public Map apply(String s) { + return Map.of(fieldName, s.split(" ")[0]); + } + } + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + Supplier>> expEval = () -> new FirstWord("test"); + return new StringExtractOperator.StringExtractOperatorFactory(new String[] { "test" }, () -> page -> page.getBlock(0), expEval); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "StringExtractOperator[fields=[test]]"; + } + + @Override + protected String expectedToStringOfSimple() { + return expectedDescriptionOfSimple(); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + BytesRef buffer = new BytesRef(); + int pos = 0; + for (var page : results) { + BytesRefBlock block1 = page.getBlock(1); + + for (int i = 0; i < page.getPositionCount(); i++) { + assertEquals(new BytesRef("word1_" + pos).utf8ToString(), block1.getBytesRef(i, buffer).utf8ToString()); + pos++; + } + } + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("doesn't use big arrays so can't break", false); + return null; + } + + public void testMultivalueDissectInput() { + + StringExtractOperator operator = new StringExtractOperator( + new String[] { "test" }, + (page) -> page.getBlock(0), + new FirstWord("test") + ); + + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(1); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("foo1 bar1")); + builder.appendBytesRef(new BytesRef("foo2 bar2")); + builder.endPositionEntry(); + builder.beginPositionEntry(); + builder.appendBytesRef(new BytesRef("foo3 bar3")); + builder.appendBytesRef(new BytesRef("foo4 bar4")); + builder.appendBytesRef(new BytesRef("foo5 bar5")); + builder.endPositionEntry(); + Page page = new Page(builder.build()); + + Page result = operator.process(page); + Block resultBlock = result.getBlock(1); + assertThat(resultBlock.getPositionCount(), equalTo(2)); + assertThat(resultBlock.getValueCount(0), equalTo(2)); + assertThat(resultBlock.getValueCount(1), equalTo(3)); + BytesRefBlock brb = (BytesRefBlock) resultBlock; + BytesRef spare = new BytesRef(""); + int idx = brb.getFirstValueIndex(0); + assertThat(brb.getBytesRef(idx, spare).utf8ToString(), equalTo("foo1")); + assertThat(brb.getBytesRef(idx + 1, spare).utf8ToString(), equalTo("foo2")); + idx = brb.getFirstValueIndex(1); + assertThat(brb.getBytesRef(idx, spare).utf8ToString(), equalTo("foo3")); + assertThat(brb.getBytesRef(idx + 1, spare).utf8ToString(), equalTo("foo4")); + assertThat(brb.getBytesRef(idx + 2, spare).utf8ToString(), equalTo("foo5")); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TopNOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TopNOperatorTests.java new file mode 100644 index 0000000000000..776a8f61632ef --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TopNOperatorTests.java @@ -0,0 +1,412 @@ +/* + * 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.compute.operator; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static org.elasticsearch.compute.data.BlockTestUtils.append; +import static org.elasticsearch.compute.data.BlockTestUtils.randomValue; +import static org.elasticsearch.compute.data.BlockTestUtils.readInto; +import static org.elasticsearch.core.Tuple.tuple; +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +public class TopNOperatorTests extends OperatorTestCase { + + private final int pageSize = randomPageSize(); + + @Override + protected Operator.OperatorFactory simple(BigArrays bigArrays) { + return new TopNOperator.TopNOperatorFactory(4, List.of(new TopNOperator.SortOrder(0, true, false)), pageSize); + } + + @Override + protected String expectedDescriptionOfSimple() { + return "TopNOperator[count = 4, sortOrders = [SortOrder[channel=0, asc=true, nullsFirst=false]]]"; + } + + @Override + protected String expectedToStringOfSimple() { + return "TopNOperator[count = 0/4, sortOrder = SortOrder[channel=0, asc=true, nullsFirst=false]]"; + } + + @Override + protected SourceOperator simpleInput(int size) { + return new SequenceLongBlockSourceOperator(LongStream.range(0, size).map(l -> ESTestCase.randomLong())); + } + + @Override + protected void assertSimpleOutput(List input, List results) { + for (int i = 0; i < results.size() - 1; i++) { + assertThat(results.get(i).getPositionCount(), equalTo(pageSize)); + } + assertThat(results.get(results.size() - 1).getPositionCount(), lessThanOrEqualTo(pageSize)); + long[] topN = input.stream() + .flatMapToLong( + page -> IntStream.range(0, page.getPositionCount()) + .filter(p -> false == page.getBlock(0).isNull(p)) + .mapToLong(p -> ((LongBlock) page.getBlock(0)).getLong(p)) + ) + .sorted() + .limit(4) + .toArray(); + assertThat( + results.stream() + .flatMapToLong(page -> IntStream.range(0, page.getPositionCount()).mapToLong(i -> page.getBlock(0).getLong(i))) + .toArray(), + equalTo(topN) + ); + } + + @Override + protected ByteSizeValue smallEnoughToCircuitBreak() { + assumeTrue("TopN doesn't break the circuit breaker for now", false); + return ByteSizeValue.ZERO; + } + + public void testRandomTopN() { + for (boolean asc : List.of(true, false)) { + int limit = randomIntBetween(1, 20); + List inputValues = randomList(0, 5000, ESTestCase::randomLong); + Comparator comparator = asc ? Comparator.naturalOrder() : Comparator.reverseOrder(); + List expectedValues = inputValues.stream().sorted(comparator).limit(limit).toList(); + List outputValues = topN(inputValues, limit, asc, false); + assertThat(outputValues, equalTo(expectedValues)); + } + } + + public void testBasicTopN() { + List values = Arrays.asList(2L, 1L, 4L, null, 5L, 10L, null, 20L, 4L, 100L); + assertThat(topN(values, 1, true, false), equalTo(Arrays.asList(1L))); + assertThat(topN(values, 1, false, false), equalTo(Arrays.asList(100L))); + assertThat(topN(values, 2, true, false), equalTo(Arrays.asList(1L, 2L))); + assertThat(topN(values, 2, false, false), equalTo(Arrays.asList(100L, 20L))); + assertThat(topN(values, 3, true, false), equalTo(Arrays.asList(1L, 2L, 4L))); + assertThat(topN(values, 3, false, false), equalTo(Arrays.asList(100L, 20L, 10L))); + assertThat(topN(values, 4, true, false), equalTo(Arrays.asList(1L, 2L, 4L, 4L))); + assertThat(topN(values, 4, false, false), equalTo(Arrays.asList(100L, 20L, 10L, 5L))); + assertThat(topN(values, 100, true, false), equalTo(Arrays.asList(1L, 2L, 4L, 4L, 5L, 10L, 20L, 100L, null, null))); + assertThat(topN(values, 100, false, false), equalTo(Arrays.asList(100L, 20L, 10L, 5L, 4L, 4L, 2L, 1L, null, null))); + assertThat(topN(values, 1, true, true), equalTo(Arrays.asList(new Long[] { null }))); + assertThat(topN(values, 1, false, true), equalTo(Arrays.asList(new Long[] { null }))); + assertThat(topN(values, 2, true, true), equalTo(Arrays.asList(null, null))); + assertThat(topN(values, 2, false, true), equalTo(Arrays.asList(null, null))); + assertThat(topN(values, 3, true, true), equalTo(Arrays.asList(null, null, 1L))); + assertThat(topN(values, 3, false, true), equalTo(Arrays.asList(null, null, 100L))); + assertThat(topN(values, 4, true, true), equalTo(Arrays.asList(null, null, 1L, 2L))); + assertThat(topN(values, 4, false, true), equalTo(Arrays.asList(null, null, 100L, 20L))); + assertThat(topN(values, 100, true, true), equalTo(Arrays.asList(null, null, 1L, 2L, 4L, 4L, 5L, 10L, 20L, 100L))); + assertThat(topN(values, 100, false, true), equalTo(Arrays.asList(null, null, 100L, 20L, 10L, 5L, 4L, 4L, 2L, 1L))); + } + + public void testCompareInts() { + Block[] bs = new Block[] { + IntBlock.newBlockBuilder(2).appendInt(Integer.MIN_VALUE).appendInt(randomIntBetween(-1000, -1)).build(), + IntBlock.newBlockBuilder(2).appendInt(randomIntBetween(-1000, -1)).appendInt(0).build(), + IntBlock.newBlockBuilder(2).appendInt(0).appendInt(randomIntBetween(1, 1000)).build(), + IntBlock.newBlockBuilder(2).appendInt(randomIntBetween(1, 1000)).appendInt(Integer.MAX_VALUE).build(), + IntBlock.newBlockBuilder(2).appendInt(Integer.MAX_VALUE).appendInt(0).build() }; + + Page page = new Page(bs); + TopNOperator.RowFactory rowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row bRow0 = rowFactory.row(page, 0, null); + TopNOperator.Row bRow1 = rowFactory.row(page, 1, null); + + Block nullBlock = Block.constantNullBlock(1); + Block[] nullBs = new Block[] { nullBlock, nullBlock, nullBlock, nullBlock, nullBlock }; + Page nullPage = new Page(nullBs); + TopNOperator.RowFactory nullRowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row nullRow = nullRowFactory.row(nullPage, 0, null); + + for (int i = 0; i < bs.length; i++) { + assertEquals(0, TopNOperator.comparePositions(randomBoolean(), randomBoolean(), bRow0, bRow0, i)); + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), true, bRow0, nullRow, i)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), false, bRow0, nullRow, i)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), true, nullRow, bRow0, i)); + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), false, nullRow, bRow0, i)); + } + for (int i = 0; i < bs.length - 1; i++) { + assertEquals(1, TopNOperator.comparePositions(true, randomBoolean(), bRow0, bRow1, i)); + assertEquals(-1, TopNOperator.comparePositions(true, randomBoolean(), bRow1, bRow0, i)); + assertEquals(-1, TopNOperator.comparePositions(false, randomBoolean(), bRow0, bRow1, i)); + assertEquals(1, TopNOperator.comparePositions(false, randomBoolean(), bRow1, bRow0, i)); + } + } + + public void testCompareBytesRef() { + Block[] bs = new Block[] { + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("bye")).appendBytesRef(new BytesRef("hello")).build() }; + Page page = new Page(bs); + TopNOperator.RowFactory rowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row bRow0 = rowFactory.row(page, 0, null); + TopNOperator.Row bRow1 = rowFactory.row(page, 1, null); + + assertEquals(0, TopNOperator.comparePositions(false, randomBoolean(), bRow0, bRow0, 0)); + assertEquals(0, TopNOperator.comparePositions(false, randomBoolean(), bRow1, bRow1, 0)); + assertThat(TopNOperator.comparePositions(true, randomBoolean(), bRow0, bRow1, 0), greaterThan(0)); + assertThat(TopNOperator.comparePositions(true, randomBoolean(), bRow1, bRow0, 0), lessThan(0)); + assertThat(TopNOperator.comparePositions(false, randomBoolean(), bRow0, bRow1, 0), lessThan(0)); + assertThat(TopNOperator.comparePositions(false, rarely(), bRow1, bRow0, 0), greaterThan(0)); + } + + public void testCompareBooleans() { + Block[] bs = new Block[] { + BooleanBlock.newBlockBuilder(2).appendBoolean(false).appendBoolean(true).build(), + BooleanBlock.newBlockBuilder(2).appendBoolean(true).appendBoolean(false).build() }; + + Page page = new Page(bs); + TopNOperator.RowFactory rowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row bRow0 = rowFactory.row(page, 0, null); + TopNOperator.Row bRow1 = rowFactory.row(page, 1, null); + + Block nullBlock = Block.constantNullBlock(2); + Block[] nullBs = new Block[] { nullBlock, nullBlock }; + Page nullPage = new Page(nullBs); + TopNOperator.RowFactory nullRowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row nullRow = nullRowFactory.row(nullPage, 0, null); + + assertEquals(0, TopNOperator.comparePositions(randomBoolean(), randomBoolean(), bRow0, bRow0, 0)); + assertEquals(0, TopNOperator.comparePositions(randomBoolean(), randomBoolean(), bRow1, bRow1, 0)); + + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), true, bRow0, nullRow, 0)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), false, bRow0, nullRow, 0)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), true, nullRow, bRow0, 0)); + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), false, nullRow, bRow0, 0)); + + for (int i = 0; i < bs.length - 1; i++) { + assertEquals(1, TopNOperator.comparePositions(true, randomBoolean(), bRow0, bRow1, 0)); + assertEquals(-1, TopNOperator.comparePositions(true, randomBoolean(), bRow1, bRow0, 0)); + assertEquals(-1, TopNOperator.comparePositions(false, randomBoolean(), bRow0, bRow1, 0)); + assertEquals(1, TopNOperator.comparePositions(false, randomBoolean(), bRow1, bRow0, 0)); + } + } + + public void testCompareWithNulls() { + Block i1 = IntBlock.newBlockBuilder(2).appendInt(100).appendNull().build(); + + Page page = new Page(i1); + TopNOperator.RowFactory rowFactory = new TopNOperator.RowFactory(page); + TopNOperator.Row bRow0 = rowFactory.row(page, 0, null); + TopNOperator.Row bRow1 = rowFactory.row(page, 1, null); + + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), true, bRow0, bRow1, 0)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), true, bRow1, bRow0, 0)); + assertEquals(1, TopNOperator.comparePositions(randomBoolean(), false, bRow0, bRow1, 0)); + assertEquals(-1, TopNOperator.comparePositions(randomBoolean(), false, bRow1, bRow0, 0)); + } + + private List topN(List inputValues, int limit, boolean ascendingOrder, boolean nullsFirst) { + return topNTwoColumns( + inputValues.stream().map(v -> tuple(v, 0L)).toList(), + limit, + List.of(new TopNOperator.SortOrder(0, ascendingOrder, nullsFirst)) + ).stream().map(Tuple::v1).toList(); + } + + public void testTopNTwoColumns() { + List> values = Arrays.asList(tuple(1L, 1L), tuple(1L, 2L), tuple(null, null), tuple(null, 1L), tuple(1L, null)); + assertThat( + topNTwoColumns(values, 5, List.of(new TopNOperator.SortOrder(0, true, false), new TopNOperator.SortOrder(1, true, false))), + equalTo(List.of(tuple(1L, 1L), tuple(1L, 2L), tuple(1L, null), tuple(null, 1L), tuple(null, null))) + ); + assertThat( + topNTwoColumns(values, 5, List.of(new TopNOperator.SortOrder(0, true, true), new TopNOperator.SortOrder(1, true, false))), + equalTo(List.of(tuple(null, 1L), tuple(null, null), tuple(1L, 1L), tuple(1L, 2L), tuple(1L, null))) + ); + assertThat( + topNTwoColumns(values, 5, List.of(new TopNOperator.SortOrder(0, true, false), new TopNOperator.SortOrder(1, true, true))), + equalTo(List.of(tuple(1L, null), tuple(1L, 1L), tuple(1L, 2L), tuple(null, null), tuple(null, 1L))) + ); + } + + public void testCollectAllValues() { + int size = 10; + int topCount = 3; + List blocks = new ArrayList<>(); + List> expectedTop = new ArrayList<>(); + + IntBlock keys = new IntArrayVector(IntStream.range(0, size).toArray(), size).asBlock(); + List topKeys = new ArrayList<>(IntStream.range(size - topCount, size).boxed().toList()); + Collections.reverse(topKeys); + expectedTop.add(topKeys); + blocks.add(keys); + + for (ElementType e : ElementType.values()) { + if (e == ElementType.UNKNOWN) { + continue; + } + List eTop = new ArrayList<>(); + Block.Builder builder = e.newBlockBuilder(size); + for (int i = 0; i < size; i++) { + Object value = randomValue(e); + append(builder, value); + if (i >= size - topCount) { + eTop.add(value); + } + } + Collections.reverse(eTop); + blocks.add(builder.build()); + expectedTop.add(eTop); + } + + List> actualTop = new ArrayList<>(); + DriverContext driverContext = new DriverContext(); + try ( + Driver driver = new Driver( + driverContext, + new CannedSourceOperator(List.of(new Page(blocks.toArray(Block[]::new))).iterator()), + List.of(new TopNOperator(topCount, List.of(new TopNOperator.SortOrder(0, false, false)), randomPageSize())), + new PageConsumerOperator(page -> readInto(actualTop, page)), + () -> {} + ) + ) { + runDriver(driver); + } + + assertMap(actualTop, matchesList(expectedTop)); + assertDriverContext(driverContext); + } + + public void testCollectAllValues_RandomMultiValues() { + int rows = 10; + int topCount = 3; + int blocksCount = 20; + List blocks = new ArrayList<>(); + List> expectedTop = new ArrayList<>(); + + IntBlock keys = new IntArrayVector(IntStream.range(0, rows).toArray(), rows).asBlock(); + List topKeys = new ArrayList<>(IntStream.range(rows - topCount, rows).boxed().toList()); + Collections.reverse(topKeys); + expectedTop.add(topKeys); + blocks.add(keys); + + for (int type = 0; type < blocksCount; type++) { + ElementType e = randomFrom(ElementType.values()); + if (e == ElementType.UNKNOWN) { + continue; + } + List eTop = new ArrayList<>(); + Block.Builder builder = e.newBlockBuilder(rows); + for (int i = 0; i < rows; i++) { + if (e != ElementType.DOC && e != ElementType.NULL && randomBoolean()) { + // generate a multi-value block + int mvCount = randomIntBetween(5, 10); + List eTopList = new ArrayList<>(mvCount); + builder.beginPositionEntry(); + for (int j = 0; j < mvCount; j++) { + Object value = randomValue(e); + append(builder, value); + if (i >= rows - topCount) { + eTopList.add(value); + } + } + builder.endPositionEntry(); + if (i >= rows - topCount) { + eTop.add(eTopList); + } + } else { + Object value = randomValue(e); + append(builder, value); + if (i >= rows - topCount) { + eTop.add(value); + } + } + } + Collections.reverse(eTop); + blocks.add(builder.build()); + expectedTop.add(eTop); + } + + DriverContext driverContext = new DriverContext(); + List> actualTop = new ArrayList<>(); + try ( + Driver driver = new Driver( + driverContext, + new CannedSourceOperator(List.of(new Page(blocks.toArray(Block[]::new))).iterator()), + List.of(new TopNOperator(topCount, List.of(new TopNOperator.SortOrder(0, false, false)), randomPageSize())), + new PageConsumerOperator(page -> readInto(actualTop, page)), + () -> {} + ) + ) { + runDriver(driver); + } + + assertMap(actualTop, matchesList(expectedTop)); + assertDriverContext(driverContext); + } + + private List> topNTwoColumns( + List> inputValues, + int limit, + List sortOrders + ) { + DriverContext driverContext = new DriverContext(); + List> outputValues = new ArrayList<>(); + try ( + Driver driver = new Driver( + driverContext, + new TupleBlockSourceOperator(inputValues, randomIntBetween(1, 1000)), + List.of(new TopNOperator(limit, sortOrders, randomPageSize())), + new PageConsumerOperator(page -> { + LongBlock block1 = page.getBlock(0); + LongBlock block2 = page.getBlock(1); + for (int i = 0; i < block1.getPositionCount(); i++) { + outputValues.add(tuple(block1.isNull(i) ? null : block1.getLong(i), block2.isNull(i) ? null : block2.getLong(i))); + } + }), + () -> {} + ) + ) { + runDriver(driver); + } + assertThat(outputValues, hasSize(Math.min(limit, inputValues.size()))); + assertDriverContext(driverContext); + return outputValues; + } + + public void testTopNManyDescriptionAndToString() { + TopNOperator.TopNOperatorFactory factory = new TopNOperator.TopNOperatorFactory( + 10, + List.of(new TopNOperator.SortOrder(1, false, false), new TopNOperator.SortOrder(3, false, true)), + randomPageSize() + ); + String sorts = List.of("SortOrder[channel=1, asc=false, nullsFirst=false]", "SortOrder[channel=3, asc=false, nullsFirst=true]") + .stream() + .collect(Collectors.joining(", ")); + assertThat(factory.describe(), equalTo("TopNOperator[count = 10, sortOrders = [" + sorts + "]]")); + try (Operator operator = factory.get(new DriverContext())) { + assertThat(operator.toString(), equalTo("TopNOperator[count = 0/10, sortOrders = [" + sorts + "]]")); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java new file mode 100644 index 0000000000000..0bcaac0e5b646 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java @@ -0,0 +1,70 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Tuple; + +import java.util.List; +import java.util.stream.Stream; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class TupleBlockSourceOperator extends AbstractBlockSourceOperator { + + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + + public TupleBlockSourceOperator(Stream> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public TupleBlockSourceOperator(Stream> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values.toList(); + } + + public TupleBlockSourceOperator(List> values) { + this(values, DEFAULT_MAX_PAGE_POSITIONS); + } + + public TupleBlockSourceOperator(List> values, int maxPagePositions) { + super(maxPagePositions); + this.values = values; + } + + @Override + protected Page createPage(int positionOffset, int length) { + var blockBuilder1 = LongBlock.newBlockBuilder(length); + var blockBuilder2 = LongBlock.newBlockBuilder(length); + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + blockBuilder1.appendLong(item.v1()); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + blockBuilder2.appendLong(item.v2()); + } + } + currentPosition += length; + return new Page(blockBuilder1.build(), blockBuilder2.build()); + } + + @Override + protected int remaining() { + return values.size() - currentPosition; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java new file mode 100644 index 0000000000000..c096052ce0de3 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java @@ -0,0 +1,450 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.ClusterModule; +import org.elasticsearch.cluster.node.VersionInformation; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.DriverRunner; +import org.elasticsearch.compute.operator.SinkOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.test.transport.StubbableTransport; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.transport.AbstractSimpleTransportTestCase; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportResponse; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class ExchangeServiceTests extends ESTestCase { + + private TestThreadPool threadPool; + + @Before + public void setThreadPool() { + int numThreads = randomBoolean() ? 1 : between(2, 16); + threadPool = new TestThreadPool( + "test", + new FixedExecutorBuilder(Settings.EMPTY, "esql_test_executor", numThreads, 1024, "esql", EsExecutors.TaskTrackingConfig.DEFAULT) + ); + } + + @After + public void shutdownThreadPool() { + terminate(threadPool); + } + + public void testBasic() throws Exception { + Page[] pages = new Page[7]; + for (int i = 0; i < pages.length; i++) { + pages[i] = new Page(new ConstantIntVector(i, 2).asBlock()); + } + ExchangeSinkHandler sinkExchanger = new ExchangeSinkHandler(2, threadPool::relativeTimeInMillis); + ExchangeSink sink1 = sinkExchanger.createExchangeSink(); + ExchangeSink sink2 = sinkExchanger.createExchangeSink(); + ExchangeSourceHandler sourceExchanger = new ExchangeSourceHandler(3, threadPool.executor("esql_test_executor")); + assertThat(sourceExchanger.refCount(), equalTo(1)); + ExchangeSource source = sourceExchanger.createExchangeSource(); + assertThat(sourceExchanger.refCount(), equalTo(2)); + sourceExchanger.addRemoteSink(sinkExchanger::fetchPageAsync, 1); + assertThat(sourceExchanger.refCount(), equalTo(3)); + ListenableActionFuture waitForReading = source.waitForReading(); + assertFalse(waitForReading.isDone()); + assertNull(source.pollPage()); + assertTrue(sink1.waitForWriting().isDone()); + randomFrom(sink1, sink2).addPage(pages[0]); + randomFrom(sink1, sink2).addPage(pages[1]); + // source and sink buffers can store 5 pages + for (Page p : List.of(pages[2], pages[3], pages[4])) { + ExchangeSink sink = randomFrom(sink1, sink2); + assertBusy(() -> assertTrue(sink.waitForWriting().isDone())); + sink.addPage(p); + } + // sink buffer is full + assertFalse(randomFrom(sink1, sink2).waitForWriting().isDone()); + assertBusy(() -> assertTrue(source.waitForReading().isDone())); + assertEquals(pages[0], source.pollPage()); + assertBusy(() -> assertTrue(source.waitForReading().isDone())); + assertEquals(pages[1], source.pollPage()); + // sink can write again + assertBusy(() -> assertTrue(randomFrom(sink1, sink2).waitForWriting().isDone())); + randomFrom(sink1, sink2).addPage(pages[5]); + assertBusy(() -> assertTrue(randomFrom(sink1, sink2).waitForWriting().isDone())); + randomFrom(sink1, sink2).addPage(pages[6]); + // sink buffer is full + assertFalse(randomFrom(sink1, sink2).waitForWriting().isDone()); + sink1.finish(); + assertTrue(sink1.isFinished()); + for (int i = 0; i < 5; i++) { + assertBusy(() -> assertTrue(source.waitForReading().isDone())); + assertEquals(pages[2 + i], source.pollPage()); + } + // source buffer is empty + assertFalse(source.waitForReading().isDone()); + assertBusy(() -> assertTrue(sink2.waitForWriting().isDone())); + sink2.finish(); + assertTrue(sink2.isFinished()); + assertTrue(source.isFinished()); + assertBusy(() -> assertThat(sourceExchanger.refCount(), equalTo(2))); + source.finish(); + assertThat(sourceExchanger.refCount(), equalTo(1)); + CountDownLatch latch = new CountDownLatch(1); + sourceExchanger.addCompletionListener(ActionListener.releasing(latch::countDown)); + sourceExchanger.decRef(); + assertTrue(latch.await(1, TimeUnit.SECONDS)); + ESTestCase.terminate(threadPool); + } + + /** + * Generates sequence numbers up to the {@code maxInputSeqNo} (exclusive) + */ + static class SeqNoGenerator implements SourceOperator.SourceOperatorFactory { + final AtomicInteger nextSeqNo = new AtomicInteger(-1); + final int maxInputSeqNo; + + SeqNoGenerator(int maxInputSeqNo) { + this.maxInputSeqNo = maxInputSeqNo; + } + + @Override + public String describe() { + return "SeqNoGenerator(maxInputSeqNo=" + maxInputSeqNo + ")"; + } + + @Override + public SourceOperator get(DriverContext driverContext) { + return new SourceOperator() { + @Override + public void finish() { + + } + + @Override + public boolean isFinished() { + return nextSeqNo.get() >= maxInputSeqNo; + } + + @Override + public Page getOutput() { + if (randomInt(100) < 5) { + return null; + } + int size = randomIntBetween(1, 10); + IntBlock.Builder builder = IntBlock.newBlockBuilder(size); + for (int i = 0; i < size; i++) { + int seqNo = nextSeqNo.incrementAndGet(); + if (seqNo < maxInputSeqNo) { + builder.appendInt(seqNo); + } + } + return new Page(builder.build()); + } + + @Override + public void close() { + + } + }; + } + } + + /** + * Collects the received sequence numbers that are less than {@code maxOutputSeqNo}. + */ + static final class SeqNoCollector implements SinkOperator.SinkOperatorFactory { + final long maxOutputSeqNo; + final Set receivedSeqNos = ConcurrentCollections.newConcurrentSet(); + + SeqNoCollector(long maxOutputSeqNo) { + this.maxOutputSeqNo = maxOutputSeqNo; + } + + @Override + public String describe() { + return "SeqNoCollector(maxOutputSeqNo=" + maxOutputSeqNo + ")"; + } + + @Override + public SinkOperator get(DriverContext driverContext) { + return new SinkOperator() { + private boolean finished = false; + + @Override + public boolean needsInput() { + return isFinished() == false; + } + + @Override + public void addInput(Page page) { + assertFalse("already finished", finished); + IntBlock block = page.getBlock(0); + for (int i = 0; i < block.getPositionCount(); i++) { + int v = block.getInt(i); + if (v < maxOutputSeqNo) { + assertTrue(receivedSeqNos.add(v)); + // Early termination + if (receivedSeqNos.size() >= maxOutputSeqNo && randomBoolean()) { + finished = true; + } + } + } + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean isFinished() { + return finished; + } + + @Override + public void close() { + + } + }; + } + } + + void runConcurrentTest( + int maxInputSeqNo, + int maxOutputSeqNo, + Supplier exchangeSource, + Supplier exchangeSink + ) { + final SeqNoCollector seqNoCollector = new SeqNoCollector(maxOutputSeqNo); + final SeqNoGenerator seqNoGenerator = new SeqNoGenerator(maxInputSeqNo); + int numSinks = randomIntBetween(1, 8); + int numSources = randomIntBetween(1, 8); + List drivers = new ArrayList<>(numSinks + numSources); + for (int i = 0; i < numSinks; i++) { + String description = "sink-" + i; + ExchangeSinkOperator sinkOperator = new ExchangeSinkOperator(exchangeSink.get(), Function.identity()); + DriverContext dc = new DriverContext(); + Driver d = new Driver("test-session:1", dc, () -> description, seqNoGenerator.get(dc), List.of(), sinkOperator, () -> {}); + drivers.add(d); + } + for (int i = 0; i < numSources; i++) { + String description = "source-" + i; + ExchangeSourceOperator sourceOperator = new ExchangeSourceOperator(exchangeSource.get()); + DriverContext dc = new DriverContext(); + Driver d = new Driver("test-session:2", dc, () -> description, sourceOperator, List.of(), seqNoCollector.get(dc), () -> {}); + drivers.add(d); + } + PlainActionFuture future = new PlainActionFuture<>(); + new DriverRunner() { + @Override + protected void start(Driver driver, ActionListener listener) { + Driver.start(threadPool.executor("esql_test_executor"), driver, between(1, 10000), listener); + } + }.runToCompletion(drivers, future); + future.actionGet(TimeValue.timeValueMinutes(1)); + var expectedSeqNos = IntStream.range(0, Math.min(maxInputSeqNo, maxOutputSeqNo)).boxed().collect(Collectors.toSet()); + assertThat(seqNoCollector.receivedSeqNos, hasSize(expectedSeqNos.size())); + assertThat(seqNoCollector.receivedSeqNos, equalTo(expectedSeqNos)); + } + + public void testConcurrentWithHandlers() { + var sourceExchanger = new ExchangeSourceHandler(randomExchangeBuffer(), threadPool.executor("esql_test_executor")); + List sinkHandlers = new ArrayList<>(); + Supplier exchangeSink = () -> { + final ExchangeSinkHandler sinkHandler; + if (sinkHandlers.isEmpty() == false && randomBoolean()) { + sinkHandler = randomFrom(sinkHandlers); + } else { + sinkHandler = new ExchangeSinkHandler(randomExchangeBuffer(), threadPool::relativeTimeInMillis); + sourceExchanger.addRemoteSink(sinkHandler::fetchPageAsync, randomIntBetween(1, 3)); + sinkHandlers.add(sinkHandler); + } + return sinkHandler.createExchangeSink(); + }; + final int maxInputSeqNo = rarely() ? -1 : randomIntBetween(0, 50_000); + final int maxOutputSeqNo = rarely() ? -1 : randomIntBetween(0, 50_000); + runConcurrentTest(maxInputSeqNo, maxOutputSeqNo, sourceExchanger::createExchangeSource, exchangeSink); + } + + public void testEarlyTerminate() { + IntBlock block = new ConstantIntVector(1, 2).asBlock(); + Page p1 = new Page(block); + Page p2 = new Page(block); + ExchangeSinkHandler sinkExchanger = new ExchangeSinkHandler(2, threadPool::relativeTimeInMillis); + ExchangeSink sink = sinkExchanger.createExchangeSink(); + sink.addPage(p1); + sink.addPage(p2); + assertFalse(sink.waitForWriting().isDone()); + PlainActionFuture future = new PlainActionFuture<>(); + sinkExchanger.fetchPageAsync(true, future); + ExchangeResponse resp = future.actionGet(); + assertTrue(resp.finished()); + assertNull(resp.page()); + assertTrue(sink.waitForWriting().isDone()); + assertTrue(sink.isFinished()); + } + + public void testConcurrentWithTransportActions() throws Exception { + MockTransportService node0 = newTransportService(); + ExchangeService exchange0 = new ExchangeService(Settings.EMPTY, threadPool); + exchange0.registerTransportHandler(node0); + MockTransportService node1 = newTransportService(); + ExchangeService exchange1 = new ExchangeService(Settings.EMPTY, threadPool); + exchange1.registerTransportHandler(node1); + AbstractSimpleTransportTestCase.connectToNode(node0, node1.getLocalNode()); + + try (exchange0; exchange1; node0; node1) { + String exchangeId = "exchange"; + Task task = new Task(1, "", "", "", null, Collections.emptyMap()); + ExchangeSourceHandler sourceHandler = exchange0.createSourceHandler(exchangeId, randomExchangeBuffer(), "esql_test_executor"); + ExchangeSinkHandler sinkHandler = exchange1.createSinkHandler(exchangeId, randomExchangeBuffer()); + sourceHandler.addRemoteSink(exchange0.newRemoteSink(task, exchangeId, node0, node1.getLocalNode()), randomIntBetween(1, 5)); + final int maxInputSeqNo = rarely() ? -1 : randomIntBetween(0, 50_000); + final int maxOutputSeqNo = rarely() ? -1 : randomIntBetween(0, 50_000); + runConcurrentTest(maxInputSeqNo, maxOutputSeqNo, sourceHandler::createExchangeSource, sinkHandler::createExchangeSink); + } + } + + public void testFailToRespondPage() throws Exception { + Settings settings = Settings.builder().build(); + MockTransportService node0 = newTransportService(); + ExchangeService exchange0 = new ExchangeService(settings, threadPool); + exchange0.registerTransportHandler(node0); + MockTransportService node1 = newTransportService(); + ExchangeService exchange1 = new ExchangeService(settings, threadPool); + exchange1.registerTransportHandler(node1); + AbstractSimpleTransportTestCase.connectToNode(node0, node1.getLocalNode()); + final int maxSeqNo = randomIntBetween(1000, 5000); + final int disconnectOnSeqNo = randomIntBetween(100, 500); + node1.addRequestHandlingBehavior(ExchangeService.EXCHANGE_ACTION_NAME, new StubbableTransport.RequestHandlingBehavior<>() { + @Override + public void messageReceived( + TransportRequestHandler handler, + TransportRequest request, + TransportChannel channel, + Task task + ) throws Exception { + FilterTransportChannel filterChannel = new FilterTransportChannel(channel) { + @Override + public void sendResponse(TransportResponse response) throws IOException { + ExchangeResponse exchangeResponse = (ExchangeResponse) response; + if (exchangeResponse.page() != null) { + IntBlock block = exchangeResponse.page().getBlock(0); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.getInt(i) == disconnectOnSeqNo) { + throw new IOException("page is too large"); + } + } + } + super.sendResponse(response); + } + }; + handler.messageReceived(request, filterChannel, task); + } + }); + try (exchange0; exchange1; node0; node1) { + String exchangeId = "exchange"; + Task task = new Task(1, "", "", "", null, Collections.emptyMap()); + ExchangeSourceHandler sourceHandler = exchange0.createSourceHandler(exchangeId, randomIntBetween(1, 128), "esql_test_executor"); + ExchangeSinkHandler sinkHandler = exchange1.createSinkHandler(exchangeId, randomIntBetween(1, 128)); + sourceHandler.addRemoteSink(exchange0.newRemoteSink(task, exchangeId, node0, node1.getLocalNode()), randomIntBetween(1, 5)); + Exception err = expectThrows( + Exception.class, + () -> runConcurrentTest(maxSeqNo, maxSeqNo, sourceHandler::createExchangeSource, sinkHandler::createExchangeSink) + ); + Throwable cause = ExceptionsHelper.unwrap(err, IOException.class); + assertNotNull(cause); + assertThat(cause.getMessage(), equalTo("page is too large")); + } + } + + private MockTransportService newTransportService() { + List namedWriteables = new ArrayList<>(ClusterModule.getNamedWriteables()); + namedWriteables.addAll(Block.getNamedWriteables()); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables); + MockTransportService service = MockTransportService.createNewService( + Settings.EMPTY, + MockTransportService.newMockTransport(Settings.EMPTY, TransportVersion.current(), threadPool, namedWriteableRegistry), + VersionInformation.CURRENT, + threadPool, + null, + Collections.emptySet() + ); + service.start(); + service.acceptIncomingRequests(); + return service; + } + + private int randomExchangeBuffer() { + return randomBoolean() ? randomIntBetween(1, 3) : randomIntBetween(1, 128); + } + + private static class FilterTransportChannel implements TransportChannel { + private final TransportChannel in; + + FilterTransportChannel(TransportChannel in) { + this.in = in; + } + + @Override + public String getProfileName() { + return in.getProfileName(); + } + + @Override + public String getChannelType() { + return in.getChannelType(); + } + + @Override + public void sendResponse(TransportResponse response) throws IOException { + in.sendResponse(response); + } + + @Override + public void sendResponse(Exception exception) throws IOException { + in.sendResponse(exception); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperatorStatusTests.java new file mode 100644 index 0000000000000..f342720b99903 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSinkOperatorStatusTests.java @@ -0,0 +1,38 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; + +public class ExchangeSinkOperatorStatusTests extends AbstractWireSerializingTestCase { + public void testToXContent() { + assertThat(Strings.toString(new ExchangeSinkOperator.Status(10)), equalTo(""" + {"pages_accepted":10}""")); + } + + @Override + protected Writeable.Reader instanceReader() { + return ExchangeSinkOperator.Status::new; + } + + @Override + protected ExchangeSinkOperator.Status createTestInstance() { + return new ExchangeSinkOperator.Status(between(0, Integer.MAX_VALUE)); + } + + @Override + protected ExchangeSinkOperator.Status mutateInstance(ExchangeSinkOperator.Status instance) throws IOException { + return new ExchangeSinkOperator.Status(randomValueOtherThan(instance.pagesAccepted(), () -> between(0, Integer.MAX_VALUE))); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperatorStatusTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperatorStatusTests.java new file mode 100644 index 0000000000000..2c5f7eebbaf3d --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceOperatorStatusTests.java @@ -0,0 +1,51 @@ +/* + * 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.compute.operator.exchange; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +import java.io.IOException; + +import static org.hamcrest.Matchers.equalTo; + +public class ExchangeSourceOperatorStatusTests extends AbstractWireSerializingTestCase { + public void testToXContent() { + assertThat(Strings.toString(new ExchangeSourceOperator.Status(0, 10)), equalTo(""" + {"pages_waiting":0,"pages_emitted":10}""")); + } + + @Override + protected Writeable.Reader instanceReader() { + return ExchangeSourceOperator.Status::new; + } + + @Override + protected ExchangeSourceOperator.Status createTestInstance() { + return new ExchangeSourceOperator.Status(between(0, Integer.MAX_VALUE), between(0, Integer.MAX_VALUE)); + } + + @Override + protected ExchangeSourceOperator.Status mutateInstance(ExchangeSourceOperator.Status instance) throws IOException { + switch (between(0, 1)) { + case 0: + return new ExchangeSourceOperator.Status( + randomValueOtherThan(instance.pagesWaiting(), () -> between(0, Integer.MAX_VALUE)), + instance.pagesEmitted() + ); + case 1: + return new ExchangeSourceOperator.Status( + instance.pagesWaiting(), + randomValueOtherThan(instance.pagesEmitted(), () -> between(0, Integer.MAX_VALUE)) + ); + default: + throw new UnsupportedOperationException(); + } + } +} diff --git a/x-pack/plugin/esql/qa/build.gradle b/x-pack/plugin/esql/qa/build.gradle new file mode 100644 index 0000000000000..0b7d210bcd99e --- /dev/null +++ b/x-pack/plugin/esql/qa/build.gradle @@ -0,0 +1,5 @@ +description = 'Integration tests for ESQL' + +subprojects { + tasks.withType(Javadoc).all { enabled = false } +} diff --git a/x-pack/plugin/esql/qa/security/build.gradle b/x-pack/plugin/esql/qa/security/build.gradle new file mode 100644 index 0000000000000..4a1b32587da61 --- /dev/null +++ b/x-pack/plugin/esql/qa/security/build.gradle @@ -0,0 +1,14 @@ +apply plugin: 'elasticsearch.legacy-java-rest-test' + +testClusters.configureEach { + testDistribution = 'DEFAULT' + setting 'xpack.license.self_generated.type', 'trial' + setting 'xpack.monitoring.collection.enabled', 'false' + setting 'xpack.security.enabled', 'true' + numberOfNodes = 1 + extraConfigFile 'roles.yml', file('roles.yml') + user username: "test-admin", password: 'x-pack-test-password', role: "test-admin" + user username: "user1", password: 'x-pack-test-password', role: "user1" + user username: "user2", password: 'x-pack-test-password', role: "user2" + user username: "user3", password: 'x-pack-test-password', role: "user3" +} diff --git a/x-pack/plugin/esql/qa/security/roles.yml b/x-pack/plugin/esql/qa/security/roles.yml new file mode 100644 index 0000000000000..d18389dc58879 --- /dev/null +++ b/x-pack/plugin/esql/qa/security/roles.yml @@ -0,0 +1,46 @@ +# All cluster rights +# All operations on all indices +# Run as all users +test-admin: + cluster: + - all + indices: + - names: '*' + privileges: [ all ] + run_as: + - '*' + +user1: + cluster: + - cluster:monitor/main + indices: + - names: ['index-user1', 'index', "test-enrich" ] + privileges: + - read + - write + - create_index + - indices:admin/refresh + +user2: + cluster: + - cluster:monitor/main + indices: + - names: [ 'index-user2', 'index' ] + privileges: + - read + - write + - create_index + - indices:admin/refresh + +user3: + cluster: + - cluster:monitor/main + indices: + - names: [ 'index' ] + privileges: [ 'read' ] + query: | + { + "term": { + "org": "sales" + } + } diff --git a/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java new file mode 100644 index 0000000000000..3d637e30da4c1 --- /dev/null +++ b/x-pack/plugin/esql/qa/security/src/javaRestTest/java/org/elasticsearch/xpack/esql/EsqlSecurityIT.java @@ -0,0 +1,224 @@ +/* + * 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; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.junit.Before; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class EsqlSecurityIT extends ESRestTestCase { + + @Override + protected Settings restClientSettings() { + String token = basicAuthHeaderValue("test-admin", new SecureString("x-pack-test-password".toCharArray())); + return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + } + + private void indexDocument(String index, int id, double value, String org) throws IOException { + Request indexDoc = new Request("PUT", index + "/_doc/" + id); + indexDoc.setJsonEntity("{\"value\":" + value + ",\"org\":\"" + org + "\"}"); + client().performRequest(indexDoc); + } + + @Before + public void indexDocuments() throws IOException { + String mapping = """ + "properties":{"value": {"type": "double"}, "org": {"type": "keyword"}} + """; + createIndex("index", Settings.EMPTY, mapping); + indexDocument("index", 1, 10.0, "sales"); + indexDocument("index", 2, 20.0, "engineering"); + refresh("index"); + + createIndex("index-user1", Settings.EMPTY, mapping); + indexDocument("index-user1", 1, 12.0, "engineering"); + indexDocument("index-user1", 2, 31.0, "sales"); + refresh("index-user1"); + + createIndex("index-user2", Settings.EMPTY, mapping); + indexDocument("index-user2", 1, 32.0, "marketing"); + indexDocument("index-user2", 2, 40.0, "sales"); + refresh("index-user2"); + } + + public void testAllowedIndices() throws Exception { + for (String user : List.of("test-admin", "user1", "user2")) { + Response resp = runESQLCommand(user, "from index | stats sum=sum(value)"); + assertOK(resp); + Map respMap = entityAsMap(resp); + assertThat(respMap.get("columns"), equalTo(List.of(Map.of("name", "sum", "type", "double")))); + assertThat(respMap.get("values"), equalTo(List.of(List.of(30.0)))); + } + + for (String user : List.of("test-admin", "user1")) { + Response resp = runESQLCommand(user, "from index-user1 | stats sum=sum(value)"); + assertOK(resp); + Map respMap = entityAsMap(resp); + assertThat(respMap.get("columns"), equalTo(List.of(Map.of("name", "sum", "type", "double")))); + assertThat(respMap.get("values"), equalTo(List.of(List.of(43.0)))); + } + + for (String user : List.of("test-admin", "user2")) { + Response resp = runESQLCommand(user, "from index-user2 | stats sum=sum(value)"); + assertOK(resp); + Map respMap = entityAsMap(resp); + assertThat(respMap.get("columns"), equalTo(List.of(Map.of("name", "sum", "type", "double")))); + assertThat(respMap.get("values"), equalTo(List.of(List.of(72.0)))); + } + } + + public void testUnauthorizedIndices() { + ResponseException error; + error = expectThrows(ResponseException.class, () -> runESQLCommand("user1", "from index-user2 | stats sum(value)")); + assertThat(error.getResponse().getStatusLine().getStatusCode(), equalTo(400)); + + error = expectThrows(ResponseException.class, () -> runESQLCommand("user2", "from index-user1 | stats sum(value)")); + assertThat(error.getResponse().getStatusLine().getStatusCode(), equalTo(400)); + } + + public void testDLS() throws Exception { + Response resp = runESQLCommand("user3", "from index | stats sum=sum(value)"); + assertOK(resp); + Map respMap = entityAsMap(resp); + assertThat(respMap.get("columns"), equalTo(List.of(Map.of("name", "sum", "type", "double")))); + assertThat(respMap.get("values"), equalTo(List.of(List.of(10.0)))); + } + + public void testRowCommand() throws Exception { + String user = randomFrom("test-admin", "user1", "user2"); + Response resp = runESQLCommand(user, "row a = 5, b = 2 | stats count=sum(b) by a"); + assertOK(resp); + Map respMap = entityAsMap(resp); + assertThat( + respMap.get("columns"), + equalTo(List.of(Map.of("name", "count", "type", "long"), Map.of("name", "a", "type", "integer"))) + ); + assertThat(respMap.get("values"), equalTo(List.of(List.of(2, 5)))); + } + + public void testEnrich() throws Exception { + createEnrichPolicy(); + try { + createIndex("test-enrich", Settings.EMPTY, """ + "properties":{"timestamp": {"type": "long"}, "song_id": {"type": "keyword"}, "duration": {"type": "double"}} + """); + record Listen(long timestamp, String songId, double duration) { + + } + var listens = List.of( + new Listen(1, "s1", 1.0), + new Listen(2, "s2", 2.0), + new Listen(3, "s1", 3.0), + new Listen(4, "s3", 1.0), + new Listen(5, "s4", 1.5), + new Listen(6, "s1", 2.5), + new Listen(7, "s1", 3.5), + new Listen(8, "s2", 5.0), + new Listen(8, "s1", 0.5), + new Listen(8, "s3", 0.25), + new Listen(8, "s4", 1.25) + ); + for (int i = 0; i < listens.size(); i++) { + Listen listen = listens.get(i); + Request indexDoc = new Request("PUT", "/test-enrich/_doc/" + i); + String doc = Strings.toString( + JsonXContent.contentBuilder() + .startObject() + .field("timestamp", listen.timestamp) + .field("song_id", listen.songId) + .field("duration", listen.duration) + .endObject() + ); + indexDoc.setJsonEntity(doc); + client().performRequest(indexDoc); + } + refresh("test-enrich"); + Response resp = runESQLCommand( + "user1", + "FROM test-enrich | ENRICH songs ON song_id | stats total_duration = sum(duration) by artist | sort artist" + ); + Map respMap = entityAsMap(resp); + assertThat( + respMap.get("values"), + equalTo(List.of(List.of(2.75, "Disturbed"), List.of(10.5, "Eagles"), List.of(8.25, "Linkin Park"))) + ); + } finally { + removeEnrichPolicy(); + } + } + + private void createEnrichPolicy() throws Exception { + createIndex("songs", Settings.EMPTY, """ + "properties":{"song_id": {"type": "keyword"}, "title": {"type": "keyword"}, "artist": {"type": "keyword"} } + """); + record Song(String id, String title, String artist) { + + } + + var songs = List.of( + new Song("s1", "Hotel California", "Eagles"), + new Song("s2", "In The End", "Linkin Park"), + new Song("s3", "Numb", "Linkin Park"), + new Song("s4", "The Sound Of Silence", "Disturbed") + ); + for (int i = 0; i < songs.size(); i++) { + var song = songs.get(i); + Request indexDoc = new Request("PUT", "/songs/_doc/" + i); + String doc = Strings.toString( + JsonXContent.contentBuilder() + .startObject() + .field("song_id", song.id) + .field("title", song.title) + .field("artist", song.artist) + .endObject() + ); + indexDoc.setJsonEntity(doc); + client().performRequest(indexDoc); + } + refresh("songs"); + + Request createEnrich = new Request("PUT", "/_enrich/policy/songs"); + createEnrich.setJsonEntity(""" + { + "match": { + "indices": "songs", + "match_field": "song_id", + "enrich_fields": ["title", "artist"] + } + } + """); + client().performRequest(createEnrich); + client().performRequest(new Request("PUT", "_enrich/policy/songs/_execute")); + } + + private void removeEnrichPolicy() throws Exception { + client().performRequest(new Request("DELETE", "_enrich/policy/songs")); + } + + private Response runESQLCommand(String user, String command) throws IOException { + Request request = new Request("POST", "_query"); + request.setJsonEntity("{\"query\":\"" + command + "\"}"); + request.setOptions(RequestOptions.DEFAULT.toBuilder().addHeader("es-security-runas-user", user)); + return client().performRequest(request); + } + +} diff --git a/x-pack/plugin/esql/qa/server/build.gradle b/x-pack/plugin/esql/qa/server/build.gradle new file mode 100644 index 0000000000000..f8a43c52f5ca7 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/build.gradle @@ -0,0 +1,58 @@ +apply plugin: 'elasticsearch.java' + +description = 'Integration tests for ESQL' + +dependencies { + api project(":test:framework") + + // Common utilities from QL + api project(xpackModule('ql:test-fixtures')) + api project(xpackModule('esql:qa:testFixtures')) +} + +subprojects { + if (subprojects.isEmpty()) { + // leaf project + } else { + apply plugin: 'elasticsearch.java' + apply plugin: 'elasticsearch.standalone-rest-test' + } + + + if (project.name != 'security') { + // The security project just configures its subprojects + apply plugin: 'elasticsearch.legacy-java-rest-test' + + testClusters.matching { it.name == "javaRestTest" }.configureEach { + testDistribution = 'DEFAULT' + setting 'xpack.ml.enabled', 'false' + setting 'xpack.watcher.enabled', 'false' + } + + + dependencies { + configurations.javaRestTestRuntimeClasspath { + resolutionStrategy.force "org.slf4j:slf4j-api:1.7.25" + } + configurations.javaRestTestRuntimeOnly { + // This is also required to make resolveAllDependencies work + resolutionStrategy.force "org.slf4j:slf4j-api:1.7.25" + } + + /* Since we're a standalone rest test we actually get transitive + * dependencies but we don't really want them because they cause + * all kinds of trouble with the jar hell checks. So we suppress + * them explicitly for non-es projects. */ + javaRestTestImplementation(project(':x-pack:plugin:esql:qa:server')) { + transitive = false + } + javaRestTestImplementation project(":test:framework") + javaRestTestRuntimeOnly project(xpackModule('ql:test-fixtures')) + + javaRestTestRuntimeOnly "org.slf4j:slf4j-api:1.7.25" + javaRestTestRuntimeOnly "net.sf.supercsv:super-csv:${versions.supercsv}" + + javaRestTestImplementation project(path: xpackModule('ql:test-fixtures')) + } + } +} diff --git a/x-pack/plugin/esql/qa/server/single-node/build.gradle b/x-pack/plugin/esql/qa/server/single-node/build.gradle new file mode 100644 index 0000000000000..6f913100e0fd7 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/build.gradle @@ -0,0 +1,22 @@ +apply plugin: 'elasticsearch.legacy-yaml-rest-test' + +dependencies { + javaRestTestImplementation project(xpackModule('esql:qa:testFixtures')) +} + +restResources { + restApi { + include '_common', 'bulk', 'indices', 'esql', 'xpack', 'enrich' + } +} + +artifacts { + restXpackTests(new File(projectDir, "src/yamlRestTest/resources/rest-api-spec/test")) +} + +testClusters.configureEach { + testDistribution = 'DEFAULT' + setting 'xpack.license.self_generated.type', 'trial' + setting 'xpack.monitoring.collection.enabled', 'true' + setting 'xpack.security.enabled', 'false' +} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlSpecIT.java b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlSpecIT.java new file mode 100644 index 0000000000000..d3a4d7a14a0f1 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlSpecIT.java @@ -0,0 +1,17 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.qa.single_node; + +import org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase; +import org.elasticsearch.xpack.ql.CsvSpecReader.CsvTestCase; + +public class EsqlSpecIT extends EsqlSpecTestCase { + public EsqlSpecIT(String fileName, String groupName, String testName, Integer lineNumber, CsvTestCase testCase) { + super(fileName, groupName, testName, lineNumber, testCase); + } +} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java new file mode 100644 index 0000000000000..efb7192bbc3e8 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.esql.qa.single_node; + +import org.apache.http.util.EntityUtils; +import org.elasticsearch.Build; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase; +import org.junit.Assert; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; + +public class RestEsqlIT extends RestEsqlTestCase { + + public void testBasicEsql() throws IOException { + StringBuilder b = new StringBuilder(); + for (int i = 0; i < 1000; i++) { + b.append(String.format(Locale.ROOT, """ + {"create":{"_index":"%s"}} + {"@timestamp":"2020-12-12","test":"value%s","value":%d} + """, testIndexName(), i, i)); + } + Request bulk = new Request("POST", "/_bulk"); + bulk.addParameter("refresh", "true"); + bulk.addParameter("filter_path", "errors"); + bulk.setJsonEntity(b.toString()); + Response response = client().performRequest(bulk); + Assert.assertEquals("{\"errors\":false}", EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8)); + + RequestObjectBuilder builder = new RequestObjectBuilder().query(fromIndex() + " | stats avg(value)"); + if (Build.current().isSnapshot()) { + builder.pragmas(Settings.builder().put("data_partitioning", "shard").build()); + } + builder.build(); + Map result = runEsql(builder); + assertEquals(2, result.size()); + Map colA = Map.of("name", "avg(value)", "type", "double"); + assertEquals(List.of(colA), result.get("columns")); + assertEquals(List.of(List.of(499.5d)), result.get("values")); + } + + public void testInvalidPragma() throws IOException { + assumeTrue("pragma only enabled on snapshot builds", Build.current().isSnapshot()); + RequestObjectBuilder builder = new RequestObjectBuilder().query("row a = 1, b = 2"); + builder.pragmas(Settings.builder().put("data_partitioning", "invalid-option").build()); + builder.build(); + ResponseException re = expectThrows(ResponseException.class, () -> runEsql(builder)); + assertThat(EntityUtils.toString(re.getResponse().getEntity()), containsString("No enum constant")); + } + + public void testPragmaNotAllowed() throws IOException { + assumeFalse("pragma only disabled on release builds", Build.current().isSnapshot()); + RequestObjectBuilder builder = new RequestObjectBuilder().query("row a = 1, b = 2"); + builder.pragmas(Settings.builder().put("data_partitioning", "shard").build()); + builder.build(); + ResponseException re = expectThrows(ResponseException.class, () -> runEsql(builder)); + assertThat(EntityUtils.toString(re.getResponse().getEntity()), containsString("[pragma] only allowed in snapshot builds")); + } +} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlClientYamlIT.java b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlClientYamlIT.java new file mode 100644 index 0000000000000..64aaf547e5468 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/EsqlClientYamlIT.java @@ -0,0 +1,25 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.qa.single_node; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; + +public class EsqlClientYamlIT extends ESClientYamlSuiteTestCase { + + public EsqlClientYamlIT(final ClientYamlTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws Exception { + return createParameters(); + } +} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml new file mode 100644 index 0000000000000..9716004c3fbc1 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml @@ -0,0 +1,299 @@ +--- +setup: + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 5 + mappings: + properties: + data: + type: long + data_d: + type: double + count: + type: long + count_d: + type: double + time: + type: long + color: + type: keyword + - do: + bulk: + index: "test" + refresh: true + body: + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275187, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275188, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275189, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275190, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275191, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275192, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275193, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275194, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275195, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275196, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275197, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275198, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275199, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275200, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275201, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275202, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275203, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275204, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275205, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275206, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275207, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275208, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275209, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275210, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275211, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275212, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275213, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275214, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275215, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275216, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275217, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275218, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275219, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275220, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275221, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275222, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275223, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275224, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275225, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275226, "color": "red" } + +--- +"Test From": + - do: + esql.query: + body: + query: 'from test' + + - match: {columns.0.name: "color"} + - match: {columns.0.type: "keyword"} + - match: {columns.1.name: "count"} + - match: {columns.1.type: "long"} + - match: {columns.2.name: "count_d"} + - match: {columns.2.type: "double"} + - match: {columns.3.name: "data"} + - match: {columns.3.type: "long"} + - match: {columns.4.name: "data_d"} + - match: {columns.4.type: "double"} + - match: {columns.5.name: "time"} + - match: {columns.5.type: "long"} + - length: {values: 40} + +--- +"Test From Sort Limit": + - do: + esql.query: + body: + query: 'from test | sort count | limit 1' + + - match: {columns.1.name: "count"} + - match: {columns.1.type: "long"} + - match: {values.0.1: 40} + +--- +"Basic ESQL query": + - do: + esql.query: + body: + query: 'from test | keep data | sort data | limit 2' + columnar: true + + - match: {columns.0.name: "data"} + - match: {columns.0.type: "long"} + - match: {values.0: [1, 1]} + +--- +"Test From Eval Sort Limit": + - do: + esql.query: + body: + query: 'from test | eval x = count + 7 | sort x | limit 1' + + - match: {columns.0.name: "color"} + - match: {columns.1.name: "count"} + - match: {columns.2.name: "count_d"} + - match: {columns.3.name: "data"} + - match: {columns.4.name: "data_d"} + - match: {columns.5.name: "time"} + - match: {columns.6.name: "x"} + - match: {values.0.6: 47} + - length: {values: 1} + +--- +"Test Eval With Multiple Expressions": + - do: + esql.query: + body: + query: 'from test | sort time | eval x = data + 1, y = data_d + count, z = x + y | keep data, x, y, z, time | limit 2' + + - match: {columns.0.name: "data"} + - match: {columns.0.type: "long"} + - match: {columns.1.name: "x"} + - match: {columns.1.type: "long"} + - match: {columns.2.name: "y"} + - match: {columns.2.type: "double"} + - match: {columns.3.name: "z"} + - match: {columns.3.type: "double"} + - match: {columns.4.name: "time"} + - match: {columns.4.type: "long"} + - length: {values: 2} + - length: {values.0: 5} + - match: {values.0.0: 1} + - match: {values.0.1: 2} + - match: {values.0.2: 41.0} + - match: {values.0.3: 43.0} + - length: {values.1: 5} + - match: {values.1.0: 2} + - match: {values.1.1: 3} + - match: {values.1.2: 44.0} + - match: {values.1.3: 47.0} + +--- +"Test keep After TopN": + - do: + esql.query: + body: + query: 'from test | sort time | limit 2 | keep count' + columnar: true + + - length: {columns: 1} + - match: {columns.0.name: "count"} + - match: {columns.0.type: "long"} + - match: {values.0: [40, 42]} + +--- +"Test keep After TopN Desc": + - do: + esql.query: + body: + query: 'from test | sort time desc | limit 2 | keep count' + columnar: true + + - length: {columns: 1} + - match: {columns.0.name: "count"} + - match: {columns.0.type: "long"} + - match: {values.0: [46, 44]} + +--- +"Test TopN keep Eval": + - do: + esql.query: + body: + query: 'from test | sort time | limit 2 | keep count | eval x = count + 1' + columnar: true + + - length: {columns: 2} + - match: {columns.0.name: "count"} + - match: {columns.0.type: "long"} + - match: {columns.1.name: "x"} + - match: {columns.1.type: "long"} + - length: {values: 2} + - match: {values.0: [40, 42]} + - match: {values.1: [41, 43]} + +--- +"Test TopN keep Eval Project": + - do: + esql.query: + body: + query: 'from test | sort time | limit 2 | keep count | eval x = count + 1 | keep x' + columnar: true + + - length: {columns: 1} + - match: {columns.0.name: "x"} + - match: {columns.0.type: "long"} + - length: {values: 1} + - match: {values.0: [41, 43]} + +--- +"Test Multi Limit Project": + - do: + esql.query: + body: + query: 'from test | limit 10 | sort time | limit 1' + + - length: {columns: 6} + - length: {values: 1} + +--- +"Test Input Params": + - do: + esql.query: + body: + query: 'row a = ? | eval b = ?, c = 1 + ?' + params: ["foo", 15, 10] + + - length: {columns: 3} + - match: {columns.0.name: "a"} + - match: {columns.0.type: "keyword"} + - match: {columns.1.name: "b"} + - match: {columns.1.type: "integer"} + - match: {columns.2.name: "c"} + - match: {columns.2.type: "integer"} + - length: {values: 1} + - match: {values.0: ["foo", 15, 11]} + + + - do: + esql.query: + body: + query: 'from test | where color == ? and count == ? and time == ? | keep data, count, color' + params: ["green", 44, 1674835275193] + + - length: {columns: 3} + - match: {columns.0.name: "data"} + - match: {columns.0.type: "long"} + - match: {columns.1.name: "count"} + - match: {columns.1.type: "long"} + - match: {columns.2.name: "color"} + - match: {columns.2.type: "keyword"} + - length: {values: 1} + - match: {values.0: [1, 44, "green"]} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/20_aggs.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/20_aggs.yml new file mode 100644 index 0000000000000..6e8c0eb120ddd --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/20_aggs.yml @@ -0,0 +1,455 @@ +--- +setup: + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 5 + mappings: + properties: + data: + type: long + data_d: + type: double + count: + type: long + count_d: + type: double + time: + type: long + color: + type: keyword + - do: + bulk: + index: "test" + refresh: true + body: + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275187, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275188, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275189, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275190, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275191, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275192, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275193, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275194, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275195, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275196, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275197, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275198, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275199, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275200, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275201, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275202, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275203, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275204, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275205, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275206, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275207, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275208, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275209, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275210, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275211, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275212, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275213, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275214, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275215, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275216, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275217, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275218, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275219, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275220, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275221, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275222, "color": "red" } + - { "index": { } } + - { "data": 1, "count": 40, "data_d": 1, "count_d": 40, "time": 1674835275223, "color": "red" } + - { "index": { } } + - { "data": 2, "count": 42, "data_d": 2, "count_d": 42, "time": 1674835275224, "color": "blue" } + - { "index": { } } + - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275225, "color": "green" } + - { "index": { } } + - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275226, "color": "red" } + +--- +"Test From": + - do: + esql.query: + body: + query: 'from test' + + - match: {columns.0.name: "color"} + - match: {columns.0.type: "keyword"} + - match: {columns.1.name: "count"} + - match: {columns.1.type: "long"} + - match: {columns.2.name: "count_d"} + - match: {columns.2.type: "double"} + - match: {columns.3.name: "data"} + - match: {columns.3.type: "long"} + - match: {columns.4.name: "data_d"} + - match: {columns.4.type: "double"} + - match: {columns.5.name: "time"} + - match: {columns.5.type: "long"} + - length: {values: 40} + +--- +"Test simple grouping avg": + - do: + esql.query: + body: + query: 'from test | where color == "red" | stats avg(data) by color' + columnar: true + + - match: {columns.0.name: "avg(data)"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0.0: 1.5} # ( 1 + 2 / 2 = 1.5 ) + +--- +"Test From Stats Avg": + - do: + esql.query: + body: + query: 'from test | stats avg(count)' + columnar: true + + - match: {columns.0.name: "avg(count)"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 43} + +--- +"Test From Stats Avg With Alias": + - do: + esql.query: + body: + query: 'from test | stats f1 = avg(count)' + columnar: true + + - match: {columns.0.name: "f1"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 43} + +--- +"Test From Stats Count": + - do: + esql.query: + body: + query: 'from test | stats count(data)' + columnar: true + + - match: {columns.0.name: "count(data)"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 40} + +--- +"Test From Stats Count With Alias": + - do: + esql.query: + body: + query: 'from test | stats dataCount = count(data)' + columnar: true + + - match: {columns.0.name: "dataCount"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 40} + +--- +"Test From Stats Min": + - do: + esql.query: + body: + query: 'from test | stats min(count)' + columnar: true + + - match: {columns.0.name: "min(count)"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 40} + +--- +"Test From Stats Min With Alias": + - do: + esql.query: + body: + query: 'from test | stats minCount=min(count)' + columnar: true + + - match: {columns.0.name: "minCount"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 40} + +--- +"Test From Stats Max": + - do: + esql.query: + body: + query: 'from test | stats max(count)' + columnar: true + + - match: {columns.0.name: "max(count)"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 46} + +--- +"Test From Stats Max With Alias": + - do: + esql.query: + body: + query: 'from test | stats maxCount=max(count)' + columnar: true + + - match: {columns.0.name: "maxCount"} + - match: {columns.0.type: "long"} + - match: {values.0.0: 46} + +--- +"Test Sort With Keyword Field": + - do: + esql.query: + body: + query: 'from test | stats avg(count) by color | sort color | limit 2' + columnar: true + + - match: {columns.0.name: "avg(count)"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0: [42.0, 44.0]} + - match: {values.1: ["blue", "green"]} + +--- +"Test Median On Long": + - do: + esql.query: + body: + query: 'from test | stats med=median(count)' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 43.0} + +--- +"Test Median On Double": + - do: + esql.query: + body: + query: 'from test | stats med=median(count_d)' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 43.0} + +--- +"Test Grouping Median On Long": + - do: + esql.query: + body: + query: 'from test | stats med=median(count) by color | sort med' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0: [42.0, 43.0, 44.0]} + - match: {values.1: ["blue", "red", "green"]} + +--- +"Test Grouping Median On Double": + - do: + esql.query: + body: + query: 'from test | stats med=median(count_d) by color | sort med' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0: [42.0, 43.0, 44.0]} + - match: {values.1: ["blue", "red", "green"]} + +--- +"Test Median Absolute Deviation On Long": + - do: + esql.query: + body: + query: 'from test | stats med=median_absolute_deviation(count)' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 2.0} + +--- +"Test Median Absolute Deviation On Double": + - do: + esql.query: + body: + query: 'from test | stats med=median_absolute_deviation(count_d)' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {values.0.0: 2.0} + +--- +"Test Grouping Median Absolute Deviation On Long": + - do: + esql.query: + body: + query: 'from test | stats med=median_absolute_deviation(count) by color | sort color' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0: [0.0, 0.0, 3.0]} + - match: {values.1: ["blue", "green", "red"]} + +--- +"Test Grouping Median Absolute Deviation On Double": + - do: + esql.query: + body: + query: 'from test | stats med=median_absolute_deviation(count_d) by color | sort color' + columnar: true + + - match: {columns.0.name: "med"} + - match: {columns.0.type: "double"} + - match: {columns.1.name: "color"} + - match: {columns.1.type: "keyword"} + - match: {values.0: [0.0, 0.0, 3.0]} + - match: {values.1: ["blue", "green", "red"]} + +--- +"Test From Stats Eval": + - do: + esql.query: + body: + query: 'from test | stats avg_count = avg(count) | eval x = avg_count + 7' + + - length: {values: 1} + - length: {values.0: 2} + - match: {values.0.1: 50.0} + +--- +"Test Stats Where": + - do: + esql.query: + body: + query: 'from test | stats x = avg(count) | where x > 100' + + - length: {values: 0} + +--- +"Test Eval With Null": + - do: + esql.query: + body: + query: 'from test | eval nullsum = count_d + null | sort nullsum | limit 1' + + - length: {columns: 7} + - length: {values: 1} + - match: {columns.6.name: "nullsum"} + - match: {columns.6.type: "double"} + - match: {values.0.6: null} + +--- +"Test Eval Row With Null": + - do: + esql.query: + body: + query: 'row a = 1, b = 2, c = null | eval z = c + b + a' + + - length: {columns: 4} + - length: {values: 1} + - length: {values.0: 4} + - match: {columns.0.name: "a"} + - match: {columns.0.type: "integer"} + - match: {columns.1.name: "b"} + - match: {columns.1.type: "integer"} + - match: {columns.2.name: "c"} + - match: {columns.2.type: "null"} + - match: {columns.3.name: "z"} + - match: {columns.3.type: "integer"} + - match: {values.0.0: 1} + - match: {values.0.1: 2} + - match: {values.0.2: null} + - match: {values.0.3: null} + +--- +"Test Eval With Null And Count": + - do: + esql.query: + body: + query: 'from test | eval nullsum = count_d + null | stats count(nullsum)' + + - length: {columns: 1} + - length: {values: 1} + - match: {columns.0.name: "count(nullsum)"} + - match: {columns.0.type: "long"} + - length: {values.0: 1} + - match: {values.0.0: 0} + +--- +"Test Eval With Multiple Expressions": + - do: + esql.query: + body: + query: 'row l=1, d=1.0, ln=1 + null, dn=1.0 + null | stats sum(l), sum(d), sum(ln), sum(dn)' + + - length: {columns: 4} + - length: {values: 1} + - length: {values.0: 4} + - match: {columns.0.name: "sum(l)"} + - match: {columns.0.type: "long"} + - match: {columns.1.name: "sum(d)"} + - match: {columns.1.type: "double"} + - match: {columns.2.name: "sum(ln)"} + - match: {columns.2.type: "long"} + - match: {columns.3.name: "sum(dn)"} + - match: {columns.3.type: "double"} + - match: {values.0.0: 1} + - match: {values.0.1: 1} + - match: {values.0.2: null} + - match: {values.0.3: null} + diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml new file mode 100644 index 0000000000000..8669244165385 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml @@ -0,0 +1,564 @@ +--- +constant_keyword: + - do: + indices.create: + index: test + body: + mappings: + properties: + kind: + type: constant_keyword + value: wow such constant + color: + type: keyword + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "color": "red" } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: color } + - match: { columns.0.type: keyword } + - match: { columns.1.name: kind } + - match: { columns.1.type: keyword } + - length: { values: 1 } + - match: { values.0.0: red } + - match: { values.0.1: wow such constant } + + - do: + esql.query: + body: + query: 'from test | eval l=length(kind) | keep l' + + - match: {columns.0.name: l} + - match: {columns.0.type: integer} + - length: {values: 1} + - match: {values.0.0: 17} + +--- +multivalued keyword: + - do: + indices.create: + index: test + body: + mappings: + properties: + card: + type: keyword + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "card": ["jack", "of", "diamonds"] } + + - do: + esql.query: + body: + query: 'from test' + - match: {columns.0.name: card} + - match: {columns.0.type: keyword} + - length: {values: 1} + - match: {values.0.0: [diamonds, jack, of]} + +--- +wildcard: + - do: + indices.create: + index: test + body: + mappings: + properties: + card: + type: wildcard + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "card": "jack of diamonds" } + + - do: + esql.query: + body: + query: 'from test' + - match: {columns.0.name: card} + - match: {columns.0.type: keyword} + - length: {values: 1} + - match: {values.0.0: jack of diamonds} + + - do: + esql.query: + body: + query: 'from test | eval l=length(card) | keep l' + - match: {columns.0.name: l} + - match: {columns.0.type: integer} + - length: {values: 1} + - match: {values.0.0: 16} + +--- +numbers: + - do: + indices.create: + index: test + body: + mappings: + properties: + i: + type: integer + l: + type: long + d: + type: double + mv_i: + type: integer + mv_l: + type: long + mv_d: + type: double + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { i: 123, l: -1234567891011121131, d: 1.234567891234568, mv_i: [123456, -123456], mv_l: [1234567891011121131, -1234567891011121131], mv_d: [1.234567891234568, -1.234567891234568] } + + - do: + esql.query: + body: + query: 'from test' + - match: {columns.0.name: d} + - match: {columns.0.type: double} + - match: {columns.1.name: i} + - match: {columns.1.type: integer} + - match: {columns.2.name: l} + - match: {columns.2.type: long} + - match: {columns.3.name: mv_d} + - match: {columns.3.type: double} + - match: {columns.4.name: mv_i} + - match: {columns.4.type: integer} + - match: {columns.5.name: mv_l} + - match: {columns.5.type: long} + - length: {values: 1} + - match: {values.0.0: 1.234567891234568} + - match: {values.0.1: 123} + - match: {values.0.2: -1234567891011121131} + - match: {values.0.3: [-1.234567891234568, 1.234567891234568]} + - match: {values.0.4: [-123456, 123456]} + - match: {values.0.5: [-1234567891011121131, 1234567891011121131]} + +--- +small_numbers: + - do: + indices.create: + index: test + body: + mappings: + properties: + b: + type: byte + s: + type: short + hf: + type: half_float + f: + type: float + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { b: 1, s: 1245, hf: 12.01, f: 112.0 } + + - do: + esql.query: + body: + query: 'from test' + - match: {columns.0.name: b} + - match: {columns.0.type: integer} + - match: {columns.1.name: f} + - match: {columns.1.type: double} + - match: {columns.2.name: hf} + - match: {columns.2.type: double} + - match: {columns.3.name: s} + - match: {columns.3.type: integer} + - length: {values: 1} + - match: {values.0.0: 1} + - match: {values.0.1: 112.0} + - match: {values.0.2: 12.0078125} + - match: {values.0.3: 1245} + + - do: + esql.query: + body: + query: 'from test | eval sum_d = b + f + hf + s, sum_i = b + s | keep sum_d, sum_i' + - match: {columns.0.name: sum_d} + - match: {columns.0.type: double} + - match: {columns.1.name: sum_i} + - match: {columns.1.type: integer} + - length: {values: 1} + - match: {values.0.0: 1370.0078125} + - match: {values.0.1: 1246} + + - do: + esql.query: + body: + query: 'from test | eval r_f = round(f), r_hf = round(hf) | keep r_f, r_hf' + - match: {columns.0.name: r_f} + - match: {columns.0.type: double} + - match: {columns.1.name: r_hf} + - match: {columns.1.type: double} + - length: {values: 1} + - match: {values.0.0: 112.0} + - match: {values.0.1: 12.0} + +--- +scaled_float: + - do: + indices.create: + index: test + body: + mappings: + properties: + f: + type: scaled_float + scaling_factor: 100 + d: + type: double + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { f: 112.01, d: 1.0 } + + - do: + esql.query: + body: + query: 'from test' + - match: {columns.0.name: d} + - match: {columns.0.type: double} + - match: {columns.1.name: f} + - match: {columns.1.type: double} + - length: {values: 1} + - match: {values.0.0: 1.0} + - match: {values.0.1: 112.01} + + - do: + esql.query: + body: + query: 'from test | eval sum = d + f | keep sum' + - match: {columns.0.name: sum} + - match: {columns.0.type: double} + - length: {values: 1} + - match: {values.0.0: 113.01} + +--- +multivalued boolean: + - do: + indices.create: + index: test + body: + mappings: + properties: + booleans: + type: boolean + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "booleans": [ true, false, false, false ] } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: booleans } + - match: { columns.0.type: boolean } + - length: { values: 1 } + - match: { values.0.0: [ false, false, false, true ] } + +--- +ip: + - do: + indices.create: + index: test + body: + mappings: + properties: + ip: + type: ip + keyword: + type: keyword + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "ip": "127.0.0.1", "keyword": "127.0.0.2" } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: ip } + - match: { columns.0.type: ip } + - match: { columns.1.name: keyword } + - match: { columns.1.type: keyword } + - length: { values: 1 } + - match: { values.0.0: "127.0.0.1" } + - match: { values.0.1: "127.0.0.2" } + + - do: + esql.query: + body: + query: 'from test | where keyword == "127.0.0.2" | rename ip as IP | drop keyword' + + - match: {columns.0.name: IP } + - match: {columns.0.type: ip } + - length: {values: 1 } + - match: {values.0.0: "127.0.0.1" } + +--- +alias: + - do: + indices.create: + index: test + body: + mappings: + properties: + foo: + type: keyword + bar: + type: alias + path: foo + level1: + properties: + level2: + type: long + level2_alias: + type: alias + path: level1.level2 + some_long: + type: long + some_long_alias: + type: alias + path: some_long + some_long_alias2: + type: alias + path: some_long + some_date: + type: date + some_date_alias: + type: alias + path: some_date + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "foo": "abc", "level1": {"level2": 10}, "some_long": 12, "some_date": "2015-01-01T12:00:00.000Z" } + - { "index": { } } + - { "foo": "def", "level1": {"level2": 50}, "some_long": 15, "some_date": "2015-01-01T12:00:00.000Z" } + + - do: + esql.query: + body: + query: 'from test | keep foo, bar, level1.level2, level2_alias, some_long, some_long_alias, some_long_alias2, some_date, some_date_alias | sort level2_alias' + - match: { columns.0.name: foo } + - match: { columns.0.type: keyword } + - match: { columns.1.name: bar } + - match: { columns.1.type: keyword } + - match: { columns.2.name: level1.level2 } + - match: { columns.2.type: long } + - match: { columns.3.name: level2_alias } + - match: { columns.3.type: long } + - match: { columns.4.name: some_long } + - match: { columns.4.type: long } + - match: { columns.5.name: some_long_alias } + - match: { columns.5.type: long } + - match: { columns.6.name: some_long_alias2 } + - match: { columns.6.type: long } + - match: { columns.7.name: some_date } + - match: { columns.7.type: date } + - match: { columns.8.name: some_date_alias } + - match: { columns.8.type: date } + - length: { values: 2 } + - match: { values.0.0: abc } + - match: { values.0.1: abc } + - match: { values.0.2: 10 } + - match: { values.0.3: 10 } + - match: { values.0.4: 12 } + - match: { values.0.5: 12 } + - match: { values.0.6: 12 } + - match: { values.0.7: 2015-01-01T12:00:00.000Z } + - match: { values.0.8: 2015-01-01T12:00:00.000Z } + - match: { values.1.0: def } + - match: { values.1.1: def } + - match: { values.1.2: 50 } + - match: { values.1.3: 50 } + - match: { values.1.4: 15 } + - match: { values.1.5: 15 } + - match: { values.1.6: 15 } + - match: { values.1.7: 2015-01-01T12:00:00.000Z } + - match: { values.1.8: 2015-01-01T12:00:00.000Z } + + - do: + esql.query: + body: + query: 'from test | where bar == "abc" | keep foo, bar, level1.level2, level2_alias' + - match: { columns.0.name: foo } + - match: { columns.0.type: keyword } + - match: { columns.1.name: bar } + - match: { columns.1.type: keyword } + - match: { columns.2.name: level1.level2 } + - match: { columns.2.type: long } + - match: { columns.3.name: level2_alias } + - match: { columns.3.type: long } + - length: { values: 1 } + - match: { values.0.0: abc } + - match: { values.0.1: abc } + - match: { values.0.2: 10 } + - match: { values.0.3: 10 } + + - do: + esql.query: + body: + query: 'from test | where level2_alias == 10 | keep foo, bar, level1.level2, level2_alias' + - match: { columns.0.name: foo } + - match: { columns.0.type: keyword } + - match: { columns.1.name: bar } + - match: { columns.1.type: keyword } + - match: { columns.2.name: level1.level2 } + - match: { columns.2.type: long } + - match: { columns.3.name: level2_alias } + - match: { columns.3.type: long } + - length: { values: 1 } + - match: { values.0.0: abc } + - match: { values.0.1: abc } + - match: { values.0.2: 10 } + - match: { values.0.3: 10 } + + - do: + esql.query: + body: + query: 'from test | where level2_alias == 20' + - length: { values: 0 } + + - do: + esql.query: + body: + query: 'from test | stats x = max(level2_alias)' + - match: { columns.0.name: x } + - match: { columns.0.type: long } + - length: { values: 1 } + - match: { values.0.0: 50 } + +--- +version: + - do: + indices.create: + index: test + body: + mappings: + properties: + version: + type: version + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "version": [ "1.2.3", "4.5.6-SNOOPY" ] } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: version } + - match: { columns.0.type: version } + - length: { values: 1 } + - match: { values.0.0: [ "1.2.3", "4.5.6-SNOOPY" ] } + +--- +id: + - do: + indices.create: + index: test + body: + mappings: + properties: + kw: + type: keyword + + - do: + bulk: + index: test + refresh: true + body: + - { "index" : { "_index" : "test", "_id" : "id-1" } } + - { "kw": "keyword1" } + + - do: + esql.query: + body: + query: 'from test [metadata _id] | keep _id, kw' + - match: { columns.0.name: _id } + - match: { columns.0.type: keyword } + - length: { values: 1 } + - match: { values.0.0: "id-1" } + - match: { values.0.1: "keyword1" } + +--- +unsigned_long: + - do: + indices.create: + index: test + body: + mappings: + properties: + number: + type: unsigned_long + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { "number": [ "1", "9223372036854775808", "0", "18446744073709551615" ] } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: number } + - match: { columns.0.type: unsigned_long } + - length: { values: 1 } + - match: { values.0.0: [ 0, 1, 9223372036854775808, 18446744073709551615 ] } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml new file mode 100644 index 0000000000000..f37c897d77b4b --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml @@ -0,0 +1,97 @@ +setup: + - do: + indices.create: + index: test + body: + settings: + index: + mode: time_series + routing_path: [metricset, k8s.pod.uid] + time_series: + start_time: 2021-04-28T00:00:00Z + end_time: 2021-04-29T00:00:00Z + mappings: + properties: + "@timestamp": + type: date + metricset: + type: keyword + time_series_dimension: true + k8s: + properties: + pod: + properties: + uid: + type: keyword + time_series_dimension: true + name: + type: keyword + ip: + type: ip + network: + properties: + tx: + type: long + rx: + type: long + - do: + bulk: + refresh: true + index: test + body: + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:04.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2001818691, "rx": 802133794}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:24.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2005177954, "rx": 801479970}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:44.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.1", "network": {"tx": 2006223737, "rx": 802337279}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:51:04.467Z", "metricset": "pod", "k8s": {"pod": {"name": "cat", "uid":"947e4ced-1786-4e53-9e0c-5c447e959507", "ip": "10.10.55.2", "network": {"tx": 2012916202, "rx": 803685721}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:03.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434521831, "rx": 530575198}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:23.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434577921, "rx": 530600088}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:50:53.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434587694, "rx": 530604797}}}}' + - '{"index": {}}' + - '{"@timestamp": "2021-04-28T18:51:03.142Z", "metricset": "pod", "k8s": {"pod": {"name": "dog", "uid":"df3145b3-0563-4d3b-a0f7-897eb2876ea9", "ip": "10.10.55.3", "network": {"tx": 1434595272, "rx": 530605511}}}}' + +--- +load everything: + - do: + esql.query: + body: + query: 'from test' + + - match: {columns.0.name: "@timestamp"} + - match: {columns.0.type: "date"} + - match: {columns.1.name: "k8s.pod.ip"} + - match: {columns.1.type: "ip"} + - match: {columns.2.name: "k8s.pod.name"} + - match: {columns.2.type: "keyword"} + - match: {columns.3.name: "k8s.pod.network.rx"} + - match: {columns.3.type: "long"} + - match: {columns.4.name: "k8s.pod.network.tx"} + - match: {columns.4.type: "long"} + - match: {columns.5.name: "k8s.pod.uid"} + - match: {columns.5.type: "keyword"} + - match: {columns.6.name: "metricset"} + - match: {columns.6.type: "keyword"} + - length: {values: 8} + +--- +load a document: + - do: + esql.query: + body: + query: 'from test | where k8s.pod.network.tx == 1434577921' + + - length: {values: 1} + - length: {values.0: 7} + - match: {values.0.0: "2021-04-28T18:50:23.142Z"} + - match: {values.0.1: "10.10.55.3"} + - match: {values.0.2: "dog"} + - match: {values.0.3: 530600088} + - match: {values.0.4: 1434577921} + - match: {values.0.5: "df3145b3-0563-4d3b-a0f7-897eb2876ea9"} + - match: {values.0.6: "pod"} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_unsupported_types.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_unsupported_types.yml new file mode 100644 index 0000000000000..52f7460ea727e --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_unsupported_types.yml @@ -0,0 +1,259 @@ +--- +unsupported: + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 5 + mappings: + properties: + aggregate_metric_double: + type: aggregate_metric_double + metrics: [ min, max ] + default_metric: max + binary: + type: binary + completion: + type: completion + date_nanos: + type: date_nanos + date_range: + type: date_range + format: "yyyy-MM-dd HH:mm:ss||yyyy-MM-dd||epoch_millis" + dense_vector: + type: dense_vector + dims: 3 + double_range: + type: double_range + float_range: + type: float_range + histogram: + type: histogram + integer_range: + type: integer_range + name: + type: keyword + geo_point: + type: geo_point + geo_point_alias: + type: alias + path: geo_point + ip_range: + type: ip_range + long_range: + type: long_range + match_only_text: + type: match_only_text + nested: + type: nested + rank_feature: + type: rank_feature + rank_features: + type: rank_features + search_as_you_type: + type: search_as_you_type + shape: + type: shape + some_doc: + properties: + foo: + type: keyword + bar: + type: long + text: + type: text + token_count: + type: token_count + analyzer: standard + + - do: + bulk: + index: test + refresh: true + body: + - { "index": { } } + - { + "aggregate_metric_double": { "min": 1.0, "max": 3.0 }, + "binary": "U29tZSBiaW5hcnkgYmxvYg==", + "completion": "foo bar", + "date_nanos": "2015-01-01T12:10:30.123456789Z", + "date_range": { "gte": "2015-10-31 12:00:00", "lte": "2050-12-31 12:00:00" }, + "dense_vector": [ 0.5, 10, 6 ], + "double_range": { "gte": 1.0, "lte": 2.0 }, + "float_range": { "gte": 1.0, "lte": 2.0 }, + "geo_point": [ 10.0, 12.0 ], + "histogram": { "values": [ 0.1, 0.25, 0.35, 0.4, 0.45, 0.5 ], "counts": [ 8, 17, 8, 7, 6, 2 ] }, + "integer_range": { "gte": 1, "lte": 2 }, + "ip_range": "127.0.0.1/16", + "long_range": { "gte": 1, "lte": 2 }, + "match_only_text": "foo bar baz", + "name": "Alice", + "rank_feature": 10, + "rank_features": { "foo": 10, "bar": 20 }, + "search_as_you_type": "foo bar baz", + "shape": "LINESTRING (-377.03653 389.897676, -377.009051 389.889939)", + "text": "foo bar", + "token_count": "foo bar baz", + "some_doc": { "foo": "xy", "bar": 12 } + } + + - do: + esql.query: + body: + query: 'from test' + - match: { columns.0.name: aggregate_metric_double } + - match: { columns.0.type: unsupported } + - match: { columns.1.name: binary } + - match: { columns.1.type: unsupported } + - match: { columns.2.name: completion } + - match: { columns.2.type: unsupported } + - match: { columns.3.name: date_nanos } + - match: { columns.3.type: unsupported } + - match: { columns.4.name: date_range } + - match: { columns.4.type: unsupported } + - match: { columns.5.name: dense_vector } + - match: { columns.5.type: unsupported } + - match: { columns.6.name: double_range } + - match: { columns.6.type: unsupported } + - match: { columns.7.name: float_range } + - match: { columns.7.type: unsupported } + - match: { columns.8.name: geo_point } + - match: { columns.8.type: unsupported } + - match: { columns.9.name: geo_point_alias } + - match: { columns.9.type: unsupported } + - match: { columns.10.name: histogram } + - match: { columns.10.type: unsupported } + - match: { columns.11.name: integer_range } + - match: { columns.11.type: unsupported } + - match: { columns.12.name: ip_range } + - match: { columns.12.type: unsupported } + - match: { columns.13.name: long_range } + - match: { columns.13.type: unsupported } + - match: { columns.14.name: match_only_text } + - match: { columns.14.type: text } + - match: { columns.15.name: name } + - match: { columns.15.type: keyword } + - match: { columns.16.name: rank_feature } + - match: { columns.16.type: unsupported } + - match: { columns.17.name: rank_features } + - match: { columns.17.type: unsupported } + - match: { columns.18.name: search_as_you_type } + - match: { columns.18.type: unsupported } + - match: { columns.19.name: search_as_you_type._2gram } + - match: { columns.19.type: unsupported } + - match: { columns.20.name: search_as_you_type._3gram } + - match: { columns.20.type: unsupported } + - match: { columns.21.name: search_as_you_type._index_prefix } + - match: { columns.21.type: unsupported } + - match: { columns.22.name: shape } + - match: { columns.22.type: unsupported } + - match: { columns.23.name: some_doc.bar } + - match: { columns.23.type: long } + - match: { columns.24.name: some_doc.foo } + - match: { columns.24.type: keyword } + - match: { columns.25.name: text } + - match: { columns.25.type: text } + - match: { columns.26.name: token_count } + - match: { columns.26.type: integer } + + - length: { values: 1 } + - match: { values.0.0: "" } + - match: { values.0.1: "" } + - match: { values.0.2: "" } + - match: { values.0.3: "" } + - match: { values.0.4: "" } + - match: { values.0.5: "" } + - match: { values.0.6: "" } + - match: { values.0.7: "" } + - match: { values.0.8: "" } + - match: { values.0.9: "" } + - match: { values.0.10: "" } + - match: { values.0.11: "" } + - match: { values.0.12: "" } + - match: { values.0.13: "" } + - match: { values.0.14: "foo bar baz" } + - match: { values.0.15: Alice } + - match: { values.0.16: "" } + - match: { values.0.17: "" } + - match: { values.0.18: "" } + - match: { values.0.19: "" } + - match: { values.0.20: "" } + - match: { values.0.21: "" } + - match: { values.0.22: "" } + - match: { values.0.23: 12 } + - match: { values.0.24: xy } + - match: { values.0.25: "foo bar" } + - match: { values.0.26: 3 } + + +# limit 0 + - do: + esql.query: + body: + query: 'from test | limit 0' + - match: { columns.0.name: aggregate_metric_double } + - match: { columns.0.type: unsupported } + - match: { columns.1.name: binary } + - match: { columns.1.type: unsupported } + - match: { columns.2.name: completion } + - match: { columns.2.type: unsupported } + - match: { columns.3.name: date_nanos } + - match: { columns.3.type: unsupported } + - match: { columns.4.name: date_range } + - match: { columns.4.type: unsupported } + - match: { columns.5.name: dense_vector } + - match: { columns.5.type: unsupported } + - match: { columns.6.name: double_range } + - match: { columns.6.type: unsupported } + - match: { columns.7.name: float_range } + - match: { columns.7.type: unsupported } + - match: { columns.8.name: geo_point } + - match: { columns.8.type: unsupported } + - match: { columns.9.name: geo_point_alias } + - match: { columns.9.type: unsupported } + - match: { columns.10.name: histogram } + - match: { columns.10.type: unsupported } + - match: { columns.11.name: integer_range } + - match: { columns.11.type: unsupported } + - match: { columns.12.name: ip_range } + - match: { columns.12.type: unsupported } + - match: { columns.13.name: long_range } + - match: { columns.13.type: unsupported } + - match: { columns.14.name: match_only_text } + - match: { columns.14.type: text } + - match: { columns.15.name: name } + - match: { columns.15.type: keyword } + - match: { columns.16.name: rank_feature } + - match: { columns.16.type: unsupported } + - match: { columns.17.name: rank_features } + - match: { columns.17.type: unsupported } + - match: { columns.18.name: search_as_you_type } + - match: { columns.18.type: unsupported } + - match: { columns.19.name: search_as_you_type._2gram } + - match: { columns.19.type: unsupported } + - match: { columns.20.name: search_as_you_type._3gram } + - match: { columns.20.type: unsupported } + - match: { columns.21.name: search_as_you_type._index_prefix } + - match: { columns.21.type: unsupported } + - match: { columns.22.name: shape } + - match: { columns.22.type: unsupported } + - match: { columns.23.name: some_doc.bar } + - match: { columns.23.type: long } + - match: { columns.24.name: some_doc.foo } + - match: { columns.24.type: keyword } + - match: { columns.25.name: text } + - match: { columns.25.type: text } + - match: { columns.26.name: token_count } + - match: { columns.26.type: integer } + + - length: { values: 0 } + +# keep unsupported and limit 0 + - do: + esql.query: + body: + query: 'from test | keep shape | limit 0' + - match: { columns.0.name: shape } + - match: { columns.0.type: unsupported } + - length: { values: 0 } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/50_index_patterns.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/50_index_patterns.yml new file mode 100644 index 0000000000000..280a32aa10cd3 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/50_index_patterns.yml @@ -0,0 +1,369 @@ +--- +disjoint_mappings: + - do: + indices.create: + index: test1 + body: + mappings: + properties: + message1: + type: keyword + + - do: + indices.create: + index: test2 + body: + mappings: + properties: + message2: + type: long + + + - do: + bulk: + index: test1 + refresh: true + body: + - { "index": {} } + - { "message1": "foo1"} + - { "index": {} } + - { "message1": "foo2" } + + - do: + bulk: + index: test2 + refresh: true + body: + - { "index": {} } + - { "message2": 1 } + - { "index": {} } + - { "message2": 2 } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 4 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + - match: { values.2.0: null } + - match: { values.3.0: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 | limit 2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 2 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 desc nulls last | limit 1' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 1 } + - match: { values.0.0: foo2 } + - match: { values.0.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1, message2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 4 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + - match: { values.2.0: null } + - match: { values.2.1: 1 } + - match: { values.3.0: null } + - match: { values.3.1: 2 } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1, message2 | limit 3' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 3 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + - match: { values.2.0: null } + - match: { values.2.1: 1 } + + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 desc nulls first, message2 | limit 3' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 3 } + - match: { values.0.0: null } + - match: { values.0.1: 1 } + - match: { values.1.0: null } + - match: { values.1.1: 2 } + - match: { values.2.0: foo2 } + - match: { values.2.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1, message2 | limit 2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 2 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 nulls first, message2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 4 } + - match: { values.0.0: null } + - match: { values.0.1: 1 } + - match: { values.1.0: null } + - match: { values.1.1: 2 } + - match: { values.2.0: foo1 } + - match: { values.2.1: null } + - match: { values.3.0: foo2 } + - match: { values.3.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 nulls first, message2 nulls first' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 4 } + - match: { values.0.0: null } + - match: { values.0.1: 1 } + - match: { values.1.0: null } + - match: { values.1.1: 2 } + - match: { values.2.0: foo1 } + - match: { values.2.1: null } + - match: { values.3.0: foo2 } + - match: { values.3.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | keep message1, message2 | sort message1 desc nulls first, message2 desc nulls first' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 4 } + - match: { values.0.0: null } + - match: { values.0.1: 2 } + - match: { values.1.0: null } + - match: { values.1.1: 1 } + - match: { values.2.0: foo2 } + - match: { values.2.1: null } + - match: { values.3.0: foo1 } + - match: { values.3.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | where message1 == "foo1" | keep message1, message2 | sort message1, message2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 1 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + + - do: + esql.query: + body: + query: 'from test1,test2 | where message1 == "foo1" or message2 == 2 | keep message1, message2 | sort message1, message2' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - length: { values: 2 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.1.0: null } + - match: { values.1.1: 2 } + + - do: + esql.query: + body: + query: 'from test1,test2 | stats x = max(message2)' + - match: { columns.0.name: x } + - match: { columns.0.type: long } + - length: { values: 1 } + - match: { values.0.0: 2 } + + - do: + esql.query: + body: + query: 'from test1,test2 | sort message1, message2 | eval x = message1, y = message2 + 1 | keep message1, message2, x, y' + - match: { columns.0.name: message1 } + - match: { columns.0.type: keyword } + - match: { columns.1.name: message2 } + - match: { columns.1.type: long } + - match: { columns.2.name: x } + - match: { columns.2.type: keyword } + - match: { columns.3.name: y } + - match: { columns.3.type: long } + - length: { values: 4 } + - match: { values.0.0: foo1 } + - match: { values.0.1: null } + - match: { values.0.2: foo1 } + - match: { values.0.3: null } + - match: { values.1.0: foo2 } + - match: { values.1.1: null } + - match: { values.1.2: foo2 } + - match: { values.1.3: null } + - match: { values.2.0: null } + - match: { values.2.1: 1 } + - match: { values.2.2: null } + - match: { values.2.3: 2 } + - match: { values.3.0: null } + - match: { values.3.1: 2 } + - match: { values.3.2: null } + - match: { values.3.3: 3 } + +--- +same_name_different_type: + - do: + indices.create: + index: test1 + body: + mappings: + properties: + message: + type: keyword + + - do: + indices.create: + index: test2 + body: + mappings: + properties: + message: + type: long + + + - do: + bulk: + index: test1 + refresh: true + body: + - { "index": {} } + - { "message": "foo1"} + - { "index": {} } + - { "message": "foo2" } + + - do: + bulk: + index: test2 + refresh: true + body: + - { "index": {} } + - { "message": 1 } + - { "index": {} } + - { "message": 2 } + + - do: + esql.query: + body: + query: 'from test1,test2 ' + - match: { columns.0.name: message } + - match: { columns.0.type: unsupported } + - length: { values: 4 } + +--- +same_name_different_type_same_family: + - do: + indices.create: + index: test1 + body: + mappings: + properties: + message: + type: keyword + + - do: + indices.create: + index: test2 + body: + mappings: + properties: + message: + type: wildcard + + + - do: + bulk: + index: test1 + refresh: true + body: + - { "index": {} } + - { "message": "foo1"} + - { "index": {} } + - { "message": "foo2" } + + - do: + bulk: + index: test2 + refresh: true + body: + - { "index": {} } + - { "message": "foo3" } + - { "index": {} } + - { "message": "foo4" } + + - do: + esql.query: + body: + query: 'from test1,test2 | sort message | keep message' + - match: { columns.0.name: message } + - match: { columns.0.type: keyword } + - length: { values: 4 } + - match: { values.0.0: foo1 } + - match: { values.1.0: foo2 } + - match: { values.2.0: foo3 } + - match: { values.3.0: foo4 } + diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_enrich.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_enrich.yml new file mode 100644 index 0000000000000..701bd63c3d35d --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_enrich.yml @@ -0,0 +1,121 @@ +--- +setup: + - do: + indices.create: + index: cities + body: + settings: + number_of_shards: 5 + mappings: + properties: + city_code: + type: keyword + city: + type: keyword + country: + type: keyword + + - do: + bulk: + index: "cities" + refresh: true + body: + - { "index": { } } + - { "city_code": "nyc", "city": "New York", "country": "USA" } + - { "index": { } } + - { "city_code": "rom", "city": "Rome", "country": "Italy" } + + - do: + enrich.put_policy: + name: cities_policy + body: + match: + indices: ["cities"] + match_field: "city_code" + enrich_fields: ["city", "country"] + + - do: + enrich.execute_policy: + name: cities_policy + + - do: + indices.create: + index: test + body: + mappings: + properties: + name: + type: keyword + city_id: + type: keyword + - do: + bulk: + index: "test" + refresh: true + body: + - { "index": { } } + - { "name": "Alice", "city_id": "nyc" } + - { "index": { } } + - { "name": "Bob", "city_id": "nyc" } + - { "index": { } } + - { "name": "Mario", "city_id": "rom" } + - { "index": { } } + - { "name": "Denise", "city_id": "sgn" } + +--- +"Basic": + - do: + esql.query: + body: + query: 'from test | enrich cities_policy on city_id | keep name, city, country | sort name' + + - match: { columns.0.name: "name" } + - match: { columns.0.type: "keyword" } + - match: { columns.1.name: "city" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "country" } + - match: { columns.2.type: "keyword" } + + - length: { values: 4 } + - match: { values.0: [ "Alice", "New York", "USA" ] } + - match: { values.1: [ "Bob", "New York", "USA" ] } + - match: { values.2: [ "Denise", null, null ] } + - match: { values.3: [ "Mario", "Rome", "Italy" ] } + + + - do: + esql.query: + body: + query: 'from test | keep name, city_id | enrich cities_policy on city_id with country | sort name' + + - match: { columns.0.name: "name" } + - match: { columns.0.type: "keyword" } + - match: { columns.1.name: "city_id" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "country" } + - match: { columns.2.type: "keyword" } + + - length: { values: 4 } + - match: { values.0: [ "Alice", "nyc", "USA" ] } + - match: { values.1: [ "Bob", "nyc", "USA" ] } + - match: { values.2: [ "Denise", "sgn", null ] } + - match: { values.3: [ "Mario", "rom", "Italy" ] } + + + - do: + esql.query: + body: + query: 'from test | keep name, city_id | enrich cities_policy on city_id with country_name = country | sort name' + + - match: { columns.0.name: "name" } + - match: { columns.0.type: "keyword" } + - match: { columns.1.name: "city_id" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "country_name" } + - match: { columns.2.type: "keyword" } + + - length: { values: 4 } + - match: { values.0: [ "Alice", "nyc", "USA" ] } + - match: { values.1: [ "Bob", "nyc", "USA" ] } + - match: { values.2: [ "Denise", "sgn", null ] } + - match: { values.3: [ "Mario", "rom", "Italy" ] } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_usage.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_usage.yml new file mode 100644 index 0000000000000..d7998651540d8 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/60_usage.yml @@ -0,0 +1,58 @@ +--- +setup: + - do: + indices.create: + index: test + body: + settings: + number_of_shards: 1 + mappings: + properties: + data: + type: long + count: + type: integer + +--- +"Basic ESQL usage output (telemetry)": + + - do: {xpack.usage: {}} + - match: { esql.available: true } + - match: { esql.enabled: true } + - length: { esql.features: 7 } + - set: {esql.features.dissect: dissect_counter} + - set: {esql.features.eval: eval_counter} + - set: {esql.features.grok: grok_counter} + - set: {esql.features.limit: limit_counter} + - set: {esql.features.sort: sort_counter} + - set: {esql.features.stats: stats_counter} + - set: {esql.features.where: where_counter} + - length: { esql.queries: 3 } + - set: {esql.queries.rest.total: rest_total_counter} + - set: {esql.queries.rest.failed: rest_failed_counter} + - set: {esql.queries.kibana.total: kibana_total_counter} + - set: {esql.queries.kibana.failed: kibana_failed_counter} + - set: {esql.queries._all.total: all_total_counter} + - set: {esql.queries._all.failed: all_failed_counter} + + - do: + esql.query: + body: + query: 'from test | where data > 2 | sort count desc | limit 5 | stats m = max(data)' + + - do: {xpack.usage: {}} + - match: { esql.available: true } + - match: { esql.enabled: true } + - match: {esql.features.dissect: $dissect_counter} + - match: {esql.features.eval: $eval_counter} + - match: {esql.features.grok: $grok_counter} + - gt: {esql.features.limit: $limit_counter} + - gt: {esql.features.sort: $sort_counter} + - gt: {esql.features.stats: $stats_counter} + - gt: {esql.features.where: $where_counter} + - gt: {esql.queries.rest.total: $rest_total_counter} + - match: {esql.queries.rest.failed: $rest_failed_counter} + - match: {esql.queries.kibana.total: $kibana_total_counter} + - match: {esql.queries.kibana.failed: $kibana_failed_counter} + - gt: {esql.queries._all.total: $all_total_counter} + - match: {esql.queries._all.failed: $all_failed_counter} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/61_enrich_ip.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/61_enrich_ip.yml new file mode 100644 index 0000000000000..225e58e55ea0e --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/61_enrich_ip.yml @@ -0,0 +1,85 @@ +--- +setup: + - do: + indices.create: + index: networks + body: + mappings: + properties: + range: + type: "ip_range" + name: + type: keyword + department: + type: keyword + + - do: + bulk: + index: networks + refresh: true + body: + - { "index": { } } + - { "range": "10.100.0.0/16", "name": "Production", "department": "OPS" } + - { "index": { } } + - { "range": "10.101.0.0/16", "name": "QA", "department": "Engineering" } + + - do: + enrich.put_policy: + name: networks-policy + body: + range: + indices: [ "networks" ] + match_field: "range" + enrich_fields: [ "name", "department" ] + + - do: + enrich.execute_policy: + name: networks-policy + + - do: + indices.create: + index: events + body: + mappings: + properties: + "@timestamp": + type: date + ip: + type: "ip" + message: + type: keyword + + - do: + bulk: + index: events + refresh: true + body: + - { "index": { } } + - { "@timestamp": "2023-06-20", "ip": "10.100.0.21", "message": "network connected" } + - { "index": { } } + - { "@timestamp": "2023-06-21", "ip": [ "10.100.0.21", "10.101.0.107" ], "message": "sending messages" } + - { "index": { } } + - { "@timestamp": "2023-06-22", "ip": "10.101.0.107", "message": "network disconnected" } + - { "index": { } } + - { "@timestamp": "2023-06-24", "ip": "13.101.0.114", "message": "authentication failed" } +--- +"IP strings": + - do: + esql.query: + body: + query: 'FROM events | eval ip_str = to_string(ip) | ENRICH networks-policy ON ip_str | sort @timestamp | KEEP ip, name, department, message' + + - match: { columns.0.name: "ip" } + - match: { columns.0.type: "ip" } + - match: { columns.1.name: "name" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "department" } + - match: { columns.2.type: "keyword" } + - match: { columns.3.name: "message" } + - match: { columns.3.type: "keyword" } + + - length: { values: 4 } + - match: { values.0: [ "10.100.0.21", "Production", "OPS", "network connected" ] } + - match: { values.1: [ [ "10.100.0.21", "10.101.0.107" ], [ "Production", "QA" ], [ "OPS","Engineering" ], "sending messages" ] } + - match: { values.2: [ "10.101.0.107" , "QA", "Engineering", "network disconnected" ] } + - match: { values.3: [ "13.101.0.114" , null, null, "authentication failed" ] } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/70_locale.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/70_locale.yml new file mode 100644 index 0000000000000..06d2b5e461822 --- /dev/null +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/70_locale.yml @@ -0,0 +1,60 @@ +--- +setup: + + - do: + indices.create: + index: events + body: + mappings: + properties: + "@timestamp": + type: date + format: + type: keyword + + - do: + bulk: + index: events + refresh: true + body: + - { "index": { } } + - { "@timestamp": "2023-06-20", "format": "MMMM" } + - { "index": { } } + - { "@timestamp": "2022-05-21", "format": "yy" } +--- +"Date format with default locale": + - do: + esql.query: + body: + query: 'FROM events | eval fixed_format = date_format(@timestamp, "MMMM"), variable_format = date_format(@timestamp, format) | sort @timestamp | keep @timestamp, fixed_format, variable_format' + + - match: { columns.0.name: "@timestamp" } + - match: { columns.0.type: "date" } + - match: { columns.1.name: "fixed_format" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "variable_format" } + - match: { columns.2.type: "keyword" } + + - length: { values: 2 } + - match: { values.0: [ "2022-05-21T00:00:00.000Z", "May", "22"] } + - match: { values.1: [ "2023-06-20T00:00:00.000Z", "June", "June"] } + + +--- +"Date format with Italian locale": + - do: + esql.query: + body: + query: 'FROM events | eval fixed_format = date_format(@timestamp, "MMMM"), variable_format = date_format(@timestamp, format) | sort @timestamp | keep @timestamp, fixed_format, variable_format' + locale: "it-IT" + + - match: { columns.0.name: "@timestamp" } + - match: { columns.0.type: "date" } + - match: { columns.1.name: "fixed_format" } + - match: { columns.1.type: "keyword" } + - match: { columns.2.name: "variable_format" } + - match: { columns.2.type: "keyword" } + + - length: { values: 2 } + - match: { values.0: [ "2022-05-21T00:00:00.000Z", "maggio", "22"] } + - match: { values.1: [ "2023-06-20T00:00:00.000Z", "giugno", "giugno"] } diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java new file mode 100644 index 0000000000000..6001fd90e087a --- /dev/null +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/EsqlSpecTestCase.java @@ -0,0 +1,120 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.esql.qa.rest; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.RequestObjectBuilder; +import org.elasticsearch.xpack.ql.CsvSpecReader.CsvTestCase; +import org.elasticsearch.xpack.ql.SpecReader; +import org.junit.AfterClass; +import org.junit.Before; + +import java.io.IOException; +import java.net.URL; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.xpack.esql.CsvAssert.assertData; +import static org.elasticsearch.xpack.esql.CsvAssert.assertMetadata; +import static org.elasticsearch.xpack.esql.CsvTestUtils.isEnabled; +import static org.elasticsearch.xpack.esql.CsvTestUtils.loadCsvSpecValues; +import static org.elasticsearch.xpack.esql.CsvTestsDataLoader.CSV_DATASET_MAP; +import static org.elasticsearch.xpack.esql.CsvTestsDataLoader.loadDataSetIntoEs; +import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.runEsql; +import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser; +import static org.elasticsearch.xpack.ql.TestUtils.classpathResources; + +public abstract class EsqlSpecTestCase extends ESRestTestCase { + + private static final Logger LOGGER = LogManager.getLogger(EsqlSpecTestCase.class); + private final String fileName; + private final String groupName; + private final String testName; + private final Integer lineNumber; + private final CsvTestCase testCase; + + @ParametersFactory(argumentFormatting = "%2$s.%3$s") + public static List readScriptSpec() throws Exception { + List urls = classpathResources("/*.csv-spec"); + assertTrue("Not enough specs found " + urls, urls.size() > 0); + return SpecReader.readScriptSpec(urls, specParser()); + } + + public EsqlSpecTestCase(String fileName, String groupName, String testName, Integer lineNumber, CsvTestCase testCase) { + this.fileName = fileName; + this.groupName = groupName; + this.testName = testName; + this.lineNumber = lineNumber; + this.testCase = testCase; + } + + @Before + public void setup() throws IOException { + if (indexExists(CSV_DATASET_MAP.keySet().iterator().next()) == false) { + loadDataSetIntoEs(client()); + } + } + + @AfterClass + public static void wipeTestData() throws IOException { + try { + adminClient().performRequest(new Request("DELETE", "/*")); + } catch (ResponseException e) { + // 404 here just means we had no indexes + if (e.getResponse().getStatusLine().getStatusCode() != 404) { + throw e; + } + } + } + + public final void test() throws Throwable { + try { + assumeTrue("Test " + testName + " is not enabled", isEnabled(testName)); + doTest(); + } catch (Exception e) { + throw reworkException(e); + } + } + + protected final void doTest() throws Throwable { + RequestObjectBuilder builder = new RequestObjectBuilder(randomFrom(XContentType.values())); + Map answer = runEsql(builder.query(testCase.query).build(), testCase.expectedWarnings); + var expectedColumnsWithValues = loadCsvSpecValues(testCase.expectedResults); + + assertNotNull(answer.get("columns")); + @SuppressWarnings("unchecked") + var actualColumns = (List>) answer.get("columns"); + assertMetadata(expectedColumnsWithValues, actualColumns, LOGGER); + + assertNotNull(answer.get("values")); + @SuppressWarnings("unchecked") + List> actualValues = (List>) answer.get("values"); + assertData(expectedColumnsWithValues, actualValues, LOGGER, value -> value == null ? "null" : value.toString()); + } + + private Throwable reworkException(Throwable th) { + StackTraceElement[] stackTrace = th.getStackTrace(); + StackTraceElement[] redone = new StackTraceElement[stackTrace.length + 1]; + System.arraycopy(stackTrace, 0, redone, 1, stackTrace.length); + redone[0] = new StackTraceElement(getClass().getName(), groupName + "." + testName, fileName, lineNumber); + + th.setStackTrace(redone); + return th; + } + + @Override + protected boolean preserveClusterUponCompletion() { + return true; + } +} diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java new file mode 100644 index 0000000000000..d925eb9a1fb5c --- /dev/null +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java @@ -0,0 +1,457 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.qa.rest; + +import org.apache.http.HttpEntity; +import org.apache.http.entity.ContentType; +import org.apache.http.nio.entity.NByteArrayEntity; +import org.apache.http.util.EntityUtils; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.client.WarningsHandler; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentType; +import org.junit.After; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static java.util.Collections.emptySet; +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.elasticsearch.test.MapMatcher.matchesMap; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class RestEsqlTestCase extends ESRestTestCase { + + // Test runner will run multiple suites in parallel, with some of them requiring preserving state between + // tests (like EsqlSpecTestCase), so test data (like index name) needs not collide and cleanup must be done locally. + private static final String TEST_INDEX_NAME = "rest-esql-test"; + + public static class RequestObjectBuilder { + private final XContentBuilder builder; + private boolean isBuilt = false; + + public RequestObjectBuilder() throws IOException { + this(randomFrom(XContentType.values())); + } + + public RequestObjectBuilder(XContentType type) throws IOException { + builder = XContentBuilder.builder(type, emptySet(), emptySet()); + builder.startObject(); + } + + public RequestObjectBuilder query(String query) throws IOException { + builder.field("query", query); + return this; + } + + public RequestObjectBuilder columnar(boolean columnar) throws IOException { + builder.field("columnar", columnar); + return this; + } + + public RequestObjectBuilder timeZone(ZoneId zoneId) throws IOException { + builder.field("time_zone", zoneId); + return this; + } + + public RequestObjectBuilder pragmas(Settings pragmas) throws IOException { + builder.startObject("pragma"); + pragmas.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + return this; + } + + public RequestObjectBuilder build() throws IOException { + if (isBuilt == false) { + builder.endObject(); + isBuilt = true; + } + return this; + } + + public OutputStream getOutputStream() throws IOException { + if (isBuilt == false) { + throw new IllegalStateException("object not yet built"); + } + builder.flush(); + return builder.getOutputStream(); + } + + public XContentType contentType() { + return builder.contentType(); + } + + public static RequestObjectBuilder jsonBuilder() throws IOException { + return new RequestObjectBuilder(XContentType.JSON); + } + } + + public void testGetAnswer() throws IOException { + Map answer = runEsql(builder().query("row a = 1, b = 2").build()); + assertEquals(2, answer.size()); + Map colA = Map.of("name", "a", "type", "integer"); + Map colB = Map.of("name", "b", "type", "integer"); + assertEquals(List.of(colA, colB), answer.get("columns")); + assertEquals(List.of(List.of(1, 2)), answer.get("values")); + } + + public void testUseUnknownIndex() throws IOException { + ResponseException e = expectThrows(ResponseException.class, () -> runEsql(builder().query("from doesNotExist").build())); + assertEquals(400, e.getResponse().getStatusLine().getStatusCode()); + assertThat(e.getMessage(), containsString("verification_exception")); + assertThat(e.getMessage(), containsString("Unknown index [doesNotExist]")); + } + + public void testNullInAggs() throws IOException { + StringBuilder b = new StringBuilder(); + for (int i = 0; i < 1000; i++) { + b.append(String.format(Locale.ROOT, """ + {"create":{"_index":"%s"}} + """, testIndexName())); + if (i % 10 == 0) { + b.append(String.format(Locale.ROOT, """ + {"group":%d} + """, i % 2)); + } else { + b.append(String.format(Locale.ROOT, """ + {"group":%d,"value":%d} + """, i % 2, i)); + } + } + Request bulk = new Request("POST", "/_bulk"); + bulk.addParameter("refresh", "true"); + bulk.addParameter("filter_path", "errors"); + bulk.setJsonEntity(b.toString()); + Response response = client().performRequest(bulk); + assertThat(EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8), equalTo("{\"errors\":false}")); + + RequestObjectBuilder builder = new RequestObjectBuilder().query(fromIndex() + " | stats min(value)"); + Map result = runEsql(builder.build()); + assertMap( + result, + matchesMap().entry("values", List.of(List.of(1))).entry("columns", List.of(Map.of("name", "min(value)", "type", "long"))) + ); + + builder = new RequestObjectBuilder().query(fromIndex() + " | stats min(value) by group"); + result = runEsql(builder.build()); + assertMap( + result, + matchesMap().entry("values", List.of(List.of(2, 0), List.of(1, 1))) + .entry("columns", List.of(Map.of("name", "min(value)", "type", "long"), Map.of("name", "group", "type", "long"))) + ); + } + + public void testColumnarMode() throws IOException { + int docCount = randomIntBetween(3, 10); + bulkLoadTestData(docCount); + + boolean columnar = randomBoolean(); + var query = builder().query(fromIndex() + " | keep keyword, integer"); + if (columnar || randomBoolean()) { + query.columnar(columnar); + } + Map answer = runEsql(query.build()); + + Map colKeyword = Map.of("name", "keyword", "type", "keyword"); + Map colInteger = Map.of("name", "integer", "type", "integer"); + assertEquals(List.of(colKeyword, colInteger), answer.get("columns")); + + if (columnar) { + List valKeyword = new ArrayList<>(); + List valInteger = new ArrayList<>(); + for (int i = 0; i < docCount; i++) { + valKeyword.add("keyword" + i); + valInteger.add(i); + } + assertEquals(List.of(valKeyword, valInteger), answer.get("values")); + } else { + List rows = new ArrayList<>(); + for (int i = 0; i < docCount; i++) { + rows.add(List.of("keyword" + i, i)); + } + assertEquals(rows, answer.get("values")); + } + } + + public void testTextMode() throws IOException { + int count = randomIntBetween(0, 100); + bulkLoadTestData(count); + var builder = builder().query(fromIndex() + " | keep keyword, integer").build(); + assertEquals(expectedTextBody("txt", count, null), runEsqlAsTextWithFormat(builder, "txt", null)); + } + + public void testCSVMode() throws IOException { + int count = randomIntBetween(0, 100); + bulkLoadTestData(count); + var builder = builder().query(fromIndex() + " | keep keyword, integer").build(); + assertEquals(expectedTextBody("csv", count, '|'), runEsqlAsTextWithFormat(builder, "csv", '|')); + } + + public void testTSVMode() throws IOException { + int count = randomIntBetween(0, 100); + bulkLoadTestData(count); + var builder = builder().query(fromIndex() + " | keep keyword, integer").build(); + assertEquals(expectedTextBody("tsv", count, null), runEsqlAsTextWithFormat(builder, "tsv", null)); + } + + public void testCSVNoHeaderMode() throws IOException { + bulkLoadTestData(1); + var builder = builder().query(fromIndex() + " | keep keyword, integer").build(); + Request request = prepareRequest(); + String mediaType = attachBody(builder, request); + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.addHeader("Content-Type", mediaType); + options.addHeader("Accept", "text/csv; header=absent"); + request.setOptions(options); + HttpEntity entity = performRequest(request, List.of()); + String actual = Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8)); + assertEquals("keyword0,0\r\n", actual); + } + + public void testWarningHeadersOnFailedConversions() throws IOException { + int count = randomFrom(10, 40, 60); + bulkLoadTestData(count); + + Request request = prepareRequest(); + var query = fromIndex() + " | eval asInt = to_int(case(integer % 2 == 0, to_str(integer), keyword))"; + var mediaType = attachBody(new RequestObjectBuilder().query(query).build(), request); + + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.setWarningsHandler(WarningsHandler.PERMISSIVE); + options.addHeader("Content-Type", mediaType); + options.addHeader("Accept", mediaType); + + request.setOptions(options); + Response response = client().performRequest(request); + assertThat(response.getStatusLine().getStatusCode(), is(200)); + + int expectedWarnings = Math.min(count / 2, 20); + var warnings = response.getWarnings(); + assertThat(warnings.size(), is(1 + expectedWarnings)); + var firstHeader = "Line 1:36: evaluation of [to_int(case(integer %25 2 == 0, to_str(integer), keyword))] failed, " + + "treating result as null. Only first 20 failures recorded."; + assertThat(warnings.get(0), containsString(firstHeader)); + for (int i = 1; i <= expectedWarnings; i++) { + assertThat( + warnings.get(i), + containsString("java.lang.NumberFormatException: For input string: \\\"keyword" + (2 * i - 1) + "\\\"") + ); + } + } + + public void testMetadataFieldsOnMultipleIndices() throws IOException { + var request = new Request("POST", "/" + testIndexName() + "-1/_doc/id-1"); + request.addParameter("refresh", "true"); + request.setJsonEntity("{\"a\": 1}"); + assertEquals(201, client().performRequest(request).getStatusLine().getStatusCode()); + request = new Request("POST", "/" + testIndexName() + "-1/_doc/id-1"); + request.addParameter("refresh", "true"); + request.setJsonEntity("{\"a\": 2}"); + assertEquals(200, client().performRequest(request).getStatusLine().getStatusCode()); + request = new Request("POST", "/" + testIndexName() + "-2/_doc/id-2"); + request.addParameter("refresh", "true"); + request.setJsonEntity("{\"a\": 3}"); + assertEquals(201, client().performRequest(request).getStatusLine().getStatusCode()); + + var query = fromIndex() + "* [metadata _index, _version, _id] | sort _version"; + Map result = runEsql(new RequestObjectBuilder().query(query).build()); + var columns = List.of( + Map.of("name", "a", "type", "long"), + Map.of("name", "_index", "type", "keyword"), + Map.of("name", "_version", "type", "long"), + Map.of("name", "_id", "type", "keyword") + ); + var values = List.of(List.of(3, testIndexName() + "-2", 1, "id-2"), List.of(2, testIndexName() + "-1", 2, "id-1")); + + assertMap(result, matchesMap().entry("columns", columns).entry("values", values)); + } + + private static String expectedTextBody(String format, int count, @Nullable Character csvDelimiter) { + StringBuilder sb = new StringBuilder(); + switch (format) { + case "txt" -> { + sb.append(" keyword | integer \n"); + sb.append("---------------+---------------\n"); + } + case "csv" -> sb.append("keyword").append(csvDelimiter).append("integer\r\n"); + case "tsv" -> sb.append("keyword\tinteger\n"); + default -> { + assert false : "unexpected format type [" + format + "]"; + } + } + for (int i = 0; i < count; i++) { + sb.append("keyword").append(i); + int iLen = String.valueOf(i).length(); + switch (format) { + case "txt" -> sb.append(" ".repeat(8 - iLen)).append("|"); + case "csv" -> sb.append(csvDelimiter); + case "tsv" -> sb.append('\t'); + } + sb.append(i); + if (format.equals("txt")) { + sb.append(" ".repeat(15 - iLen)); + } + sb.append(format.equals("csv") ? "\r\n" : "\n"); + } + return sb.toString(); + } + + public static Map runEsql(RequestObjectBuilder requestObject) throws IOException { + return runEsql(requestObject, List.of()); + } + + public static Map runEsql(RequestObjectBuilder requestObject, List expectedWarnings) throws IOException { + Request request = prepareRequest(); + String mediaType = attachBody(requestObject, request); + + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.setWarningsHandler(WarningsHandler.PERMISSIVE); // We assert the warnings ourselves + options.addHeader("Content-Type", mediaType); + + if (randomBoolean()) { + options.addHeader("Accept", mediaType); + } else { + request.addParameter("format", requestObject.contentType().queryParameter()); + } + request.setOptions(options); + + HttpEntity entity = performRequest(request, expectedWarnings); + try (InputStream content = entity.getContent()) { + XContentType xContentType = XContentType.fromMediaType(entity.getContentType().getValue()); + assertEquals(requestObject.contentType(), xContentType); + return XContentHelper.convertToMap(xContentType.xContent(), content, false); + } + } + + static String runEsqlAsTextWithFormat(RequestObjectBuilder builder, String format, @Nullable Character delimiter) throws IOException { + Request request = prepareRequest(); + String mediaType = attachBody(builder, request); + + RequestOptions.Builder options = request.getOptions().toBuilder(); + options.addHeader("Content-Type", mediaType); + + if (randomBoolean()) { + request.addParameter("format", format); + } else { + switch (format) { + case "txt" -> options.addHeader("Accept", "text/plain"); + case "csv" -> options.addHeader("Accept", "text/csv"); + case "tsv" -> options.addHeader("Accept", "text/tab-separated-values"); + } + } + if (delimiter != null) { + request.addParameter("delimiter", String.valueOf(delimiter)); + } + request.setOptions(options); + + HttpEntity entity = performRequest(request, List.of()); + return Streams.copyToString(new InputStreamReader(entity.getContent(), StandardCharsets.UTF_8)); + } + + private static Request prepareRequest() { + Request request = new Request("POST", "/_query"); + request.addParameter("error_trace", "true"); // Helps with debugging in case something crazy happens on the server. + request.addParameter("pretty", "true"); // Improves error reporting readability + return request; + } + + private static String attachBody(RequestObjectBuilder requestObject, Request request) throws IOException { + String mediaType = requestObject.contentType().mediaTypeWithoutParameters(); + try (ByteArrayOutputStream bos = (ByteArrayOutputStream) requestObject.getOutputStream()) { + request.setEntity(new NByteArrayEntity(bos.toByteArray(), ContentType.getByMimeType(mediaType))); + } + return mediaType; + } + + private static HttpEntity performRequest(Request request, List allowedWarnings) throws IOException { + Response response = client().performRequest(request); + assertEquals(200, response.getStatusLine().getStatusCode()); + assertMap(response.getWarnings(), matchesList(allowedWarnings)); + return response.getEntity(); + } + + private static void bulkLoadTestData(int count) throws IOException { + Request request = new Request("PUT", "/" + testIndexName()); + request.setJsonEntity(""" + { + "mappings": { + "properties": { + "keyword": { + "type": "keyword" + }, + "integer": { + "type": "integer" + } + } + } + }"""); + assertEquals(200, client().performRequest(request).getStatusLine().getStatusCode()); + + if (count > 0) { + request = new Request("POST", "/" + testIndexName() + "/_bulk"); + request.addParameter("refresh", "true"); + StringBuilder bulk = new StringBuilder(); + for (int i = 0; i < count; i++) { + bulk.append(org.elasticsearch.core.Strings.format(""" + {"index":{"_id":"%s"}} + {"keyword":"keyword%s", "integer":%s} + """, i, i, i)); + } + request.setJsonEntity(bulk.toString()); + assertEquals(200, client().performRequest(request).getStatusLine().getStatusCode()); + } + } + + private static RequestObjectBuilder builder() throws IOException { + return new RequestObjectBuilder(); + } + + @After + public void wipeTestData() throws IOException { + try { + var response = client().performRequest(new Request("DELETE", "/" + testIndexName())); + assertEquals(200, response.getStatusLine().getStatusCode()); + } catch (ResponseException re) { + assertEquals(404, re.getResponse().getStatusLine().getStatusCode()); + } + } + + protected static String testIndexName() { + return TEST_INDEX_NAME; + } + + protected static String fromIndex() { + return "from " + testIndexName(); + } + + @Override + protected boolean preserveClusterUponCompletion() { + return true; + } +} diff --git a/x-pack/plugin/esql/qa/server/src/main/resources/explain.csv-spec b/x-pack/plugin/esql/qa/server/src/main/resources/explain.csv-spec new file mode 100644 index 0000000000000..edddd82f75ffe --- /dev/null +++ b/x-pack/plugin/esql/qa/server/src/main/resources/explain.csv-spec @@ -0,0 +1,19 @@ +explainFrom-Ignore +explain [ from foo ]; + +plan:keyword | type:keyword +"?foo" | PARSED +"org.elasticsearch.xpack.esql.analysis.VerificationException: Found 1 problem + line 1:11: Unknown index [foo]" | ANALYZED +; + + +explainCompositeQuery-Ignore +explain [ row a = 1 | where b > 0 ]; + +plan:keyword | type:keyword +"Filter[?b > 0[INTEGER]] +\_Row[[1[INTEGER] AS a]]" | PARSED +"org.elasticsearch.xpack.esql.analysis.VerificationException: Found 1 problem + line 1:29: Unknown column [b]" | ANALYZED +; diff --git a/x-pack/plugin/esql/qa/testFixtures/build.gradle b/x-pack/plugin/esql/qa/testFixtures/build.gradle new file mode 100644 index 0000000000000..e94ebf5f5b08c --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/build.gradle @@ -0,0 +1,14 @@ +apply plugin: 'elasticsearch.java' + + +dependencies { + implementation project(':x-pack:plugin:esql:compute') + compileOnly project(':x-pack:plugin:esql') + implementation project(":libs:elasticsearch-x-content") + implementation project(':client:rest') + implementation project(':libs:elasticsearch-logging') + implementation project(':test:framework') + api(testArtifact(project(xpackModule('ql')))) + implementation project(':server') + api "net.sf.supercsv:super-csv:${versions.supercsv}" +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java new file mode 100644 index 0000000000000..dba1d29656c95 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java @@ -0,0 +1,226 @@ +/* + * 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; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.xpack.esql.CsvTestUtils.ActualResults; +import org.elasticsearch.xpack.versionfield.Version; +import org.hamcrest.Matchers; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.esql.CsvTestUtils.ExpectedResults; +import static org.elasticsearch.xpack.esql.CsvTestUtils.Type; +import static org.elasticsearch.xpack.esql.CsvTestUtils.Type.UNSIGNED_LONG; +import static org.elasticsearch.xpack.esql.CsvTestUtils.logMetaData; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public final class CsvAssert { + private CsvAssert() {} + + static void assertResults(ExpectedResults expected, ActualResults actual, Logger logger) { + assertMetadata(expected, actual, logger); + assertData(expected, actual, logger); + } + + static void assertMetadata(ExpectedResults expected, ActualResults actual, Logger logger) { + assertMetadata(expected, actual.columnNames(), actual.columnTypes(), actual.pages(), logger); + } + + public static void assertMetadata(ExpectedResults expected, List> actualColumns, Logger logger) { + var actualColumnNames = new ArrayList(actualColumns.size()); + var actualColumnTypes = actualColumns.stream() + .peek(c -> actualColumnNames.add(c.get("name"))) + .map(c -> CsvTestUtils.Type.asType(c.get("type"))) + .toList(); + assertMetadata(expected, actualColumnNames, actualColumnTypes, List.of(), logger); + } + + private static void assertMetadata( + ExpectedResults expected, + List actualNames, + List actualTypes, + List pages, + Logger logger + ) { + if (logger != null) { + logMetaData(actualNames, actualTypes, logger); + } + + var expectedNames = expected.columnNames(); + var expectedTypes = expected.columnTypes(); + + assertThat( + format( + null, + "Different number of columns returned; expected [{}] but actual was [{}]", + expectedNames.size(), + actualNames.size() + ), + actualNames, + Matchers.hasSize(expectedNames.size()) + ); + + // loop through the metadata + // first check the column names + // then check the actual types + for (int column = 0; column < expectedNames.size(); column++) { + String expectedName = expectedNames.get(column); + String actualName = actualNames.get(column); + + if (expectedName.equals(actualName) == false) { + // to help debugging, indicate the previous column (which also happened to match and thus was correct) + String expectedSet = expectedName; + String actualSet = actualName; + if (column > 1) { + expectedSet = expectedNames.get(column - 1) + "," + expectedName; + actualSet = actualNames.get(column - 1) + "," + actualName; + } + + assertEquals("Different column name [" + column + "]", expectedSet, actualSet); + } + + var expectedType = expectedTypes.get(column); + var actualType = actualTypes.get(column); + + if (actualType == Type.INTEGER && expectedType == Type.LONG) { + actualType = Type.LONG; + } + + assertEquals( + "Different column type for column [" + expectedName + "] (" + expectedType + " != " + actualType + ")", + expectedType, + actualType + ); + + // perform another check against each returned page to make sure they have the same metadata + for (int pageIndex = 0; pageIndex < pages.size(); pageIndex++) { + var page = pages.get(pageIndex); + var block = page.getBlock(column); + var blockType = Type.asType(block.elementType()); + + if (blockType == Type.LONG && (expectedType == Type.DATETIME || expectedType == UNSIGNED_LONG)) { + continue; + } + if (blockType == Type.KEYWORD && (expectedType == Type.IP || expectedType == Type.VERSION || expectedType == Type.TEXT)) { + // Type.asType translates all bytes references into keywords + continue; + } + if (blockType == Type.NULL) { + // Null pages don't have any real type information beyond "it's all null, man" + continue; + } + + assertEquals( + format( + null, + "Different column type for column [{}][{}] as block inside page [{}]; ({} != {})", + expectedName, + column, + pageIndex, + expectedType, + blockType + ), + expectedType, + blockType + ); + } + } + } + + static void assertData(ExpectedResults expected, ActualResults actual, Logger logger) { + assertData(expected, actual.values(), logger, Function.identity()); + } + + public static void assertData( + ExpectedResults expected, + List> actualValues, + Logger logger, + Function valueTransformer + ) { + var expectedValues = expected.values(); + + for (int row = 0; row < expectedValues.size(); row++) { + try { + assertTrue("Expected more data but no more entries found after [" + row + "]", row < actualValues.size()); + + if (logger != null) { + logger.info(row(actualValues, row)); + } + + var expectedRow = expectedValues.get(row); + var actualRow = actualValues.get(row); + + for (int column = 0; column < expectedRow.size(); column++) { + var expectedValue = expectedRow.get(column); + var actualValue = actualRow.get(column); + + if (expectedValue != null) { + var expectedType = expected.columnTypes().get(column); + // convert the long from CSV back to its STRING form + if (expectedType == Type.DATETIME) { + expectedValue = rebuildExpected(expectedValue, Long.class, x -> UTC_DATE_TIME_FORMATTER.formatMillis((long) x)); + } else if (expectedType == Type.IP) { + // convert BytesRef-packed IP to String, allowing subsequent comparison with what's expected + expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> DocValueFormat.IP.format((BytesRef) x)); + } else if (expectedType == Type.VERSION) { + // convert BytesRef-packed Version to String + expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> new Version((BytesRef) x).toString()); + } else if (expectedType == UNSIGNED_LONG) { + expectedValue = rebuildExpected(expectedValue, Long.class, x -> unsignedLongAsNumber((long) x)); + } + } + assertEquals(valueTransformer.apply(expectedValue), valueTransformer.apply(actualValue)); + } + + var delta = actualRow.size() - expectedRow.size(); + if (delta > 0) { + fail("Plan has extra columns, returned [" + actualRow.size() + "], expected [" + expectedRow.size() + "]"); + } + } catch (AssertionError ae) { + if (logger != null && row + 1 < actualValues.size()) { + logger.info("^^^ Assertion failure ^^^"); + logger.info(row(actualValues, row + 1)); + } + throw ae; + } + } + if (expectedValues.size() < actualValues.size()) { + fail( + "Elasticsearch still has data after [" + expectedValues.size() + "] entries:\n" + row(actualValues, expectedValues.size()) + ); + } + } + + private static Object rebuildExpected(Object expectedValue, Class clazz, Function mapper) { + if (List.class.isAssignableFrom(expectedValue.getClass())) { + assertThat(((List) expectedValue).get(0), instanceOf(clazz)); + return ((List) expectedValue).stream().map(mapper).toList(); + } else { + assertThat(expectedValue, instanceOf(clazz)); + return mapper.apply(expectedValue); + } + } + + static String row(List> values, int row) { + return values.get(row).toString(); + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java new file mode 100644 index 0000000000000..9a88d28163939 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java @@ -0,0 +1,452 @@ +/* + * 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; + +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.time.DateFormatters; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.BlockUtils.BuilderWrapper; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Booleans; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; +import org.elasticsearch.xpack.ql.util.StringUtils; +import org.elasticsearch.xpack.versionfield.Version; +import org.supercsv.io.CsvListReader; +import org.supercsv.prefs.CsvPreference; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.StringReader; +import java.math.BigDecimal; +import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; + +import static org.elasticsearch.common.Strings.delimitedListToStringArray; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.ql.SpecReader.shouldSkipLine; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToUnsignedLong; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; + +public final class CsvTestUtils { + private static final int MAX_WIDTH = 20; + private static final CsvPreference CSV_SPEC_PREFERENCES = new CsvPreference.Builder('"', '|', "\r\n").build(); + private static final String NULL_VALUE = "null"; + + private CsvTestUtils() {} + + public static boolean isEnabled(String testName) { + return testName.endsWith("-Ignore") == false; + } + + public static Tuple> loadPageFromCsv(URL source) throws Exception { + + record CsvColumn(String name, Type type, BuilderWrapper builderWrapper) { + void append(String stringValue) { + if (stringValue.contains(",")) {// multi-value field + builderWrapper().builder().beginPositionEntry(); + + String[] arrayOfValues = delimitedListToStringArray(stringValue, ","); + List convertedValues = new ArrayList<>(arrayOfValues.length); + for (String value : arrayOfValues) { + convertedValues.add(type.convert(value)); + } + convertedValues.stream().sorted().forEach(v -> builderWrapper().append().accept(v)); + builderWrapper().builder().endPositionEntry(); + + return; + } + + var converted = stringValue.length() == 0 ? null : type.convert(stringValue); + builderWrapper().append().accept(converted); + } + } + + CsvColumn[] columns = null; + + try (BufferedReader reader = org.elasticsearch.xpack.ql.TestUtils.reader(source)) { + String line; + int lineNumber = 1; + + while ((line = reader.readLine()) != null) { + line = line.trim(); + // ignore comments + if (shouldSkipLine(line) == false) { + String[] entries = multiValuesAwareCsvToStringArray(line, lineNumber); + // the schema row + if (columns == null) { + columns = new CsvColumn[entries.length]; + for (int i = 0; i < entries.length; i++) { + int split = entries[i].indexOf(":"); + String name, typeName; + + if (split < 0) { + throw new IllegalArgumentException( + "A type is always expected in the schema definition; found " + entries[i] + ); + } else { + name = entries[i].substring(0, split).trim(); + typeName = entries[i].substring(split + 1).trim(); + if (typeName.length() == 0) { + throw new IllegalArgumentException( + "A type is always expected in the schema definition; found " + entries[i] + ); + } + } + Type type = Type.asType(typeName); + if (type == null) { + throw new IllegalArgumentException("Can't find type for " + entries[i]); + } + if (type == Type.NULL) { + throw new IllegalArgumentException("Null type is not allowed in the test data; found " + entries[i]); + } + columns[i] = new CsvColumn(name, type, BlockUtils.wrapperFor(type.clazz(), 8)); + } + } + // data rows + else { + if (entries.length != columns.length) { + throw new IllegalArgumentException( + format( + "Error line [{}]: Incorrect number of entries; expected [{}] but found [{}]", + lineNumber, + columns.length, + entries.length + ) + ); + } + for (int i = 0; i < entries.length; i++) { + var entry = entries[i]; + try { + columns[i].append(entry); + } catch (Exception e) { + throw new IllegalArgumentException( + format("Error line [{}]: Cannot parse entry [{}] with value [{}]", lineNumber, i + 1, entry), + e + ); + } + } + } + } + lineNumber++; + } + } + var columnNames = new ArrayList(columns.length); + var blocks = Arrays.stream(columns) + .peek(b -> columnNames.add(b.name)) + .map(b -> b.builderWrapper.builder().build()) + .toArray(Block[]::new); + return new Tuple<>(new Page(blocks), columnNames); + } + + /** + * Takes a csv String and converts it to a String array. Also, it recognizes an opening bracket "[" in one string and a closing "]" + * in another string and it creates a single concatenated comma-separated String of all the values between the opening bracket entry + * and the closing bracket entry. In other words, entries enclosed by "[]" are returned as a single element. + */ + static String[] multiValuesAwareCsvToStringArray(String csvLine, int lineNumber) { + var mvCompressedEntries = new ArrayList(); + String previousMvValue = null; // just helping out with error messaging + StringBuilder mvValue = null; + + int pos = 0; // current position in the csv String + int commaPos; // current "," character position + while ((commaPos = csvLine.indexOf(",", pos)) != -1 || pos <= csvLine.length()) { + boolean isLastElement = commaPos == -1; + String entry = csvLine.substring(pos, isLastElement ? csvLine.length() : commaPos).trim(); + if (entry.startsWith("[")) { + if (previousMvValue != null || (isLastElement && entry.endsWith("]") == false)) { + String message = "Error line [{}:{}]: Unexpected start of a multi-value field value; current token [{}], " + + (isLastElement ? "no closing point" : "previous token [{}]"); + throw new IllegalArgumentException(format(message, lineNumber, pos, entry, previousMvValue)); + } + if (entry.endsWith("]")) { + if (entry.length() > 2) {// single-valued multivalue field :shrug: + mvCompressedEntries.add(entry.substring(1, entry.length() - 1)); + } else {// empty multivalue field + mvCompressedEntries.add(""); + } + } else { + mvValue = new StringBuilder(); + previousMvValue = entry.substring(1); + mvValue.append(previousMvValue); + } + } else if (entry.endsWith("]")) { + if (previousMvValue == null) { + throw new IllegalArgumentException( + format( + "Error line [{}:{}]: Unexpected end of a multi-value field value (no previous starting point); found [{}]", + lineNumber, + pos, + entry + ) + ); + } + mvValue.append("," + entry.substring(0, entry.length() - 1)); + mvCompressedEntries.add(mvValue.toString()); + mvValue = null; + previousMvValue = null; + } else { + if (mvValue != null) {// mid-MV value + if (entry.length() == 0) {// this means there shouldn't be any null value in a multi-value field ie [a,,b,c] + throw new IllegalArgumentException( + format( + "Error line [{}:{}]: Unexpected missing value in a multi-value column; found [{}]", + lineNumber, + pos, + csvLine.substring(pos - 1) + ) + ); + } + mvValue.append("," + entry); + } else { + mvCompressedEntries.add(entry);// regular comma separated value + } + } + pos = 1 + (isLastElement ? csvLine.length() : commaPos);// break out of the loop if it reached its last element + } + return mvCompressedEntries.toArray(String[]::new); + } + + public record ExpectedResults(List columnNames, List columnTypes, List> values) {} + + public static ExpectedResults loadCsvSpecValues(String csv) { + List columnNames; + List columnTypes; + + try (CsvListReader listReader = new CsvListReader(new StringReader(csv), CSV_SPEC_PREFERENCES)) { + String[] header = listReader.getHeader(true); + columnNames = new ArrayList<>(header.length); + columnTypes = new ArrayList<>(header.length); + + for (String c : header) { + String[] nameWithType = Strings.split(c, ":"); + if (nameWithType == null || nameWithType.length != 2) { + throw new IllegalArgumentException("Invalid CSV header " + c); + } + String typeName = nameWithType[1].trim(); + if (typeName.length() == 0) { + throw new IllegalArgumentException("A type is always expected in the csv file; found " + nameWithType); + } + String name = nameWithType[0].trim(); + columnNames.add(name); + Type type = Type.asType(typeName); + if (type == null) { + throw new IllegalArgumentException("Unknown type name: [" + typeName + "]"); + } + columnTypes.add(type); + } + + List> values = new ArrayList<>(); + List row; + while ((row = listReader.read()) != null) { + List rowValues = new ArrayList<>(row.size()); + for (int i = 0; i < row.size(); i++) { + String value = row.get(i); + if (value == null || value.trim().equalsIgnoreCase(NULL_VALUE)) { + value = null; + rowValues.add(columnTypes.get(i).convert(value)); + continue; + } + + value = value.trim(); + if (value.startsWith("[") ^ value.endsWith("]")) { + throw new IllegalArgumentException("Incomplete multi-value (opening and closing square brackets) found " + value); + } + if (value.contains(",") && value.startsWith("[")) {// commas outside a multi-value should be ok + List listOfMvValues = new ArrayList<>(); + for (String mvValue : delimitedListToStringArray(value.substring(1, value.length() - 1), ",")) { + listOfMvValues.add(columnTypes.get(i).convert(mvValue.trim())); + } + rowValues.add(listOfMvValues); + } else { + rowValues.add(columnTypes.get(i).convert(value)); + } + } + values.add(rowValues); + } + + return new ExpectedResults(columnNames, columnTypes, values); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public enum Type { + INTEGER(Integer::parseInt, Integer.class), + LONG(Long::parseLong, Long.class), + UNSIGNED_LONG(s -> asLongUnsigned(safeToUnsignedLong(s)), Long.class), + DOUBLE(Double::parseDouble, Double.class), + FLOAT( + // Simulate writing the index as `float` precision by parsing as a float and rounding back to double + s -> (double) Float.parseFloat(s), + Double.class + ), + HALF_FLOAT( + s -> (double) HalfFloatPoint.sortableShortToHalfFloat(HalfFloatPoint.halfFloatToSortableShort(Float.parseFloat(s))), + Double.class + ), + // we currently only support a hard-coded scaling factor, since we're not querying the mapping of a field when reading CSV values + // for it, so the scaling_factor isn't available + SCALED_FLOAT(s -> s == null ? null : scaledFloat(s, "100"), Double.class), + KEYWORD(Object::toString, BytesRef.class), + TEXT(Object::toString, BytesRef.class), + IP(StringUtils::parseIP, BytesRef.class), + VERSION(v -> new Version(v).toBytesRef(), BytesRef.class), + NULL(s -> null, Void.class), + DATETIME(x -> x == null ? null : DateFormatters.from(UTC_DATE_TIME_FORMATTER.parse(x)).toInstant().toEpochMilli(), Long.class), + BOOLEAN(Booleans::parseBoolean, Boolean.class); + + private static final Map LOOKUP = new HashMap<>(); + + static { + for (Type value : Type.values()) { + LOOKUP.put(value.name(), value); + } + // widen smaller types + LOOKUP.put("SHORT", INTEGER); + LOOKUP.put("BYTE", INTEGER); + + // add also the types with short names + LOOKUP.put("I", INTEGER); + LOOKUP.put("L", LONG); + LOOKUP.put("UL", UNSIGNED_LONG); + LOOKUP.put("D", DOUBLE); + LOOKUP.put("K", KEYWORD); + LOOKUP.put("S", KEYWORD); + LOOKUP.put("STRING", KEYWORD); + LOOKUP.put("TXT", TEXT); + LOOKUP.put("N", NULL); + LOOKUP.put("DATE", DATETIME); + LOOKUP.put("DT", DATETIME); + LOOKUP.put("V", VERSION); + } + + private final Function converter; + private final Class clazz; + + Type(Function converter, Class clazz) { + this.converter = converter; + this.clazz = clazz; + } + + public static Type asType(String name) { + return LOOKUP.get(name.toUpperCase(Locale.ROOT)); + } + + public static Type asType(ElementType elementType) { + return switch (elementType) { + case INT -> INTEGER; + case LONG -> LONG; + case DOUBLE -> DOUBLE; + case NULL -> NULL; + case BYTES_REF -> KEYWORD; + case BOOLEAN -> BOOLEAN; + case DOC -> throw new IllegalArgumentException("can't assert on doc blocks"); + case UNKNOWN -> throw new IllegalArgumentException("Unknown block types cannot be handled"); + }; + } + + Object convert(String value) { + if (value == null) { + return null; + } + return converter.apply(value); + } + + Class clazz() { + return clazz; + } + } + + record ActualResults( + List columnNames, + List columnTypes, + List dataTypes, + List pages, + Map> responseHeaders + ) { + List> values() { + return EsqlQueryResponse.pagesToValues(dataTypes(), pages); + } + } + + static void logMetaData(List actualColumnNames, List actualColumnTypes, Logger logger) { + // header + StringBuilder sb = new StringBuilder(); + StringBuilder column = new StringBuilder(); + + for (int i = 0; i < actualColumnNames.size(); i++) { + if (i > 0) { + sb.append(" | "); + } + column.setLength(0); + column.append(actualColumnNames.get(i)); + column.append("("); + column.append(actualColumnTypes.get(i)); + column.append(")"); + + sb.append(trimOrPad(column)); + } + + int l = sb.length(); + logger.info(sb.toString()); + sb.setLength(0); + sb.append("-".repeat(Math.max(0, l))); + + logger.info(sb.toString()); + } + + static void logData(List> values, Logger logger) { + for (List list : values) { + logger.info(rowAsString(list)); + } + } + + private static String rowAsString(List list) { + StringBuilder sb = new StringBuilder(); + StringBuilder column = new StringBuilder(); + for (int i = 0; i < list.size(); i++) { + column.setLength(0); + if (i > 0) { + sb.append(" | "); + } + sb.append(trimOrPad(column.append(list.get(i)))); + } + return sb.toString(); + } + + private static StringBuilder trimOrPad(StringBuilder buffer) { + if (buffer.length() > MAX_WIDTH) { + buffer.setLength(MAX_WIDTH - 1); + buffer.append("~"); + } else { + buffer.append(" ".repeat(Math.max(0, MAX_WIDTH - buffer.length()))); + } + return buffer; + } + + private static double scaledFloat(String value, String factor) { + double scalingFactor = Double.parseDouble(factor); + // this extra division introduces extra imprecision in the following multiplication, but this is how ScaledFloatFieldMapper works. + double scalingFactorInverse = 1d / scalingFactor; + return new BigDecimal(value).multiply(BigDecimal.valueOf(scalingFactor)).longValue() * scalingFactorInverse; + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java new file mode 100644 index 0000000000000..5a8e269ac6bc3 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java @@ -0,0 +1,339 @@ +/* + * 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; + +import org.apache.http.HttpEntity; +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.apache.logging.log4j.core.config.plugins.util.PluginManager; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.cluster.ClusterModule; +import org.elasticsearch.common.CheckedBiFunction; +import org.elasticsearch.common.logging.LogConfigurator; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.XContent; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.ql.TestUtils; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.Strings.delimitedListToStringArray; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.esql.CsvTestUtils.multiValuesAwareCsvToStringArray; + +public class CsvTestsDataLoader { + private static final TestsDataset EMPLOYEES = new TestsDataset("employees", "mapping-default.json", "employees.csv"); + private static final TestsDataset HOSTS = new TestsDataset("hosts", "mapping-hosts.json", "hosts.csv"); + private static final TestsDataset APPS = new TestsDataset("apps", "mapping-apps.json", "apps.csv"); + private static final TestsDataset LANGUAGES = new TestsDataset("languages", "mapping-languages.json", "languages.csv"); + private static final TestsDataset UL_LOGS = new TestsDataset("ul_logs", "mapping-ul_logs.json", "ul_logs.csv"); + + public static final Map CSV_DATASET_MAP = Map.of( + EMPLOYEES.indexName, + EMPLOYEES, + HOSTS.indexName, + HOSTS, + APPS.indexName, + APPS, + LANGUAGES.indexName, + LANGUAGES, + UL_LOGS.indexName, + UL_LOGS + ); + + private static final EnrichConfig LANGUAGES_ENRICH = new EnrichConfig("languages_policy", "enricy-policy-languages.json"); + + public static final List ENRICH_POLICIES = List.of(LANGUAGES_ENRICH); + + /** + *

+ * Loads spec data on a local ES server. + *

+ *

+ * Accepts an URL as first argument, eg. http://localhost:9200 or http://user:pass@localhost:9200 + *

+ *

+ * If no arguments are specified, the default URL is http://localhost:9200 without authentication + *

+ *

+ * It also supports HTTPS + *

+ * @param args the URL to connect + * @throws IOException + */ + public static void main(String[] args) throws IOException { + // Need to setup the log configuration properly to avoid messages when creating a new RestClient + PluginManager.addPackage(LogConfigurator.class.getPackage().getName()); + LogConfigurator.configureESLogging(); + + String protocol = "http"; + String host = "localhost"; + int port = 9200; + String username = null; + String password = null; + if (args.length > 0) { + URL url = URI.create(args[0]).toURL(); + protocol = url.getProtocol(); + host = url.getHost(); + port = url.getPort(); + if (port < 0 || port > 65535) { + throw new IllegalArgumentException("Please specify a valid port [0 - 65535], found [" + port + "]"); + } + String userInfo = url.getUserInfo(); + if (userInfo != null) { + if (userInfo.contains(":") == false || userInfo.split(":").length != 2) { + throw new IllegalArgumentException("Invalid user credentials [username:password], found [" + userInfo + "]"); + } + String[] userPw = userInfo.split(":"); + username = userPw[0]; + password = userPw[1]; + } + } + RestClientBuilder builder = RestClient.builder(new HttpHost(host, port, protocol)); + if (username != null) { + CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + builder = builder.setHttpClientConfigCallback( + httpClientBuilder -> httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider) + ); + } + + try (RestClient client = builder.build()) { + loadDataSetIntoEs(client); + } + } + + public static void loadDataSetIntoEs(RestClient client) throws IOException { + loadDataSetIntoEs(client, LogManager.getLogger(CsvTestsDataLoader.class)); + } + + public static void loadDataSetIntoEs(RestClient client, Logger logger) throws IOException { + for (var dataSet : CSV_DATASET_MAP.values()) { + load(client, dataSet.indexName, "/" + dataSet.mappingFileName, "/" + dataSet.dataFileName, logger); + } + for (var policy : ENRICH_POLICIES) { + loadEnrichPolicy(client, policy.policyName, policy.policyFileName, logger); + } + } + + private static void loadEnrichPolicy(RestClient client, String policyName, String policyFileName, Logger logger) throws IOException { + URL policyMapping = CsvTestsDataLoader.class.getResource("/" + policyFileName); + if (policyMapping == null) { + throw new IllegalArgumentException("Cannot find resource " + policyFileName); + } + String entity = readTextFile(policyMapping); + Request request = new Request("PUT", "/_enrich/policy/" + policyName); + request.setJsonEntity(entity); + client.performRequest(request); + + request = new Request("POST", "/_enrich/policy/" + policyName + "/_execute"); + client.performRequest(request); + } + + private static void load(RestClient client, String indexName, String mappingName, String dataName, Logger logger) throws IOException { + URL mapping = CsvTestsDataLoader.class.getResource(mappingName); + if (mapping == null) { + throw new IllegalArgumentException("Cannot find resource " + mappingName); + } + URL data = CsvTestsDataLoader.class.getResource(dataName); + if (data == null) { + throw new IllegalArgumentException("Cannot find resource " + dataName); + } + createTestIndex(client, indexName, readTextFile(mapping)); + loadCsvData(client, indexName, data, CsvTestsDataLoader::createParser, logger); + } + + private static void createTestIndex(RestClient client, String indexName, String mapping) throws IOException { + ESRestTestCase.createIndex(client, indexName, null, mapping, null); + } + + public static String readTextFile(URL resource) throws IOException { + try (BufferedReader reader = TestUtils.reader(resource)) { + StringBuilder b = new StringBuilder(); + String line; + while ((line = reader.readLine()) != null) { + b.append(line); + } + return b.toString(); + } + } + + @SuppressWarnings("unchecked") + private static void loadCsvData( + RestClient client, + String indexName, + URL resource, + CheckedBiFunction p, + Logger logger + ) throws IOException { + Request request = new Request("POST", "/_bulk"); + StringBuilder builder = new StringBuilder(); + try (BufferedReader reader = org.elasticsearch.xpack.ql.TestUtils.reader(resource)) { + String line; + int lineNumber = 1; + String[] columns = null; // list of column names. If one column name contains dot, it is a subfield and its value will be null + List subFieldsIndices = new ArrayList<>(); // list containing the index of a subfield in "columns" String[] + + while ((line = reader.readLine()) != null) { + line = line.trim(); + // ignore comments + if (line.isEmpty() == false && line.startsWith("//") == false) { + String[] entries = multiValuesAwareCsvToStringArray(line, lineNumber); + // the schema row + if (columns == null) { + columns = new String[entries.length]; + for (int i = 0; i < entries.length; i++) { + int split = entries[i].indexOf(":"); + String name, typeName; + + if (split < 0) { + throw new IllegalArgumentException( + "A type is always expected in the schema definition; found " + entries[i] + ); + } else { + name = entries[i].substring(0, split).trim(); + if (name.indexOf(".") < 0) { + typeName = entries[i].substring(split + 1).trim(); + if (typeName.length() == 0) { + throw new IllegalArgumentException( + "A type is always expected in the schema definition; found " + entries[i] + ); + } + } else {// if it's a subfield, ignore it in the _bulk request + name = null; + subFieldsIndices.add(i); + } + } + columns[i] = name; + } + } + // data rows + else { + if (entries.length != columns.length) { + throw new IllegalArgumentException( + format( + null, + "Error line [{}]: Incorrect number of entries; expected [{}] but found [{}]", + lineNumber, + columns.length, + entries.length + ) + ); + } + StringBuilder row = new StringBuilder(); + String idField = null; + for (int i = 0; i < entries.length; i++) { + // ignore values that belong to subfields and don't add them to the bulk request + if (subFieldsIndices.contains(i) == false) { + if ("".equals(entries[i])) { + // Value is null, skip + continue; + } + if ("_id".equals(columns[i])) { + // Value is an _id + idField = entries[i]; + continue; + } + try { + // add a comma after the previous value, only when there was actually a value before + if (i > 0 && row.length() > 0) { + row.append(","); + } + if (entries[i].contains(",")) {// multi-value + StringBuilder rowStringValue = new StringBuilder("["); + for (String s : delimitedListToStringArray(entries[i], ",")) { + rowStringValue.append("\"" + s + "\","); + } + // remove the last comma and put a closing bracket instead + rowStringValue.replace(rowStringValue.length() - 1, rowStringValue.length(), "]"); + entries[i] = rowStringValue.toString(); + } else { + entries[i] = "\"" + entries[i] + "\""; + } + row.append("\"" + columns[i] + "\":" + entries[i]); + } catch (Exception e) { + throw new IllegalArgumentException( + format( + null, + "Error line [{}]: Cannot parse entry [{}] with value [{}]", + lineNumber, + i + 1, + entries[i] + ), + e + ); + } + } + } + String idPart = idField != null ? "\", \"_id\": \"" + idField : ""; + builder.append("{\"index\": {\"_index\":\"" + indexName + idPart + "\"}}\n"); + builder.append("{" + row + "}\n"); + } + } + lineNumber++; + } + builder.append("\n"); + } + + request.setJsonEntity(builder.toString()); + request.addParameter("refresh", "wait_for"); + Response response = client.performRequest(request); + if (response.getStatusLine().getStatusCode() == 200) { + HttpEntity entity = response.getEntity(); + try (InputStream content = entity.getContent()) { + XContentType xContentType = XContentType.fromMediaType(entity.getContentType().getValue()); + Map result = XContentHelper.convertToMap(xContentType.xContent(), content, false); + Object errors = result.get("errors"); + if (Boolean.FALSE.equals(errors)) { + logger.info("Data loading OK"); + request = new Request("POST", "/" + indexName + "/_forcemerge?max_num_segments=1"); + response = client.performRequest(request); + if (response.getStatusLine().getStatusCode() != 200) { + logger.warn("Force-merge to 1 segment failed: " + response.getStatusLine()); + } else { + logger.info("Forced-merge to 1 segment"); + } + } else { + logger.error("Data loading FAILED"); + } + } + } else { + logger.error("Error loading data: " + response.getStatusLine()); + } + } + + private static XContentParser createParser(XContent xContent, InputStream data) throws IOException { + NamedXContentRegistry contentRegistry = new NamedXContentRegistry(ClusterModule.getNamedXWriteables()); + XContentParserConfiguration config = XContentParserConfiguration.EMPTY.withRegistry(contentRegistry) + .withDeprecationHandler(LoggingDeprecationHandler.INSTANCE); + return xContent.createParser(config, data); + } + + public record TestsDataset(String indexName, String mappingFileName, String dataFileName) {} + + public record EnrichConfig(String policyName, String policyFileName) {} +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/apps.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/apps.csv new file mode 100644 index 0000000000000..f3db534e19091 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/apps.csv @@ -0,0 +1,15 @@ +id:integer,version:version,name:keyword,_id:keyword +1,1,aaaaa,1 +2,2.1,bbbbb,2 +3,2.3.4,ccccc,3 +4,2.12.0,ddddd,4 +5,1.11.0,eeeee,5 +6,5.2.9,fffff,6 +7,5.2.9-SNAPSHOT,ggggg,7 +8,1.2.3.4,hhhhh,8 +9,bad,iiiii,9 +10,5.2.9,jjjjj,10 +11,,kkkkk,11 +12,1.2.3.4,aaaaa,12 +13,,lllll,13 +14,5.2.9,mmmmm,14 diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/blog-ignoreCsvTests.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/blog-ignoreCsvTests.csv-spec new file mode 100644 index 0000000000000..f670738bd3c49 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/blog-ignoreCsvTests.csv-spec @@ -0,0 +1,31 @@ +2023-08-08.full-blown-query + + FROM employees +| WHERE still_hired == true +| EVAL hired = DATE_FORMAT(hire_date, "YYYY") +| STATS avg_salary = AVG(salary) BY languages +| EVAL avg_salary = ROUND(avg_salary) +| EVAL lang_code = TO_STRING(languages) +| ENRICH languages_policy ON lang_code WITH lang = language_name +| WHERE lang IS NOT NULL +| KEEP avg_salary, lang +| SORT avg_salary ASC +| LIMIT 3 +; + +avg_salary:d | lang:k +43760.0 | Spanish +48644.0 | French +48832.0 | German +; + +2023-08-08.multiple-agg + + FROM employees +| STATS c = COUNT(emp_no) BY languages +| STATS largest_group = MAX(c) +; + +largest_group:l +21 +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/boolean.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/boolean.csv-spec new file mode 100644 index 0000000000000..6489d363ed77d --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/boolean.csv-spec @@ -0,0 +1,215 @@ +simple +from employees | sort emp_no | keep emp_no, still_hired | limit 3; + +emp_no:integer | still_hired:boolean +10001 | true +10002 | true +10003 | false +; + +directFilter +from employees | sort emp_no | where still_hired | keep emp_no | limit 3; + +emp_no:integer +10001 +10002 +10004 +; + +sort +from employees | sort still_hired, emp_no | keep emp_no, still_hired | limit 3; + +emp_no:integer | still_hired:boolean +10003 | false +10006 | false +10009 | false +; + +statsBy +from employees | stats avg(salary) by still_hired | sort still_hired; + +avg(salary):double | still_hired:boolean +50625.163636363635 | false + 45343.8 | true +; + +statsByAlwaysTrue +from employees | where first_name is not null | eval always_true = starts_with(first_name, "") | stats avg(salary) by always_true; + +avg(salary):double | always_true:boolean + 48353.72222222222 | true +; + +statsByAlwaysFalse +from employees | where first_name is not null | eval always_false = starts_with(first_name, "nonestartwiththis") | stats avg(salary) by always_false; + +avg(salary):double | always_false:boolean + 48353.72222222222 | false +; + +in +from employees | keep emp_no, is_rehired, still_hired | where is_rehired in (still_hired, true) | where is_rehired != still_hired; + +emp_no:integer |is_rehired:boolean |still_hired:boolean +10021 |true |false +10029 |true |false +10033 |true |false +10075 |true |false +10088 |true |false +; + +trueTrue +row lhs=true, rhs=true | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +true | true +; + +trueFalse +row lhs=true, rhs=false | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +false | true +; + +trueNull +row lhs=true, rhs=null | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +null | true +; + +falseTrue +row lhs=false, rhs=true | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +false | true +; + +falseFalse +row lhs=false, rhs=false | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +false | false +; + +falseNull +row lhs=false, rhs=null | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +false | null +; + +nullTrue +row lhs=null, rhs=true | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +null | true +; + +nullFalse +row lhs=null, rhs=false | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +false | null +; + +nullNull +row lhs=null, rhs=null | eval aa=lhs AND rhs, oo=lhs OR rhs | keep aa, oo; + +aa:boolean | oo:boolean +null | null +; + +notTrue +row v=true | eval v=NOT v | keep v; + +v:boolean +false +; + +notFalse +row v=false | eval v=NOT v | keep v; + +v:boolean +true +; + +notNull +row v=null | eval v=NOT v | keep v; + +v:boolean +null +; + +convertFromBoolean +row tf = [true, false] | eval tt = to_boolean(true), ff = to_boolean(false), ttff = to_boolean(tf); + +tf:boolean |tt:boolean |ff:boolean |ttff:boolean +[true, false] |true |false |[true, false] +; + +convertFromString +from employees | keep emp_no, is_rehired, first_name | eval rehired_str = to_string(is_rehired) | eval rehired_bool = to_boolean(rehired_str) | eval all_false = to_boolean(first_name) | drop first_name | limit 5; +emp_no:integer |is_rehired:boolean |rehired_str:keyword |rehired_bool:boolean |all_false:boolean +10001 |[false, true] |[false, true] |[false, true] |false +10002 |[false, false] |[false, false] |[false, false] |false +10003 |null |null |null |false +10004 |true |true |true |false +10005 |[false, false, false, true]|[false, false, false, true]|[false, false, false, true] |false +; + +convertFromStringForDocs +// tag::to_boolean[] +ROW str = ["true", "TRuE", "false", "", "yes", "1"] +| EVAL bool = TO_BOOLEAN(str) +// end::to_boolean[] +; + +// tag::to_boolean-result[] +str:keyword | bool:boolean +["true", "TRuE", "false", "", "yes", "1"] | [true, true, false, false, false, false] +// end::to_boolean-result[] +; + +convertFromDouble +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; + +emp_no:integer |height:double |double2bool:boolean +10036 |1.61 |true +10037 |2.0 |false +10038 |1.53 |true +; + +convertFromUnsignedLong +row ul = [9223372036854775808, 9223372036854775807, 1, 0] | eval bool = to_bool(ul); + + ul:ul | bool:boolean +[9223372036854775808, 9223372036854775807, 1, 0]|[true, true, true, false] +; + +convertFromIntAndLong +from employees | keep emp_no, salary_change* | eval int2bool = to_boolean(salary_change.int), long2bool = to_boolean(salary_change.long) | limit 10; + +emp_no:integer |salary_change:double |salary_change.int:integer |salary_change.long:long |int2bool:boolean |long2bool:boolean +10001 |1.19 |1 |1 |true |true +10002 |[-7.23, 11.17] |[-7, 11] |[-7, 11] |[true, true] |[true, true] +10003 |[12.82, 14.68] |[12, 14] |[12, 14] |[true, true] |[true, true] +10004 |[-0.35, 1.13, 3.65, 13.48]|[0, 1, 3, 13] |[0, 1, 3, 13] |[false, true, true, true] |[false, true, true, true] +10005 |[-2.14, 13.07] |[-2, 13] |[-2, 13] |[true, true] |[true, true] +10006 |-3.9 |-3 |-3 |true |true +10007 |[-7.06, 0.57, 1.99] |[-7, 0, 1] |[-7, 0, 1] |[true, false, true] |[true, false, true] +10008 |[-2.92, 0.75, 3.54, 12.68]|[-2, 0, 3, 12] |[-2, 0, 3, 12] |[true, false, true, true] |[true, false, true, true] +10009 |null |null |null |null |null +10010 |[-6.77, 4.69, 5.05, 12.15]|[-6, 4, 5, 12] |[-6, 4, 5, 12] |[true, true, true, true] |[true, true, true, true] +; + +// short and byte aren't actually tested, these are loaded as int blocks atm +convertFromByteAndShort +from employees | eval byte2bool = to_boolean(languages.byte), short2bool = to_boolean(languages.short) | where emp_no in (10019, 10020, 10030) | keep emp_no, languages, *2bool; + +emp_no:integer |languages:integer |byte2bool:boolean |short2bool:boolean +10019 |1 |true |true +10020 |null |null |null +10030 |3 |true |true +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/comparison.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/comparison.csv-spec new file mode 100644 index 0000000000000..b07259e01ddf2 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/comparison.csv-spec @@ -0,0 +1,116 @@ +intToInt +from employees +| where emp_no < 10002 +| keep emp_no; + +emp_no:integer +10001 +; + +longToLong +from employees +| where languages.long < avg_worked_seconds +| limit 1 +| keep emp_no; + +emp_no:integer +10001 +; + +doubleToDouble +from employees +| where height < 10.0 +| limit 1 +| keep emp_no; + +emp_no:integer +10001 +; + +intToLong +from employees +| where emp_no > languages.long +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10001 +; + +longToInt +from employees +| where languages.long < emp_no +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10001 +; + +doubleToLong +from employees +| where 2.0 > languages.long +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10005 +; + +longToDouble +from employees +| where languages.long < 2.0 +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10005 +; + +intToLong +from employees +| where 2.0 > languages +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10005 +; + +intToDouble +from employees +| where languages < 2.0 +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10005 +; + +boolToBool +from employees +| where still_hired == false +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10003 +; + +dateToDate +from employees +| where birth_date < hire_date +| keep emp_no +| sort emp_no +| limit 1; + +emp_no:integer +10001 +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/conditional.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/conditional.csv-spec new file mode 100644 index 0000000000000..7bc5f4b13ae0e --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/conditional.csv-spec @@ -0,0 +1,149 @@ +twoConditionsWithDefault +from employees +| eval type = case( + languages <= 1, "monolingual", + languages <= 2, "bilingual", + "polyglot") +| keep emp_no, type +| limit 10; + +emp_no:integer | type:keyword +10001 |bilingual +10002 |polyglot +10003 |polyglot +10004 |polyglot +10005 |monolingual +10006 |polyglot +10007 |polyglot +10008 |bilingual +10009 |monolingual +10010 |polyglot +; + +singleCondition +from employees +| eval g = case(gender == "F", true) +| keep gender, g +| limit 10; + +gender:keyword |g:boolean +M |null +F |true +M |null +M |null +M |null +F |true +F |true +M |null +F |true +null |null +; + +conditionIsNull +from employees +| eval g = case( + gender == "F", 1, + languages > 1, 2, + 3) +| keep gender, languages, g +| limit 25; + +gender:keyword |languages:integer|g:integer +M |2 |2 +F |5 |1 +M |4 |2 +M |5 |2 +M |1 |3 +F |3 |1 +F |4 |1 +M |2 |2 +F |1 |1 +null |4 |2 +null |5 |2 +null |5 |2 +null |1 |3 +null |5 |2 +null |5 |2 +null |2 |2 +null |2 |2 +null |2 |2 +null |1 |3 +M |null |3 +M |null |3 +M |null |3 +F |null |1 +F |null |1 +M |null |3 +; + +nullValue +from employees +| eval g = case(gender == "F", 1 + null, 10) +| keep gender, g +| limit 5; + +gender:keyword |g:integer +M |10 +F |null +M |10 +M |10 +M |10 +; + +isNull +from employees +| where gender is null +| sort first_name +| keep first_name, gender +| limit 3; + +first_name:keyword|gender:keyword +Berni |null +Cristinel |null +Duangkaew |null +; + +notIsNull +from employees +| where gender is not null +| sort first_name +| keep first_name, gender +| limit 3; + +first_name:keyword|gender:keyword +Alejandro |F +Amabile |M +Anneke |F +; + +isNullForDocs +// tag::is-null[] +FROM employees +| WHERE birth_date IS NULL +| KEEP first_name, last_name +| SORT first_name +| LIMIT 3 +// end::is-null[] +; + +// tag::is-null-result[] +first_name:keyword|last_name:keyword +Basil |Tramer +Florian |Syrotiuk +Lucien |Rosenbaum +// end::is-null-result[] +; + +isNotNullForDocs +// tag::is-not-null[] +FROM employees +| WHERE is_rehired IS NOT NULL +| STATS count(emp_no) +// end::is-not-null[] +; + +// tag::is-not-null-result[] +count(emp_no):long +84 +// end::is-not-null-result[] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec new file mode 100644 index 0000000000000..334884ac6f4bb --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec @@ -0,0 +1,455 @@ +simple +from employees | sort emp_no | keep emp_no, hire_date | limit 1; + +emp_no:integer | hire_date:date +10001 | 1986-06-26T00:00:00.000Z +; + + +sort +from employees | sort hire_date | keep emp_no, hire_date | limit 5; + +emp_no:integer | hire_date:date +10009 | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z +; + + + +sortDesc +from employees | sort hire_date desc | keep emp_no, hire_date | limit 5; + +emp_no:integer | hire_date:date +10019 | 1999-04-30T00:00:00.000Z +10024 | 1997-05-19T00:00:00.000Z +10093 | 1996-11-05T00:00:00.000Z +10084 | 1995-12-15T00:00:00.000Z +10022 | 1995-08-22T00:00:00.000Z +; + + +evalAssign +from employees | sort hire_date | eval x = hire_date | keep emp_no, x | limit 5; + +emp_no:integer | x:date +10009 | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z +; + + + +evalDateFormat +from employees | sort hire_date | eval x = date_format(hire_date), y = date_format(hire_date, "YYYY-MM-dd") | keep emp_no, x, y | limit 5; + +emp_no:integer | x:keyword | y:keyword +10009 | 1985-02-18T00:00:00.000Z | 1985-02-18 +10048 | 1985-02-24T00:00:00.000Z | 1985-02-24 +10098 | 1985-05-13T00:00:00.000Z | 1985-05-13 +10076 | 1985-07-09T00:00:00.000Z | 1985-07-09 +10061 | 1985-09-17T00:00:00.000Z | 1985-09-17 +; + + +compareToString +from employees | where hire_date < "1985-03-01T00:00:00Z" | keep emp_no, hire_date; + +emp_no:integer | hire_date:date +10009 | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z +; + + +compareToDatetime +from employees | where hire_date < birth_date | keep emp_no, hire_date; + +emp_no:integer | hire_date:date +; + + +nullDate +from employees | where emp_no == 10040 | eval x = date_format(birth_date) | keep emp_no, birth_date, hire_date, x; + +emp_no:integer | birth_date:date | hire_date:date | x:keyword +10040 | null | 1993-02-14T00:00:00.000Z | null +; + +minMax +from employees | stats min = min(hire_date), max = max(hire_date); + +min:date | max:date +1985-02-18T00:00:00.000Z | 1999-04-30T00:00:00.000Z +; + +evalDateTruncIntervalExpressionPeriod +from employees | sort hire_date | eval x = date_trunc(hire_date, 1 month) | keep emp_no, hire_date, x | limit 5; + +emp_no:integer | hire_date:date | x:date +10009 | 1985-02-18T00:00:00.000Z | 1985-02-01T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z | 1985-02-01T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z | 1985-05-01T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z | 1985-07-01T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z | 1985-09-01T00:00:00.000Z +; + +evalDateTruncIntervalExpressionDuration +from employees | sort hire_date | eval x = date_trunc(hire_date, 240 hours) | keep emp_no, hire_date, x | limit 5; + +emp_no:integer | hire_date:date | x:date +10009 | 1985-02-18T00:00:00.000Z | 1985-02-11T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z | 1985-02-21T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z | 1985-05-12T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z | 1985-07-01T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z | 1985-09-09T00:00:00.000Z +; + +evalDateTruncWeeklyInterval +from employees | sort hire_date | eval x = date_trunc(hire_date, 1 week) | keep emp_no, hire_date, x | limit 5; + +emp_no:integer | hire_date:date | x:date +10009 | 1985-02-18T00:00:00.000Z | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z | 1985-02-18T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z | 1985-05-13T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z | 1985-07-08T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z | 1985-09-16T00:00:00.000Z +; + +evalDateTruncQuarterlyInterval +from employees | sort hire_date | eval x = date_trunc(hire_date, 3 month) | keep emp_no, hire_date, x | limit 5; + +emp_no:integer | hire_date:date | x:date +10009 | 1985-02-18T00:00:00.000Z | 1985-01-01T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z | 1985-01-01T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z | 1985-04-01T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z | 1985-07-01T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z | 1985-07-01T00:00:00.000Z +; + +evalDateTruncNullDate +from employees | where emp_no == 10040 | eval x = date_trunc(birth_date, 1 day) | keep emp_no, birth_date, x; + +emp_no:integer | birth_date:date | x:date +10040 | null | null +; + +evalDateTruncGrouping +from employees | eval y = date_trunc(hire_date, 1 year) | stats count(emp_no) by y | sort y | keep y, count(emp_no) | limit 5; + +y:date | count(emp_no):long +1985-01-01T00:00:00.000Z | 11 +1986-01-01T00:00:00.000Z | 11 +1987-01-01T00:00:00.000Z | 15 +1988-01-01T00:00:00.000Z | 9 +1989-01-01T00:00:00.000Z | 13 +; + +in +from employees | eval x = date_trunc(hire_date, 1 year) | where birth_date not in (x, hire_date) | keep x, hire_date | sort x desc | limit 4; + +x:date |hire_date:date +1999-01-01T00:00:00.000Z|1999-04-30T00:00:00.000Z +1997-01-01T00:00:00.000Z|1997-05-19T00:00:00.000Z +1996-01-01T00:00:00.000Z|1996-11-05T00:00:00.000Z +1995-01-01T00:00:00.000Z|1995-01-27T00:00:00.000Z +; + +convertFromDatetime +from employees| keep birth_date | eval bd = to_datetime(birth_date) | limit 2; + +birth_date:date |bd:date +1953-09-02T00:00:00.000Z|1953-09-02T00:00:00.000Z +1964-06-02T00:00:00.000Z|1964-06-02T00:00:00.000Z +; + +convertFromString +// tag::to_datetime-str[] +ROW string = ["1953-09-02T00:00:00.000Z", "1964-06-02T00:00:00.000Z", "1964-06-02 00:00:00"] +| EVAL datetime = TO_DATETIME(string) +// end::to_datetime-str[] +; +warning:Line 2:19: evaluation of [TO_DATETIME(string)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.IllegalArgumentException: failed to parse date field [1964-06-02 00:00:00] with format [yyyy-MM-dd'T'HH:mm:ss.SSS'Z'] + +// tag::to_datetime-str-result[] +string:keyword |datetime:date +["1953-09-02T00:00:00.000Z", "1964-06-02T00:00:00.000Z", "1964-06-02 00:00:00"]|[1953-09-02T00:00:00.000Z, 1964-06-02T00:00:00.000Z] +// end::to_datetime-str-result[] +; + +convertFromUnsignedLong +row ul = [9223372036854775808, 520128000000] | eval dt = to_datetime(ul); +warning:Line 1:58: evaluation of [to_datetime(ul)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [9223372036854775808] out of [long] range + + ul:ul | dt:date +[9223372036854775808, 520128000000]|1986-06-26T00:00:00.000Z +; + +convertFromLong +row long = [501379200000, 520128000000] | eval dt = to_datetime(long); + +long:long |dt:date +[501379200000, 520128000000] |[1985-11-21T00:00:00.000Z, 1986-06-26T00:00:00.000Z] +; + +convertFromDouble +row dbl = [501379200000.1, 520128000000.2] | eval dt = to_datetime(dbl); + +dbl:double |dt:date +[5.013792000001E11, 5.201280000002E11]|[1985-11-21T00:00:00.000Z, 1986-06-26T00:00:00.000Z] +; + +convertFromInt +row int = [501379200, 520128000] | eval dt = to_datetime(int); + +int:integer |dt:date +[501379200, 520128000]|[1970-01-06T19:16:19.200Z, 1970-01-07T00:28:48.000Z] +; + +// TODO: add a -1, once https://github.com/elastic/elasticsearch-internal/issues/1203 is fixed +convertFromIntForDocs +// tag::to_datetime-int[] +ROW int = [0, 1] +| EVAL dt = TO_DATETIME(int) +// end::to_datetime-int[] +; + +// tag::to_datetime-int-result[] +int:integer |dt:date +[0, 1] |[1970-01-01T00:00:00.000Z, 1970-01-01T00:00:00.001Z] +// end::to_datetime-int-result[] +; + +autoBucketSimpleMonth +// tag::auto_bucket_month[] +ROW date=TO_DATETIME("1985-07-09T00:00:00.000Z") +| EVAL bucket=AUTO_BUCKET(date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") +// end::auto_bucket_month[] +; + +// tag::auto_bucket_month-result[] + date:datetime | bucket:datetime +1985-07-09T00:00:00.000Z | 1985-07-01T00:00:00.000Z +// end::auto_bucket_month-result[] +; + +autoBucketSimpleWeek +// tag::auto_bucket_week[] +ROW date=TO_DATETIME("1985-07-09T00:00:00.000Z") +| EVAL bucket=AUTO_BUCKET(date, 100, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") +// end::auto_bucket_week[] +; + +// tag::auto_bucket_week-result[] + date:datetime | bucket:datetime +1985-07-09T00:00:00.000Z | 1985-07-08T00:00:00.000Z +// end::auto_bucket_week-result[] +; + +autoBucketMonth +from employees +| where hire_date >= "1985-01-01T00:00:00Z" and hire_date < "1986-01-01T00:00:00Z" +| eval hd = auto_bucket(hire_date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") +| sort hire_date +| keep hire_date, hd; + +hire_date:date | hd:date +1985-02-18T00:00:00.000Z | 1985-02-01T00:00:00.000Z +1985-02-24T00:00:00.000Z | 1985-02-01T00:00:00.000Z +1985-05-13T00:00:00.000Z | 1985-05-01T00:00:00.000Z +1985-07-09T00:00:00.000Z | 1985-07-01T00:00:00.000Z +1985-09-17T00:00:00.000Z | 1985-09-01T00:00:00.000Z +1985-10-14T00:00:00.000Z | 1985-10-01T00:00:00.000Z +1985-10-20T00:00:00.000Z | 1985-10-01T00:00:00.000Z +1985-11-19T00:00:00.000Z | 1985-11-01T00:00:00.000Z +1985-11-20T00:00:00.000Z | 1985-11-01T00:00:00.000Z +1985-11-20T00:00:00.000Z | 1985-11-01T00:00:00.000Z +1985-11-21T00:00:00.000Z | 1985-11-01T00:00:00.000Z +; + +autoBucketWeek +from employees +| where hire_date >= "1985-01-01T00:00:00Z" and hire_date < "1986-01-01T00:00:00Z" +| eval hd = auto_bucket(hire_date, 55, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") +| sort hire_date +| keep hire_date, hd; + +hire_date:date | hd:date +1985-02-18T00:00:00.000Z | 1985-02-18T00:00:00.000Z +1985-02-24T00:00:00.000Z | 1985-02-18T00:00:00.000Z +1985-05-13T00:00:00.000Z | 1985-05-13T00:00:00.000Z +1985-07-09T00:00:00.000Z | 1985-07-08T00:00:00.000Z +1985-09-17T00:00:00.000Z | 1985-09-16T00:00:00.000Z +1985-10-14T00:00:00.000Z | 1985-10-14T00:00:00.000Z +1985-10-20T00:00:00.000Z | 1985-10-14T00:00:00.000Z +1985-11-19T00:00:00.000Z | 1985-11-18T00:00:00.000Z +1985-11-20T00:00:00.000Z | 1985-11-18T00:00:00.000Z +1985-11-20T00:00:00.000Z | 1985-11-18T00:00:00.000Z +1985-11-21T00:00:00.000Z | 1985-11-18T00:00:00.000Z +; + +now +row a = now() | eval x = a == now(), y = substring(date_format(a, "yyyy"), 0, 2) | keep x, y; + +x:boolean | y:keyword +true | 20 +; + +born_before_today +from employees | where birth_date < now() | sort emp_no asc | keep emp_no, birth_date| limit 1; + +emp_no:integer | birth_date:date +10001 | 1953-09-02T00:00:00Z +; + + +born_after_today +from employees | where birth_date > now() | sort emp_no asc | keep emp_no, birth_date| limit 1; + +emp_no:integer | birth_date:date +; + + +autoBucketMonthInAgg +// tag::auto_bucket_in_agg[] +FROM employees +| WHERE hire_date >= "1985-01-01T00:00:00Z" AND hire_date < "1986-01-01T00:00:00Z" +| EVAL bucket = AUTO_BUCKET(hire_date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z") +| STATS AVG(salary) BY bucket +| SORT bucket +// end::auto_bucket_in_agg[] +; + +// tag::auto_bucket_in_agg-result[] +AVG(salary):double | bucket:date + 46305.0 | 1985-02-01T00:00:00.000Z + 44817.0 | 1985-05-01T00:00:00.000Z + 62405.0 | 1985-07-01T00:00:00.000Z + 49095.0 | 1985-09-01T00:00:00.000Z + 51532.0 | 1985-10-01T00:00:00.000Z + 54539.75 | 1985-11-01T00:00:00.000Z +// end::auto_bucket_in_agg-result[] +; + +evalDateParseWithSimpleDate +row a = "2023-02-01" | eval b = date_parse(a, "yyyy-MM-dd") | keep b; + +b:datetime +2023-02-01T00:00:00.000Z +; + +evalDateParseWithDateTime +row a = "2023-02-01 12:15:55" | eval b = date_parse(a, "yyyy-MM-dd HH:mm:ss") | keep b; + +b:datetime +2023-02-01T12:15:55.000Z +; + +evalDateParseWithDateTimeDefaultFormat +row a = "2023-02-01T12:15:55.000Z" | eval b = date_parse(a) | keep b; + +b:datetime +2023-02-01T12:15:55.000Z +; + +evalDateParseWrongDate +row a = "2023-02-01 foo" | eval b = date_parse(a, "yyyy-MM-dd") | keep b; +warning:Line 1:37: evaluation of [date_parse(a, \"yyyy-MM-dd\")] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.IllegalArgumentException: failed to parse date field [2023-02-01 foo] with format [yyyy-MM-dd] + +b:datetime +null +; + +evalDateParseNotMatching +row a = "2023-02-01" | eval b = date_parse(a, "yyyy-MM") | keep b; +warning:Line 1:33: evaluation of [date_parse(a, \"yyyy-MM\")] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.IllegalArgumentException: failed to parse date field [2023-02-01] with format [yyyy-MM] +b:datetime +null +; + +evalDateParseNotMatching2 +row a = "2023-02-01" | eval b = date_parse(a, "yyyy-MM-dd HH:mm:ss") | keep b; +warning:Line 1:33: evaluation of [date_parse(a, \"yyyy-MM-dd HH:mm:ss\")] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.IllegalArgumentException: failed to parse date field [2023-02-01] with format [yyyy-MM-dd HH:mm:ss] + +b:datetime +null +; + +evalDateParseNullPattern +row a = "2023-02-01" | eval b = date_parse(a, null) | keep b; + +b:datetime +null +; + +evalDateParseDynamic +from employees | where emp_no == 10039 or emp_no == 10040 | sort emp_no +| eval birth_date_string = date_format(birth_date, "yyyy-MM-dd") +| eval new_date = date_parse(birth_date_string, "yyyy-MM-dd") | eval bool = new_date == birth_date | keep emp_no, new_date, birth_date, bool; + +emp_no:integer | new_date:datetime | birth_date:datetime | bool:boolean +10039 | 1959-10-01 | 1959-10-01 | true +10040 | null | null | null +; + +evalDateParseDynamic2 +from employees | where emp_no >= 10047 | sort emp_no | where emp_no <= 10051 +| eval birth_date_string = date_format(birth_date, "yyyy-MM-dd") +| eval new_date = date_parse(birth_date_string, "yyyy-MM-dd") +| keep emp_no, new_date, birth_date | eval bool = new_date == birth_date; + +emp_no:integer | new_date:datetime | birth_date:datetime | bool:boolean +10047 | null | null | null +10048 | null | null | null +10049 | null | null | null +10050 | 1958-05-21T00:00:00.000Z | 1958-05-21T00:00:00.000Z | true +10051 | 1953-07-28T00:00:00.000Z | 1953-07-28T00:00:00.000Z | true +; + + +evalDateParseDynamicDateAndPattern +from employees | where emp_no == 10049 or emp_no == 10050 | sort emp_no +| eval pattern = "yyyy-MM-dd", birth_date_string = date_format(birth_date, pattern) +| eval new_date = date_parse(birth_date_string, "yyyy-MM-dd") | eval bool = new_date == birth_date | keep emp_no, new_date, birth_date, bool; + +emp_no:integer | new_date:datetime | birth_date:datetime | bool:boolean +10049 | null | null | null +10050 | 1958-05-21 | 1958-05-21 | true +; + +evalDateFormatParse +from employees | where emp_no == 10049 or emp_no == 10050 | sort emp_no +| eval new_date = date_parse(date_format(birth_date)) | eval bool = new_date == birth_date | keep emp_no, new_date, birth_date, bool; + +emp_no:integer | new_date:datetime | birth_date:datetime | bool:boolean +10049 | null | null | null +10050 | 1958-05-21T00:00:00.000Z | 1958-05-21T00:00:00.000Z | true +; + +dateFields +from employees | where emp_no == 10049 or emp_no == 10050 +| eval year = date_extract(birth_date, "year"), month = date_extract(birth_date, "month_of_year"), day = date_extract(birth_date, "day_of_month") +| keep emp_no, year, month, day; + +emp_no:integer | year:long | month:long | day:long +10049 | null | null | null +10050 | 1958 | 5 | 21 +; + + +dateFormatLocale +from employees | where emp_no == 10049 or emp_no == 10050 | sort emp_no +| eval birth_month = date_format(birth_date, "MMMM") | keep emp_no, birth_date, birth_month; + +emp_no:integer | birth_date:datetime | birth_month:keyword +10049 | null | null +10050 | 1958-05-21T00:00:00.000Z | May +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec new file mode 100644 index 0000000000000..54bc481c54b48 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec @@ -0,0 +1,157 @@ +simpleDissect +row a = "foo bar" | dissect a "%{b} %{c}"; + +a:keyword | b:keyword | c:keyword +foo bar | foo | bar +; + + +simpleDissectNoMatch +row a = "foo bar" | dissect a "%{b} %{c} baz"; + +a:keyword | b:keyword | c:keyword +foo bar | null | null +; + + +complexPattern +// tag::dissect[] +ROW a = "1953-01-23T12:15:00Z - some text - 127.0.0.1;" +| DISSECT a "%{Y}-%{M}-%{D}T%{h}:%{m}:%{s}Z - %{msg} - %{ip};" +| KEEP Y, M, D, h, m, s, msg, ip +// end::dissect[] +; + +// tag::dissect-result[] +Y:keyword | M:keyword | D:keyword | h:keyword | m:keyword | s:keyword | msg:keyword | ip:keyword +1953 | 01 | 23 | 12 | 15 | 00 | some text | 127.0.0.1 +// end::dissect-result[] +; + + +append +row a = "foo 1 bar 2 baz" | dissect a "%{+b} %{c} %{+b} %{d} %{+b}"; + +a:keyword | b:keyword | c:keyword | d:keyword +foo 1 bar 2 baz | foobarbaz | 1 | 2 +; + + +appendWithOrder +row a = "foo 1 bar 2 baz" | dissect a "%{+b/3} %{c} %{+b/2} %{d} %{+b/1}"; + +a:keyword | b:keyword | c:keyword | d:keyword +foo 1 bar 2 baz | bazbarfoo | 1 | 2 +; + + +appendSeparator +row a = "foo 1 bar 2 baz" | dissect a "%{+b} %{c} %{+b} %{d} %{+b}" append_separator=","; + +a:keyword | b:keyword | c:keyword | d:keyword +foo 1 bar 2 baz | foo,bar,baz | 1 | 2 +; + + +namedSkip +row a = "foo bar baz" | dissect a "%{b} %{?c} %{d}"; + +a:keyword | b:keyword | d:keyword +foo bar baz | foo | baz +; + + +padding +row a = "foo bar" | dissect a "%{b->} %{c}"; + +a:keyword | b:keyword | c:keyword +foo bar | foo | bar +; + + +evalDissect +from employees | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{a} %{b}" | sort emp_no asc | keep full_name, a, b | limit 3; + +full_name:keyword | a:keyword | b:keyword +Georgi Facello | Georgi | Facello +Bezalel Simmel | Bezalel | Simmel +Parto Bamford | Parto | Bamford +; + + +dissectExpression +from employees | dissect concat(first_name, " ", last_name) "%{a} %{b}" | sort emp_no asc | keep a, b | limit 3; + +a:keyword | b:keyword +Georgi | Facello +Bezalel | Simmel +Parto | Bamford +; + + +evalDissectSort +from employees | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{a} %{b}" | sort a asc | keep full_name, a, b | limit 3; + +full_name:keyword | a:keyword | b:keyword +Alejandro McAlpine | Alejandro | McAlpine +Amabile Gomatam | Amabile | Gomatam +Anneke Preusig | Anneke | Preusig +; + + +dissectStats +from employees | eval x = concat(gender, " foobar") | dissect x "%{a} %{b}" | stats n = max(emp_no) by a | keep a, n | sort a asc; + +a:keyword | n:integer +F | 10100 +M | 10097 +null | 10019 +; + + +nullOnePattern +from employees | where emp_no == 10030 | dissect first_name "%{a}" | keep first_name, a; + +first_name:keyword | a:keyword +null | null +; + + +nullTwoPatterns +from employees | where emp_no == 10030 | dissect first_name "%{a} %{b}" | keep first_name, a, b; + +first_name:keyword | a:keyword | b:keyword +null | null | null +; + + +overwriteName +from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{emp_no} %{b}" | keep full_name, emp_no, b | limit 3; + +full_name:keyword | emp_no:keyword | b:keyword +Georgi Facello | Georgi | Facello +Bezalel Simmel | Bezalel | Simmel +Parto Bamford | Parto | Bamford +; + + +overwriteNameWhere +from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{emp_no} %{b}" | where emp_no == "Bezalel" | keep full_name, emp_no, b | limit 3; + +full_name:keyword | emp_no:keyword | b:keyword +Bezalel Simmel | Bezalel | Simmel +; + + +# for now it calculates only based on the first value +multivalueInput +from employees | where emp_no <= 10006 | dissect job_positions "%{a} %{b} %{c}" | sort emp_no | keep emp_no, a, b, c; + +emp_no:integer | a:keyword | b:keyword | c:keyword +10001 | Senior | Python | Developer +10002 | Senior | Team | Lead +10003 | null | null | null +10004 | Head | Human | Resources +10005 | null | null | null +10006 | [Principal, Senior] | [Support, Team] | [Engineer, Lead] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs-ignoreCsvTests.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs-ignoreCsvTests.csv-spec new file mode 100644 index 0000000000000..ee8e5e0d784ce --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs-ignoreCsvTests.csv-spec @@ -0,0 +1,54 @@ +enrich +// tag::enrich[] +ROW language_code = "1" +| ENRICH languages_policy +// end::enrich[] +; + +// tag::enrich-result[] +language_code:keyword | language_name:keyword +1 | English +// end::enrich-result[] +; + + +enrichOn +// tag::enrich_on[] +ROW a = "1" +| ENRICH languages_policy ON a +// end::enrich_on[] +; + +// tag::enrich_on-result[] +a:keyword | language_name:keyword +1 | English +// end::enrich_on-result[] +; + + +enrichWith +// tag::enrich_with[] +ROW a = "1" +| ENRICH languages_policy ON a WITH language_name +// end::enrich_with[] +; + +// tag::enrich_with-result[] +a:keyword | language_name:keyword +1 | English +// end::enrich_with-result[] +; + + +enrichRename +// tag::enrich_rename[] +ROW a = "1" +| ENRICH languages_policy ON a WITH name = language_name +// end::enrich_rename[] +; + +// tag::enrich_rename-result[] +a:keyword | name:keyword +1 | English +// end::enrich_rename-result[] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs.csv-spec new file mode 100644 index 0000000000000..b386e546f9cb8 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/docs.csv-spec @@ -0,0 +1,411 @@ +docsDropHeight +// tag::dropheight[] +FROM employees +| DROP height +// end::dropheight[] +| LIMIT 0; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean +; + +docsDropHeightWithWildcard +// tag::dropheightwithwildcard[] +FROM employees +| DROP height* +// end::dropheightwithwildcard[] +| LIMIT 0; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean +; + +docsEval +// tag::eval[] +FROM employees +| KEEP first_name, last_name, height +| EVAL height_feet = height * 3.281, height_cm = height * 100 +// end::eval[] +| WHERE first_name == "Georgi" +| LIMIT 1; + +// tag::eval-result[] +first_name:keyword | last_name:keyword | height:double | height_feet:double | height_cm:double +Georgi |Facello | 2.03 | 6.66043 | 202.99999999999997 +// end::eval-result[] +; + +docsEvalReplace +// tag::evalReplace[] +FROM employees +| KEEP first_name, last_name, height +| EVAL height = height * 3.281 +// end::evalReplace[] +| WHERE first_name == "Georgi" +| LIMIT 1; + +// tag::evalReplace-result[] +first_name:keyword | last_name:keyword | height:double +Georgi | Facello | 6.66043 +// end::evalReplace-result[] +; + +docsLimit +// tag::limit[] +FROM employees +| LIMIT 5 +// end::limit[] +| KEEP emp_no +| SORT emp_no ASC +; + +emp_no:integer +10001 +10002 +10003 +10004 +10005 +; + +docsKeep +// tag::keep[] +FROM employees +| KEEP emp_no, first_name, last_name, height +// end::keep[] +| SORT emp_no ASC +| LIMIT 5 +; + +// tag::keep-result[] +emp_no:integer | first_name:keyword | last_name:keyword | height:double +10001 |Georgi |Facello |2.03 +10002 |Bezalel |Simmel |2.08 +10003 |Parto |Bamford |1.83 +10004 |Chirstian |Koblick |1.78 +10005 |Kyoichi |Maliniak |2.05 +// end::keep-result[] +; + +docsKeepWildcard +// tag::keepWildcard[] +FROM employees +| KEEP h* +// end::keepWildcard[] +| LIMIT 0; + +height:double | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date +; + +docsKeepDoubleWildcard +// tag::keepDoubleWildcard[] +FROM employees +| KEEP h*, * +// end::keepDoubleWildcard[] +| LIMIT 0; + +height:double | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date | avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean +; + +docsRename +// tag::rename[] +FROM employees +| KEEP first_name, last_name, still_hired +| RENAME still_hired AS employed +// end::rename[] +| LIMIT 0; + +first_name:keyword | last_name:keyword | employed:boolean +; + +docsRenameMultipleColumns +// tag::renameMultipleColumns[] +FROM employees +| KEEP first_name, last_name +| RENAME first_name AS fn, last_name AS ln +// end::renameMultipleColumns[] +| LIMIT 0; + +fn:keyword | ln:keyword +; + +docsSort +// tag::sort[] +FROM employees +| KEEP first_name, last_name, height +| SORT height +// end::sort[] +| SORT height, first_name +| LIMIT 3; + +first_name:keyword | last_name:keyword | height:double +Mayuko |Warwick |1.41 +Breannda |Billingsley |1.42 +Vishv |Zockler |1.42 +; + +docsSortDesc +// tag::sortDesc[] +FROM employees +| KEEP first_name, last_name, height +| SORT height DESC +// end::sortDesc[] +| SORT height DESC, first_name ASC +| LIMIT 3; + +first_name:keyword | last_name:keyword | height:double +Arumugam |Ossenbruggen |2.1 +Kwee |Schusler |2.1 +Saniya |Kalloufi |2.1 +; + +docsSortTie +// tag::sortTie[] +FROM employees +| KEEP first_name, last_name, height +| SORT height DESC, first_name ASC +// end::sortTie[] +| LIMIT 3; + +first_name:keyword | last_name:keyword | height:double +Arumugam |Ossenbruggen |2.1 +Kwee |Schusler |2.1 +Saniya |Kalloufi |2.1 +; + +docsSortNullsFirst +// tag::sortNullsFirst[] +FROM employees +| KEEP first_name, last_name, height +| SORT first_name ASC NULLS FIRST +// end::sortNullsFirst[] +| SORT first_name ASC NULLS FIRST, height +| LIMIT 3; + +first_name:keyword | last_name:keyword | height:double +null |Swan |1.46 +null |Lortz |1.53 +null |Brender |1.55 +; + +docsStats +// tag::stats[] +FROM employees +| STATS count = COUNT(emp_no) BY languages +| SORT languages +// end::stats[] +; + +// tag::stats-result[] + count:long | languages:integer +15 |1 +19 |2 +17 |3 +18 |4 +21 |5 +10 |null +// end::stats-result[] +; + +docsStatsWithoutBy +// tag::statsWithoutBy[] +FROM employees +| STATS avg_lang = AVG(languages) +// end::statsWithoutBy[] +; + +// tag::statsWithoutBy-result[] +avg_lang:double +3.1222222222222222 +// end::statsWithoutBy-result[] +; + +docsStatsMultiple +// tag::statsCalcMultipleValues[] +FROM employees +| STATS avg_lang = AVG(languages), max_lang = MAX(languages) +// end::statsCalcMultipleValues[] +; + +avg_lang:double | max_lang:integer +3.1222222222222222|5 +; + +docsStatsGroupByMultipleValues +// tag::statsGroupByMultipleValues[] +FROM employees +| EVAL hired = DATE_FORMAT(hire_date, "YYYY") +| STATS avg_salary = AVG(salary) BY hired, languages.long +| EVAL avg_salary = ROUND(avg_salary) +| SORT hired, languages.long +// end::statsGroupByMultipleValues[] +| LIMIT 4 +; + +hired:keyword |languages.long:long | avg_salary:double +1985 |1 |54668.0 +1985 |3 |47723.0 +1985 |4 |44817.0 +1985 |5 |47720.0 +; + +docsWhere +// tag::where[] +FROM employees +| KEEP first_name, last_name, still_hired +| WHERE still_hired == true +// end::where[] +| STATS count = COUNT(last_name) BY still_hired +; + +count:long | still_hired:boolean +45 |true +; + +docsWhereBoolean +// tag::whereBoolean[] +FROM employees +| KEEP first_name, last_name, still_hired +| WHERE still_hired +// end::whereBoolean[] +| STATS count = COUNT(last_name) BY still_hired +; + +count:long | still_hired:boolean +45 |true +; + +docsWhereFunction +// tag::whereFunction[] +FROM employees +| KEEP first_name, last_name, height +| WHERE length(first_name) < 4 +// end::whereFunction[] +| SORT first_name +; + +first_name:keyword | last_name:keyword | height:double +Gao |Dolinsky |1.94 +Tse |Herber |1.45 +Udi |Jansch |1.93 +Uri |Lenart |1.75 +; + + +dateExtract +// tag::dateExtract[] +ROW date = DATE_PARSE("2022-05-06", "yyyy-MM-dd") +| EVAL year = DATE_EXTRACT(date, "year") +// end::dateExtract[] +; + +// tag::dateExtract-result[] +date:date | year:long +2022-05-06T00:00:00.000Z | 2022 +// end::dateExtract-result[] +; + +docsSubstring +// tag::substring[] +FROM employees +| KEEP last_name +| EVAL ln_sub = SUBSTRING(last_name, 1, 3) +// end::substring[] +| SORT last_name ASC +| LIMIT 5 +; + +// tag::substring-result[] +last_name:keyword | ln_sub:keyword +Awdeh |Awd +Azuma |Azu +Baek |Bae +Bamford |Bam +Bernatsky |Ber +// end::substring-result[] +; + +docsSubstringEnd +// tag::substringEnd[] +FROM employees +| KEEP last_name +| EVAL ln_sub = SUBSTRING(last_name, -3, 3) +// end::substringEnd[] +| SORT last_name ASC +| LIMIT 5 +; + +// tag::substringEnd-result[] +last_name:keyword | ln_sub:keyword +Awdeh |deh +Azuma |uma +Baek |aek +Bamford |ord +Bernatsky |sky +// end::substringEnd-result[] +; + +docsSubstringRemainder +// tag::substringRemainder[] +FROM employees +| KEEP last_name +| EVAL ln_sub = SUBSTRING(last_name, 2) +// end::substringRemainder[] +| SORT last_name ASC +| LIMIT 5 +; + +// tag::substringRemainder-result[] +last_name:keyword | ln_sub:keyword +Awdeh |wdeh +Azuma |zuma +Baek |aek +Bamford |amford +Bernatsky |ernatsky +// end::substringRemainder-result[] +; + +docsStartsWith +// tag::startsWith[] +FROM employees +| KEEP last_name +| EVAL ln_S = STARTS_WITH(last_name, "B") +// end::startsWith[] +| SORT last_name ASC +| LIMIT 5 +; + +// tag::startsWith-result[] +last_name:keyword | ln_S:boolean +Awdeh |false +Azuma |false +Baek |true +Bamford |true +Bernatsky |true +// end::startsWith-result[] +; + +docsRound +// tag::round[] +FROM employees +| KEEP first_name, last_name, height +| EVAL height_ft = ROUND(height * 3.281, 1) +// end::round[] +| SORT height DESC, first_name ASC +| LIMIT 3; + +// tag::round-result[] +first_name:keyword | last_name:keyword | height:double | height_ft:double +Arumugam |Ossenbruggen |2.1 |6.9 +Kwee |Schusler |2.1 |6.9 +Saniya |Kalloufi |2.1 |6.9 +// end::round-result[] +; + +dateParse +// tag::dateParse[] +ROW date_string = "2022-05-06" +| EVAL date = DATE_PARSE(date_string, "yyyy-MM-dd") +// end::dateParse[] +; + +date_string:keyword | date:date +2022-05-06 | 2022-05-06T00:00:00.000Z +; 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 new file mode 100644 index 0000000000000..2fdd5ff9bf324 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/drop.csv-spec @@ -0,0 +1,56 @@ +sortWithLimitOne_DropHeight +from employees | sort languages | limit 1 | drop height*; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean +244294991 |1955-01-21T00:00:00.000Z|10005 |Kyoichi |M |1989-09-12T00:00:00.000Z|[false, false, false, true]|null |1 |1 |1 |1 |Maliniak |63528 |[-2.14, 13.07] |[-2, 13] |[-2, 13] |true +; + +simpleEvalWithSortAndLimitOne_DropHeight +from employees | eval x = languages + 7 | sort x | limit 1 | drop height*; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean | x:integer +244294991 |1955-01-21T00:00:00.000Z|10005 |Kyoichi |M |1989-09-12T00:00:00.000Z|[false, false, false, true]|null |1 |1 |1 |1 |Maliniak |63528 |[-2.14, 13.07] |[-2, 13] |[-2, 13] |true |8 +; + +whereWithEvalGeneratedValue_DropHeight +from employees | eval x = salary / 2 | where x > 37000 | drop height*; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean | x:integer +393084805 |1957-05-23T00:00:00.000Z|10007 |Tzvetan |F |1989-02-10T00:00:00.000Z|[false, false, true, true]|null |4 |4 |4 |4 |Zielinski |74572 |[-7.06, 0.57, 1.99] |[-7, 0, 1] |[-7, 0, 1] |true |37286 +257694181 |1956-12-13T00:00:00.000Z|10029 |Otmar |M |1985-11-20T00:00:00.000Z|true |[Data Scientist, Principal Support Engineer, Senior Python Developer] |null |null |null |null |Herbst |74999 |[-8.19, -1.9, -0.32]|[-8, -1, 0] |[-8, -1, 0] |false |37499 +371418933 |null |10045 |Moss |M |1989-09-02T00:00:00.000Z|[false, true] |[Accountant, Junior Developer, Principal Support Engineer, Purchase Manager]|3 |3 |3 |3 |Shanbhogue |74970 |null |null |null |false |37485 +; + +projectDropWithWildcardKeepOthers +row a = 1+3, b = 2, ab = 5 | eval x = 1 + b + 5 | eval abc = x * 2 | drop a* | keep b,x; + +b:integer | x:integer +2 | 8 +; + +dropAllColumns +from employees | keep height | drop height | eval x = 1; + +x:integer +; + +dropAllColumns_WithLimit +from employees | keep height | drop height | eval x = 1 | limit 3; + +x:integer +; + +dropAllColumns_WithCount +from employees | keep height | drop height | eval x = 1 | stats c=count(x); + +c:long +0 +; + +dropAllColumns_WithStats +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 +; + diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/employees.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/employees.csv new file mode 100644 index 0000000000000..32a8173f06d48 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/employees.csv @@ -0,0 +1,101 @@ +birth_date:date ,emp_no:integer,first_name:keyword,gender:keyword,hire_date:date,languages:integer,languages.long:long,languages.short:short,languages.byte:byte,last_name:keyword,salary:integer,height:double,height.float:float,height.scaled_float:scaled_float,height.half_float:half_float,still_hired:boolean,avg_worked_seconds:long,job_positions:keyword,is_rehired:boolean,salary_change:double,salary_change.int:integer,salary_change.long:long +1953-09-02T00:00:00Z,10001,Georgi ,M,1986-06-26T00:00:00Z,2,2,2,2,Facello ,57305,2.03,2.03,2.03,2.03,true ,268728049,[Senior Python Developer,Accountant],[false,true],[1.19],[1],[1] +1964-06-02T00:00:00Z,10002,Bezalel ,F,1985-11-21T00:00:00Z,5,5,5,5,Simmel ,56371,2.08,2.08,2.08,2.08,true ,328922887,[Senior Team Lead],[false,false],[-7.23,11.17],[-7,11],[-7,11] +1959-12-03T00:00:00Z,10003,Parto ,M,1986-08-28T00:00:00Z,4,4,4,4,Bamford ,61805,1.83,1.83,1.83,1.83,false,200296405,[],[],[14.68,12.82],[14,12],[14,12] +1954-05-01T00:00:00Z,10004,Chirstian ,M,1986-12-01T00:00:00Z,5,5,5,5,Koblick ,36174,1.78,1.78,1.78,1.78,true ,311267831,[Reporting Analyst,Tech Lead,Head Human Resources,Support Engineer],[true],[3.65,-0.35,1.13,13.48],[3,0,1,13],[3,0,1,13] +1955-01-21T00:00:00Z,10005,Kyoichi ,M,1989-09-12T00:00:00Z,1,1,1,1,Maliniak ,63528,2.05,2.05,2.05,2.05,true ,244294991,[],[false,false,false,true],[-2.14,13.07],[-2,13],[-2,13] +1953-04-20T00:00:00Z,10006,Anneke ,F,1989-06-02T00:00:00Z,3,3,3,3,Preusig ,60335,1.56,1.56,1.56,1.56,false,372957040,[Tech Lead,Principal Support Engineer,Senior Team Lead],[],[-3.90],[-3],[-3] +1957-05-23T00:00:00Z,10007,Tzvetan ,F,1989-02-10T00:00:00Z,4,4,4,4,Zielinski ,74572,1.70,1.70,1.70,1.70,true ,393084805,[],[true,false,true,false],[-7.06,1.99,0.57],[-7,1,0],[-7,1,0] +1958-02-19T00:00:00Z,10008,Saniya ,M,1994-09-15T00:00:00Z,2,2,2,2,Kalloufi ,43906,2.10,2.10,2.10,2.10,true ,283074758,[Senior Python Developer,Junior Developer,Purchase Manager,Internship],[true,false],[12.68,3.54,0.75,-2.92],[12,3,0,-2],[12,3,0,-2] +1952-04-19T00:00:00Z,10009,Sumant ,F,1985-02-18T00:00:00Z,1,1,1,1,Peac ,66174,1.85,1.85,1.85,1.85,false,236805489,[Senior Python Developer,Internship],[],[],[],[] +1963-06-01T00:00:00Z,10010,Duangkaew , ,1989-08-24T00:00:00Z,4,4,4,4,Piveteau ,45797,1.70,1.70,1.70,1.70,false,315236372,[Architect,Reporting Analyst,Tech Lead,Purchase Manager],[true,true,false,false],[5.05,-6.77,4.69,12.15],[5,-6,4,12],[5,-6,4,12] +1953-11-07T00:00:00Z,10011,Mary , ,1990-01-22T00:00:00Z,5,5,5,5,Sluis ,31120,1.50,1.50,1.50,1.50,true ,239615525,[Architect,Reporting Analyst,Tech Lead,Senior Team Lead],[true,true],[10.35,-7.82,8.73,3.48],[10,-7,8,3],[10,-7,8,3] +1960-10-04T00:00:00Z,10012,Patricio , ,1992-12-18T00:00:00Z,5,5,5,5,Bridgland ,48942,1.97,1.97,1.97,1.97,false,365510850,[Head Human Resources,Accountant],[false,true,true,false],[0.04],[0],[0] +1963-06-07T00:00:00Z,10013,Eberhardt , ,1985-10-20T00:00:00Z,1,1,1,1,Terkki ,48735,1.94,1.94,1.94,1.94,true ,253864340,[Reporting Analyst],[true,true],[],[],[] +1956-02-12T00:00:00Z,10014,Berni , ,1987-03-11T00:00:00Z,5,5,5,5,Genin ,37137,1.99,1.99,1.99,1.99,false,225049139,[Reporting Analyst,Data Scientist,Head Human Resources],[],[-1.89,9.07],[-1,9],[-1,9] +1959-08-19T00:00:00Z,10015,Guoxiang , ,1987-07-02T00:00:00Z,5,5,5,5,Nooteboom ,25324,1.66,1.66,1.66,1.66,true ,390266432,[Principal Support Engineer,Junior Developer,Head Human Resources,Support Engineer],[true,false,false,false],[14.25,12.40],[14,12],[14,12] +1961-05-02T00:00:00Z,10016,Kazuhito , ,1995-01-27T00:00:00Z,2,2,2,2,Cappelletti ,61358,1.54,1.54,1.54,1.54,false,253029411,[Reporting Analyst,Python Developer,Accountant,Purchase Manager],[false,false],[-5.18,7.69],[-5,7],[-5,7] +1958-07-06T00:00:00Z,10017,Cristinel , ,1993-08-03T00:00:00Z,2,2,2,2,Bouloucos ,58715,1.74,1.74,1.74,1.74,false,236703986,[Data Scientist,Head Human Resources,Purchase Manager],[true,false,true,true],[-6.33],[-6],[-6] +1954-06-19T00:00:00Z,10018,Kazuhide , ,1987-04-03T00:00:00Z,2,2,2,2,Peha ,56760,1.97,1.97,1.97,1.97,false,309604079,[Junior Developer],[false,false,true,true],[-1.64,11.51,-5.32],[-1,11,-5],[-1,11,-5] +1953-01-23T00:00:00Z,10019,Lillian , ,1999-04-30T00:00:00Z,1,1,1,1,Haddadi ,73717,2.06,2.06,2.06,2.06,false,342855721,[Purchase Manager],[false,false],[-6.84,8.42,-7.26],[-6,8,-7],[-6,8,-7] +1952-12-24T00:00:00Z,10020,Mayuko ,M,1991-01-26T00:00:00Z, , , , ,Warwick ,40031,1.41,1.41,1.41,1.41,false,373309605,[Tech Lead],[true,true,false],[-5.81],[-5],[-5] +1960-02-20T00:00:00Z,10021,Ramzi ,M,1988-02-10T00:00:00Z, , , , ,Erde ,60408,1.47,1.47,1.47,1.47,false,287654610,[Support Engineer],[true],[],[],[] +1952-07-08T00:00:00Z,10022,Shahaf ,M,1995-08-22T00:00:00Z, , , , ,Famili ,48233,1.82,1.82,1.82,1.82,false,233521306,[Reporting Analyst,Data Scientist,Python Developer,Internship],[true,false],[12.09,2.85],[12,2],[12,2] +1953-09-29T00:00:00Z,10023,Bojan ,F,1989-12-17T00:00:00Z, , , , ,Montemayor ,47896,1.75,1.75,1.75,1.75,true ,330870342,[Accountant,Support Engineer,Purchase Manager],[true,true,false],[14.63,0.80],[14,0],[14,0] +1958-09-05T00:00:00Z,10024,Suzette ,F,1997-05-19T00:00:00Z, , , , ,Pettey ,64675,2.08,2.08,2.08,2.08,true ,367717671,[Junior Developer],[true,true,true,true],[],[],[] +1958-10-31T00:00:00Z,10025,Prasadram ,M,1987-08-17T00:00:00Z, , , , ,Heyers ,47411,1.87,1.87,1.87,1.87,false,371270797,[Accountant],[true,false],[-4.33,-2.90,12.06,-3.46],[-4,-2,12,-3],[-4,-2,12,-3] +1953-04-03T00:00:00Z,10026,Yongqiao ,M,1995-03-20T00:00:00Z, , , , ,Berztiss ,28336,2.10,2.10,2.10,2.10,true ,359208133,[Reporting Analyst],[false,true],[-7.37,10.62,11.20],[-7,10,11],[-7,10,11] +1962-07-10T00:00:00Z,10027,Divier ,F,1989-07-07T00:00:00Z, , , , ,Reistad ,73851,1.53,1.53,1.53,1.53,false,374037782,[Senior Python Developer],[false],[],[],[] +1963-11-26T00:00:00Z,10028,Domenick ,M,1991-10-22T00:00:00Z, , , , ,Tempesti ,39356,2.07,2.07,2.07,2.07,true ,226435054,[Tech Lead,Python Developer,Accountant,Internship],[true,false,false,true],[],[],[] +1956-12-13T00:00:00Z,10029,Otmar ,M,1985-11-20T00:00:00Z, , , , ,Herbst ,74999,1.99,1.99,1.99,1.99,false,257694181,[Senior Python Developer,Data Scientist,Principal Support Engineer],[true],[-0.32,-1.90,-8.19],[0,-1,-8],[0,-1,-8] +1958-07-14T00:00:00Z,10030, ,M,1994-02-17T00:00:00Z,3,3,3,3,Demeyer ,67492,1.92,1.92,1.92,1.92,false,394597613,[Tech Lead,Data Scientist,Senior Team Lead],[true,false,false],[-0.40],[0],[0] +1959-01-27T00:00:00Z,10031, ,M,1991-09-01T00:00:00Z,4,4,4,4,Joslin ,37716,1.68,1.68,1.68,1.68,false,348545109,[Architect,Senior Python Developer,Purchase Manager,Senior Team Lead],[false],[],[],[] +1960-08-09T00:00:00Z,10032, ,F,1990-06-20T00:00:00Z,3,3,3,3,Reistad ,62233,2.10,2.10,2.10,2.10,false,277622619,[Architect,Senior Python Developer,Junior Developer,Purchase Manager],[false,false],[9.32,-4.92],[9,-4],[9,-4] +1956-11-14T00:00:00Z,10033, ,M,1987-03-18T00:00:00Z,1,1,1,1,Merlo ,70011,1.63,1.63,1.63,1.63,false,208374744,[],[true],[],[],[] +1962-12-29T00:00:00Z,10034, ,M,1988-09-21T00:00:00Z,1,1,1,1,Swan ,39878,1.46,1.46,1.46,1.46,false,214393176,[Business Analyst,Data Scientist,Python Developer,Accountant],[false],[-8.46],[-8],[-8] +1953-02-08T00:00:00Z,10035, ,M,1988-09-05T00:00:00Z,5,5,5,5,Chappelet ,25945,1.81,1.81,1.81,1.81,false,203838153,[Senior Python Developer,Data Scientist],[false],[-2.54,-6.58],[-2,-6],[-2,-6] +1959-08-10T00:00:00Z,10036, ,M,1992-01-03T00:00:00Z,4,4,4,4,Portugali ,60781,1.61,1.61,1.61,1.61,false,305493131,[Senior Python Developer],[true,false,false],[],[],[] +1963-07-22T00:00:00Z,10037, ,M,1990-12-05T00:00:00Z,2,2,2,2,Makrucki ,37691,2.00,2.00,2.00,2.00,true ,359217000,[Senior Python Developer,Tech Lead,Accountant],[false],[-7.08],[-7],[-7] +1960-07-20T00:00:00Z,10038, ,M,1989-09-20T00:00:00Z,4,4,4,4,Lortz ,35222,1.53,1.53,1.53,1.53,true ,314036411,[Senior Python Developer,Python Developer,Support Engineer],[],[],[],[] +1959-10-01T00:00:00Z,10039, ,M,1988-01-19T00:00:00Z,2,2,2,2,Brender ,36051,1.55,1.55,1.55,1.55,false,243221262,[Business Analyst,Python Developer,Principal Support Engineer],[true,true],[-6.90],[-6],[-6] + ,10040,Weiyi ,F,1993-02-14T00:00:00Z,4,4,4,4,Meriste ,37112,1.90,1.90,1.90,1.90,false,244478622,[Principal Support Engineer],[true,false,true,true],[6.97,14.74,-8.94,1.92],[6,14,-8,1],[6,14,-8,1] + ,10041,Uri ,F,1989-11-12T00:00:00Z,1,1,1,1,Lenart ,56415,1.75,1.75,1.75,1.75,false,287789442,[Data Scientist,Head Human Resources,Internship,Senior Team Lead],[],[9.21,0.05,7.29,-2.94],[9,0,7,-2],[9,0,7,-2] + ,10042,Magy ,F,1993-03-21T00:00:00Z,3,3,3,3,Stamatiou ,30404,1.44,1.44,1.44,1.44,true ,246355863,[Architect,Business Analyst,Junior Developer,Internship],[],[-9.28,9.42],[-9,9],[-9,9] + ,10043,Yishay ,M,1990-10-20T00:00:00Z,1,1,1,1,Tzvieli ,34341,1.52,1.52,1.52,1.52,true ,287222180,[Data Scientist,Python Developer,Support Engineer],[false,true,true],[-5.17,4.62,7.42],[-5,4,7],[-5,4,7] + ,10044,Mingsen ,F,1994-05-21T00:00:00Z,1,1,1,1,Casley ,39728,2.06,2.06,2.06,2.06,false,387408356,[Tech Lead,Principal Support Engineer,Accountant,Support Engineer],[true,true],[8.09],[8],[8] + ,10045,Moss ,M,1989-09-02T00:00:00Z,3,3,3,3,Shanbhogue ,74970,1.70,1.70,1.70,1.70,false,371418933,[Principal Support Engineer,Junior Developer,Accountant,Purchase Manager],[true,false],[],[],[] + ,10046,Lucien ,M,1992-06-20T00:00:00Z,4,4,4,4,Rosenbaum ,50064,1.52,1.52,1.52,1.52,true ,302353405,[Principal Support Engineer,Junior Developer,Head Human Resources,Internship],[true,true,false,true],[2.39],[2],[2] + ,10047,Zvonko ,M,1989-03-31T00:00:00Z,4,4,4,4,Nyanchama ,42716,1.52,1.52,1.52,1.52,true ,306369346,[Architect,Data Scientist,Principal Support Engineer,Senior Team Lead],[true],[-6.36,12.12],[-6,12],[-6,12] + ,10048,Florian ,M,1985-02-24T00:00:00Z,3,3,3,3,Syrotiuk ,26436,2.00,2.00,2.00,2.00,false,248451647,[Internship],[true,true],[],[],[] + ,10049,Basil ,F,1992-05-04T00:00:00Z,5,5,5,5,Tramer ,37853,1.52,1.52,1.52,1.52,true ,320725709,[Senior Python Developer,Business Analyst],[],[-1.05],[-1],[-1] +1958-05-21T00:00:00Z,10050,Yinghua ,M,1990-12-25T00:00:00Z,2,2,2,2,Dredge ,43026,1.96,1.96,1.96,1.96,true ,242731798,[Reporting Analyst,Junior Developer,Accountant,Support Engineer],[true],[8.70,10.94],[8,10],[8,10] +1953-07-28T00:00:00Z,10051,Hidefumi ,M,1992-10-15T00:00:00Z,3,3,3,3,Caine ,58121,1.89,1.89,1.89,1.89,true ,374753122,[Business Analyst,Accountant,Purchase Manager],[],[],[],[] +1961-02-26T00:00:00Z,10052,Heping ,M,1988-05-21T00:00:00Z,1,1,1,1,Nitsch ,55360,1.79,1.79,1.79,1.79,true ,299654717,[],[true,true,false],[-0.55,-1.89,-4.22,-6.03],[0,-1,-4,-6],[0,-1,-4,-6] +1954-09-13T00:00:00Z,10053,Sanjiv ,F,1986-02-04T00:00:00Z,3,3,3,3,Zschoche ,54462,1.58,1.58,1.58,1.58,false,368103911,[Support Engineer],[true,false,true,false],[-7.67,-3.25],[-7,-3],[-7,-3] +1957-04-04T00:00:00Z,10054,Mayumi ,M,1995-03-13T00:00:00Z,4,4,4,4,Schueller ,65367,1.82,1.82,1.82,1.82,false,297441693,[Principal Support Engineer],[false,false],[],[],[] +1956-06-06T00:00:00Z,10055,Georgy ,M,1992-04-27T00:00:00Z,5,5,5,5,Dredge ,49281,2.04,2.04,2.04,2.04,false,283157844,[Senior Python Developer,Head Human Resources,Internship,Support Engineer],[false,false,true],[7.34,12.99,3.17],[7,12,3],[7,12,3] +1961-09-01T00:00:00Z,10056,Brendon ,F,1990-02-01T00:00:00Z,2,2,2,2,Bernini ,33370,1.57,1.57,1.57,1.57,true ,349086555,[Senior Team Lead],[true,false,false],[10.99,-5.17],[10,-5],[10,-5] +1954-05-30T00:00:00Z,10057,Ebbe ,F,1992-01-15T00:00:00Z,4,4,4,4,Callaway ,27215,1.59,1.59,1.59,1.59,true ,324356269,[Python Developer,Head Human Resources],[],[-6.73,-2.43,-5.27,1.03],[-6,-2,-5,1],[-6,-2,-5,1] +1954-10-01T00:00:00Z,10058,Berhard ,M,1987-04-13T00:00:00Z,3,3,3,3,McFarlin ,38376,1.83,1.83,1.83,1.83,false,268378108,[Principal Support Engineer],[],[-4.89],[-4],[-4] +1953-09-19T00:00:00Z,10059,Alejandro ,F,1991-06-26T00:00:00Z,2,2,2,2,McAlpine ,44307,1.48,1.48,1.48,1.48,false,237368465,[Architect,Principal Support Engineer,Purchase Manager,Senior Team Lead],[false],[5.53,13.38,-4.69,6.27],[5,13,-4,6],[5,13,-4,6] +1961-10-15T00:00:00Z,10060,Breannda ,M,1987-11-02T00:00:00Z,2,2,2,2,Billingsley ,29175,1.42,1.42,1.42,1.42,true ,341158890,[Business Analyst,Data Scientist,Senior Team Lead],[false,false,true,false],[-1.76,-0.85],[-1,0],[-1,0] +1962-10-19T00:00:00Z,10061,Tse ,M,1985-09-17T00:00:00Z,1,1,1,1,Herber ,49095,1.45,1.45,1.45,1.45,false,327550310,[Purchase Manager,Senior Team Lead],[false,true],[14.39,-2.58,-0.95],[14,-2,0],[14,-2,0] +1961-11-02T00:00:00Z,10062,Anoosh ,M,1991-08-30T00:00:00Z,3,3,3,3,Peyn ,65030,1.70,1.70,1.70,1.70,false,203989706,[Python Developer,Senior Team Lead],[false,true,true],[-1.17],[-1],[-1] +1952-08-06T00:00:00Z,10063,Gino ,F,1989-04-08T00:00:00Z,3,3,3,3,Leonhardt ,52121,1.78,1.78,1.78,1.78,true ,214068302,[],[true],[],[],[] +1959-04-07T00:00:00Z,10064,Udi ,M,1985-11-20T00:00:00Z,5,5,5,5,Jansch ,33956,1.93,1.93,1.93,1.93,false,307364077,[Purchase Manager],[false,false,true,false],[-8.66,-2.52],[-8,-2],[-8,-2] +1963-04-14T00:00:00Z,10065,Satosi ,M,1988-05-18T00:00:00Z,2,2,2,2,Awdeh ,50249,1.59,1.59,1.59,1.59,false,372660279,[Business Analyst,Data Scientist,Principal Support Engineer],[false,true],[-1.47,14.44,-9.81],[-1,14,-9],[-1,14,-9] +1952-11-13T00:00:00Z,10066,Kwee ,M,1986-02-26T00:00:00Z,5,5,5,5,Schusler ,31897,2.10,2.10,2.10,2.10,true ,360906451,[Senior Python Developer,Data Scientist,Accountant,Internship],[true,true,true],[5.94],[5],[5] +1953-01-07T00:00:00Z,10067,Claudi ,M,1987-03-04T00:00:00Z,2,2,2,2,Stavenow ,52044,1.77,1.77,1.77,1.77,true ,347664141,[Tech Lead,Principal Support Engineer],[false,false],[8.72,4.44],[8,4],[8,4] +1962-11-26T00:00:00Z,10068,Charlene ,M,1987-08-07T00:00:00Z,3,3,3,3,Brattka ,28941,1.58,1.58,1.58,1.58,true ,233999584,[Architect],[true],[3.43,-5.61,-5.29],[3,-5,-5],[3,-5,-5] +1960-09-06T00:00:00Z,10069,Margareta ,F,1989-11-05T00:00:00Z,5,5,5,5,Bierman ,41933,1.77,1.77,1.77,1.77,true ,366512352,[Business Analyst,Junior Developer,Purchase Manager,Support Engineer],[false],[-3.34,-6.33,6.23,-0.31],[-3,-6,6,0],[-3,-6,6,0] +1955-08-20T00:00:00Z,10070,Reuven ,M,1985-10-14T00:00:00Z,3,3,3,3,Garigliano ,54329,1.77,1.77,1.77,1.77,true ,347188604,[],[true,true,true],[-5.90],[-5],[-5] +1958-01-21T00:00:00Z,10071,Hisao ,M,1987-10-01T00:00:00Z,2,2,2,2,Lipner ,40612,2.07,2.07,2.07,2.07,false,306671693,[Business Analyst,Reporting Analyst,Senior Team Lead],[false,false,false],[-2.69],[-2],[-2] +1952-05-15T00:00:00Z,10072,Hironoby ,F,1988-07-21T00:00:00Z,5,5,5,5,Sidou ,54518,1.82,1.82,1.82,1.82,true ,209506065,[Architect,Tech Lead,Python Developer,Senior Team Lead],[false,false,true,false],[11.21,-2.30,2.22,-5.44],[11,-2,2,-5],[11,-2,2,-5] +1954-02-23T00:00:00Z,10073,Shir ,M,1991-12-01T00:00:00Z,4,4,4,4,McClurg ,32568,1.66,1.66,1.66,1.66,false,314930367,[Principal Support Engineer,Python Developer,Junior Developer,Purchase Manager],[true,false],[-5.67],[-5],[-5] +1955-08-28T00:00:00Z,10074,Mokhtar ,F,1990-08-13T00:00:00Z,5,5,5,5,Bernatsky ,38992,1.64,1.64,1.64,1.64,true ,382397583,[Senior Python Developer,Python Developer],[true,false,false,true],[6.70,1.98,-5.64,2.96],[6,1,-5,2],[6,1,-5,2] +1960-03-09T00:00:00Z,10075,Gao ,F,1987-03-19T00:00:00Z,5,5,5,5,Dolinsky ,51956,1.94,1.94,1.94,1.94,false,370238919,[Purchase Manager],[true],[9.63,-3.29,8.42],[9,-3,8],[9,-3,8] +1952-06-13T00:00:00Z,10076,Erez ,F,1985-07-09T00:00:00Z,3,3,3,3,Ritzmann ,62405,1.83,1.83,1.83,1.83,false,376240317,[Architect,Senior Python Developer],[false],[-6.90,-1.30,8.75],[-6,-1,8],[-6,-1,8] +1964-04-18T00:00:00Z,10077,Mona ,M,1990-03-02T00:00:00Z,5,5,5,5,Azuma ,46595,1.68,1.68,1.68,1.68,false,351960222,[Internship],[],[-0.01],[0],[0] +1959-12-25T00:00:00Z,10078,Danel ,F,1987-05-26T00:00:00Z,2,2,2,2,Mondadori ,69904,1.81,1.81,1.81,1.81,true ,377116038,[Architect,Principal Support Engineer,Internship],[true],[-7.88,9.98,12.52],[-7,9,12],[-7,9,12] +1961-10-05T00:00:00Z,10079,Kshitij ,F,1986-03-27T00:00:00Z,2,2,2,2,Gils ,32263,1.59,1.59,1.59,1.59,false,320953330,[],[false],[7.58],[7],[7] +1957-12-03T00:00:00Z,10080,Premal ,M,1985-11-19T00:00:00Z,5,5,5,5,Baek ,52833,1.80,1.80,1.80,1.80,false,239266137,[Senior Python Developer],[],[-4.35,7.36,5.56],[-4,7,5],[-4,7,5] +1960-12-17T00:00:00Z,10081,Zhongwei ,M,1986-10-30T00:00:00Z,2,2,2,2,Rosen ,50128,1.44,1.44,1.44,1.44,true ,321375511,[Accountant,Internship],[false,false,false],[],[],[] +1963-09-09T00:00:00Z,10082,Parviz ,M,1990-01-03T00:00:00Z,4,4,4,4,Lortz ,49818,1.61,1.61,1.61,1.61,false,232522994,[Principal Support Engineer],[false],[1.19,-3.39],[1,-3],[1,-3] +1959-07-23T00:00:00Z,10083,Vishv ,M,1987-03-31T00:00:00Z,1,1,1,1,Zockler ,39110,1.42,1.42,1.42,1.42,false,331236443,[Head Human Resources],[],[],[],[] +1960-05-25T00:00:00Z,10084,Tuval ,M,1995-12-15T00:00:00Z,1,1,1,1,Kalloufi ,28035,1.51,1.51,1.51,1.51,true ,359067056,[Principal Support Engineer],[false],[],[],[] +1962-11-07T00:00:00Z,10085,Kenroku ,M,1994-04-09T00:00:00Z,5,5,5,5,Malabarba ,35742,2.01,2.01,2.01,2.01,true ,353404008,[Senior Python Developer,Business Analyst,Tech Lead,Accountant],[],[11.67,6.75,8.40],[11,6,8],[11,6,8] +1962-11-19T00:00:00Z,10086,Somnath ,M,1990-02-16T00:00:00Z,1,1,1,1,Foote ,68547,1.74,1.74,1.74,1.74,true ,328580163,[Senior Python Developer],[false,true],[13.61],[13],[13] +1959-07-23T00:00:00Z,10087,Xinglin ,F,1986-09-08T00:00:00Z,5,5,5,5,Eugenio ,32272,1.74,1.74,1.74,1.74,true ,305782871,[Junior Developer,Internship],[false,false],[-2.05],[-2],[-2] +1954-02-25T00:00:00Z,10088,Jungsoon ,F,1988-09-02T00:00:00Z,5,5,5,5,Syrzycki ,39638,1.91,1.91,1.91,1.91,false,330714423,[Reporting Analyst,Business Analyst,Tech Lead],[true],[],[],[] +1963-03-21T00:00:00Z,10089,Sudharsan ,F,1986-08-12T00:00:00Z,4,4,4,4,Flasterstein,43602,1.57,1.57,1.57,1.57,true ,232951673,[Junior Developer,Accountant],[true,false,false,false],[],[],[] +1961-05-30T00:00:00Z,10090,Kendra ,M,1986-03-14T00:00:00Z,2,2,2,2,Hofting ,44956,2.03,2.03,2.03,2.03,true ,212460105,[],[false,false,false,true],[7.15,-1.85,3.60],[7,-1,3],[7,-1,3] +1955-10-04T00:00:00Z,10091,Amabile ,M,1992-11-18T00:00:00Z,3,3,3,3,Gomatam ,38645,2.09,2.09,2.09,2.09,true ,242582807,[Reporting Analyst,Python Developer],[true,true,false,false],[-9.23,7.50,5.85,5.19],[-9,7,5,5],[-9,7,5,5] +1964-10-18T00:00:00Z,10092,Valdiodio ,F,1989-09-22T00:00:00Z,1,1,1,1,Niizuma ,25976,1.75,1.75,1.75,1.75,false,313407352,[Junior Developer,Accountant],[false,false,true,true],[8.78,0.39,-6.77,8.30],[8,0,-6,8],[8,0,-6,8] +1964-06-11T00:00:00Z,10093,Sailaja ,M,1996-11-05T00:00:00Z,3,3,3,3,Desikan ,45656,1.69,1.69,1.69,1.69,false,315904921,[Reporting Analyst,Tech Lead,Principal Support Engineer,Purchase Manager],[],[-0.88],[0],[0] +1957-05-25T00:00:00Z,10094,Arumugam ,F,1987-04-18T00:00:00Z,5,5,5,5,Ossenbruggen,66817,2.10,2.10,2.10,2.10,false,332920135,[Senior Python Developer,Principal Support Engineer,Accountant],[true,false,true],[2.22,7.92],[2,7],[2,7] +1965-01-03T00:00:00Z,10095,Hilari ,M,1986-07-15T00:00:00Z,4,4,4,4,Morton ,37702,1.55,1.55,1.55,1.55,false,321850475,[],[true,true,false,false],[-3.93,-6.66],[-3,-6],[-3,-6] +1954-09-16T00:00:00Z,10096,Jayson ,M,1990-01-14T00:00:00Z,4,4,4,4,Mandell ,43889,1.94,1.94,1.94,1.94,false,204381503,[Architect,Reporting Analyst],[false,false,false],[],[],[] +1952-02-27T00:00:00Z,10097,Remzi ,M,1990-09-15T00:00:00Z,3,3,3,3,Waschkowski ,71165,1.53,1.53,1.53,1.53,false,206258084,[Reporting Analyst,Tech Lead],[true,false],[-1.12],[-1],[-1] +1961-09-23T00:00:00Z,10098,Sreekrishna,F,1985-05-13T00:00:00Z,4,4,4,4,Servieres ,44817,2.00,2.00,2.00,2.00,false,272392146,[Architect,Internship,Senior Team Lead],[false],[-2.83,8.31,4.38],[-2,8,4],[-2,8,4] +1956-05-25T00:00:00Z,10099,Valter ,F,1988-10-18T00:00:00Z,2,2,2,2,Sullins ,73578,1.81,1.81,1.81,1.81,true ,377713748,[],[true,true],[10.71,14.26,-8.78,-3.98],[10,14,-8,-3],[10,14,-8,-3] +1953-04-21T00:00:00Z,10100,Hironobu ,F,1987-09-21T00:00:00Z,4,4,4,4,Haraldson ,68431,1.77,1.77,1.77,1.77,true ,223910853,[Purchase Manager],[false,true,true,false],[13.97,-7.49],[13,-7],[13,-7] diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich-ignoreCsvTests.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich-ignoreCsvTests.csv-spec new file mode 100644 index 0000000000000..e107fc2ffea63 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich-ignoreCsvTests.csv-spec @@ -0,0 +1,143 @@ +simple +row language_code = "1" +| enrich languages_policy +; + +language_code:keyword | language_name:keyword +1 | English +; + + +enrichOn +from employees | sort emp_no | limit 1 | eval x = to_string(languages) | enrich languages_policy on x | keep emp_no, language_name; + +emp_no:integer | language_name:keyword +10001 | French +; + + +enrichOn2 +from employees | eval x = to_string(languages) | enrich languages_policy on x | keep emp_no, language_name | sort emp_no | limit 1 ; + +emp_no:integer | language_name:keyword +10001 | French +; + +simpleSortLimit +from employees | eval x = to_string(languages) | enrich languages_policy on x | keep emp_no, language_name | sort emp_no | limit 1; + +emp_no:integer | language_name:keyword +10001 | French +; + + +with +from employees | eval x = to_string(languages) | keep emp_no, x | sort emp_no | limit 1 +| enrich languages_policy on x with language_name; + +emp_no:integer | x:keyword | language_name:keyword +10001 | 2 | French +; + + +withAlias +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; + +emp_no:integer | x:keyword | lang:keyword +10001 | 2 | French +10002 | 5 | null +10003 | 4 | German +; + + + +withAliasSort +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; + +emp_no:integer | x:keyword | lang:keyword +10001 | 2 | French +10002 | 5 | null +10003 | 4 | German +; + + +withAliasAndPlain +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; + +emp_no:integer | x:keyword | lang:keyword | language_name:keyword +10100 | 4 | German | German +10099 | 2 | French | French +10098 | 4 | German | German +; + + +withTwoAliasesSameProp +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; + +emp_no:integer | x:keyword | lang:keyword | lang2:keyword +10001 | 2 | French | French +; + + +redundantWith +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; + +emp_no:integer | x:keyword | language_name:keyword +10001 | 2 | French +; + + +nullInput +from employees | where emp_no == 10017 | keep emp_no, gender +| enrich languages_policy on gender with language_name, language_name; + +emp_no:integer | gender:keyword | language_name:keyword +10017 | null | null +; + + +constantNullInput +from employees | where emp_no == 10020 | eval x = to_string(languages) | keep emp_no, x +| enrich languages_policy on x with language_name, language_name; + +emp_no:integer | x:keyword | language_name:keyword +10020 | null | null +; + + +multipleEnrich +row a = "1", b = "2", c = "10" +| enrich languages_policy on a with a_lang = language_name +| enrich languages_policy on b with b_lang = language_name +| enrich languages_policy on c with c_lang = language_name; + +a:keyword | b:keyword | c:keyword | a_lang:keyword | b_lang:keyword | c_lang:keyword +1 | 2 | 10 | English | French | null +; + + +enrichEval +from employees | eval x = to_string(languages) +| enrich languages_policy on x with lang = language_name +| eval language = concat(x, "-", lang) +| keep emp_no, x, lang, language +| sort emp_no desc | limit 3; + +emp_no:integer | x:keyword | lang:keyword | language:keyword +10100 | 4 | German | 4-German +10099 | 2 | French | 2-French +10098 | 4 | German | 4-German +; + + +multivalue +row a = ["1", "2"] | enrich languages_policy on a with a_lang = language_name; + +a:keyword | a_lang:keyword +["1", "2"] | ["English", "French"] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich.csv-spec new file mode 100644 index 0000000000000..796a7bceca55d --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enrich.csv-spec @@ -0,0 +1,5 @@ +simple +from employees | eval x = 1, y = to_string(languages) | enrich languages_policy on y | where x > 1 | keep emp_no, language_name | limit 1; + +emp_no:integer | language_name:keyword +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enricy-policy-languages.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enricy-policy-languages.json new file mode 100644 index 0000000000000..fcf404fbe2111 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/enricy-policy-languages.json @@ -0,0 +1,7 @@ +{ + "match": { + "indices": "languages", + "match_field": "language_code", + "enrich_fields": ["language_name"] + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec new file mode 100644 index 0000000000000..ba89685716059 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec @@ -0,0 +1,110 @@ +simpleEval +row a = 1 | eval b = 2; + +a:integer | b:integer +1 | 2 +; + +withMath +row a = 1 | eval b = 2 + 3; + +a:integer | b:integer +1 | 5 +; + + +withMathAndVariables +row a = 1 | eval b = a + 2; + +a:integer | b:integer +1 | 3 +; + + +evalEval +row a = 1 | eval b = a + 1 | eval c = b + 2; + +a:integer | b:integer | c:integer +1 | 2 | 4 +; + + +multiple +row a = 1 | eval b = a + 1, c = b + 2; + +a:integer | b:integer | c:integer +1 | 2 | 4 +; + + +multiple2 +row a = 1 | eval b = a + 1, c = b + 2, d = a + b + c | eval e = a + d - 2; + +a:integer | b:integer | c:integer | d:integer | e:integer +1 | 2 | 4 | 7 | 6 +; + + +multipleDuplicateInterleaved1 +row a = 1 | eval b = a, c = 1, c = 3, d = b + 1, b = c * 2, c = 2, c = d * c + b | keep a, b, c, d; + +a:integer | b:integer | c:integer | d:integer +1 | 6 | 10 | 2 +; + + +multipleDuplicateInterleaved2 +row a = 1 | eval b = a, c = 1 | eval c = 3, d = b + 1 | eval b = c * 2, c = 2 | eval c = d * c + b | keep a, b, c, d; + +a:integer | b:integer | c:integer | d:integer +1 | 6 | 10 | 2 +; + + +multipleDuplicateInterleaved3 +row a = 1 | eval b = a, c = 1, c = 3 | eval d = b + 1 | eval b = c * 2, c = 2, c = d * c + b | keep a, b, c, d; + +a:integer | b:integer | c:integer | d:integer +1 | 6 | 10 | 2 +; + +multipleDuplicateInterleaved4 +row a = 1 | eval b = a | eval c = 1 | eval c = 3 | eval d = b + 1 | eval b = c * 2 | eval c = 2 | eval c = d * c + b | keep a, b, c, d; + +a:integer | b:integer | c:integer | d:integer +1 | 6 | 10 | 2 +; + + +projectEval +row x = 1 | keep x | eval a1 = x + 1, a2 = x + 1, a3 = a1 + a2, a1 = a1 + a2; + +x:integer | a2:integer | a3:integer | a1:integer +1 | 2 | 4 | 4 +; + +evalNullSort +from employees | eval x = null | sort x asc, emp_no desc | keep emp_no, x, last_name | limit 2; + +emp_no:integer | x:null | last_name:keyword +10100 | null | Haraldson +10099 | null | Sullins +; + + +filterEvalFilter +from employees | where emp_no < 100010 | eval name_len = length(first_name) | where name_len < 4 | keep first_name | sort first_name; + +first_name:keyword +Gao +Tse +Udi +Uri +; + +evalWithIsNullIsNotNull +from employees | eval true_bool = null is null, false_bool = null is not null, negated_true = not(null is null), negated_false = not(null is not null) | sort emp_no | limit 1 | keep *true*, *false*, first_name, last_name; + +true_bool:boolean | negated_true:boolean | false_bool:boolean | negated_false:boolean | first_name:keyword | last_name:keyword +true | false | false | true | Georgi | Facello +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/floats.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/floats.csv-spec new file mode 100644 index 0000000000000..39758a3f21d7f --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/floats.csv-spec @@ -0,0 +1,405 @@ +// Floating point types-specific tests + +inDouble +from employees | keep emp_no, height, height.float, height.half_float, height.scaled_float | where height in (2.03, 2.0299999713897705, 2.029296875, 2.0300000000000002); + +emp_no:integer |height:double |height.float:double |height.half_float:double |height.scaled_float:double +10001 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +10090 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +; + +inFloat +from employees | keep emp_no, height, height.float, height.half_float, height.scaled_float | where height.float in (2.03, 2.0299999713897705, 2.029296875, 2.0300000000000002); + +emp_no:integer |height:double |height.float:double |height.half_float:double |height.scaled_float:double +10001 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +10090 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +; + +inHalfFloat +from employees | keep emp_no, height, height.float, height.half_float, height.scaled_float | where height.half_float in (2.03, 2.0299999713897705, 2.029296875, 2.0300000000000002); + +emp_no:integer |height:double |height.float:double |height.half_float:double |height.scaled_float:double +10001 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +10090 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +; + +inScaledFloat +from employees | keep emp_no, height, height.float, height.half_float, height.scaled_float | where height.scaled_float in (2.03, 2.0299999713897705, 2.029296875, 2.0300000000000002); + +emp_no:integer |height:double |height.float:double |height.half_float:double |height.scaled_float:double +10001 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +10090 |2.03 |2.0299999713897705 |2.029296875 |2.0300000000000002 +; + +convertFromDouble +row d = [123.4, 567.8] | eval dd = to_double(d); + +d:double |dd:double +[123.4, 567.8] |[123.4, 567.8] +; + +convertFromBoolean +row ft = [false, true] | eval fd = to_double(false), td = to_double(true), ftd = to_double(ft); + +ft:boolean |fd:double |td:double |ftd:double +[false, true] |0 |1 |[0, 1] +; + +convertFromDatetime +from employees | sort emp_no | eval hire_double = to_double(hire_date) | keep emp_no, hire_date, hire_double | limit 3; + +emp_no:integer |hire_date:date |hire_double:double +10001 |1986-06-26T00:00:00.000Z|5.20128E11 +10002 |1985-11-21T00:00:00.000Z|5.013792E11 +10003 |1986-08-28T00:00:00.000Z|5.255712E11 +; + +convertFromString +// tag::to_double-str[] +ROW str1 = "5.20128E11", str2 = "foo" +| EVAL dbl = TO_DOUBLE("520128000000"), dbl1 = TO_DOUBLE(str1), dbl2 = TO_DOUBLE(str2) +// end::to_double-str[] +; +warning:Line 2:72: evaluation of [TO_DOUBLE(str2)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"foo\" + +// tag::to_double-str-result[] +str1:keyword |str2:keyword |dbl:double |dbl1:double |dbl2:double +5.20128E11 |foo |5.20128E11 |5.20128E11 |null +// end::to_double-str-result[] +; + +convertFromUnsignedLong +row ul = 9223372036854775808 | eval dbl = to_double(ul); + + ul:ul | dbl:double +9223372036854775808|9.223372036854776E18 +; + +convertFromLong +row long = 520128000000 | eval dbl = to_double(long); + +long:long |dbl:double +520128000000 |520128000000 +; + +convertFromInt +row int = 520128 | eval dbl = to_double(int); + +int:integer |dbl:double +520128 |520128 +; + +lessThanMultivalue +from employees | where salary_change < 1 | keep emp_no, salary_change | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10006 |-3.9 +10012 | 0.04 +10017 |-6.33 +10020 |-5.81 +10030 |-0.4 +; + +greaterThanMultivalue +from employees | where salary_change > 1 | keep emp_no, salary_change | sort emp_no | limit 5; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10001 |1.19 +10044 |8.09 +10046 |2.39 +10066 |5.94 +10079 |7.58 +; + +equalToMultivalue +from employees | where salary_change == 1.19 | keep emp_no, salary_change | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10001 |1.19 +; + +equalToOrEqualToMultivalue +from employees | where salary_change == 1.19 or salary_change == 7.58 | keep emp_no, salary_change | sort emp_no; + +// Note that multivalued salaries are filtered out +emp_no:integer |salary_change:double +10001 |1.19 +10079 |7.58 +; + +inMultivalue +from employees | where salary_change in (1.19, 7.58) | keep emp_no, salary_change | sort emp_no; + +// Note that multivalued salaries are filtered out +emp_no:integer |salary_change:double +10001 |1.19 +10079 |7.58 +; + +notLessThanMultivalue +from employees | where not(salary_change < 1) | keep emp_no, salary_change | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10001 | 1.19 +10044 | 8.09 +10046 | 2.39 +10066 | 5.94 +10079 | 7.58 +; + +notGreaterThanMultivalue +from employees | where not(salary_change > 1) | keep emp_no, salary_change | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10006 | -3.9 +10012 | 0.04 +10017 | -6.33 +10020 | -5.81 +10030 | -0.4 +; + +notEqualToMultivalue +from employees | where not(salary_change == 1.19) | keep emp_no, salary_change | sort emp_no | limit 5; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10006 | -3.9 +10012 | 0.04 +10017 | -6.33 +10020 | -5.81 +10030 | -0.4 +; + +notEqualToAndEqualToMultivalue-Ignore +from employees | where not(salary_change == 1.19 or salary_change == -3.9) | keep emp_no, salary_change | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10012 | 0.04 +10017 | -6.33 +10020 | -5.81 +10030 | -0.4 +10030 | -0.4 +; + +notInMultivalue-Ignore +from employees | where not(salary_change in (1.19, -3.9)) | keep emp_no, salary_change | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change:double +10012 | 0.04 +10017 | -6.33 +10020 | -5.81 +10030 | -0.4 +10030 | -0.4 +; + +mvDedupe +row a = [1.1, 2.1, 2.1] | eval da = mv_dedupe(a); + + a:double | da:double +[1.1, 2.1, 2.1] | [1.1, 2.1] +; + +autoBucket +FROM employees +| WHERE hire_date >= "1985-01-01T00:00:00Z" AND hire_date < "1986-01-01T00:00:00Z" +| EVAL bh = auto_bucket(height, 20, 1.41, 2.10) +| SORT hire_date +| KEEP hire_date, height, bh +; + +hire_date:date | height:double | bh:double +1985-02-18T00:00:00.000Z | 1.85 | 1.85 +1985-02-24T00:00:00.000Z | 2.0 | 2.0 +1985-05-13T00:00:00.000Z | 2.0 | 2.0 +1985-07-09T00:00:00.000Z | 1.83 | 1.8 +1985-09-17T00:00:00.000Z | 1.45 | 1.4000000000000001 +1985-10-14T00:00:00.000Z | 1.77 | 1.75 +1985-10-20T00:00:00.000Z | 1.94 | 1.9000000000000001 +1985-11-19T00:00:00.000Z | 1.8 | 1.8 +1985-11-20T00:00:00.000Z | 1.99 | 1.9500000000000002 +1985-11-20T00:00:00.000Z | 1.93 | 1.9000000000000001 +1985-11-21T00:00:00.000Z | 2.08 | 2.0500000000000003 +; + +cos +// tag::cos[] +ROW a=1.8 +| EVAL cos=COS(a) +// end::cos[] +; + +// tag::cos-result[] +a:double | cos:double + 1.8 | -0.2272020946930871 +// end::cos-result[] +; + +cosh +// tag::cosh[] +ROW a=1.8 +| EVAL cosh=COSH(a) +// end::cosh[] +; + +// tag::cosh-result[] +a:double | cosh:double + 1.8 | 3.1074731763172667 +// end::cosh-result[] +; + +acos +// tag::acos[] +ROW a=.9 +| EVAL acos=ACOS(a) +// end::acos[] +; + +// tag::acos-result[] +a:double | acos:double + .9 | 0.45102681179626236 +// end::acos-result[] +; + +acosNan +ROW a=12.0 +| EVAL acos=ACOS(a) +; + +a:double | acos:double + 12 | NaN +; + +sin +// tag::sin[] +ROW a=1.8 +| EVAL sin=SIN(a) +// end::sin[] +; + +// tag::sin-result[] +a:double | sin:double + 1.8 | 0.9738476308781951 +// end::sin-result[] +; + +sinh +// tag::sinh[] +ROW a=1.8 +| EVAL sinh=SINH(a) +// end::sinh[] +; + +// tag::sinh-result[] +a:double | sinh:double + 1.8 | 2.94217428809568 +// end::sinh-result[] +; + +asin +// tag::asin[] +ROW a=.9 +| EVAL asin=ASIN(a) +// end::asin[] +; + +// tag::asin-result[] +a:double | asin:double + .9 | 1.1197695149986342 +// end::asin-result[] +; + +asinNan +ROW a=12.0 +| EVAL asin=ASIN(a) +; + +a:double | asin:double + 12 | NaN +; + +tan +// tag::tan[] +ROW a=1.8 +| EVAL tan=TAN(a) +// end::tan[] +; + +// tag::tan-result[] +a:double | tan:double + 1.8 | -4.286261674628062 +// end::tan-result[] +; + +tanh +// tag::tanh[] +ROW a=1.8 +| EVAL tanh=TANH(a) +// end::tanh[] +; + +// tag::tanh-result[] +a:double | tanh:double + 1.8 | 0.9468060128462683 +// end::tanh-result[] +; + +atan +// tag::atan[] +ROW a=12.9 +| EVAL atan=ATAN(a) +// end::atan[] +; + +// tag::atan-result[] +a:double | atan:double + 12.9 | 1.4934316673669235 +// end::atan-result[] +; + +atan2 +// tag::atan2[] +ROW y=12.9, x=.6 +| EVAL atan2=ATAN2(y, x) +// end::atan2[] +; + +// tag::atan2-result[] +y:double | x:double | atan2:double + 12.9 | 0.6 | 1.5243181954438936 +// end::atan2-result[] +; + +toDegrees +// tag::to_degrees[] +ROW rad = [1.57, 3.14, 4.71] +| EVAL deg = TO_DEGREES(rad) +// end::to_degrees[] +; + +// tag::to_degrees-result[] + rad:double | deg:double +[1.57, 3.14, 4.71] | [89.95437383553924, 179.9087476710785, 269.86312150661774] +// end::to_degrees-result[] +; + +toRadians +// tag::to_radians[] +ROW deg = [90.0, 180.0, 270.0] +| EVAL rad = TO_RADIANS(deg) +// end::to_radians[] +; + +// tag::to_radians-result[] + deg:double | rad:double +[90.0, 180.0, 270.0] | [1.5707963267948966, 3.141592653589793, 4.71238898038469] +// end::to_radians-result[] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/folding.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/folding.csv-spec new file mode 100644 index 0000000000000..4e35e0ba9efcf --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/folding.csv-spec @@ -0,0 +1,105 @@ +# +# CSV specs for plans executed locally - plans that can be evaluated (folded) without hitting the storage +# + +# source + +localSourceWithNumeric +row a = 1; + +a:i +1 +; + +localSourceWithString +row s = "string"; + +s:s +"string" +; + +localSourceMultiAttributeNumeric +row a = 1, b = 2; + +a:i | b:i +1 | 2 +; + +localSourceMultiAttributeString +row a = "a", b = "b"; + +a:s | b:s +a | b +; + +localSourceMultiAttributeMixed +row a = 1, b = "b"; + +a:i | b:s +1 | b +; + +# filter + +filterAlwaysFalse +row a = 1 | where true == false; + +a:i +; + +filterEvaluationFalseNumeric +row a = 1 | where a > 1; + +a:i +; + +filterEvaluationTrueNumeric +row a = 1 | where a == 1; + +a:i +1 +; + +filterEvaluationTrueString +row s = "abc" | where starts_with(s, "a"); + +s:s +"abc" +; + +filterEvaluationFalseString +row s = "abc" | where starts_with(s, "c"); + +s:s +; + +filterDisjunctionMatches +row a = 1, b = 2 | where a > 1 or b == 2; + +a:i | b:i +1 | 2 +; + +filterConjunctionMatches +row a = 1, b = 2 | where a > 0 and b > 1; + +a:i | b:i +1 | 2 +; + +# eval + +evalArithmetic +row a = 1, b = 2 | eval x = a + b; + +a:i | b:i | x:i +1 | 2 | 3 +; + +evalStringFunction +row a = "some", b = "string" | eval x = length(a), y = concat(a, b), z = concat("another", b); + +a:s | b:s | x:i | y:s | z:s +"some" | "string" | 4 | "somestring" | "anotherstring" +; + diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec new file mode 100644 index 0000000000000..9dc9444de0155 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec @@ -0,0 +1,190 @@ +simpleGrok +row a = "foo bar" | grok a "%{WORD:b} .*"; + +a:keyword | b:keyword +foo bar | foo +; + + +simpleGrokNoMatch +row a = "foo bar" | grok a "%{NUMBER:b:int} .*"; + +a:keyword | b:integer +foo bar | null +; + + +complexPattern +// tag::grok[] +ROW a = "1953-01-23T12:15:00Z 127.0.0.1 some.email@foo.com 42" +| GROK a "%{TIMESTAMP_ISO8601:date} %{IP:ip} %{EMAILADDRESS:email} %{NUMBER:num:int}" +| KEEP date, ip, email, num +// end::grok[] +; + +// tag::grok-result[] +date:keyword | ip:keyword | email:keyword | num:integer +1953-01-23T12:15:00Z | 127.0.0.1 | some.email@foo.com | 42 +// end::grok-result[] +; + + +typeConversions +row a = "12 15.5 15.6 true" | grok a "%{NUMBER:b:int} %{NUMBER:c:float} %{NUMBER:d:double} %{WORD:e:boolean}"; + +a:keyword | b:integer | c:double | d:double | e:boolean +12 15.5 15.6 true |12 | 15.5 | 15.6 | true +; + + +manualPattern +row a = "asdf bar" | grok a "(?[0-9a-z]{4,11}) (?[0-9a-z]{1,11})"; + +a:keyword | b:keyword | c:keyword +asdf bar | asdf | bar +; + + +manualPatternNoMatch +row a = "foo bar" | grok a "(?[0-9a-z]{4,11}) (?[0-9a-z]{1,11})"; + +a:keyword | b:keyword | c:keyword +foo bar | null | null +; + + +evalGrok +from employees | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:a} %{WORD:b}" | sort emp_no asc | keep full_name, a, b | limit 3; + +full_name:keyword | a:keyword | b:keyword +Georgi Facello | Georgi | Facello +Bezalel Simmel | Bezalel | Simmel +Parto Bamford | Parto | Bamford +; + + +grokExpression +from employees | grok concat(first_name, " ", last_name) "%{WORD:a} %{WORD:b}" | sort emp_no asc | keep a, b | limit 3; + +a:keyword | b:keyword +Georgi | Facello +Bezalel | Simmel +Parto | Bamford +; + + +evalGrokSort +from employees | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:a} %{WORD:b}" | sort a asc | keep full_name, a, b | limit 3; + +full_name:keyword | a:keyword | b:keyword +Alejandro McAlpine | Alejandro | McAlpine +Amabile Gomatam | Amabile | Gomatam +Anneke Preusig | Anneke | Preusig +; + + +grokStats +from employees | eval x = concat(gender, " foobar") | grok x "%{WORD:a} %{WORD:b}" | stats n = max(emp_no) by a | keep a, n | sort a asc; + +a:keyword | n:integer +F | 10100 +M | 10097 +null | 10019 +; + + +nullOnePattern +from employees | where emp_no == 10030 | grok first_name "%{WORD:a}" | keep first_name, a; + +first_name:keyword | a:keyword +null | null +; + + +nullTwoPatterns +from employees | where emp_no == 10030 | grok first_name "%{WORD:a} %{WORD:b}" | keep first_name, a, b; + +first_name:keyword | a:keyword | b:keyword +null | null | null +; + + +overwriteName +from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:emp_no} %{WORD:b}" | keep full_name, emp_no, b | limit 3; + +full_name:keyword | emp_no:keyword | b:keyword +Georgi Facello | Georgi | Facello +Bezalel Simmel | Bezalel | Simmel +Parto Bamford | Parto | Bamford +; + + +overwriteNameWhere +from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:emp_no} %{WORD:b}" | where emp_no == "Bezalel" | keep full_name, emp_no, b | limit 3; + +full_name:keyword | emp_no:keyword | b:keyword +Bezalel Simmel | Bezalel | Simmel +; + + +multivalueOutput +row a = "foo bar" | grok a "%{WORD:b} %{WORD:b}"; + +a:keyword | b:keyword +foo bar | ["foo", "bar"] +; + + +multivalueInput +from employees | where emp_no <= 10006 | grok job_positions "%{WORD:a} %{WORD:b} %{WORD:c}" | sort emp_no | keep emp_no, a, b, c, job_positions; + +emp_no:integer | a:keyword | b:keyword | c:keyword | job_positions:keyword +10001 | Senior | Python | Developer | [Accountant, Senior Python Developer] +10002 | Senior | Team | Lead | Senior Team Lead +10003 | null | null | null | null +10004 | Head | Human | Resources | [Head Human Resources, Reporting Analyst, Support Engineer, Tech Lead] +10005 | null | null | null | null +10006 | [Principal, Senior] | [Support, Team] | [Engineer, Lead] | [Principal Support Engineer,Senior Team Lead, Tech Lead] +; + + +matchAtTheBegin +from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name, " 123 456") | grok full_name "%{WORD:first_name} %{WORD:last_name} %{NUMBER:num:int}" | keep full_name, first_name, last_name, num | limit 3; + +full_name:keyword | first_name:keyword | last_name:keyword | num:integer +Georgi Facello 123 456 | Georgi | Facello | 123 +Bezalel Simmel 123 456 | Bezalel | Simmel | 123 +Parto Bamford 123 456 | Parto | Bamford | 123 +; + + +matchAtTheEnd +from employees | sort emp_no asc | eval full_name = concat("123 ", first_name, " ", last_name, " 123") | grok full_name "%{WORD:first_name} %{WORD:last_name} %{NUMBER:num:int}" | keep full_name, first_name, last_name, num | limit 3; + +full_name:keyword | first_name:keyword | last_name:keyword | num:integer +123 Georgi Facello 123 | Georgi | Facello | 123 +123 Bezalel Simmel 123 | Bezalel | Simmel | 123 +123 Parto Bamford 123 | Parto | Bamford | 123 +; + + +matchInBetween +from employees | sort emp_no asc | eval full_name = concat("123 ", first_name, " ", last_name, " 123 456") | grok full_name "%{WORD:first_name} %{WORD:last_name} %{NUMBER:num:int}" | keep full_name, first_name, last_name, num | limit 3; + +full_name:keyword | first_name:keyword | last_name:keyword | num:integer +123 Georgi Facello 123 456 | Georgi | Facello | 123 +123 Bezalel Simmel 123 456 | Bezalel | Simmel | 123 +123 Parto Bamford 123 456 | Parto | Bamford | 123 +; + + +optionalMatchMv +from employees | grok job_positions "%{WORD:a}?\\s*%{WORD:b}?\\s*%{WORD:c}?" | keep emp_no, a, b, c, job_positions | sort emp_no | limit 5; + +emp_no:integer | a:keyword | b:keyword | c:keyword | job_positions:keyword +10001 | [Accountant, Senior] | Python | Developer | [Accountant, Senior Python Developer] +10002 | Senior | Team | Lead | Senior Team Lead +10003 | null | null | null | null +10004 | [Head, Reporting, Support, Tech] | [Human, Analyst, Engineer, Lead] | Resources | [Head Human Resources, Reporting Analyst, Support Engineer, Tech Lead] +10005 | null | null | null | null +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/hosts.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/hosts.csv new file mode 100644 index 0000000000000..ce898a0b55066 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/hosts.csv @@ -0,0 +1,11 @@ +host:keyword,host_group:text,description:text,card:keyword,ip0:ip,ip1:ip +alpha,DB servers,alpha db server,eth0,127.0.0.1,127.0.0.1 +alpha,DB servers,alpha db server,eth1,::1,::1 +beta,Kubernetes cluster,beta k8s server,eth0,127.0.0.1,::1 +beta,Kubernetes cluster,beta k8s server,eth1,127.0.0.1,127.0.0.2 +beta,Kubernetes cluster,[beta k8s server, beta k8s server2],eth1,127.0.0.1,128.0.0.1 +gamma,Kubernetes cluster 2,gamma k8s server,lo0,fe80::cae2:65ff:fece:feb9,fe81::cae2:65ff:fece:feb9 +gamma,Kubernetes cluster 2,gamma k8s server,eth0,fe80::cae2:65ff:fece:feb9,127.0.0.3 +epsilon,,epsilon gw instance,eth0,[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1],fe80::cae2:65ff:fece:fec1 +epsilon,Gateway instances,,eth1,,[127.0.0.1, 127.0.0.2, 127.0.0.3] +epsilon,Gateway instances,[epsilon host, epsilon2 host],eth2,[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0],[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/id.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/id.csv-spec new file mode 100644 index 0000000000000..9bbdce25f2ab8 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/id.csv-spec @@ -0,0 +1,129 @@ +// +// Tests for _id fields +// + +selectAll +FROM apps [metadata _id]; + +id:integer |name:keyword |version:version | _id:keyword +1 |aaaaa |1 | 1 +2 |bbbbb |2.1 | 2 +3 |ccccc |2.3.4 | 3 +4 |ddddd |2.12.0 | 4 +5 |eeeee |1.11.0 | 5 +6 |fffff |5.2.9 | 6 +7 |ggggg |5.2.9-SNAPSHOT | 7 +8 |hhhhh |1.2.3.4 | 8 +9 |iiiii |bad | 9 +10 |jjjjj |5.2.9 | 10 +11 |kkkkk |null | 11 +12 |aaaaa |1.2.3.4 | 12 +13 |lllll |null | 13 +14 |mmmmm |5.2.9 | 14 +; + +filterById +FROM apps [metadata _id]| WHERE _id == "4"; + +id:i |name:k |version:v | _id:k +4 |ddddd |2.12.0 | 4 +; + +keepId +FROM apps [metadata _id] | WHERE id == 3 | KEEP _id; + +_id:k +3 +; + +idRangeAndSort +FROM apps [metadata _id] | WHERE _id >= "2" AND _id <= "7" | SORT _id | keep id, name, _id; + +id:i |name:k | _id:k +2 |bbbbb | 2 +3 |ccccc | 3 +4 |ddddd | 4 +5 |eeeee | 5 +6 |fffff | 6 +7 |ggggg | 7 +; + +orderById +FROM apps [metadata _id] | KEEP _id, name | SORT _id; + +_id:k | name:s +1 | aaaaa +10 | jjjjj +11 | kkkkk +12 | aaaaa +13 | lllll +14 | mmmmm +2 | bbbbb +3 | ccccc +4 | ddddd +5 | eeeee +6 | fffff +7 | ggggg +8 | hhhhh +9 | iiiii +; + +orderByIdDesc +FROM apps [metadata _id] | KEEP _id, name | SORT _id DESC; + +_id:k | name:s + +9 | iiiii +8 | hhhhh +7 | ggggg +6 | fffff +5 | eeeee +4 | ddddd +3 | ccccc +2 | bbbbb +14 | mmmmm +13 | lllll +12 | aaaaa +11 | kkkkk +10 | jjjjj +1 | aaaaa +; + +concatId +FROM apps [metadata _id] | eval c = concat(_id, name) | SORT _id | KEEP c; + +c:k +1aaaaa +10jjjjj +11kkkkk +12aaaaa +13lllll +14mmmmm +2bbbbb +3ccccc +4ddddd +5eeeee +6fffff +7ggggg +8hhhhh +9iiiii +; + +statsOnId +FROM apps [metadata _id] | stats c = count(_id), d = count_distinct(_id); + +c:l | d:l +14 | 14 +; + + +statsOnIdByGroup +FROM apps [metadata _id] | stats c = count(_id) by name | sort c desc, name | limit 5; + +c:l | name:k +2 | aaaaa +1 | bbbbb +1 | ccccc +1 | ddddd +1 | eeeee +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ints.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ints.csv-spec new file mode 100644 index 0000000000000..0c73e24136a0f --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ints.csv-spec @@ -0,0 +1,471 @@ +// Integral types-specific tests + +inLongAndInt +from employees | where avg_worked_seconds in (372957040, salary_change.long, 236703986) | where emp_no in (10017, emp_no - 1) | keep emp_no, avg_worked_seconds; + +emp_no:integer |avg_worked_seconds:long +10017 |236703986 +; + +inShortAndByte +from employees | keep emp_no, languages.short, languages.byte | where languages.short in (2, 4, 5) and languages.byte in (4, -1) and emp_no < 10010; + +emp_no:integer |languages.short:short|languages.byte:byte +10003 |4 |4 +10007 |4 |4 +; + +inCast +from employees | keep emp_no, languages.byte, avg_worked_seconds, height | where languages.byte in (4, -1, avg_worked_seconds, 1000000000000, null, height) and emp_no < 10010; + +emp_no:integer |languages.byte:byte |avg_worked_seconds:long |height:double +10003 |4 |200296405 |1.83 +10007 |4 |393084805 |1.7 +; + +// `<= 10030` insures going over records where is_null(languages)==true; `in (.., emp_no)` prevents pushing the IN to Lucene +inOverNulls +from employees | keep emp_no, languages | where languages is null or emp_no <= 10030 | where languages in (2, 3, emp_no); + +emp_no:integer |languages:integer +10001 |2 +10006 |3 +10008 |2 +10016 |2 +10017 |2 +10018 |2 +10030 |3 +; + +// conversions to UNSIGNED_LONG + +convertULtoUL +row ul = [9223372036854775808, 18446744073709551615] | eval ulul = to_unsigned_long(ul); + + ul:ul | ulul:ul +[9223372036854775808, 18446744073709551615]|[9223372036854775808, 18446744073709551615] +; + +convertIntToUL +row int = [5013792, 520128] | eval ul = to_ulong(int); + +int:integer |ul:ul +[5013792, 520128] |[5013792, 520128] +; + +convertLongToUL +row long = [501379200000, 520128000000] | eval ul = to_ul(long); + +long:long |ul:ul +[501379200000, 520128000000] |[501379200000, 520128000000] +; + +convertDoubleToUL +row d = 123.4 | eval ul = to_ul(d), overflow = to_ul(1e20); +warning:Line 1:48: evaluation of [to_ul(1e20)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [1.0E20] out of [unsigned_long] range + +d:double |ul:ul |overflow:ul +123.4 |123 |null +; + +convertBooleanToUL +row tf = [true, false] | eval t2ul = to_ul(true), f2ul = to_ul(false), tf2ul = to_ul(tf); + +tf:boolean |t2ul:ul |f2ul:ul |tf2ul:ul +[true, false] |1 |0 |[1, 0] +; + +convertDatetimeToUL +from employees | sort emp_no | eval hired_ul = to_ul(hire_date) | keep emp_no, hire_date, hired_ul | limit 3; + +emp_no:integer |hire_date:date |hired_ul:ul +10001 |1986-06-26T00:00:00.000Z|520128000000 +10002 |1985-11-21T00:00:00.000Z|501379200000 +10003 |1986-08-28T00:00:00.000Z|525571200000 +; + +convertStringToUL +// tag::to_unsigned_long-str[] +ROW str1 = "2147483648", str2 = "2147483648.2", str3 = "foo" +| EVAL long1 = TO_UNSIGNED_LONG(str1), long2 = TO_ULONG(str2), long3 = TO_UL(str3) +// end::to_unsigned_long-str[] +; +warning:Line 2:72: evaluation of [TO_UL(str3)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: Character f is neither a decimal digit number, decimal point, nor \"e\" notation exponential mark. + + +// tag::to_unsigned_long-str-result[] +str1:keyword |str2:keyword |str3:keyword |long1:unsigned_long |long2:unsigned_long |long3:unsigned_long +2147483648 |2147483648.2 |foo |2147483648 |2147483648 |null +// end::to_unsigned_long-str-result[] +; + +// conversions to LONG + +convertLongToLong +row long = [501379200000, 520128000000] | eval ll = to_long(long); + +long:long |ll:long +[501379200000, 520128000000] |[501379200000, 520128000000] +; + +convertIntToLong +row int = [5013792, 520128] | eval long = to_long(int); + +int:integer |long:long +[5013792, 520128] |[5013792, 520128] +; + +convertULToLong +row ul = [9223372036854775807, 9223372036854775808] | eval long = to_long(ul); +warning:Line 1:67: evaluation of [to_long(ul)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [9223372036854775808] out of [long] range + + ul:ul | long:long +[9223372036854775807, 9223372036854775808]|9223372036854775807 +; + +convertDatetimeToLong +from employees | sort emp_no | eval hired_long = to_long(hire_date) | keep emp_no, hire_date, hired_long | limit 3; + +emp_no:integer |hire_date:date |hired_long:long +10001 |1986-06-26T00:00:00.000Z|520128000000 +10002 |1985-11-21T00:00:00.000Z|501379200000 +10003 |1986-08-28T00:00:00.000Z|525571200000 +; + +convertBooleanToLong +row tf = [true, false] | eval t2l = to_long(true), f2l = to_long(false), tf2l = to_long(tf); + +tf:boolean |t2l:long |f2l:long |tf2l:long +[true, false] |1 |0 |[1, 0] +; + +convertStringToLong +// tag::to_long-str[] +ROW str1 = "2147483648", str2 = "2147483648.2", str3 = "foo" +| EVAL long1 = TO_LONG(str1), long2 = TO_LONG(str2), long3 = TO_LONG(str3) +// end::to_long-str[] +; +warning:Line 2:62: evaluation of [TO_LONG(str3)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"foo\" + + +// tag::to_long-str-result[] +str1:keyword |str2:keyword |str3:keyword |long1:long |long2:long |long3:long +2147483648 |2147483648.2 |foo |2147483648 |2147483648 |null +// end::to_long-str-result[] +; + +convertDoubleToLong +row d = 123.4 | eval d2l = to_long(d), overflow = to_long(1e19); +warning:Line 1:51: evaluation of [to_long(1e19)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [1.0E19] out of [long] range + +d:double |d2l:long |overflow:long +123.4 |123 |null +; + +// conversions to INTEGER + +convertIntToInt +row int = [5013792, 520128] | eval ii = to_integer(int); + +int:integer |ii:integer +[5013792, 520128] |[5013792, 520128] +; + +convertLongToInt +// tag::to_int-long[] +ROW long = [5013792, 2147483647, 501379200000] +| EVAL int = TO_INTEGER(long) +// end::to_int-long[] +; +warning:Line 2:14: evaluation of [TO_INTEGER(long)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [501379200000] out of [integer] range + +// tag::to_int-long-result[] +long:long |int:integer +[5013792, 2147483647, 501379200000] |[5013792, 2147483647] +// end::to_int-long-result[] +; + +convertULToInt +row ul = [2147483647, 9223372036854775808] | eval int = to_int(ul); +warning:Line 1:57: evaluation of [to_int(ul)] failed, treating result as null. Only first 20 failures recorded. +// UL conversion to int dips into long; not the most efficient, but it's how SQL does it too. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [9223372036854775808] out of [long] range + + ul:ul |int:integer +[2147483647, 9223372036854775808]|2147483647 +; + +convertDatetimeToInt +row int = [5013792, 520128] | eval dt = to_datetime(int) | eval ii = to_integer(dt); + +int:integer |dt:date |ii:integer +[5013792, 520128]|[1970-01-01T01:23:33.792Z, 1970-01-01T00:08:40.128Z]|[5013792, 520128] +; + +convertBooleanToInt +row tf = [true, false] | eval t2i = to_integer(true), f2i = to_integer(false), tf2i = to_integer(tf); + +tf:boolean |t2i:integer |f2i:integer |tf2i:integer +[true, false] |1 |0 |[1, 0] +; + +convertStringToInt +row int_str = "2147483647", int_dbl_str = "2147483647.2" | eval is2i = to_integer(int_str), ids2i = to_integer(int_dbl_str), overflow = to_integer("2147483648"), no_number = to_integer("foo"); +warning:Line 1:137: evaluation of [to_integer(\"2147483648\")] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"2147483648\" +warning:Line 1:175: evaluation of [to_integer(\"foo\")] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"foo\" + +int_str:keyword |int_dbl_str:keyword |is2i:integer|ids2i:integer |overflow:integer |no_number:integer +2147483647 |2147483647.2 |2147483647 |2147483647 |null |null +; + +convertDoubleToInt +row d = 123.4 | eval d2i = to_integer(d), overflow = to_integer(1e19); +warning:Line 1:54: evaluation of [to_integer(1e19)] failed, treating result as null. Only first 20 failures recorded. +warning:org.elasticsearch.xpack.ql.QlIllegalArgumentException: [1.0E19] out of [long] range + +d:double |d2i:integer |overflow:integer +123.4 |123 |null +; + +lessThanMultivalue +from employees | where salary_change.int < 1 | keep emp_no, salary_change.int | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10006 |-3 +10012 | 0 +10017 |-6 +10020 |-5 +10030 | 0 +; + +greaterThanMultivalue +from employees | where salary_change.int > 1 | keep emp_no, salary_change.int | sort emp_no | limit 5; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10044 | 8 +10046 | 2 +10066 | 5 +10079 | 7 +10086 |13 +; + +equalToMultivalue +from employees | where salary_change.int == 0 | keep emp_no, salary_change.int | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10012 | 0 +10030 | 0 +10077 | 0 +10093 | 0 +; + +equalToOrEqualToMultivalue +from employees | where salary_change.int == 1 or salary_change.int == 8 | keep emp_no, salary_change.int | sort emp_no; + +// Note that multivalued salaries are filtered out +emp_no:integer |salary_change.int:integer +10001 |1 +10044 |8 +; + +inMultivalue +from employees | where salary_change.int in (1, 7) | keep emp_no, salary_change.int | sort emp_no; + +// Note that multivalued salaries are filtered out +emp_no:integer |salary_change.int:integer +10001 |1 +10079 |7 +; + +notLessThanMultivalue +from employees | where not(salary_change.int < 1) | keep emp_no, salary_change.int | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10001 | 1 +10044 | 8 +10046 | 2 +10066 | 5 +10079 | 7 +; + +notGreaterThanMultivalue +from employees | where not(salary_change.int > 1) | keep emp_no, salary_change.int | sort emp_no | limit 5; + +// Note that multivalued salaries aren't less than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10001 | 1 +10006 | -3 +10012 | 0 +10017 | -6 +10020 | -5 +; + +notEqualToMultivalue +from employees | where not(salary_change.int == 1) | keep emp_no, salary_change.int | sort emp_no | limit 5; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10006 | -3 +10012 | 0 +10017 | -6 +10020 | -5 +10030 | 0 +; + +notEqualToAndEqualToMultivalue-Ignore +from employees | where not(salary_change.int == 1 or salary_change.int == -4) | keep emp_no, salary_change.int | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10012 | 0 +10017 | -6 +10020 | -5 +10030 | 0 +10030 | 0 +; + +notInMultivalue-Ignore +from employees | where not(salary_change.int in (1, -4)) | keep emp_no, salary_change.int | sort emp_no; + +// Note that multivalued salaries aren't greater than 1 - they are null - so they aren't included +emp_no:integer |salary_change.int:integer +10012 | 0 +10017 | -6 +10020 | -6 +10030 | -0 +10030 | -0 +; + +mvDedupe +row a = [1, 2, 2, 3] | eval da = mv_dedupe(a); + + a:integer | da:integer +[1, 2, 2, 3] | [1, 2, 3] +; + +autoBucket +// tag::auto_bucket[] +FROM employees +| WHERE hire_date >= "1985-01-01T00:00:00Z" AND hire_date < "1986-01-01T00:00:00Z" +| EVAL bs = auto_bucket(salary, 20, 25324, 74999) +| SORT hire_date +| KEEP hire_date, salary, bs +// end::auto_bucket[] +; + +// tag::auto_bucket-result[] +hire_date:date | salary:integer | bs:double +1985-02-18T00:00:00.000Z | 66174 | 65000.0 +1985-02-24T00:00:00.000Z | 26436 | 25000.0 +1985-05-13T00:00:00.000Z | 44817 | 40000.0 +1985-07-09T00:00:00.000Z | 62405 | 60000.0 +1985-09-17T00:00:00.000Z | 49095 | 45000.0 +1985-10-14T00:00:00.000Z | 54329 | 50000.0 +1985-10-20T00:00:00.000Z | 48735 | 45000.0 +1985-11-19T00:00:00.000Z | 52833 | 50000.0 +1985-11-20T00:00:00.000Z | 74999 | 70000.0 +1985-11-20T00:00:00.000Z | 33956 | 30000.0 +1985-11-21T00:00:00.000Z | 56371 | 55000.0 +// end::auto_bucket-result[] +; + +cos +ROW a=2 | EVAL cos=COS(a); + +a:integer | cos:double + 2 | -0.4161468365471424 +; + +acos +ROW a=1 | EVAL acos=ACOS(a); + +a:integer | acos:double + 1 | 0.0 +; + +cosh +ROW a=2 | EVAL cosh=COSH(a); + +a:integer | cosh:double + 2 | 3.7621956910836314 +; + +sin +ROW a=2 | EVAL sin=SIN(a); + +a:integer | sin:double + 2 | 0.9092974268256817 +; + +sinh +ROW a=2 | EVAL sinh=SINH(a); + +a:integer | sinh:double + 2 | 3.626860407847019 +; + +asin +ROW a=1 | EVAL asin=ASIN(a); + +a:integer | asin:double + 1 | 1.5707963267948966 +; + + +tan +ROW a=2 | EVAL tan=TAN(a); + +a:integer | tan:double + 2 | -2.185039863261519 +; + +tanh +ROW a=2 | EVAL tanh=TANH(a); + +a:integer | tanh:double + 2 | 0.9640275800758169 +; + +atan +ROW a=2 | EVAL atan=ATAN(a); + +a:integer | atan:double + 2 | 1.1071487177940904 +; + +atan2 +ROW y=2, x=12 | EVAL atan2=ATAN2(y, x); + +y:integer | x:integer | atan2:double + 2 | 12 | 0.16514867741462683 +; + +toDegrees +ROW rad = [1, 2, 3, 4] +| EVAL deg = TO_DEGREES(rad) +; + + rad:integer | deg:double + [1, 2, 3, 4] | [57.29577951308232, 114.59155902616465, 171.88733853924697, 229.1831180523293] +; + +toRadians +ROW deg = [90, 180, 270] +| EVAL rad = TO_RADIANS(deg) +; + + deg:integer | rad:double +[90, 180, 270] | [1.5707963267948966, 3.141592653589793, 4.71238898038469] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ip.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ip.csv-spec new file mode 100644 index 0000000000000..d69be91cd2f22 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ip.csv-spec @@ -0,0 +1,234 @@ +simpleProject +from hosts | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +eth0 |beta |127.0.0.1 |::1 +eth1 |beta |127.0.0.1 |127.0.0.2 +eth1 |beta |127.0.0.1 |128.0.0.1 +lo0 |gamma |fe80::cae2:65ff:fece:feb9 |fe81::cae2:65ff:fece:feb9 +eth0 |gamma |fe80::cae2:65ff:fece:feb9 |127.0.0.3 +eth0 |epsilon |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1]|fe80::cae2:65ff:fece:fec1 +eth1 |epsilon |null |[127.0.0.1, 127.0.0.2, 127.0.0.3] +eth2 |epsilon |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +equals +from hosts | sort host, card | where ip0 == ip1 | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +; + +# ignored due to unstable sort +equalityNoSort-Ignore +from hosts | where ip0 == ip1 | keep card, host, ip0, ip1 + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +eth2 |epsilon |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0]|[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +greaterThanEquals +from hosts | sort host, card | where ip0 >= ip1 | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +eth0 |beta |127.0.0.1 |::1 +eth0 |gamma |fe80::cae2:65ff:fece:feb9|127.0.0.3 +; + +# ignored due to unstable sort +greaterThanEqualsNoSort-Ignore +from hosts | where ip0 >= ip1 | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +eth0 |beta |127.0.0.1 |::1 +eth0 |gamma |fe80::cae2:65ff:fece:feb9 |127.0.0.3 +eth2 |epsilon |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0]|[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +lessThen +from hosts | sort host, card | where ip0 < ip1 | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +eth1 |beta |127.0.0.1 |128.0.0.1 +lo0 |gamma |fe80::cae2:65ff:fece:feb9|fe81::cae2:65ff:fece:feb9 +; + +notEquals +from hosts | sort host, card, ip1 | where ip0 != ip1 | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |beta |127.0.0.1 |::1 +eth1 |beta |127.0.0.1 |127.0.0.2 +eth1 |beta |127.0.0.1 |128.0.0.1 +eth0 |gamma |fe80::cae2:65ff:fece:feb9|127.0.0.3 +lo0 |gamma |fe80::cae2:65ff:fece:feb9|fe81::cae2:65ff:fece:feb9 +; + +aggAndSort +from hosts | stats c=count(ip0) by ip0 | sort ip0 | rename ip0 as ip; + +c:long |ip:ip +1 |::1 +4 |127.0.0.1 +5 |fe80::cae2:65ff:fece:feb9 +3 |fe80::cae2:65ff:fece:fec0 +3 |fe80::cae2:65ff:fece:fec1 +2 |fe81::cae2:65ff:fece:feb9 +2 |fe82::cae2:65ff:fece:fec0 +0 |null +; + +doubleSort +from hosts | sort ip0 asc nulls first, ip1 desc| keep card, host, ip0, ip1; + + card:keyword | host:keyword | ip0:ip | ip1:ip +eth1 |epsilon |null |[127.0.0.1, 127.0.0.2, 127.0.0.3] +eth1 |alpha |::1 |::1 +eth1 |beta |127.0.0.1 |128.0.0.1 +eth1 |beta |127.0.0.1 |127.0.0.2 +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth0 |beta |127.0.0.1 |::1 +lo0 |gamma |fe80::cae2:65ff:fece:feb9 |fe81::cae2:65ff:fece:feb9 +eth0 |epsilon |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1]|fe80::cae2:65ff:fece:fec1 +eth0 |gamma |fe80::cae2:65ff:fece:feb9 |127.0.0.3 +eth2 |epsilon |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +isNull +from hosts | where ip0 is null | keep ip0, ip1; + +ip0:ip |ip1:ip +null |[127.0.0.1, 127.0.0.2, 127.0.0.3] +; + +conditional +from hosts | eval eq=case(ip0==ip1, ip0, ip1) | keep eq, ip0, ip1; + +eq:ip |ip0:ip |ip1:ip +127.0.0.1 |127.0.0.1 |127.0.0.1 +::1 |::1 |::1 +::1 |127.0.0.1 |::1 +127.0.0.2 |127.0.0.1 |127.0.0.2 +128.0.0.1 |127.0.0.1 |128.0.0.1 +fe81::cae2:65ff:fece:feb9 |fe80::cae2:65ff:fece:feb9 |fe81::cae2:65ff:fece:feb9 +127.0.0.3 |fe80::cae2:65ff:fece:feb9 |127.0.0.3 +fe80::cae2:65ff:fece:fec1 |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1]|fe80::cae2:65ff:fece:fec1 +[127.0.0.1, 127.0.0.2, 127.0.0.3] |null |[127.0.0.1, 127.0.0.2, 127.0.0.3] +[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0]|[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +in +from hosts | eval eq=case(ip0==ip1, ip0, ip1) | where eq in (ip0, ip1) | keep card, host, ip0, ip1, eq; + +card:keyword |host:keyword |ip0:ip |ip1:ip |eq:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 |::1 +eth0 |beta |127.0.0.1 |::1 |::1 +eth1 |beta |127.0.0.1 |127.0.0.2 |127.0.0.2 +eth1 |beta |127.0.0.1 |128.0.0.1 |128.0.0.1 +lo0 |gamma |fe80::cae2:65ff:fece:feb9 |fe81::cae2:65ff:fece:feb9|fe81::cae2:65ff:fece:feb9 +eth0 |gamma |fe80::cae2:65ff:fece:feb9 |127.0.0.3 |127.0.0.3 +eth0 |epsilon |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1]|fe80::cae2:65ff:fece:fec1|fe80::cae2:65ff:fece:fec1 +; + +cidrMatchSimple +from hosts | where cidr_match(ip1, "127.0.0.2/32") | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +; + +cidrMatchNullField +from hosts | where cidr_match(ip0, "127.0.0.2/32") is null | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |epsilon |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1]|fe80::cae2:65ff:fece:fec1 +eth1 |epsilon |null |[127.0.0.1, 127.0.0.2, 127.0.0.3] +eth2 |epsilon |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +cdirMatchMultipleArgs +from hosts | where cidr_match(ip1, "127.0.0.2/32", "127.0.0.3/32") | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +eth0 |gamma |fe80::cae2:65ff:fece:feb9|127.0.0.3 +; + +cidrMatchFunctionArg +from hosts | where cidr_match(ip1, concat("127.0.0.2", "/32"), "127.0.0.3/32") | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +eth0 |gamma |fe80::cae2:65ff:fece:feb9|127.0.0.3 +; + +cidrMatchFieldArg +from hosts | eval cidr="127.0.0.2" | where cidr_match(ip1, cidr, "127.0.0.3/32") | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +eth0 |gamma |fe80::cae2:65ff:fece:feb9|127.0.0.3 +; + +convertFromIP +from hosts | keep ip0 | eval ip0ip = to_ip(ip0) | sort ip0ip desc | limit 2; + +ip0:ip |ip0ip:ip +null |null +[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0]|[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +convertFromString +// tag::to_ip[] +ROW str1 = "1.1.1.1", str2 = "foo" +| EVAL ip1 = TO_IP(str1), ip2 = TO_IP(str2) +| WHERE CIDR_MATCH(ip1, "1.0.0.0/8") +// end::to_ip[] +; +warning:Line 2:33: evaluation of [TO_IP(str2)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.IllegalArgumentException: 'foo' is not an IP string literal. + +// tag::to_ip-result[] +str1:keyword |str2:keyword |ip1:ip |ip2:ip +1.1.1.1 |foo |1.1.1.1 |null +// end::to_ip-result[] +; + +pushDownIP +from hosts | where ip1 == to_ip("::1") | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |alpha |::1 |::1 +eth0 |beta |127.0.0.1 |::1 +; + +pushDownIPWithIn +from hosts | where ip1 in (to_ip("::1"), to_ip("127.0.0.1")) | keep card, host, ip0, ip1; + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth0 |alpha |127.0.0.1 |127.0.0.1 +eth1 |alpha |::1 |::1 +eth0 |beta |127.0.0.1 |::1 +; + +pushDownIPWithComparision +from hosts | where ip1 > to_ip("127.0.0.1") | keep card, ip1; + +card:keyword |ip1:ip +eth1 |127.0.0.2 +eth1 |128.0.0.1 +lo0 |fe81::cae2:65ff:fece:feb9 +eth0 |127.0.0.3 +eth0 |fe80::cae2:65ff:fece:fec1 +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/keep.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/keep.csv-spec new file mode 100644 index 0000000000000..69a64a0a03b11 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/keep.csv-spec @@ -0,0 +1,522 @@ +projectFrom +from employees | keep languages, emp_no, first_name, last_name | limit 10; + +languages:integer | emp_no:integer | first_name:keyword | last_name:keyword +2 | 10001 | Georgi | Facello +5 | 10002 | Bezalel | Simmel +4 | 10003 | Parto | Bamford +5 | 10004 | Chirstian | Koblick +1 | 10005 | Kyoichi | Maliniak +3 | 10006 | Anneke | Preusig +4 | 10007 | Tzvetan | Zielinski +2 | 10008 | Saniya | Kalloufi +1 | 10009 | Sumant | Peac +4 | 10010 | Duangkaew | Piveteau +; + +projectFromWithFilter +from employees | keep languages, emp_no, first_name, last_name | eval x = emp_no + 10 | where x > 10040 and x < 10050 | limit 5; + +languages:integer | emp_no:integer | first_name:keyword | last_name:keyword | x:integer +4 | 10031 | null | Joslin | 10041 +3 | 10032 | null | Reistad | 10042 +1 | 10033 | null | Merlo | 10043 +1 | 10034 | null | Swan | 10044 +5 | 10035 | null | Chappelet | 10045 +; + +whereWithAverage +from employees | where languages == 5 | stats avg(avg_worked_seconds); + +avg(avg_worked_seconds):double +313301314.9047619 +; + +whereWithCount +from employees | where languages == 1 | keep languages | stats c=count(languages); + +c : long +15 +; + +averageByField +from employees | stats avg(avg_worked_seconds) by languages; + +avg(avg_worked_seconds):double | languages:integer + 3.181719481E8 | null + 3.0318626831578946E8 | 2 + 3.133013149047619E8 | 5 + 2.863684210555556E8 | 4 + 2.94833632E8 | 1 + 2.978159518235294E8 | 3 +; + +whereWithAverageBySubField +from employees | where languages + 1 == 6 | stats avg(avg_worked_seconds) by languages.long; + +avg(avg_worked_seconds):double | languages.long:long +313301314.9047619 | 5 +; + +statsBySubField +from employees | stats avg=avg(avg_worked_seconds),min=min(avg_worked_seconds),max=max(avg_worked_seconds) by languages.long; + + avg:double | min:long | max:long | languages.long:long +3.181719481E8 | 226435054 | 374037782 | null +3.0318626831578946E8 | 212460105 | 377713748 | 2 +3.133013149047619E8 | 203838153 | 390266432 | 5 +2.863684210555556E8 | 200296405 | 393084805 | 4 +2.94833632E8 | 208374744 | 387408356 | 1 +2.978159518235294E8 | 203989706 | 394597613 | 3 +; + +statsBySubFieldSortedByKey +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | stats avg=avg(avg_worked_seconds),min=min(avg_worked_seconds),max=max(avg_worked_seconds) by languages.long | sort languages.long; + + avg:double | min:long | max:long | languages.long:long +2.94833632E8 | 208374744 | 387408356 | 1 +3.0318626831578946E8 | 212460105 | 377713748 | 2 +2.978159518235294E8 | 203989706 | 394597613 | 3 +2.863684210555556E8 | 200296405 | 393084805 | 4 +3.133013149047619E8 | 203838153 | 390266432 | 5 +3.181719481E8 | 226435054 | 374037782 | null +; + +avgOfIntegerWithSortByGroupingKey +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | stats avg(salary) by last_name | sort last_name desc | limit 10; + +avg(salary):double | last_name:keyword +54462.0 | Zschoche +39110.0 | Zockler +74572.0 | Zielinski +71165.0 | Waschkowski +40031.0 | Warwick +34341.0 | Tzvieli +37853.0 | Tramer +48735.0 | Terkki +39356.0 | Tempesti +39638.0 | Syrzycki +; + +avgOfInteger-Ignore +// Without "sort last_name" the results are randomly returned by CSV tests infrastructure, while ES sorts them by last_name. +// The OrdinalsGroupingOperator is doing this by default (using ordinals for keywords). +// https://github.com/elastic/elasticsearch-internal/issues/767 +from employees | stats avg(salary) by last_name | limit 10; + +avg(salary):double | last_name:keyword +50249.0 | Awdeh +46595.0 | Azuma +52833.0 | Baek +61805.0 | Bamford +38992.0 | Bernatsky +33370.0 | Bernini +28336.0 | Berztiss +41933.0 | Bierman +29175.0 | Billingsley +58715.0 | Bouloucos +; + +avgOfIntegerSortedExplicitly +from employees | stats avg(salary) by last_name | sort last_name | limit 10; + +avg(salary):double | last_name:keyword +50249.0 | Awdeh +46595.0 | Azuma +52833.0 | Baek +61805.0 | Bamford +38992.0 | Bernatsky +33370.0 | Bernini +28336.0 | Berztiss +41933.0 | Bierman +29175.0 | Billingsley +58715.0 | Bouloucos +; + +statsOfInteger +from employees | where starts_with(last_name, "L") | stats a=avg(salary), s=sum(salary), c=count(last_name) by last_name; + + a:double | s:long | c:long |last_name:keyword +42520.0 |85040 |2 |Lortz +56415.0 |56415 |1 |Lenart +52121.0 |52121 |1 |Leonhardt +40612.0 |40612 |1 |Lipner +; + +medianByFieldAndSortedByValue +from employees | stats med=median(salary) by languages | sort med | limit 1; + +med:double | languages:integer +38992.0 | 5 +; + +medianByFieldAndSortedByValue2 +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | where languages > 0 | stats med=median(salary) by languages | sort med; + +med:double | languages:integer +38992.0 | 5 +44353.0 | 4 +44956.0 | 2 +49095.0 | 1 +54462.0 | 3 +; + +medianByFieldAndSortedByAggregatedValue +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | where languages > 0 | stats med=median(salary) by languages | sort languages; + +med:double | languages:integer +49095.0 | 1 +44956.0 | 2 +54462.0 | 3 +44353.0 | 4 +38992.0 | 5 +; + +multiConditionalWhere +from employees | eval abc = 1+2 | where (abc + emp_no > 10100 or languages == 1) or (abc + emp_no < 10005 and gender == "F") | keep emp_no, languages, gender, first_name, abc; + +emp_no:integer | languages:integer | gender:keyword | first_name:keyword | abc:integer +10005 | 1 | M | Kyoichi | 3 +10009 | 1 | F | Sumant | 3 +10013 | 1 | null | Eberhardt | 3 +10019 | 1 | null | Lillian | 3 +10033 | 1 | M | null | 3 +10034 | 1 | M | null | 3 +10041 | 1 | F | Uri | 3 +10043 | 1 | M | Yishay | 3 +10044 | 1 | F | Mingsen | 3 +10052 | 1 | M | Heping | 3 +10061 | 1 | M | Tse | 3 +10083 | 1 | M | Vishv | 3 +10084 | 1 | M | Tuval | 3 +10086 | 1 | M | Somnath | 3 +10092 | 1 | F | Valdiodio | 3 +10098 | 4 | F | Sreekrishna | 3 +10099 | 2 | F | Valter | 3 +10100 | 4 | F | Hironobu | 3 +; + +projectFromWithStatsAfterLimit +from employees | keep gender, avg_worked_seconds, first_name, last_name | limit 10 | stats m = max(avg_worked_seconds) by gender; + + m:long | gender:keyword +315236372 | null +311267831 | M +393084805 | F +; + +projectFromWithStatsAndSort-Ignore +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | keep gender, avg_worked_seconds, first_name, last_name | stats m = max(avg_worked_seconds) by last_name | sort m desc; + +m:long | last_name:keyword +311267831 | M +393084805 | F +315236372 | +311267831 | M +393084805 | F +; + +sortFirstProjectAfter +// https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | sort languages asc nulls last, emp_no asc | limit 3 | keep emp_no, languages, first_name, last_name; + +emp_no:integer | languages:integer | first_name:keyword | last_name:keyword +10005 | 1 | Kyoichi | Maliniak +10009 | 1 | Sumant | Peac +10013 | 1 | Eberhardt | Terkki +; + +sortWithLimitOne +from employees | sort languages | limit 1; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | height:double | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean +244294991 |1955-01-21T00:00:00.000Z|10005 |Kyoichi |M |2.05 |2.049999952316284|2.05078125 |2.05 |1989-09-12T00:00:00.000Z|[false, false, false, true]|null |1 |1 |1 |1 |Maliniak |63528 |[-2.14, 13.07] |[-2, 13] |[-2, 13] |true +; + +sortWithLimitFifteenAndProject +//https://github.com/elastic/elasticsearch-internal/issues/414 +from employees | sort height desc, languages.long nulls last, still_hired | limit 15 | keep height, languages.long, still_hired; + +height:double | languages.long:long | still_hired:boolean +2.1 | 2 | true +2.1 | 3 | false +2.1 | 5 | false +2.1 | 5 | true +2.1 | null | true +2.09 | 3 | true +2.08 | 5 | true +2.08 | null | true +2.07 | 2 | false +2.07 | null | true +2.06 | 1 | false +2.06 | 1 | false +2.05 | 1 | true +2.04 | 5 | false +2.03 | 2 | true +; + +simpleEvalWithSortAndLimitOne +from employees | eval x = languages + 7 | sort x | limit 1; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | height:double | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean | x:integer +244294991 |1955-01-21T00:00:00.000Z|10005 |Kyoichi |M |2.05 |2.049999952316284|2.05078125 |2.05 |1989-09-12T00:00:00.000Z|[false, false, false, true]|null |1 |1 |1 |1 |Maliniak |63528 |[-2.14, 13.07] |[-2, 13] |[-2, 13]|true |8 +; + +evalOfAverageValue +from employees | stats avg_salary = avg(salary) | eval x = avg_salary + 7; + +avg_salary:double | x:double +48248.55 | 48255.55 +; + +averageOfEvalValue +from employees | eval ratio = salary / height | stats avg(ratio); + +avg(ratio):double +27517.279737149947 +; + +simpleWhere +from employees | where salary > 70000 | keep first_name, last_name, salary; + +first_name:keyword | last_name:keyword | salary:integer +Tzvetan | Zielinski | 74572 +Lillian | Haddadi | 73717 +Divier | Reistad | 73851 +Otmar | Herbst | 74999 +null | Merlo | 70011 +Moss | Shanbhogue | 74970 +Remzi | Waschkowski | 71165 +Valter | Sullins | 73578 +; + +whereAfterProject +from employees | keep salary | where salary > 70000; + +salary:integer +74572 +73717 +73851 +74999 +70011 +74970 +71165 +73578 +; + +whereWithEvalGeneratedValue +// the result from running on ES is the one with many decimals the test that runs locally is the one rounded to 2 decimals +// the "height" fields have the values as 1.7, 1.7000000476837158, 1.7001953125, 1.7 +from employees | eval x = salary / 2 | where x > 37000; + +avg_worked_seconds:long | birth_date:date | emp_no:integer | first_name:keyword | gender:keyword | height:double | height.float:double | height.half_float:double | height.scaled_float:double | hire_date:date | is_rehired:boolean | job_positions:keyword | languages:integer | languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | salary:integer | salary_change:double | salary_change.int:integer |salary_change.long:long | still_hired:boolean | x:integer +393084805 |1957-05-23T00:00:00.000Z|10007 |Tzvetan |F |1.7 |1.7000000476837158|1.7001953125 |1.7 |1989-02-10T00:00:00.000Z|[false, false, true, true]|null |4 |4 |4 |4 |Zielinski |74572 |[-7.06, 0.57, 1.99] |[-7, 0, 1] |[-7, 0, 1] |true |37286 +257694181 |1956-12-13T00:00:00.000Z|10029 |Otmar |M |1.99 |1.9900000095367432|1.990234375 |1.99 |1985-11-20T00:00:00.000Z|true |[Data Scientist, Principal Support Engineer, Senior Python Developer] |null |null |null |null |Herbst |74999 |[-8.19, -1.9, -0.32]|[-8, -1, 0] |[-8, -1, 0] |false |37499 +371418933 |null |10045 |Moss |M |1.7 |1.7000000476837158|1.7001953125 |1.7 |1989-09-02T00:00:00.000Z|[false, true] |[Accountant, Junior Developer, Principal Support Engineer, Purchase Manager]|3 |3 |3 |3 |Shanbhogue |74970 |null |null |null |false |37485 +; + +whereWithStatsValue +from employees | stats x = avg(salary) | where x > 5000; + +x:double +48248.55 +; + +statsByDouble +from employees | eval abc=1+2 | where abc + languages > 4 | stats count(height) by height; + +count(height):long | height:double +2 | 2.03 +1 | 2.08 +3 | 1.83 +2 | 1.78 +1 | 1.56 +4 | 1.7 +4 | 2.1 +1 | 1.5 +2 | 1.97 +1 | 1.99 +2 | 1.66 +1 | 1.54 +2 | 1.74 +1 | 1.92 +2 | 1.68 +3 | 1.81 +2 | 1.61 +3 | 2.0 +2 | 1.53 +2 | 1.55 +1 | 1.9 +2 | 1.44 +3 | 1.52 +1 | 1.96 +1 | 1.89 +2 | 1.58 +2 | 1.82 +1 | 2.04 +2 | 1.57 +3 | 1.59 +1 | 1.48 +1 | 1.42 +1 | 1.93 +4 | 1.77 +1 | 2.07 +1 | 1.64 +2 | 1.94 +1 | 1.8 +1 | 2.01 +1 | 1.91 +1 | 2.09 +1 | 1.69 +; + +statsByEvalDouble +from employees | eval h1 = round(height, 1) | stats count(height) by h1 | sort h1 desc; + +count(height):long | h1:double +13 | 2.1 +12 | 2.0 +10 | 1.9 +20 | 1.8 +12 | 1.7 +14 | 1.6 +14 | 1.5 + 5 | 1.4 +; + + +whereNegatedCondition +from employees | eval abc=1+2 | where abc + languages > 4 and languages.long != 1 | eval x=abc+languages | keep x, languages, languages.long | limit 3; + +x:integer | languages:integer | languages.long:long +5 | 2 | 2 +8 | 5 | 5 +7 | 4 | 4 +; + +evalOverride +from employees | eval languages = languages + 1 | eval languages = languages + 1 | limit 5 | keep l*; + +languages.byte:integer | languages.long:long | languages.short:integer | last_name:keyword | languages:integer +2 | 2 | 2 | Facello | 4 +5 | 5 | 5 | Simmel | 7 +4 | 4 | 4 | Bamford | 6 +5 | 5 | 5 | Koblick | 7 +1 | 1 | 1 | Maliniak | 3 +; + +evalWithNull +from employees | eval nullsum = salary + null | sort nullsum asc, salary desc | keep nullsum, salary | limit 1; + +nullsum:integer | salary:integer +null | 74999 +; + +evalWithNullAndAvg +from employees | eval nullsum = salary + null | stats avg(nullsum), count(nullsum); + +avg(nullsum):double | count(nullsum):long + null | 0 +; + +fromStatsLimit +from employees | stats ac = avg(salary) by languages | limit 2; + +ac:double | languages:integer +52519.6 | null +48178.84210526316 | 2 +; + +fromLimit +from employees | keep first_name | limit 2; + +first_name:keyword +Georgi +Bezalel +; + +projectAfterTopN +from employees | sort salary | limit 1 | keep first_name, salary; + +first_name:keyword | salary:integer +Guoxiang | 25324 +; + +projectAfterTopNDesc +from employees | sort salary desc | limit 1 | keep first_name, salary; + +first_name:keyword | salary:integer +Otmar | 74999 +; + +topNProjectEval +from employees | sort salary | limit 1 | keep languages, salary | eval x = languages + 1; + +languages:integer | salary:integer | x:integer +5 | 25324 | 6 +; + +topNProjectEvalProject +from employees | sort salary | limit 1 | keep languages, salary | eval x = languages + 1 | keep x; + +x:integer +6 +; + +filterKeyword +from employees | where first_name != "abc" and emp_no < 10010 | keep first_name; + +first_name:keyword +Georgi +Bezalel +Parto +Chirstian +Kyoichi +Anneke +Tzvetan +Saniya +Sumant +; + +projectMultiValueKeywords +from employees | keep emp_no, job_positions, still_hired | limit 5; + +emp_no:integer | job_positions:keyword |still_hired:boolean +10001 |[Accountant, Senior Python Developer] |true +10002 |Senior Team Lead |true +10003 |null |false +10004 |[Head Human Resources, Reporting Analyst, Support Engineer, Tech Lead]|true +10005 |null |true +; + +projectMultiValueBooleans +from employees | keep emp_no, is_rehired, still_hired | limit 5; + +emp_no:integer | is_rehired:boolean |still_hired:boolean +10001 |[false, true] |true +10002 |[false, false] |true +10003 |null |false +10004 |true |true +10005 |[false, false, false, true]|true +; + +projectMultiValueNumbers +from employees | keep emp_no, salary_change, salary_change.int, salary_change.long | limit 10; + +emp_no:integer | salary_change:double |salary_change.int:integer|salary_change.long:long +10001 |1.19 |1 |1 +10002 |[-7.23, 11.17] |[-7, 11] |[-7, 11] +10003 |[12.82, 14.68] |[12, 14] |[12, 14] +10004 |[-0.35, 1.13, 3.65, 13.48]|[0, 1, 3, 13] |[0, 1, 3, 13] +10005 |[-2.14, 13.07] |[-2, 13] |[-2, 13] +10006 |-3.9 |-3 |-3 +10007 |[-7.06, 0.57, 1.99] |[-7, 0, 1] |[-7, 0, 1] +10008 |[-2.92, 0.75, 3.54, 12.68]|[-2, 0, 3, 12] |[-2, 0, 3, 12] +10009 |null |null |null +10010 |[-6.77, 4.69, 5.05, 12.15]|[-6, 4, 5, 12] |[-6, 4, 5, 12] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages.csv new file mode 100644 index 0000000000000..3ee60b79970ba --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages.csv @@ -0,0 +1,5 @@ +language_code:keyword,language_name:keyword +1,English +2,French +3,Spanish +4,German diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-apps.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-apps.json new file mode 100644 index 0000000000000..6597cfaf73ae4 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-apps.json @@ -0,0 +1,13 @@ +{ + "properties" : { + "id" : { + "type" : "integer" + }, + "version" : { + "type" : "version" + }, + "name" : { + "type" : "keyword" + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-basic.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-basic.json new file mode 100644 index 0000000000000..7edd242c50a7c --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-basic.json @@ -0,0 +1,25 @@ +{ + "properties" : { + "emp_no" : { + "type" : "integer" + }, + "first_name" : { + "type" : "keyword" + }, + "gender" : { + "type" : "text" + }, + "languages" : { + "type" : "byte" + }, + "last_name" : { + "type" : "keyword" + }, + "salary" : { + "type" : "integer" + }, + "_meta_field": { + "type" : "keyword" + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-default.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-default.json new file mode 100644 index 0000000000000..00e3e6b37a92e --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-default.json @@ -0,0 +1,77 @@ +{ + "properties" : { + "emp_no" : { + "type" : "integer" + }, + "first_name" : { + "type" : "keyword" + }, + "last_name" : { + "type" : "keyword" + }, + "gender" : { + "type" : "keyword" + }, + "birth_date": { + "type" : "date" + }, + "hire_date": { + "type" : "date" + }, + "salary" : { + "type" : "integer" + }, + "languages" : { + "type" : "integer", + "fields": { + "long": { + "type": "long" + }, + "short": { + "type": "short" + }, + "byte": { + "type": "byte" + } + } + }, + "height": { + "type" : "double", + "fields" : { + "float" : { + "type" : "float" + }, + "scaled_float": { + "type": "scaled_float", + "scaling_factor": 100 + }, + "half_float": { + "type": "half_float" + } + } + }, + "still_hired": { + "type" : "boolean" + }, + "avg_worked_seconds" : { + "type" : "long" + }, + "job_positions" : { + "type" : "keyword" + }, + "is_rehired" : { + "type" : "boolean" + }, + "salary_change": { + "type": "double", + "fields": { + "int": { + "type": "integer" + }, + "long": { + "type": "long" + } + } + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-hosts.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-hosts.json new file mode 100644 index 0000000000000..e528fe9f038ac --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-hosts.json @@ -0,0 +1,23 @@ +{ + "properties" : { + "host" : { + "type" : "keyword" + }, + "host_group" : { + "type" : "text", + "store": true + }, + "description" : { + "type" : "text" + }, + "card" : { + "type" : "keyword" + }, + "ip0" : { + "type" : "ip" + }, + "ip1" : { + "type" : "ip" + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-languages.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-languages.json new file mode 100644 index 0000000000000..0cec0caf17304 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-languages.json @@ -0,0 +1,10 @@ +{ + "properties" : { + "language_code" : { + "type" : "keyword" + }, + "language_name" : { + "type" : "keyword" + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-ul_logs.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-ul_logs.json new file mode 100644 index 0000000000000..68228b2e54d0f --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-ul_logs.json @@ -0,0 +1,19 @@ +{ + "properties" : { + "id" : { + "type" : "integer" + }, + "@timestamp" : { + "type" : "date" + }, + "bytes_in" : { + "type" : "unsigned_long" + }, + "bytes_out" : { + "type" : "unsigned_long" + }, + "status" : { + "type" : "keyword" + } + } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/math.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/math.csv-spec new file mode 100644 index 0000000000000..407192418d221 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/math.csv-spec @@ -0,0 +1,990 @@ +addIntAndInt +from employees | eval s = emp_no + languages | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:integer + 10001 | 10003 + 10002 | 10007 +; + +addLongAndLong +from employees | eval s = avg_worked_seconds + languages.long | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:long + 10001 | 268728051 + 10002 | 328922892 +; + +addLongOverflow +row max = 9223372036854775807 | eval sum = max + 1 | keep sum; + +warning:Line 1:44: evaluation of [max + 1] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +sum:long +null +; + +subLongUnderflow +row l = -9223372036854775807 | eval sub = l - 2 | keep sub; + +warning:Line 1:43: evaluation of [l - 2] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +sub:long +null +; + +mulLongOverflow +row max = 9223372036854775807 | eval mul = max * 2 | keep mul; + +warning:Line 1:44: evaluation of [max * 2] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +mul:long +null +; + +divLongByZero +row max = 9223372036854775807 | eval div = max / 0 | keep div; + +warning:Line 1:44: evaluation of [max / 0] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: / by zero + +div:long +null +; + +modLongByZero +row max = 9223372036854775807 | eval mod = max % 0 | keep mod; + +// ascii(%) == %25 +warning:Line 1:44: evaluation of [max %25 0] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: / by zero + +mod:long +null +; + +addDoubleAndDouble +from employees | eval s = height + 5 | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:double + 10001 | 7.029999999999999 + 10002 | 7.08 +; + +addIntAndLong +from employees | eval s = emp_no + languages.long | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:long + 10001 | 10003 + 10002 | 10007 +; + +addLongAndInt +from employees | eval s = languages.long + emp_no | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:long + 10001 | 10003 + 10002 | 10007 +; + +addIntAndDouble +from employees | eval s = emp_no + height | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:double + 10001 | 10003.03 + 10002 | 10004.08 +; + +addDoubleAndInt +from employees | eval s = height + emp_no | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:double + 10001 | 10003.03 + 10002 | 10004.08 +; + +addLongAndDouble +from employees | eval s = languages.long + height | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:double + 10001 | 4.029999999999999 + 10002 | 7.08 +; + +addDoubleAndLong +from employees | eval s = height + languages.long | keep emp_no, s | sort emp_no asc | limit 2; + +emp_no:integer | s:double + 10001 | 4.029999999999999 + 10002 | 7.08 +; + +negate +row x=1 | eval a = -x, b = --x * -+-+-1 * -3, c = -(-(-(-x))); + +x:integer | a:integer | b:integer | c:integer +1 | -1 | 3 | 1 +; + +negateIntLongDouble +from employees | eval negInt = -languages, negLong = -languages.long, negDouble = -height | keep emp_no, negInt, negLong, negDouble | sort emp_no asc | limit 2; + +emp_no:integer | negInt:integer | negLong:long | negDouble:double + 10001 | -2 | -2 | -2.03 + 10002 | -5 | -5 | -2.08 +; + +negateIntOverflow +// Negating Integer.MIN_VALUE overflows. +row x=-2147483648 | eval a = -x; +warning:Line 1:30: evaluation of [-x] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: integer overflow + +x:integer | a:integer +-2147483648 | null +; + +negateLongOverflow +// Negating Long.MIN_VALUE overflows. +row x=-9223372036854775808 | eval a = -x; +warning:Line 1:39: evaluation of [-x] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +x:long | a:long +-9223372036854775808 | null +; + +absUnsignedLong +row ul = [0, 1, 9223372036854775807, 9223372036854775808, 18446744073709551615] | eval abs = abs(ul); + + ul:ul | abs:ul +[0, 1, 9223372036854775807, 9223372036854775808, 18446744073709551615]|[0, 1, 9223372036854775807, 9223372036854775808, 18446744073709551615] +; + +absLong +from employees | eval l = abs(0-languages.long) | keep l | sort l asc | limit 3; + +l:long +1 +1 +1 +; + +absInt +from employees | eval s = abs(0-salary) | keep s | sort s asc | limit 3; + +s:integer +25324 +25945 +25976 +; + +absDouble +from employees | eval s = abs(0.0-salary) | keep s | sort s asc | limit 3; + +s:double +25324.0 +25945.0 +25976.0 +; + +powHeightSquared +from employees | sort height asc | limit 20 | eval s = round(pow(height, 2) - 2, 2) | keep height, s | sort s desc | limit 4; + +height:double | s:double +1.55 | 0.40 +1.54 | 0.37 +1.53 | 0.34 +1.53 | 0.34 +; + +powSalarySquared +from employees | eval s = pow(to_long(salary) - 75000, 2) + 10000 | keep salary, s | sort salary desc | limit 4; + +salary:integer | s:long +74999 | 10001 +74970 | 10900 +74572 | 193184 +73851 | 1330201 +; + +isFiniteFalse +row d = 1.0 | eval s = is_finite(d/0); + +d:double | s:boolean +1.0 | false +; + +isFiniteTrue +row d = 1.0 | eval s = is_finite(d); + +d:double | s:boolean +1.0 | true +; + +isInfiniteFalse +row d = 1.0 | eval s = is_infinite(d); + +d:double | s:boolean +1.0 | false +; + +isInfiniteTrue +row d = 1.0/0 | eval s = is_infinite(d); + +d:double | s:boolean +Infinity | true +; + +isNaN +row d = 1.0 | eval s = is_nan(d); + +d:double | s:boolean +1.0 | false +; + +isNaNTrue +row d = 0.0/0.0 | eval s = is_nan(d); + +d:double | s:boolean +NaN | true +; + +log10 +// tag::log10[] +ROW d = 1000.0 +| EVAL s = LOG10(d) +// end::log10[] +; + +// tag::log10-result[] +d: double | s:double +1000.0 | 3.0 +// end::log10-result[] +; + +log10ofNegative +row d = -1.0 | eval s = is_nan(log10(d)); + +d:double | s:boolean +-1.0 | true +; + +log10ofNan +row d = 0.0/0.0 | eval s = is_nan(log10(d)); + +d:double | s:boolean +NaN | true +; + +log10ofZero +row d = 0.0 |eval s = is_infinite(log10(d)); + +d:double | s:boolean +0.0 | true +; + +log10ofNegativeZero +row d = -0.0 |eval s = is_infinite(log10(d)); + +d:double | s:boolean +-0.0 | true +; + +log10ofInfinite +row d = 1/0.0 | eval s = is_infinite(log10(d)); + +d:double | s:boolean +Infinity | true +; + +log10ofLong +row d = 10 | eval l = log10(10); + +d:i | l:double +10 | 1 +; + +powDoubleDouble +ROW base = 2.0, exponent = 2.0 +| EVAL s = POW(base, exponent) +; + +base:double | exponent:double | s:double +2.0 | 2.0 | 4.0 +; + +powIntDouble +// tag::powID-sqrt[] +ROW base = 4, exponent = 0.5 +| EVAL s = POW(base, exponent) +// end::powID-sqrt[] +; + +// tag::powID-sqrt-result[] +base:integer | exponent:double | s:double +4 | 0.5 | 2.0 +// end::powID-sqrt-result[] +; + +powSqrtNeg +// tag::powNeg-sqrt[] +ROW base = -4, exponent = 0.5 +| EVAL s = POW(base, exponent) +// end::powNeg-sqrt[] +; +warning:Line 2:12: evaluation of [POW(base, exponent)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: invalid result: pow(-4.0, 0.5) + +// tag::powNeg-sqrt-result[] +base:integer | exponent:double | s:double +-4 | 0.5 | null +// end::powNeg-sqrt-result[] +; + +powDoubleInt +// tag::powDI[] +ROW base = 2.0, exponent = 2 +| EVAL result = POW(base, exponent) +// end::powDI[] +; + +// tag::powDI-result[] +base:double | exponent:integer | result:double +2.0 | 2 | 4.0 +// end::powDI-result[] +; + +powIntInt +// tag::powII[] +ROW base = 2, exponent = 2 +| EVAL s = POW(base, exponent) +// end::powII[] +; + +// tag::powII-result[] +base:integer | exponent:integer | s:integer +2 | 2 | 4 +// end::powII-result[] +; + +powIntIntPlusInt +row s = 1 + pow(2, 2); + +s:integer +5 +; + +powIntIntPlusDouble +row s = 1.0 + pow(2, 2); + +s:double +5 +; + +powIntUL +row x = pow(1, 9223372036854775808); + +x:long +1 +; + +powLongUL +row x = to_long(1) | eval x = pow(x, 9223372036854775808); + +x:long +1 +; + +powUnsignedLongUL +row x = to_ul(1) | eval x = pow(x, 9223372036854775808); + +x:long +1 +; + +powDoubleUL +row x = pow(1.0, 9223372036854775808); + +x:double +1.0 +; + +powIntULOverrun +row x = pow(2, 9223372036854775808); +warning:Line 1:9: evaluation of [pow(2, 9223372036854775808)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +x:long +null +; + +powULInt +row x = pow(to_unsigned_long(9223372036854775807), 1); + +x:long +9223372036854775807 +; + +powULIntOverrun +// tag::powULOverrun[] +ROW x = POW(9223372036854775808, 2) +// end::powULOverrun[] +; +// tag::powULOverrun-warning[] +warning:Line 1:9: evaluation of [POW(9223372036854775808, 2)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow +// end::powULOverrun-warning[] + +// tag::powULOverrun-result[] +x:long +null +// end::powULOverrun-result[] +; + +powULInt_2d +// tag::pow2d[] +ROW x = POW(9223372036854775808, TO_DOUBLE(1)) +// end::pow2d[] +; + +// tag::pow2d-result[] +x:double +9.223372036854776E18 +// end::pow2d-result[] +; + +powULLong +row x = to_long(10) | eval x = pow(to_unsigned_long(10), x); + +x:long +10000000000 +; + +powULLongOverrun +row x = to_long(100) | eval x = pow(to_unsigned_long(10), x); +warning:Line 1:33: evaluation of [pow(to_unsigned_long(10), x)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +x:long +null +; + +powULDouble +row x = pow(9223372036854775808, -.1); + +x:double +0.012691443693066178 +; + +roundLongAsUL +row x = to_ul(9223372036854775804) | eval x = round(x, -1); + +x:ul +9223372036854775800 +; + +roundUL +row x = round(9223372036854775808, -1); + +x:ul +9223372036854775810 +; + +roundMaxMinusOneUL +row ul = round(18446744073709551614, -1); + +ul:ul +18446744073709551610 +; + +roundULNoFold +row ul = 18446744073709551614 | eval ul = round(ul, -4); + +ul:ul +18446744073709550000 +; + +roundMaxULNoDecimals +row ul = round(18446744073709551615); + +ul:ul +18446744073709551615 +; + +mvAvg +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; + +emp_no:integer | salary_change.int:integer | salary_change:double +10009 | null | null +10010 | [-6, 4, 5, 12] | 3.7800000000000002 +10011 | [-7, 3, 8, 10] | 3.685 +10012 | 0 | 0.04 +10013 | null | null +10014 | [-1, 9] | 3.5900000000000003 +10015 | [12, 14] | 13.325 +; + +mvAvgSimple +// tag::mv_avg[] +ROW a=[3, 5, 1, 6] +| EVAL avg_a = MV_AVG(a) +// end::mv_avg[] +; + +// tag::mv_avg-result[] + a:integer | avg_a:double +[3, 5, 1, 6] | 3.75 +// end::mv_avg-result[] +; + +mvAvgIntsOverflow +ROW ints = [0, 1, 2147483647] +| EVAL mvavg = mv_avg(ints) +| KEEP mvavg; + +mvavg:double +7.158278826666666E8 +; + +mvAvgLongsOverflow +ROW longs = [0, 1, 9223372036854775807] +| EVAL mvavg = mv_avg(longs) +| KEEP mvavg; + +mvavg:double +3.0744573456182584E18 +; + +mvAvgUnsignedLongsOverflow +ROW ulongs = [0, 1, 18446744073709551615] +| EVAL mvavg = mv_avg(ulongs) +| KEEP mvavg; + +mvavg:double +6.148914691236517E18 +; + +mvCount +ROW a=[3, 5, 1, 6] +| EVAL count_a = MV_COUNT(a) +; + + a:integer | count_a:integer +[3, 5, 1, 6] | 4 +; + + +mvMax +from employees | where emp_no > 10008 | eval salary_change = mv_max(salary_change.int) | sort emp_no | keep emp_no, salary_change.int, salary_change | limit 7; + +emp_no:integer | salary_change.int:integer | salary_change:integer +10009 | null | null +10010 | [-6, 4, 5, 12] | 12 +10011 | [-7, 3, 8, 10] | 10 +10012 | 0 | 0 +10013 | null | null +10014 | [-1, 9] | 9 +10015 | [12, 14] | 14 +; + +mvMaxSimple +// tag::mv_max[] +ROW a=[3, 5, 1] +| EVAL max_a = MV_MAX(a) +// end::mv_max[] +; + +// tag::mv_max-result[] +a:integer | max_a:integer +[3, 5, 1] | 5 +// end::mv_max-result[] +; + +mvMedian +from employees | where emp_no > 10008 | eval med = mv_median(salary_change) | sort emp_no | keep emp_no, salary_change, med | limit 7; + +emp_no:integer | salary_change:double | med:double +10009 | null | null +10010 | [-6.77, 4.69, 5.05, 12.15] | 4.87 +10011 | [-7.82, 3.48, 8.73, 10.35] | 6.105 +10012 | 0.04 | 0.04 +10013 | null | null +10014 | [-1.89, 9.07] | 3.5900000000000003 +10015 | [12.4, 14.25] | 13.325 +; + +mvMedianSimple +// tag::mv_median[] +ROW a=[3, 5, 1] +| EVAL median_a = MV_MEDIAN(a) +// end::mv_median[] +; + +// tag::mv_median-result[] +a:integer | median_a:integer +[3, 5, 1] | 3 +// end::mv_median-result[] +; + +mvMedianRoundDown +// tag::mv_median_round_down[] +ROW a=[3, 7, 1, 6] +| EVAL median_a = MV_MEDIAN(a) +// end::mv_median_round_down[] +; + +// tag::mv_median_round_down-result[] + a:integer | median_a:integer +[3, 7, 1, 6] | 4 +// end::mv_median_round_down-result[] +; + +mvMin +from employees | where emp_no > 10008 | eval salary_change = mv_min(salary_change.int) | sort emp_no | keep emp_no, salary_change.int, salary_change | limit 7; + +emp_no:integer | salary_change.int:integer | salary_change:integer +10009 |null |null +10010 |[-6, 4, 5, 12] |-6 +10011 |[-7, 3, 8, 10] |-7 +10012 |0 |0 +10013 |null |null +10014 |[-1, 9] |-1 +10015 |[12, 14] |12 +; + +mvMinSimple +// tag::mv_min[] +ROW a=[2, 1] +| EVAL min_a = MV_MIN(a) +// end::mv_min[] +; + +// tag::mv_min-result[] +a:integer | min_a:integer + [2, 1] | 1 +// end::mv_min-result[] +; + +mvSum +from employees | where emp_no > 10008 | eval salary_change = mv_sum(salary_change.int) | sort emp_no | keep emp_no, salary_change.int, salary_change | limit 7; + +emp_no:integer | salary_change.int:integer | salary_change:integer +10009 | null | null +10010 | [-6, 4, 5, 12] | 15 +10011 | [-7, 3, 8, 10] | 14 +10012 | 0 | 0 +10013 | null | null +10014 | [-1, 9] | 8 +10015 | [12, 14] | 26 +; + +mvSumSimple +// tag::mv_sum[] +ROW a=[3, 5, 6] +| EVAL sum_a = MV_SUM(a) +// end::mv_sum[] +; + +// tag::mv_sum-result[] +a:integer | sum_a:integer +[3, 5, 6] | 14 +// end::mv_sum-result[] +; + +mvSumIntsOverflow +ROW ints = [0, 1, 2147483647] +| EVAL mvsum = mv_sum(ints) +| KEEP mvsum; + +warning:Line 2:16: evaluation of [mv_sum(ints)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: integer overflow + +mvsum:integer +null +; + +mvSumLongsOverflow +ROW longs = [0, 1, 9223372036854775807] +| EVAL mvsum = mv_sum(longs) +| KEEP mvsum; + +warning:Line 2:16: evaluation of [mv_sum(longs)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: long overflow + +mvsum:long +null +; + +mvSumUnsignedLongsOverflow +ROW ulongs = [0, 1, 18446744073709551615] +| EVAL mvsum = mv_sum(ulongs) +| KEEP mvsum; + +warning:Line 2:16: evaluation of [mv_sum(ulongs)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: unsigned_long overflow + +mvsum:unsigned_long +null +; + +e +// tag::e[] +ROW E() +// end::e[] +; + +// tag::e-result[] +E():double +2.718281828459045 +// end::e-result[] +; + +eInside +ROW a=2 | EVAL c = abs(a + e()); + +a:integer | c:double + 2 | 4.718281828459045 +; + +// +// unsigned_long arithmetics, MV functions +// + +ulAddition +row x = to_ul(100), y = to_ul(1) | eval x = x + y | keep x; + +x:ul +101 +; + +ulAdditionOverLongLimit +row x = to_ul(9223372036854775807), y = to_ul(1) | eval x = x + y | keep x; + +x:ul +9223372036854775808 +; + +ulAdditionToUnsignedLongLimit +row x = 18446744073709551614, y = to_ul(1) | eval x = x + y | keep x; + +x:ul +18446744073709551615 +; + +ulAdditionOverflow +row x = 18446744073709551615, y = to_ul(1) | eval x = x + y | keep x; + +warning:Line 1:55: evaluation of [x + y] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: unsigned_long overflow + +x:ul +null +; + +ulSubtraction +row x = to_ul(100), y = to_ul(1) | eval x = x - y | keep x; + +x:ul +99 +; + +ulSubtractionToLongLimit +row x = 9223372036854775808, y = to_ul(1) | eval x = x - y | keep x; + +x:ul +9223372036854775807 +; + +ulSubtractionFromUnsignedLongLimit +row x = 18446744073709551615, y = to_ul(1) | eval x = x - y | keep x; + +x:ul +18446744073709551614 +; + +ulSubtractionUnderflow +row x = to_ul(0), y = to_ul(1) | eval x = x - y | keep x; + +warning:Line 1:43: evaluation of [x - y] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: unsigned_long overflow + +x:ul +null +; + +ulMultiplication +row x = to_ul(9223372036854775807), y = to_ul(2) | eval x = x * y | keep x; + +x:ul +18446744073709551614 +; + +ulMultiplicationOverflow +row x = 9223372036854775808, two = to_ul(2) | eval times2 = x * two | keep times2; + +warning:Line 1:61: evaluation of [x * two] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: unsigned_long overflow + +times2:ul +null +; + +ulMultiplicationOverflow2 +row x = 9223372036854775808, y = 9223372036854775809 | eval x = x * y | keep x; + +warning:Line 1:65: evaluation of [x * y] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: unsigned_long overflow + +x:ul +null +; + +ulDivision +row max = 18446744073709551615, halfplus = 9223372036854775808, two = to_ul(2) | eval x = max / two, y = max / halfplus | keep x, y; + +x:ul |y:ul +9223372036854775807 |1 +; + +ulDivisionByZero +row halfplus = 9223372036854775808, zero = to_ul(0) | eval div = halfplus / zero | keep div; + +warning:Line 1:66: evaluation of [halfplus / zero] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: / by zero + +div:ul +null +; + +ulModulo +row max = 18446744073709551615, halfplus = 9223372036854775808, two = to_ul(2) | eval x = max % halfplus, y = halfplus % two | keep x, y; + +x:ul |y:ul +9223372036854775807 |0 +; + +ulModuloByZero +row halfplus = 9223372036854775808, zero = to_ul(0) | eval mod = halfplus % zero | keep mod; + +// ascii(%) == %25 +warning:Line 1:66: evaluation of [halfplus %25 zero] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.ArithmeticException: / by zero + +mod:ul +null +; + + +ulMvNonSumFunctions +row ul = [18446744073709551615, 0, 1, 9223372036854775807, 9223372036854775808] | eval mv_min(ul), mv_max(ul), mv_median(ul), mv_count(ul) | drop ul; + + mv_min(ul):ul| mv_max(ul):ul | mv_median(ul):ul| mv_count(ul):i +0 |18446744073709551615|9223372036854775807|5 +; + +ulMvSum +row ul = [1, 9223372036854775806, 9223372036854775808] | eval mv_sum(ul), mv_avg(ul) | drop ul; + + mv_sum(ul):ul | mv_avg(ul):double +18446744073709551615|6.148914691236517E18 +; + +ulMedianEvenCount +row ul = [18446744073709551615, 0, 1, 9223372036854775807, 9223372036854775808, 2] | eval mv_median(ul) | drop ul; + +mv_median(ul):ul +4611686018427387904 +; + +pi +// tag::pi[] +ROW PI() +// end::pi[] +; + +// tag::pi-result[] +PI():double +3.141592653589793 +// end::pi-result[] +; + +tau +// tag::tau[] +ROW TAU() +// end::tau[] +; + +// tag::tau-result[] +TAU():double +6.283185307179586 +// end::tau-result[] +; + +floor +// tag::floor[] +ROW a=1.8 | EVAL a=FLOOR(a) +// end::floor[] +; + +// tag::floor-result[] +a:double +1 +// end::floor-result[] +; + +sqrt +// tag::sqrt[] +ROW d = 100.0 +| EVAL s = SQRT(d); +// end::sqrt[] + +// tag::sqrt-result[] +d: double | s:double +100.0 | 10.0 +// end::sqrt-result[] +; + +sqrtOfInteger +row i = 81 | eval s = sqrt(i); + +i:integer | s:double +81 | 9 +; + +sqrtOfNegative +row d = -1.0 | eval s = is_nan(sqrt(d)); + +d:double | s:boolean +-1.0 | true +; + +sqrtOfNan +row d = 0.0/0.0 | eval s = is_nan(sqrt(d)); + +d:double | s:boolean +NaN | true +; + +sqrtOfZero +row d = 0.0 |eval s = sqrt(d); + +d:double | s:double +0.0 | 0.0 +; + +sqrtOfNegativeZero +row d = -0.0 |eval s = sqrt(d); + +d:double | s:double +-0.0 | -0.0 +; + +sqrtOfInfinite +row d = 1/0.0 | eval s = is_infinite(sqrt(d)); + +d:double | s:boolean +Infinity | true +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/metadata-ignoreCsvTests.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/metadata-ignoreCsvTests.csv-spec new file mode 100644 index 0000000000000..34935384786f1 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/metadata-ignoreCsvTests.csv-spec @@ -0,0 +1,105 @@ + +simpleKeep +from employees [metadata _index, _version] | sort emp_no | limit 2 | keep emp_no, _index, _version; + +emp_no:integer |_index:keyword |_version:long +10001 |employees |1 +10002 |employees |1 +; + +aliasWithSameName +from employees [metadata _index, _version] | sort emp_no | limit 2 | eval _index = _index, _version = _version | keep emp_no, _index, _version; + +emp_no:integer |_index:keyword |_version:long +10001 |employees |1 +10002 |employees |1 +; + +inComparison +from employees [metadata _index, _version] | sort emp_no | where _index == "employees" | where _version == 1 | keep emp_no | limit 2; + +emp_no:integer +10001 +10002 +; + +metaIndexInAggs +from employees [metadata _index] | stats max = max(emp_no) by _index; + +max:integer |_index:keyword +10100 |employees +; + +metaIndexAliasedInAggs +from employees [metadata _index] | eval _i = _index | stats max = max(emp_no) by _i; + +max:integer |_i:keyword +10100 |employees +; + +metaVersionInAggs +from employees [metadata _version] | stats min = min(emp_no) by _version; + +min:integer |_version:long +10001 |1 +; + +metaVersionAliasedInAggs +from employees [metadata _version] | eval _v = _version | stats min = min(emp_no) by _v; + +min:integer |_v:long +10001 |1 +; + +inAggsAndAsGroups +from employees [metadata _index, _version] | stats max = max(_version) by _index; + +max:long |_index:keyword +1 |employees +; + +inAggsAndAsGroupsAliased +from employees [metadata _index, _version] | eval _i = _index, _v = _version | stats max = max(_v) by _i; + +max:long |_i:keyword +1 |employees +; + +inFunction +from employees [metadata _index, _version] | sort emp_no | where length(_index) == length("employees") | where abs(_version) == 1 | keep emp_no | limit 2; + +emp_no:integer +10001 +10002 +; + +inArithmetics +from employees [metadata _index, _version] | eval i = _version + 2 | stats min = min(emp_no) by i; + +min:integer |i:long +10001 |3 +; + +inSort +from employees [metadata _index, _version] | sort _version, _index, emp_no | keep emp_no, _version, _index | limit 2; + +emp_no:integer |_version:long |_index:keyword +10001 |1 |employees +10002 |1 |employees +; + +withMvFunction +from employees [metadata _version] | eval i = mv_avg(_version) + 2 | stats min = min(emp_no) by i; + +min:integer |i:double +10001 |3.0 +; + +overwritten +from employees [metadata _index, _version] | eval _index = 3, _version = "version" | keep emp_no, _index, _version | limit 3; + +emp_no:integer |_index:integer |_version:keyword +10001 |3 |version +10002 |3 |version +10003 |3 |version +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mv_expand.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mv_expand.csv-spec new file mode 100644 index 0000000000000..7cc11c6fab5b3 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mv_expand.csv-spec @@ -0,0 +1,26 @@ +simple +// tag::simple[] +ROW a=[1,2,3], b="b", j=["a","b"] +| MV_EXPAND a +// end::simple[] +; + +// tag::simple-result[] +a:integer | b:keyword | j:keyword + 1 | b | ["a", "b"] + 2 | b | ["a", "b"] + 3 | b | ["a", "b"] +// end::simple-result[] +; + +twice +row a=[1,2,3], b="b", j=["a","b"] | mv_expand a | mv_expand j; + +a:integer | b:keyword | j:keyword + 1 | b | "a" + 1 | b | "b" + 2 | b | "a" + 2 | b | "b" + 3 | b | "a" + 3 | b | "b" +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/query_task.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/query_task.json new file mode 100644 index 0000000000000..1da628e0a3e84 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/query_task.json @@ -0,0 +1,14 @@ +{ + "node" : "2j8UKw1bRO283PMwDugNNg", + "id" : 5326, + "type" : "transport", + "action" : "indices:data/read/esql", + "description" : "FROM test | STATS MAX(d) by a, b", <1> + "start_time" : "2023-07-31T15:46:32.328Z", + "start_time_in_millis" : 1690818392328, + "running_time" : "41.7ms", <2> + "running_time_in_nanos" : 41770830, + "cancellable" : true, + "cancelled" : false, + "headers" : { } +} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/rename.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/rename.csv-spec new file mode 100644 index 0000000000000..f99788eb7d708 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/rename.csv-spec @@ -0,0 +1,149 @@ +renameFirstCol +row a = 1, b = 2 | rename a as c; + +c:integer | b:integer +1 | 2 +; + +renameSecondCol +row a = 1, b = 2 | rename b as c; + +a:integer | c:integer +1 | 2 +; + +chaining +row a = 1, b = 2 | rename a as c, c as d, d as e; + +e:integer | b:integer +1 | 2 +; + +chainReuse +row a = 1, b = 2 | rename a as c, c as d, b as c; + +d:integer | c:integer +1 | 2 +; + +effectivelyANop +row a = 1, b = 2 | rename a as c, c as a; + +a:integer | b:integer +1 | 2 +; + +reuseAlias +row a = 1, b = 2 | rename a as c, b as c; + +c:integer +2 +; + +unquotedNamesWithAt +row @a = 10 | rename @a as @b | eval @c = @b + 1; + +@b:integer | @c:integer +10 | 11 +; + +renameEval +row a = 1, b = 2 | rename a as c | eval e = b + c; + +c:integer | b:integer | e:integer +1 | 2 | 3 +; + +rowRenameEvalProject +row a = 1, b = 2 | rename a as c | keep c | eval e = 2 * c | keep e, c; + +e:integer | c:integer +2 | 1 +; + +rowRenameNop +row a = 1, b = 2 | rename a as a; + +a:integer | b:integer +1 | 2 +; + +rowRenameDrop +row a = 1, b = 2, c = 3 | rename a as d | drop b; + +d:integer | c:integer +1 | 3 +; + +renameEvalProject +from employees | rename languages as x | keep x | eval z = 2 * x | keep x, z | limit 3; + +x:integer | z:integer +2 | 4 +5 | 10 +4 | 8 +; + +renameProjectEval +from employees | eval y = languages | rename languages as x | keep x, y | eval x2 = x + 1 | eval y2 = y + 2 | limit 3; + +x:integer | y:integer | x2:integer | y2:integer +2 | 2 | 3 | 4 +5 | 5 | 6 | 7 +4 | 4 | 5 | 6 +; + +renameWithFilterPushedToES +from employees | rename emp_no as x | keep languages, first_name, last_name, x | where x > 10030 and x < 10040 | limit 5; + +languages:integer | first_name:keyword | last_name:keyword | x:integer +4 | null | Joslin | 10031 +3 | null | Reistad | 10032 +1 | null | Merlo | 10033 +1 | null | Swan | 10034 +5 | null | Chappelet | 10035 +; + +renameNopProject +from employees | rename emp_no as emp_no | keep emp_no, last_name | limit 3; + +emp_no:integer | last_name:keyword +10001 | Facello +10002 | Simmel +10003 | Bamford +; + +renameOverride +from employees | rename emp_no as languages | keep languages, last_name | limit 3; + +languages:integer | last_name:keyword +10001 | Facello +10002 | Simmel +10003 | Bamford +; + +projectRenameDate +from employees | sort hire_date | rename hire_date as x | keep emp_no, x | limit 5; + +emp_no:integer | x:date +10009 | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z +; + +renameDrop +from employees + | sort hire_date + | rename hire_date as x, emp_no as y + | drop first_name, last_name, gender, birth_date, salary, languages*, height*, still_hired, avg_worked_seconds, job_positions, is_rehired, salary_change* + | limit 5; + +y:integer | x:date +10009 | 1985-02-18T00:00:00.000Z +10048 | 1985-02-24T00:00:00.000Z +10098 | 1985-05-13T00:00:00.000Z +10076 | 1985-07-09T00:00:00.000Z +10061 | 1985-09-17T00:00:00.000Z +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/row.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/row.csv-spec new file mode 100644 index 0000000000000..0c5026413fce9 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/row.csv-spec @@ -0,0 +1,333 @@ +oneField +row a = 1; + +a:integer +1 +; + +multipleFields +// tag::example[] +ROW a = 1, b = "two", c = null +// end::example[] +; + +// tag::example-result[] +a:integer | b:keyword | c:null +1 | "two" | null +// end::example-result[] +; + +implicitNames +row 100, 10, c = 1; + +100:integer | 10:integer | c:integer +100 | 10 | 1 +; + +multivalue +// tag::multivalue[] +ROW a = [2, 1] +// end::multivalue[] +; + +// tag::multivalue-result[] +a:integer +[2, 1] +// end::multivalue-result[] +; + +unsignedLongLiteral +ROW long_max = 9223372036854775807, ul_start = 9223372036854775808, ul_end = 18446744073709551615, double=18446744073709551616; + +long_max:long |ul_start:unsigned_long |ul_end:ul |double:double +9223372036854775807 |9223372036854775808 |18446744073709551615 |1.8446744073709552E19 +; + +fieldFromFunctionEvaluation +// tag::function[] +ROW a = ROUND(1.23, 0) +// end::function[] +; + +// tag::function-result[] +a:double +1.0 +// end::function-result[] +; + +evalRow +row a = 1, b = 2 | eval c = a + b; + +a:integer | b:integer | c:integer +1 | 2 | 3 +; + +evalRowWithNull +row a = 1, b = 2, c = null | eval z = c+b+a; + +a:integer | b:integer | c:null | z:integer +1 | 2 | null | null +; + +evalRowWithNull2 +row a = 1, null, b = 2, c = null, null | eval z = a+b; + +a:integer | null:null | b:integer | c:null | null:null | z:integer +1 | null | 2 | null | null | 3 +; + +evalRowWithNull3 +row a = 1, b = 2, x = round(null) | eval z = a+b+x; + +a:integer | b:integer | x:null | z:integer +1 | 2 | null | null +; + +evalRowWithRound +row a = 1, b = 2 | eval y = round(-123.5); + +a:integer | b:integer | y:double +1 | 2 | -124.0 +; + +evalRowWithRoundNullDecimals +row a = 1, b = 2 | eval y = round(-123.5, null); + +a:integer | b:integer | y:double +1 | 2 | null +; + +evalRowWithComplexRound +row a = 1, b = 2 | eval y = round(-123.56789, 3) + ROUND(a + b / 2); + +a:integer | b:integer | y:double +1 | 2 | -121.568 +; + +evalOverride +row a = 1, b = 100 | eval b = a + 1 | eval a = b + 1; + +b:integer | a:integer +2 | 3 +; + +evalOverride2 +row a = 1, b = 10, c = 100, d = 1000, e = 10000 | eval d = a + 1 | eval b = d + 1; + +a:integer | c:integer | e:integer | d:integer | b:integer +1 | 100 | 10000 | 2 | 3 +; + +filterRow +row a = 1 | where a > 0; + +a:integer +1 +; + +filterRow2 +row a = 1 | where a > 10; + +a:integer +; + +filterRow3 +row a = 1 | eval b = a * null | where b > 10; + +a:integer | b:integer +; + +filterRow4 +row a = 1 | eval b = null * 1 | where b > 10; + +a:integer | b:integer +; + +filterRow5 +row a = 1.0 | eval b = a * null | where b > 2.0; + +a:double | b:double +; + +evalRowWithNulls +row a = 1, b = 2 | eval y = null; + +a:integer | b:integer | y:null +1 | 2 | null +; + +evalRowWithNullsInRound +row a = 1, b = 2 | eval y = round(null, 3) + ROUND(a + b / 2); + +a:integer | b:integer | y:integer +1 | 2 | null +; + +evalRowWithNullsInArithmethicOp +row a = 1, b = 2 | eval y = null + 3; + +a:integer | b:integer | y:integer +1 | 2 | null +; + +rowWithNullsInCount +row a = 1.5, b = 2.6, c = null | eval s = null + a + b | stats c = count(s); + +c:long +0 +; + +rowWithNullsInAvg +row a = 1.5, b = 2.6, c = null | eval s = null + a + b | stats c = avg(s); + +c:double +null +; + +rowWithNullsInAvg2 +row a = 1.5, b = 2.6, c = null | eval s = a - b * c | stats avg(s); + +avg(s):double +null +; + +rowStatsProjectGroupByInt +row a = 1, b = 2 | stats count(b) by a | keep a; + +a:integer +1 +; + +rowStatsProjectGroupByLong +row a = 1000000000000, b = 2 | stats count(b) by a | keep a; + +a:long +1000000000000 +; + +rowStatsProjectGroupByDouble +row a = 1.0, b = 2 | stats count(b) by a | keep a; + +a:double +1.0 +; + +rowStatsProjectGroupByLong +row a = "hello world", b = 2 | stats count(b) by a | keep a; + +a:keyword +"hello world" +; + +limitRow +row a = 1 | limit 0; + +a:integer +; + +rowWithMultipleStats +row a = 1+3, b = 2, ab = 5 | eval x = 1 + b + 5 | stats avg = avg(x), min(x), max(x), count(x), avg(x), avg(ab), avg(a); + +avg:double | min(x):integer | max(x):integer | count(x):long | avg(x):double | avg(ab):double | avg(a):double + 8.0 | 8 | 8 | 1 | 8.0 | 5.0 | 4.0 +; + +rowWithMultipleStatsOverNull +row x=1, y=2 | eval tot = null + y + x | stats c=count(tot), a=avg(tot), mi=min(tot), ma=max(tot), s=sum(tot); + +c:long | a:double | mi:integer | ma:integer | s:long + 0 | null | null | null | null +; + + +min +row l=1, d=1.0, ln=1 + null, dn=1.0 + null | stats min(l), min(d), min(ln), min(dn); + +min(l):integer | min(d):double | min(ln):integer | min(dn):double + 1 | 1.0 | null | null +; + + +sum +row l=1, d=1.0, ln=1 + null, dn=1.0 + null | stats sum(l), sum(d), sum(ln), sum(dn); + +sum(l):long | sum(d):double | sum(ln):long | sum(dn):double + 1 | 1.0 | null | null +; + +boolean +row false; + +false:boolean +false +; + +in +row a=1, b=2 | where a in (0, b-1); + +a:integer|b:integer +1 |2 +; + +inWithLiteralValue +row a=1, b=2 | where 2 in (a, b); + +a:integer|b:integer +1 |2 +; + +inWithExpressions +// tag::in-with-expressions[] +ROW a = 1, b = 4, c = 3 +| WHERE c-a IN (3, b / 2, a) +// end::in-with-expressions[] +; + +// tag::in-with-expressions-result[] +a:integer |b:integer |c:integer +1 |4 |3 +// end::in-with-expressions-result[] +; + +convertMvToMvDifferentCardinality +row strings = ["1", "2", "three"] | eval ints = to_int(strings); +warning:Line 1:49: evaluation of [to_int(strings)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"three\" + +strings:keyword |ints:integer +[1, 2, three] |[1, 2] +; + +convertMvToSv +row strings = ["1", "two"] | eval ints = to_int(strings); + +warning:Line 1:42: evaluation of [to_int(strings)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"two\" + +strings:keyword |ints:integer +[1, two] |1 +; + +convertMvToNull +row strings = ["one", "two"] | eval ints = to_int(strings); +warning:Line 1:44: evaluation of [to_int(strings)] failed, treating result as null. Only first 20 failures recorded. +warning:java.lang.NumberFormatException: For input string: \"one\" +warning:java.lang.NumberFormatException: For input string: \"two\" + +strings:keyword |ints:integer +[one, two] |null +; + +arraysWithNegatives +row integers = [+ 1, -2, -300, -2147483648], longs = [1, - 2, -2147483649], longs2 = [+1, -2, -9223372036854775808], doubles = [1, -.455, -2.43, 3.4, - 0.12, 8]; + +integers:integer | longs:long | longs2:long | doubles:double +[1, -2, -300, -2147483648] | [1, -2, -2147483649] | [1, -2, -9223372036854775808] | [1, -0.455, -2.43, 3.4, -0.12, 8] +; + +isNullIsNotNull +row true_bool = null is null, false_bool = null is not null, negated_true = not(null is null), negated_false = not(null is not null); + +true_bool:boolean | false_bool:boolean | negated_true:boolean | negated_false:boolean +true | false | false | true +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/show.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/show.csv-spec new file mode 100644 index 0000000000000..99991d39cf8d3 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/show.csv-spec @@ -0,0 +1,93 @@ +showInfo +show info | stats v = count(version); + +v:long +1 +; + +showFunctions +show functions; + + name:keyword | synopsis:keyword +abs |abs(arg1) +acos |acos(arg1) +asin |asin(arg1) +atan |atan(arg1) +atan2 |atan2(arg1, arg2) +auto_bucket |auto_bucket(arg1, arg2, arg3, arg4) +avg |avg(arg1) +case |case(arg1...) +cidr_match |cidr_match(arg1, arg2...) +concat |concat(arg1, arg2...) +cos |cos(arg1) +cosh |cosh(arg1) +count |count(arg1) +count_distinct |count_distinct(arg1, arg2) +date_extract |date_extract(arg1, arg2) +date_format |date_format(arg1, arg2) +date_parse |date_parse(arg1, arg2) +date_trunc |date_trunc(arg1, arg2) +e |e() +floor |floor(arg1) +is_finite |is_finite(arg1) +is_infinite |is_infinite(arg1) +is_nan |is_nan(arg1) +length |length(arg1) +log10 |log10(arg1) +max |max(arg1) +median |median(arg1) +median_absolute_deviation|median_absolute_deviation(arg1) +min |min(arg1) +mv_avg |mv_avg(arg1) +mv_concat |mv_concat(arg1, arg2) +mv_count |mv_count(arg1) +mv_dedupe |mv_dedupe(arg1) +mv_max |mv_max(arg1) +mv_median |mv_median(arg1) +mv_min |mv_min(arg1) +mv_sum |mv_sum(arg1) +now |now() +percentile |percentile(arg1, arg2) +pi |pi() +pow |pow(arg1, arg2) +round |round(arg1, arg2) +sin |sin(arg1) +sinh |sinh(arg1) +split |split(arg1, arg2) +sqrt |sqrt(arg1) +starts_with |starts_with(arg1, arg2) +substring |substring(arg1, arg2, arg3) +sum |sum(arg1) +tan |tan(arg1) +tanh |tanh(arg1) +tau |tau() +to_bool |to_bool(arg1) +to_boolean |to_boolean(arg1) +to_datetime |to_datetime(arg1) +to_dbl |to_dbl(arg1) +to_degrees |to_degrees(arg1) +to_double |to_double(arg1) +to_dt |to_dt(arg1) +to_int |to_int(arg1) +to_integer |to_integer(arg1) +to_ip |to_ip(arg1) +to_long |to_long(arg1) +to_radians |to_radians(arg1) +to_str |to_str(arg1) +to_string |to_string(arg1) +to_ul |to_ul(arg1) +to_ulong |to_ulong(arg1) +to_unsigned_long |to_unsigned_long(arg1) +to_ver |to_ver(arg1) +to_version |to_version(arg1) +trim |trim(arg1) +; + +showFunctionsFiltered +show functions | where starts_with(name, "is_"); + + name:keyword | synopsis:keyword +is_finite |is_finite(arg1) +is_infinite |is_infinite(arg1) +is_nan |is_nan(arg1) +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec new file mode 100644 index 0000000000000..d227bd3b7622e --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec @@ -0,0 +1,474 @@ +maxOfLong +from employees | stats l = max(languages.long); + +l:long +5 +; + +maxOfInteger +// tag::max[] +FROM employees +| STATS MAX(languages) +// end::max[] +; + +// tag::max-result[] +MAX(languages):integer +5 +// end::max-result[] +; + +minOfInteger +// tag::min[] +FROM employees +| STATS MIN(languages) +// end::min[] +; + +// tag::min-result[] +MIN(languages):integer +1 +// end::min-result[] +; + +maxOfShort +// short becomes int until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats l = max(languages.short); + +l:integer +5 +; + +maxOfByte +// byte becomes int until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats l = max(languages.byte); + +l:integer +5 +; + +maxOfDouble +from employees | stats h = max(height); + +h:double +2.1 +; + +maxOfFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = max(height.float); + +h:double +2.0999999046325684 +; + +maxOfHalfFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = max(height.half_float); + +h:double +2.099609375 +; + + +maxOfScaledFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = max(height.scaled_float); + +h:double +2.1 +; + + +maxOfManyLongs +from employees | stats l = max(salary_change.long); + +l:long +14 +; + + +maxOfManyInts +from employees | stats l = max(salary_change.int); + +l:integer +14 +; + + +maxOfManyDoubles +from employees | stats l = max(salary_change); + +l:double +14.74 +; + + +avgOfLong +from employees | stats l = avg(languages.long); + +l:double +3.1222222222222222 +; + +avgOfInteger +from employees | stats l = avg(languages); + +l:double +3.1222222222222222 +; + +avgOfShort +from employees | stats l = avg(languages.short); + +l:double +3.1222222222222222 +; + +avgOfByte +from employees | stats l = avg(languages.byte); + +l:double +3.1222222222222222 +; + +avgOfDouble +// tag::avg[] +FROM employees +| STATS AVG(height) +// end::avg[] +; + +// tag::avg-result[] +AVG(height):double +1.7682 +// end::avg-result[] +; + +avgOfFloat +from employees | stats h = avg(height.float); + +h:double +1.7681999909877777 +; + +avgOfHalfFloat +from employees | stats h = avg(height.half_float); + +h:double +1.76818359375 +; +avgOfScaledFloat +from employees | stats h = avg(height.scaled_float); + +h:double +1.7682 +; + +countOfDouble +// tag::count[] +FROM employees +| STATS COUNT(height) +// end::count[] +; + +// tag::count-result[] +COUNT(height):long +100 +// end::count-result[] +; + +sumOfLong +from employees | stats l = sum(languages.long); + +l:long +281 +; + +sumOfInteger +// tag::sum[] +FROM employees +| STATS SUM(languages) +// end::sum[] +; + +// tag::sum-result[] +SUM(languages):long +281 +// end::sum-result[] +; + +sumOfByte +from employees | stats l = sum(languages.byte); + +l:long +281 +; + +sumOfShort +from employees | stats l = sum(languages.short); + +l:long +281 +; + +sumOfDouble +from employees | stats h = sum(height); + +h:double +176.82 +; + +sumOfFloat +from employees | stats h = sum(height.float); + +h:double +176.81999909877777 +; + +sumOfHalfFloat +from employees | stats h = sum(height.half_float); + +h:double +176.818359375 +; + +sumOfScaledFloat +from employees | stats h = sum(height.scaled_float); + +h:double +176.82 +; + +groupWithMin +// declared to double check the tests below +from employees | stats m = min(height) by languages | sort languages; + +m:d | languages:i +1.42 | 1 +1.42 | 2 +1.44 | 3 +1.52 | 4 +1.5 | 5 +1.41 | null +; + +IfDuplicateNamesLastOneWins +from employees | stats h = avg(height), h = min(height) by languages | sort languages; + +h:d | languages:i +1.42 | 1 +1.42 | 2 +1.44 | 3 +1.52 | 4 +1.5 | 5 +1.41 | null +; + +groupByAlias +from employees | rename languages as l | keep l, height | stats m = min(height) by l | sort l; + +m:d | l:i +1.42 | 1 +1.42 | 2 +1.44 | 3 +1.52 | 4 +1.5 | 5 +1.41 | null +; + +IfDuplicateNamesGroupingHasPriority +from employees | stats languages = avg(height), languages = min(height) by languages | sort languages; + +languages:i +1 +2 +3 +4 +5 +null +; + +byStringAndLong +from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(gender) by gender, trunk_worked_seconds | sort c desc; + +c:long | gender:keyword | trunk_worked_seconds:long +30 | M | 300000000 +27 | M | 200000000 +22 | F | 300000000 +11 | F | 200000000 +; + +byStringAndLongWithAlias +from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | rename gender as g, trunk_worked_seconds as tws | keep g, tws | stats c = count(g) by g, tws | sort c desc; + +c:long | g:keyword | tws:long +30 | M | 300000000 +27 | M | 200000000 +22 | F | 300000000 +11 | F | 200000000 +; + +byStringAndString +from employees | eval hire_year_str = date_format(hire_date, "yyyy") | stats c = count(gender) by gender, hire_year_str | sort c desc, gender, hire_year_str | where c >= 5; + +c:long | gender:keyword | hire_year_str:keyword +8 | F | 1989 +8 | M | 1987 +8 | M | 1990 +7 | M | 1986 +6 | M | 1985 +6 | M | 1988 +5 | M | 1991 +5 | M | 1992 +; + +byLongAndLong +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; + +c:long | languages.long:long | trunk_worked_seconds:long +15 | 5 | 300000000 +11 | 2 | 300000000 +10 | 4 | 300000000 + 9 | 3 | 200000000 + 8 | 2 | 200000000 + 8 | 4 | 200000000 + 8 | 3 | 300000000 + 8 | 1 | 200000000 + 7 | 1 | 300000000 + 6 | 5 | 200000000 +; + +byUnmentionedLongAndLong +from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(gender) by languages.long, trunk_worked_seconds | sort c desc; + +c:long | languages.long:long | trunk_worked_seconds:long +13 | 5 | 300000000 +10 | 2 | 300000000 + 9 | 4 | 300000000 + 9 | 3 | 200000000 + 8 | 4 | 200000000 + 8 | 3 | 300000000 + 7 | 1 | 200000000 + 6 | 2 | 200000000 + 6 | 1 | 300000000 + 4 | 5 | 200000000 +; + +byUnmentionedIntAndLong +from employees | eval trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 | stats c = count(gender) by languages, trunk_worked_seconds | sort c desc; + +c:long | languages:integer | trunk_worked_seconds:long + 13 | 5 | 300000000 + 10 | 2 | 300000000 + 9 | 4 | 300000000 + 9 | 3 | 200000000 + 8 | 4 | 200000000 + 8 | 3 | 300000000 + 7 | 1 | 200000000 + 6 | 2 | 200000000 + 6 | null | 300000000 + 6 | 1 | 300000000 + 4 | null | 200000000 + 4 | 5 | 200000000 +; + +byUnmentionedIntAndBoolean +from employees | stats c = count(gender) by languages, still_hired | sort c desc, languages desc; + +c:long | languages:integer | still_hired:boolean + 11 | 3 | false + 11 | 2 | true + 10 | 4 | false + 9 | 5 | true + 8 | 5 | false + 8 | 1 | false + 7 | 4 | true + 6 | null | false + 6 | 3 | true + 5 | 2 | false + 5 | 1 | true + 4 | null | true +; + +byDateAndKeywordAndInt +from employees | eval d = date_trunc(hire_date, 1 year) | stats c = count(emp_no) by d, gender, languages | sort c desc, d, languages desc | limit 10; + +c:long | d:date | gender:keyword | languages:integer + 3 | 1986-01-01T00:00:00.000Z | M | 2 + 3 | 1987-01-01T00:00:00.000Z | M | 2 + 2 | 1985-01-01T00:00:00.000Z | M | 5 + 2 | 1985-01-01T00:00:00.000Z | M | 3 + 2 | 1986-01-01T00:00:00.000Z | M | 5 + 2 | 1986-01-01T00:00:00.000Z | M | 4 + 2 | 1987-01-01T00:00:00.000Z | null | 5 + 2 | 1987-01-01T00:00:00.000Z | F | 5 + 2 | 1987-01-01T00:00:00.000Z | M | 3 + 2 | 1987-01-01T00:00:00.000Z | M | 1 +; + +byDateAndKeywordAndIntWithAlias +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; + +c:long | d:date | g:keyword | l:integer + 3 | 1986-01-01T00:00:00.000Z | M | 2 + 3 | 1987-01-01T00:00:00.000Z | M | 2 + 2 | 1985-01-01T00:00:00.000Z | M | 5 + 2 | 1985-01-01T00:00:00.000Z | M | 3 + 2 | 1986-01-01T00:00:00.000Z | M | 5 + 2 | 1986-01-01T00:00:00.000Z | M | 4 + 2 | 1987-01-01T00:00:00.000Z | null | 5 + 2 | 1987-01-01T00:00:00.000Z | F | 5 + 2 | 1987-01-01T00:00:00.000Z | M | 3 + 2 | 1987-01-01T00:00:00.000Z | M | 1 +; + +byDoubleAndBoolean +from employees | stats c = count(gender) by height, still_hired | sort c desc, height | limit 10; + +c:long | height:double | still_hired:boolean +4 | 1.52 | true +4 | 1.77 | true +3 | 1.83 | false +3 | 2.1 | true +2 | 1.44 | true +2 | 1.53 | false +2 | 1.55 | false +2 | 1.57 | true +2 | 1.59 | false +2 | 1.61 | false +; + +byMvBoolean +from employees | stats min(salary), max(salary) by is_rehired | sort is_rehired; + +min(salary):integer | max(salary):integer | is_rehired:boolean +25324 | 74970 | false +25324 | 74999 | true +27215 | 66174 | null +; + +byMvInt +from employees | stats min(salary), max(salary) by salary_change.int | sort salary_change.int desc | limit 5; + +min(salary):integer | max(salary):integer | salary_change.int:integer +26436 | 74970 | null +25324 | 73578 | 14 +36174 | 68547 | 13 +25324 | 69904 | 12 +28336 | 56760 | 11 +; + +aggsWithoutGroupingCount +from employees | stats count(salary); + +count(salary):l +100 +; + +aggsWithoutGroupingMinMax +from employees | stats min(salary), max(salary), c = count(salary); + +min(salary):i | max(salary):i | c:l +25324 | 74999 | 100 +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_count_distinct.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_count_distinct.csv-spec new file mode 100644 index 0000000000000..be36a49f28057 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_count_distinct.csv-spec @@ -0,0 +1,130 @@ +countDistinctOfLong +from employees | stats l = count_distinct(languages.long); + +l:long +5 +; + +countDistinctOfInteger +from employees | stats l = count_distinct(languages); + +l:long +5 +; + +countDistinctOfShort +// short becomes int until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats l = count_distinct(languages.short); + +l:long +5 +; + +countDistinctOfByte +// byte becomes int until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats l = count_distinct(languages.byte); + +l:long +5 +; + +countDistinctOfDouble +from employees | stats h = count_distinct(height, 100); + +h:long +54 +; + +countDistinctOfFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = count_distinct(height.float); + +h:long +54 +; + +countDistinctOfHalfFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = count_distinct(height.half_float); + +h:long +54 +; + +countDistinctOfScaledFloat +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats h = count_distinct(height.scaled_float); + +h:long +54 +; + +countDistinctOfBoolean +// float becomes double until https://github.com/elastic/elasticsearch-internal/issues/724 +from employees | stats c = count_distinct(still_hired); + +c:long +2 +; + +countDistinctOfKeywords +from employees | eval hire_year_str = date_format(hire_date, "yyyy") | stats g = count_distinct(gender), h = count_distinct(hire_year_str); + +g:long | h:long +2 | 14 +; + +countDistinctOfIp +// tag::count-distinct[] +FROM hosts +| STATS COUNT_DISTINCT(ip0), COUNT_DISTINCT(ip1) +// end::count-distinct[] +; + +// tag::count-distinct-result[] +COUNT_DISTINCT(ip0):long | COUNT_DISTINCT(ip1):long +7 | 8 +// end::count-distinct-result[] +; + +countDistinctOfIpPrecision +// tag::count-distinct-precision[] +FROM hosts +| STATS COUNT_DISTINCT(ip0, 80000), COUNT_DISTINCT(ip1, 5) +// end::count-distinct-precision[] +; + +// tag::count-distinct-precision-result[] +COUNT_DISTINCT(ip0,80000):long | COUNT_DISTINCT(ip1,5):long +7 | 9 +// end::count-distinct-precision-result[] +; + +countDistinctOfDates +from employees | eval d = date_trunc(hire_date, 1 year) | stats h = count_distinct(d); + +h:long +14 +; + +countDistinctWithGroup +from employees | stats m = count_distinct(height, 9876) by languages | sort languages; + +m:long | languages:i +13 | 1 +16 | 2 +14 | 3 +15 | 4 +20 | 5 +10 | null +; + +countDistinctOfIpGroupByKeyword +from hosts | stats h0 = count_distinct(ip0), h1 = count_distinct(ip1) by host | sort host; + +h0:long | h1:long | host:keyword +2 | 2 | alpha +1 | 3 | beta +5 | 6 | epsilon +1 | 2 | gamma +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_percentile.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_percentile.csv-spec new file mode 100644 index 0000000000000..eaa0786588480 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats_percentile.csv-spec @@ -0,0 +1,144 @@ +percentileOfLong +from employees | stats p0 = percentile(salary_change.long, 0), p50 = percentile(salary_change.long, 50), p99 = percentile(salary_change.long, 99); + +p0:double | p50:double | p99:double +-9 | 0 | 14 +; + + +percentileOfInteger +// tag::percentile[] +FROM employees +| STATS p0 = PERCENTILE(salary, 0) + , p50 = PERCENTILE(salary, 50) + , p99 = PERCENTILE(salary, 99) +// end::percentile[] +; + +// tag::percentile-result[] +p0:double | p50:double | p99:double +25324 | 47003 | 74970.29 +// end::percentile-result[] +; + + +percentileOfDouble +from employees | stats p0 = percentile(salary_change, 0), p50 = percentile(salary_change, 50), p99 = percentile(salary_change, 99); + +p0:double | p50:double | p99:double +-9.81 | 0.75 | 14.639000000000001 +; + + +percentileOfLongByKeyword +from employees | stats p90 = percentile(salary_change.long, 90) by job_positions | sort p90 | limit 4; + +p90:double | job_positions:keyword + 7 | "Python Developer" +9.600000000000001 | "Business Analyst" +10.200000000000006 | "Data Scientist" +10.399999999999999 | "Senior Python Developer" +; + +percentileOfIntegerByKeyword +from employees | stats p90 = percentile(salary, 90) by job_positions | sort p90 | limit 4; + +p90:double | job_positions:keyword +50249.0 | "Business Analyst" +54462.0 | "Support Engineer" +56308.799999999996 | "Reporting Analyst" +56645.0 | "Head Human Resources" +; + + +percentileOfDoubleByKeyword +from employees | stats p90 = percentile(salary_change, 90) by job_positions | sort p90 | limit 4; + +p90:double | job_positions:keyword +7.5760000000000005 | "Python Developer" +10.095000000000002 | "Business Analyst" +10.362000000000007 | "Data Scientist" +10.964999999999998 | "Senior Python Developer" +; + + +invalidPercentile +from employees | stats x = percentile(salary_change, 110); + +x:double +NULL +; + + +medianOfLong +from employees | stats m = median(salary_change.long), p50 = percentile(salary_change.long, 50); + +m:double | p50:double +0 | 0 +; + +medianOfInteger +// tag::median[] +FROM employees +| STATS MEDIAN(salary), PERCENTILE(salary, 50) +// end::median[] +; + +// tag::median-result[] +MEDIAN(salary):double | PERCENTILE(salary,50):double +47003 | 47003 +// end::median-result[] +; + +medianOfDouble +from employees | stats m = median(salary_change), p50 = percentile(salary_change, 50); + +m:double | p50:double +0.75 | 0.75 +; + + +medianOfLongByKeyword +from employees | stats m = median(salary_change.long), p50 = percentile(salary_change.long, 50) by job_positions | sort m desc | limit 4; + +m:double | p50:double | job_positions:keyword +5 | 5 | "Accountant" +4.5 | 4.5 | "Reporting Analyst" +4 | 4 | "Support Engineer" +3.5 | 3.5 | "Architect" +; + + +medianOfIntegerByKeyword +from employees | stats m = median(salary), p50 = percentile(salary, 50) by job_positions | sort m | limit 4; + +m:double | p50:double | job_positions:keyword +38992 | 38992 | "Python Developer" +39638 | 39638 | "Business Analyst" +40031 | 40031 | "Tech Lead" +41933 | 41933 | "Support Engineer" +; + + +medianOfDoubleByKeyword +from employees | stats m = median(salary_change), p50 = percentile(salary_change, 50)by job_positions | sort m desc | limit 4; + +m:double | p50:double | job_positions:keyword +5.94 | 5.94 | "Accountant" +4.87 | 4.87 | "Reporting Analyst" +4.62 | 4.62 | "Support Engineer" +3.9299999999999997 | 3.9299999999999997 | "Architect" +; + +medianAbsoluteDeviation +// tag::median-absolute-deviation[] +FROM employees +| STATS MEDIAN(salary), MEDIAN_ABSOLUTE_DEVIATION(salary) +// end::median-absolute-deviation[] +; + +// tag::median-absolute-deviation-result[] +MEDIAN(salary):double | MEDIAN_ABSOLUTE_DEVIATION(salary):double +47003 | 10096.5 +// end::median-absolute-deviation-result[] +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/string.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/string.csv-spec new file mode 100644 index 0000000000000..2dbceb3b12335 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/string.csv-spec @@ -0,0 +1,636 @@ +rowWithStrings +row a = "hi", b = "", c = "Ünîcødé❗️"; + +a:keyword | b:keyword | c:keyword +hi | | Ünîcødé❗️ +; + +length +row a = "hello", b = "" | eval y = length(a) + length(b); + +a:keyword | b:keyword | y:integer +hello | | 5 +; + +// note, emojis are encoded with at least two unicode code points +lengthWithNonAsciiChars +row a = "¡", b = "❗️" | eval y = length(a) | eval z = length(b); + +a:keyword | b:keyword | y:integer | z:integer +¡ | ❗️ | 1 | 2 +; + +foldLength +row a = 1 | eval b = length("hello"); + +a:integer | b:integer +1 | 5 +; + + +length +from employees | sort emp_no | limit 3 | eval l = length(first_name) | keep emp_no, l; + +emp_no:integer | l:integer +10001 | 6 +10002 | 7 +10003 | 5 +; + +startsWithConstant +from employees | sort emp_no | limit 10 | eval f_S = starts_with(first_name, "S") | keep emp_no, first_name, f_S; + +emp_no:integer | first_name:keyword | f_S:boolean +10001 | Georgi | false +10002 | Bezalel | false +10003 | Parto | false +10004 | Chirstian | false +10005 | Kyoichi | false +10006 | Anneke | false +10007 | Tzvetan | false +10008 | Saniya | true +10009 | Sumant | true +10010 | Duangkaew | false +; + +startsWithField +from employees | where emp_no <= 10010 | eval f_l = starts_with(last_name, gender) | keep emp_no, last_name, gender, f_l; + +emp_no:integer | last_name:keyword | gender:keyword | f_l:boolean +10001 | Facello | M | false +10002 | Simmel | F | false +10003 | Bamford | M | false +10004 | Koblick | M | false +10005 | Maliniak | M | true +10006 | Preusig | F | false +10007 | Zielinski | F | false +10008 | Kalloufi | M | false +10009 | Peac | F | false +10010 | Piveteau | null | null +; + +substring +from employees | where emp_no <= 10010 | eval f_l = substring(last_name, 3) | keep emp_no, last_name, f_l; + +emp_no:integer | last_name:keyword | f_l:keyword +10001 | Facello | cello +10002 | Simmel | mmel +10003 | Bamford | mford +10004 | Koblick | blick +10005 | Maliniak | liniak +10006 | Preusig | eusig +10007 | Zielinski | elinski +10008 | Kalloufi | lloufi +10009 | Peac | ac +10010 | Piveteau | veteau +; + +substring with length +from employees | where emp_no <= 10010 | eval f_l = substring(last_name, 3, 1) | keep emp_no, last_name, f_l; + +emp_no:integer | last_name:keyword | f_l:keyword +10001 | Facello | c +10002 | Simmel | m +10003 | Bamford | m +10004 | Koblick | b +10005 | Maliniak | l +10006 | Preusig | e +10007 | Zielinski | e +10008 | Kalloufi | l +10009 | Peac | a +10010 | Piveteau | v +; + +substring negative start +from employees | where emp_no <= 10010 | eval f_l = substring(last_name, -3) | keep emp_no, last_name, f_l; + +emp_no:integer | last_name:keyword | f_l:keyword +10001 | Facello | llo +10002 | Simmel | mel +10003 | Bamford | ord +10004 | Koblick | ick +10005 | Maliniak | iak +10006 | Preusig | sig +10007 | Zielinski | ski +10008 | Kalloufi | ufi +10009 | Peac | eac +10010 | Piveteau | eau +; + +substring nested negative start +from employees | where emp_no <= 10010 | eval f_l = substring(substring(last_name, -3),-1) | keep emp_no, last_name, f_l; + +emp_no:integer | last_name:keyword | f_l:keyword +10001 | Facello | o +10002 | Simmel | l +10003 | Bamford | d +10004 | Koblick | k +10005 | Maliniak | k +10006 | Preusig | g +10007 | Zielinski | i +10008 | Kalloufi | i +10009 | Peac | c +10010 | Piveteau | u +; + +substring length +from employees | where emp_no <= 10010 | eval f_l = length(substring(last_name, 3)) | keep emp_no, last_name, f_l; + +emp_no:integer | last_name:keyword | f_l:integer +10001 | Facello | 5 +10002 | Simmel | 4 +10003 | Bamford | 5 +10004 | Koblick | 5 +10005 | Maliniak | 6 +10006 | Preusig | 5 +10007 | Zielinski | 7 +10008 | Kalloufi | 6 +10009 | Peac | 2 +10010 | Piveteau | 6 +; + +substring pair +from employees | where emp_no <= 10010 | eval x = substring(last_name, 1, 1), y = 1, z = substring("abcdef", y, y) | keep emp_no, last_name, x, z; + +emp_no:integer | last_name:keyword | x:keyword | z:keyword +10001 | Facello | F | a +10002 | Simmel | S | a +10003 | Bamford | B | a +10004 | Koblick | K | a +10005 | Maliniak | M | a +10006 | Preusig | P | a +10007 | Zielinski | Z | a +10008 | Kalloufi | K | a +10009 | Peac | P | a +10010 | Piveteau | P | a +; + +trim +from employees | sort emp_no | limit 10 | eval name = concat(" ", first_name) | eval name = trim(first_name) | keep emp_no, name; + +emp_no:integer | name:keyword +10001 | Georgi +10002 | Bezalel +10003 | Parto +10004 | Chirstian +10005 | Kyoichi +10006 | Anneke +10007 | Tzvetan +10008 | Saniya +10009 | Sumant +10010 | Duangkaew +; + +trimRow +// tag::trim[] +ROW message = " some text ", color = " red "| EVAL message = TRIM(message)| EVAL color = TRIM(color) +// end::trim[] +; + +// tag::trim-result[] +message:s | color:s +some text | red +// end::trim-result[] +; + +concat +from employees | sort emp_no | limit 10 | eval name = concat(first_name, " ", last_name) | keep emp_no, name; + +emp_no:integer | name:keyword +10001 | Georgi Facello +10002 | Bezalel Simmel +10003 | Parto Bamford +10004 | Chirstian Koblick +10005 | Kyoichi Maliniak +10006 | Anneke Preusig +10007 | Tzvetan Zielinski +10008 | Saniya Kalloufi +10009 | Sumant Peac +10010 | Duangkaew Piveteau +; + +concatComplex +from employees | sort emp_no | limit 10 | eval foo = " - ", x = concat(gender, foo) | eval name = concat(x, first_name, " ", last_name, ", ", concat(first_name, last_name)) | keep emp_no, name; + +emp_no:integer | name:keyword +10001 | M - Georgi Facello, GeorgiFacello +10002 | F - Bezalel Simmel, BezalelSimmel +10003 | M - Parto Bamford, PartoBamford +10004 | M - Chirstian Koblick, ChirstianKoblick +10005 | M - Kyoichi Maliniak, KyoichiMaliniak +10006 | F - Anneke Preusig, AnnekePreusig +10007 | F - Tzvetan Zielinski, TzvetanZielinski +10008 | M - Saniya Kalloufi, SaniyaKalloufi +10009 | F - Sumant Peac, SumantPeac +10010 | null +; + +// Note: no matches in MV returned +in +from employees | where job_positions in ("Internship", first_name) | keep emp_no, job_positions; + +emp_no:integer |job_positions:keyword +10048 |Internship +10077 |Internship +; + +in3VLNoNull +// filtering for SVs, since IN uses EQUALS evaluators, that turn MVs into NULL +from employees | where mv_count(job_positions) <= 1 | where emp_no >= 10024 | limit 3 | keep emp_no, job_positions | eval is_in = job_positions in ("Accountant", "Internship"); + +emp_no:integer |job_positions:keyword |is_in:boolean +10024 |Junior Developer |false +10025 |Accountant |true +10026 |Reporting Analyst |false +; + +in3VLWithNull +from employees | where mv_count(job_positions) <= 1 | where emp_no >= 10024 | limit 3 | keep emp_no, job_positions | eval is_in = job_positions in ("Accountant", "Internship", null); + +emp_no:integer |job_positions:keyword |is_in:boolean +10024 |Junior Developer |null +10025 |Accountant |true +10026 |Reporting Analyst |null +; + +in3VLWithComputedNull +from employees | where mv_count(job_positions) <= 1 | where emp_no >= 10024 | limit 3 | keep emp_no, job_positions | eval nil = concat("", null) | eval is_in = job_positions in ("Accountant", "Internship", nil); + +emp_no:integer |job_positions:keyword |nil:keyword |is_in:boolean +10024 |Junior Developer |null |null +10025 |Accountant |null |true +10026 |Reporting Analyst |null |null +; + +in3VLWithNullAsValue +from employees | where mv_count(job_positions) <= 1 | where emp_no >= 10024 | limit 3 | keep emp_no, job_positions | eval is_in = null in ("Accountant", "Internship", null); + +emp_no:integer |job_positions:keyword |is_in:boolean +10024 |Junior Developer |null +10025 |Accountant |null +10026 |Reporting Analyst |null +; + +in3VLWithComputedNullAsValue +from employees | where mv_count(job_positions) <= 1 | where emp_no >= 10024 | limit 3 | keep emp_no, job_positions | eval nil = concat("", null) | eval is_in = nil in ("Accountant", "Internship", null); + +emp_no:integer |job_positions:keyword |nil:keyword |is_in:boolean +10024 |Junior Developer |null |null +10025 |Accountant |null |null +10026 |Reporting Analyst |null |null +; + +split +// tag::split[] +ROW words="foo;bar;baz;qux;quux;corge" +| EVAL word = SPLIT(words, ";") +// end::split[] +; + +// tag::split-result[] + words:keyword | word:keyword +foo;bar;baz;qux;quux;corge | [foo,bar,baz,qux,quux,corge] +// end::split-result[] +; + +mvCount +// tag::mv_count[] +ROW a=["foo", "zoo", "bar"] +| EVAL count_a = MV_COUNT(a) +// end::mv_count[] +; + +// tag::mv_count-result[] + a:keyword | count_a:integer +["foo", "zoo", "bar"] | 3 +// end::mv_count-result[] +; + +mvDedupe +// tag::mv_dedupe[] +ROW a=["foo", "foo", "bar", "foo"] +| EVAL dedupe_a = MV_DEDUPE(a) +// end::mv_dedupe[] +; + +// tag::mv_dedupe-result[] + a:keyword | dedupe_a:keyword +["foo", "foo", "bar", "foo"] | ["foo", "bar"] +// end::mv_dedupe-result[] +; + +mvJoin +// tag::mv_concat[] +ROW a=["foo", "zoo", "bar"] +| EVAL j = MV_CONCAT(a, ", ") +// end::mv_concat[] +; + +// tag::mv_concat-result[] + a:keyword | j:keyword +["foo", "zoo", "bar"] | "foo, zoo, bar" +// end::mv_concat-result[] +; + +mvMax +// tag::mv_max[] +ROW a=["foo", "zoo", "bar"] +| EVAL max_a = MV_MAX(a) +// end::mv_max[] +; + +// tag::mv_max-result[] + a:keyword | max_a:keyword +["foo", "zoo", "bar"] | "zoo" +// end::mv_max-result[] +; + +mvMin +// tag::mv_min[] +ROW a=["foo", "bar"] +| EVAL min_a = MV_MIN(a) +// end::mv_min[] +; + +// tag::mv_min-result[] + a:keyword | min_a:keyword +["foo", "bar"] | "bar" +// end::mv_min-result[] +; + +groupByMv +from employees | stats min(salary), max(salary) by job_positions | sort job_positions | limit 5; + +min(salary):integer | max(salary):integer | job_positions:keyword +25976 | 74970 | Accountant +28941 | 69904 | Architect +29175 | 58121 | Business Analyst +25945 | 74999 | Data Scientist +25324 | 58715 | Head Human Resources +; + +convertFromString +from employees | eval positions = to_string(job_positions) | keep emp_no, positions, job_positions | limit 5; + +emp_no:integer |positions:keyword |job_positions:keyword +10001 |[Accountant, Senior Python Developer] |[Accountant, Senior Python Developer] +10002 |Senior Team Lead |Senior Team Lead +10003 |null |null +10004 |[Head Human Resources, Reporting Analyst, Support Engineer, Tech Lead]|[Head Human Resources, Reporting Analyst, Support Engineer, Tech Lead] +10005 |null |null +; + +lessThanMultivalue +from employees | where job_positions < "C" | keep emp_no, job_positions | sort emp_no; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10025 |Accountant +10068 |Architect +; + +greaterThanMultivalue +from employees | where job_positions > "C" | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10002 |Senior Team Lead +10013 |Reporting Analyst +10018 |Junior Developer +10019 |Purchase Manager +10020 |Tech Lead +10021 |Support Engineer +; + +equalToMultivalue +from employees | where job_positions == "Accountant" | keep emp_no, job_positions | sort emp_no; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10025 |Accountant +; + +equalToOrEqualToMultivalue +from employees | where job_positions == "Accountant" or job_positions == "Tech Lead" | keep emp_no, job_positions | sort emp_no; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10020 |Tech Lead +10025 |Accountant +; + +inMultivalue +from employees | where job_positions in ("Accountant", "Tech Lead") | keep emp_no, job_positions | sort emp_no; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10020 |Tech Lead +10025 |Accountant +; + +notLessThanMultivalue +from employees | where not(job_positions < "C") | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10002 |Senior Team Lead +10013 |Reporting Analyst +10018 |Junior Developer +10019 |Purchase Manager +10020 |Tech Lead +10021 |Support Engineer +; + +notGreaterThanMultivalue +from employees | where not(job_positions > "C") | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10025 |Accountant +10068 |Architect +; + +notEqualToMultivalue +from employees | where not(job_positions == "Accountant") | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10002 |Senior Team Lead +10013 |Reporting Analyst +10018 |Junior Developer +10019 |Purchase Manager +10020 |Tech Lead +10021 |Support Engineer +; + +notEqualToOrEqualToMultivalue-Ignore +from employees | where not(job_positions == "Accountant" or job_positions == "Tech Lead") | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10020 |Tech Lead // TODO flip results +10025 |Accountant +; + +notInMultivalue-Ignore +from employees | where not(job_positions in ("Accountant", "Tech Lead")) | keep emp_no, job_positions | sort emp_no | limit 6; + +// Note that multivalued job_positions aren't included because they aren't less than or greater than C - that comparison is null +emp_no:integer |job_positions:keyword +10020 |Tech Lead // TODO flip results +10025 |Accountant +; + +convertFromBoolean +from employees | eval rehired = to_string(is_rehired) | keep emp_no, rehired, is_rehired | limit 5; + +emp_no:integer |rehired:string |is_rehired:boolean +10001 |[false, true] |[false, true] +10002 |[false, false] |[false, false] +10003 |null |null +10004 |true |true +10005 |[false, false, false, true] |[false, false, false, true] +; + +convertFromDatetime +from employees | sort emp_no| eval hired_at = to_string(hire_date) | keep emp_no, hired_at, hire_date | limit 1; + +emp_no:integer |hired_at:keyword |hire_date:date +10001 |1986-06-26T00:00:00.000Z |1986-06-26T00:00:00.000Z +; + +convertFromIP +from hosts | where host=="epsilon" | eval str0 = to_string(ip0) | keep str0, ip0; + +str0:keyword |ip0:ip +["fe80::cae2:65ff:fece:feb9", "fe80::cae2:65ff:fece:fec0", "fe80::cae2:65ff:fece:fec1"] |[fe80::cae2:65ff:fece:feb9, fe80::cae2:65ff:fece:fec0, fe80::cae2:65ff:fece:fec1] +null |null +["fe81::cae2:65ff:fece:feb9", "fe82::cae2:65ff:fece:fec0"] |[fe81::cae2:65ff:fece:feb9, fe82::cae2:65ff:fece:fec0] +; + +convertFromFloats +from employees | sort emp_no| eval double = to_string(height), float = to_string(height.float), scaled_float = to_string(height.scaled_float), half_float = to_string(height.half_float) | keep emp_no, double, float, scaled_float, half_float, height | limit 2; + +emp_no:integer |double:keyword |float:keyword |scaled_float:keyword |half_float:keyword |height:double +10001 |2.03 |2.0299999713897705|2.0300000000000002 |2.029296875 |2.03 +10002 |2.08 |2.0799999237060547|2.08 |2.080078125 |2.08 +; + +convertFromInts +from employees | sort emp_no| eval byte = to_string(languages.byte), short = to_string(languages.short), long = to_string(languages.long), int = to_string(languages) | keep emp_no, byte, short, long, int, languages | limit 2; + +emp_no:integer |byte:keyword |short:keyword |long:keyword |int:keyword |languages:integer +10001 |2 |2 |2 |2 |2 +10002 |5 |5 |5 |5 |5 +; + +convertFromUnsignedLong +row ul = [9223372036854775808, 9223372036854775807, 1, 0] | eval str = to_str(ul); + + ul:ul | str:keyword +[9223372036854775808, 9223372036854775807, 1, 0]|[9223372036854775808, 9223372036854775807, 1, 0] +; + +convertFromIntSimple +// tag::to_string[] +ROW a=10 +| EVAL j = TO_STRING(a) +// end::to_string[] +; + +// tag::to_string-result[] +a:integer | j:keyword + 10 | "10" +// end::to_string-result[] +; + +convertFromIntMultivalue +// tag::to_string_multivalue[] +ROW a=[10, 9, 8] +| EVAL j = TO_STRING(a) +// end::to_string_multivalue[] +; + +// tag::to_string_multivalue-result[] + a:integer | j:keyword +[10, 9, 8] | ["10", "9", "8"] +// end::to_string_multivalue-result[] +; + +mvConcatToString +// tag::mv_concat-to_string[] +ROW a=[10, 9, 8] +| EVAL j = MV_CONCAT(TO_STRING(a), ", ") +// end::mv_concat-to_string[] +; + +// tag::mv_concat-to_string-result[] + a:integer | j:keyword +[10, 9, 8] | "10, 9, 8" +// end::mv_concat-to_string-result[] +; + +showTextFields +from hosts | where host == "beta" | keep host, host_group, description; + +host:keyword | host_group:text | description:text +beta | Kubernetes cluster | beta k8s server +beta | Kubernetes cluster | beta k8s server +beta | Kubernetes cluster | [beta k8s server, beta k8s server2] +; + +lengthOfText +from hosts | where host=="epsilon" | eval l1 = length(host_group), l2 = length(description) | keep l1, l2; + +l1:integer | l2:integer +null | 19 +17 | null +17 | null +; + +startsWithText +from hosts | where host=="epsilon" | eval l1 = starts_with(host_group, host), l2 = starts_with(description, host) | keep l1, l2; + +l1:boolean | l2:boolean +null | true +false | null +false | null +; + +substringOfText +from hosts | where host=="epsilon" | eval l1 = substring(host_group, 0, 5), l2 = substring(description, 0, 5) | keep l1, l2; + +l1:keyword | l2:keyword +null | epsil +Gatew | null +Gatew | null +; + +concatOfText +from hosts | where host == "epsilon" | eval l1 = concat(host,"/", host_group), l2 = concat(host_group,"/", description) | sort l1 | keep l1, l2; + +l1:keyword | l2:keyword +epsilon/Gateway instances | null +epsilon/Gateway instances | null +null | null +; + + +grok +from hosts | where host == "epsilon" | grok host_group "%{WORD:l1} %{WORD:l2}"| sort l1 | keep l1, l2; + +l1:keyword | l2:keyword +Gateway | instances +Gateway | instances +null | null +; + + +dissect +from hosts | where host == "epsilon" | dissect host_group "%{l1} %{l2}"| sort l1 | keep l1, l2; + +l1:keyword | l2:keyword +Gateway | instances +Gateway | instances +null | null +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ul_logs.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ul_logs.csv new file mode 100644 index 0000000000000..e826148f51c2e --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/ul_logs.csv @@ -0,0 +1,102 @@ +id:integer,@timestamp:date,bytes_in:unsigned_long,bytes_out:unsigned_long,status:keyword +1,2017-11-10T21:15:54Z,4348801185987554667,12749081495402663265,OK +2,2017-11-10T21:15:39Z,11054572642507476486,2215793005711196537,OK +3,2017-11-10T21:15:39Z,7239423344688551324,4747671420480199905,OK +4,2017-11-10T21:15:39Z,12880875341157998416,10347160802894727455,OK +5,2017-11-10T21:15:40Z,6480569113728286781,4628689249901172357,OK +6,2017-11-10T21:15:40Z,8847365258155648277,18107197698386620672,OK +7,2017-11-10T21:15:40Z,18081123477485622121,6254036056888007861,OK +8,2017-11-10T21:15:41Z,17159009460398071592,6041947699951197416,OK +9,2017-11-10T21:15:41Z,18317075104972913640,3738987414350619907,OK +10,2017-11-10T20:36:07Z,9223372036854775807,13014552081688587417,OK +11,2017-11-10T20:36:08Z,10618481193158417699,7645257133789254601,OK +12,2017-11-10T20:36:07Z,14243423136348863449,1851693232606252132,OK +13,2017-11-10T20:36:07Z,8014838889043461601,12855878692699288887,OK +14,2017-11-10T20:36:15Z,9704166250476073712,9243820354371174974,OK +15,2017-11-10T20:36:15Z,16673466483681919036,17281501450843634251,OK +16,2017-11-10T20:35:54Z,11414099303186823563,4552407785188434877,OK +17,2017-11-10T20:35:54Z,9614024902524991937,583785103161450865,OK +18,2017-11-10T20:35:55Z,2703254959364209157,15688732125935676003,OK +19,2017-11-10T17:54:43Z,16907772202142018796,1978055896356244912,OK +20,2017-11-10T23:23:24Z,18446744073709551614,9891957732954625161,OK +21,2017-11-10T17:54:59Z,18098466156271475039,10560599221675458546,OK +22,2017-11-10T21:13:27Z,12113814789427553914,17695317925249333633,OK +23,2017-11-10T22:37:41Z,369412756671598363,4454824974559554214,OK +24,2017-11-10T20:34:43Z,17764691215469285192,751496841062464739,OK +25,2017-11-10T23:30:46Z,316080452389500167,13471731928228498458,OK +26,2017-11-10T21:13:16Z,3987249898147090269,857017108209908030,OK +27,2017-11-10T23:36:32Z,9343007301895818617,13652755194722568502,OK +28,2017-11-10T23:36:33Z,12951716972543168268,9336652471323200906,OK +29,2017-11-10T20:35:26Z,16002960716282089759,6754707638562449159,OK +30,2017-11-10T23:36:41Z,18446744073709550591,14393839423240122480,OK +31,2017-11-10T23:56:36Z,5495907774457032585,8384790841458113028,OK +32,2017-11-10T20:29:25Z,905851433235877972,11682551086136399874,Error +33,2017-11-10T21:35:01Z,4368413537705409055,10386906319745215430,OK +34,2017-11-10T21:12:17Z,16002960716282089759,16002960716282089759,OK +35,2017-11-10T23:17:14Z,9188929021194043442,991636820083925493,OK +36,2017-11-10T23:28:11Z,16002960716282089759,17953153966527637143,OK +37,2017-11-10T22:36:27Z,8156660980420095219,901610289258538340,OK +38,2017-11-10T20:35:55Z,2408213296071189837,419872666232023984,OK +39,2017-11-10T20:35:55Z,17460378829280278708,10724795375261191248,OK +40,2017-11-10T20:35:55Z,18446744073709551614,14524142879756567901,OK +41,2017-11-10T20:35:55Z,,,Error +42,2017-11-10T21:34:49Z,[154551962150890564, 154551962150890561],4317649615355527138,Error +43,2017-11-10T20:35:55Z,6713823401157015713,768392740554438381,OK +44,2017-11-10T20:14:04Z,13007085541148329579,1262767764958640849,OK +45,2017-11-10T19:38:06Z,4008445367955620676,2444837981761911481,OK +46,2017-11-10T21:14:18Z,9056948257586320738,3660006000364826492,OK +47,2017-11-10T20:35:56Z,10640542847470647209,3071012467454913482,OK +48,2017-11-10T20:53:05Z,14463699407888333801,16193000254773667372,OK +49,2017-11-10T21:25:42Z,4691003749418709874,16735032755695343779,OK +50,2017-11-10T21:14:44Z,18446744073709551615,8359170160363687272,OK +51,2017-11-10T21:28:34Z,10414368669933920698,17857609920324506371,OK +52,2017-11-10T20:35:55Z,14591698995327831783,837800054257171070,OK +53,2017-11-10T20:15:24Z,9149768745019330607,9934783425401329847,OK +54,2017-11-10T20:35:57Z,5826090293715995525,13263580863583654980,OK +55,2017-11-10T17:14:10Z,15352019942832250739,1498178946494790227,OK +56,2017-11-10T20:35:57Z,9732690250707058359,2520919358333960813,OK +57,2017-11-10T23:22:13Z,8914368988247035466,16187631537609304549,OK +58,2017-11-10T20:32:57Z,8420006392678593250,14938622925960605968,OK +59,2017-11-10T21:24:00Z,17056885385468285787,9973198429366930442,OK +60,2017-11-10T20:35:56Z,9223372036854775808,6620615504579533702,OK +61,2017-11-10T23:43:10Z,2390976293435536689,16020561580624977312,OK +62,2017-11-10T20:35:57Z,10993546521190430203,18184253384683076090,OK +63,2017-11-10T20:21:58Z,154551962150890564,9382204513185396493,OK +64,2017-11-10T20:35:57Z,9983398877364735609,10626289664367265415,OK +65,2017-11-10T20:33:06Z,5480608687137202404,6895880056122579688,Error +66,2017-11-10T20:35:57Z,7538807943450220608,11745980216826561015,OK +67,2017-11-10T20:26:21Z,17067060651018256448,1722789377000665830,OK +68,2017-11-10T21:23:25Z,16873365461162643186,10056378788277261033,OK +69,2017-11-10T21:23:54Z,9991932520184465636,16110121334900810541,OK +70,2017-11-10T20:35:57Z,0,2507200025082562692,OK +71,2017-11-10T00:27:03Z,0,18223615477147360166,OK +72,2017-11-10T00:27:46Z,0,11206857258468587792,OK +73,2017-11-10T20:35:58Z,13986802678251316321,1330575423003442317,OK +74,2017-11-10T20:35:57Z,13922094693483143156,14343149449348005776,OK +75,2017-11-10T22:27:09Z,13999070515664268533,8422074124513216267,OK +76,2017-11-10T20:35:58Z,15968566213936682639,3784845108080773823,OK +77,2017-11-10T22:26:44Z,1729864283282545225,11105009496753939058,OK +78,2017-11-10T22:27:31Z,14241624006161076477,11563896463355414928,OK +79,2017-11-10T20:35:52Z,2294690022638798960,14564159158999105001,OK +80,2017-11-10T00:00:22Z,0,11060623717086222747,OK +81,2017-11-10T20:35:52Z,7470203340634956368,7490193999241578548,OK +82,2017-11-10T00:01:20Z,74330435873664882,4875216609683497742,OK +83,2017-11-10T00:01:04Z,9636626466125797351,14208813483941526550,OK +84,2017-11-10T00:32:48Z,11949176856304796477,8190769023162854115,OK +85,2017-11-10T00:01:45Z,[754822992931077409, 154551962150890564],12647826153259487490,OK +86,2017-11-10T20:36:08Z,16424089095262982944,12394320926003300611,OK +87,2017-11-10T21:17:37Z,10580536762493152413,13605535835272740587,OK +88,2017-11-10T20:06:49Z,195161570976258241,15395084776572180858,Error +89,2017-11-10T21:17:37Z,15084788733189711518,6353233118260828721,OK +90,2017-11-10T19:51:38Z,,,Error +91,2017-11-10T19:51:38Z,11628588779507401305,8500236459902170712,Error +92,2017-11-10T20:06:50Z,2706408999083639864,594246218266628121,OK +93,2017-11-10T21:17:46Z,9007528787465012783,15931740851225178582,OK +94,2017-11-10T19:51:38Z,18345360876889252152,16119381686035586648,Error +95,2017-11-10T21:17:46Z,2788944430410706777,11087293691148056886,OK +96,2017-11-10T00:04:50Z,9932469097722733505,14925592145374204307,OK +97,2017-11-10T21:17:48Z,11620953158540412267,3809712277266935082,OK +98,2017-11-10T21:12:24Z,3448205404634246112,5409549730889481641,OK +99,2017-11-10T21:17:37Z,1957665857956635540,352442273299370793,OK +100,2017-11-10T03:21:36Z,16462768484251021236,15616395223975497926,OK +101,2017-11-10T23:22:36Z,,,Error diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/unsigned_long.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/unsigned_long.csv-spec new file mode 100644 index 0000000000000..2566fc5845f86 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/unsigned_long.csv-spec @@ -0,0 +1,156 @@ + +simpleLoad +from ul_logs | sort id | where id in (40, 41, 42, 43); + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k +2017-11-10T20:35:55.000Z|18446744073709551614 |14524142879756567901|40 |OK +2017-11-10T20:35:55.000Z|null |null |41 |Error +2017-11-10T21:34:49.000Z|[154551962150890561, 154551962150890564]|4317649615355527138 |42 |Error +2017-11-10T20:35:55.000Z|6713823401157015713 |768392740554438381 |43 |OK +; + +sortAsc +from ul_logs | sort bytes_in nulls first, id | limit 12; + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k +2017-11-10T20:35:55.000Z|null |null |41 |Error +2017-11-10T19:51:38.000Z|null |null |90 |Error +2017-11-10T23:22:36.000Z|null |null |101 |Error +2017-11-10T20:35:57.000Z|0 |2507200025082562692 |70 |OK +2017-11-10T00:27:03.000Z|0 |18223615477147360166|71 |OK +2017-11-10T00:27:46.000Z|0 |11206857258468587792|72 |OK +2017-11-10T00:00:22.000Z|0 |11060623717086222747|80 |OK +2017-11-10T00:01:20.000Z|74330435873664882 |4875216609683497742 |82 |OK +2017-11-10T21:34:49.000Z|[154551962150890561, 154551962150890564]|4317649615355527138 |42 |Error +2017-11-10T20:21:58.000Z|154551962150890564 |9382204513185396493 |63 |OK +2017-11-10T00:01:45.000Z|[154551962150890564, 754822992931077409]|12647826153259487490|85 |OK +2017-11-10T20:06:49.000Z|195161570976258241 |15395084776572180858|88 |Error +; + +sortDesc +from ul_logs | sort bytes_in desc nulls last, id | limit 12; + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k +2017-11-10T21:14:44.000Z|18446744073709551615|8359170160363687272 |50 |OK +2017-11-10T23:23:24.000Z|18446744073709551614|9891957732954625161 |20 |OK +2017-11-10T20:35:55.000Z|18446744073709551614|14524142879756567901|40 |OK +2017-11-10T23:36:41.000Z|18446744073709550591|14393839423240122480|30 |OK +2017-11-10T19:51:38.000Z|18345360876889252152|16119381686035586648|94 |Error +2017-11-10T21:15:41.000Z|18317075104972913640|3738987414350619907 |9 |OK +2017-11-10T17:54:59.000Z|18098466156271475039|10560599221675458546|21 |OK +2017-11-10T21:15:40.000Z|18081123477485622121|6254036056888007861 |7 |OK +2017-11-10T20:34:43.000Z|17764691215469285192|751496841062464739 |24 |OK +2017-11-10T20:35:55.000Z|17460378829280278708|10724795375261191248|39 |OK +2017-11-10T21:15:41.000Z|17159009460398071592|6041947699951197416 |8 |OK +2017-11-10T20:26:21.000Z|17067060651018256448|1722789377000665830 |67 |OK +; + +filterPushDownGT +from ul_logs | where bytes_in >= to_ul(74330435873664882) | sort bytes_in | eval div = bytes_in / to_ul(pow(10., 15)) | keep bytes_in, div, id | limit 12; + + bytes_in:ul | div:ul |id:i +74330435873664882 |74 |82 +154551962150890564 |154 |63 +195161570976258241 |195 |88 +316080452389500167 |316 |25 +369412756671598363 |369 |23 +905851433235877972 |905 |32 +1729864283282545225|1729 |77 +1957665857956635540|1957 |99 +2294690022638798960|2294 |79 +2390976293435536689|2390 |61 +2408213296071189837|2408 |38 +2703254959364209157|2703 |18 +; + +filterPushDownRange +from ul_logs | where bytes_in >= to_ul(74330435873664882) | where bytes_in <= to_ul(316080452389500167) | sort bytes_in | eval div = bytes_in / to_ul(pow(10., 15)) | keep bytes_in, div, id | limit 12; + + bytes_in:ul | div:ul |id:i +74330435873664882 |74 |82 +154551962150890564 |154 |63 +195161570976258241 |195 |88 +316080452389500167 |316 |25 +; + +filterPushDownIn +// TODO: testing framework doesn't perform implicit conversion to UL of given values, needs explicit conversion +from ul_logs | where bytes_in in (to_ul(74330435873664882), to_ul(154551962150890564), to_ul(195161570976258241)) | sort bytes_in | keep bytes_in, id; + + bytes_in:ul |id:i +74330435873664882 |82 +154551962150890564 |63 +195161570976258241 |88 +; + +filterOnFieldsEquality +from ul_logs | where bytes_in == bytes_out; + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k +2017-11-10T21:12:17.000Z|16002960716282089759|16002960716282089759|34 |OK +; + +filterOnFieldsInequality +from ul_logs | where bytes_in < bytes_out | eval b_in = bytes_in / to_ul(pow(10.,15)), b_out = bytes_out / to_ul(pow(10.,15)) | limit 5; + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k | b_in:ul | b_out:ul +2017-11-10T21:15:54.000Z|4348801185987554667 |12749081495402663265|1 |OK |4348 |12749 +2017-11-10T21:15:40.000Z|8847365258155648277 |18107197698386620672|6 |OK |8847 |18107 +2017-11-10T20:36:07.000Z|9223372036854775807 |13014552081688587417|10 |OK |9223 |13014 +2017-11-10T20:36:07.000Z|8014838889043461601 |12855878692699288887|13 |OK |8014 |12855 +2017-11-10T20:36:15.000Z|16673466483681919036|17281501450843634251|15 |OK |16673 |17281 +; + +groupBy +from ul_logs | stats c = count(bytes_in) by bytes_in | sort c desc, bytes_in desc | limit 10; + + c:l | bytes_in:ul +5 | 154551962150890564 +4 | 0 +3 |16002960716282089759 +2 |18446744073709551614 +2 | 754822992931077409 +2 | 154551962150890561 +1 |18446744073709551615 +1 |18446744073709550591 +1 |18345360876889252152 +1 |18317075104972913640 +; + +case +from ul_logs | where case(bytes_in == to_ul(154551962150890564), true, false); + + @timestamp:date | bytes_in:ul | bytes_out:ul | id:i | status:k +2017-11-10T20:21:58.000Z|154551962150890564|9382204513185396493|63 |OK +; + +autoBucket +FROM ul_logs +| WHERE @timestamp >= "2017-11-10T20:30:00Z" AND @timestamp < "2017-11-10T20:35:00Z" +| EVAL bh = auto_bucket(bytes_in, 20, 5480608687137202404, 17764691215469285192) +| SORT @timestamp +| KEEP @timestamp, bytes_in, bh +; + +@timestamp:date | bytes_in:ul | bh:double +2017-11-10T20:32:57.000Z | 8420006392678593250 | 8.0E18 +2017-11-10T20:33:06.000Z | 5480608687137202404 | 5.0E18 +2017-11-10T20:34:43.000Z | 17764691215469285192 | 1.75E19 +; + +toDegrees +FROM ul_logs | WHERE bytes_in == bytes_out | EVAL deg = TO_DEGREES(bytes_in) | KEEP bytes_in, deg +; + + bytes_in:ul | deg:double +16002960716282089759 | 9.169021087566165E20 +; + + +toRadians +FROM ul_logs | WHERE bytes_in == bytes_out | EVAL rad = TO_RADIANS(bytes_in) | KEEP bytes_in, rad +; + + bytes_in:ul | rad:double +16002960716282089759 | 2.79304354566432608E17 +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/version.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/version.csv-spec new file mode 100644 index 0000000000000..b7b3ca1c99d3c --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/version.csv-spec @@ -0,0 +1,311 @@ +// To mute tests follow example in file: example.csv-spec + +// +// Tests for VERSION fields +// + +selectAll +FROM apps; + +id:integer |name:keyword |version:version +1 |aaaaa |1 +2 |bbbbb |2.1 +3 |ccccc |2.3.4 +4 |ddddd |2.12.0 +5 |eeeee |1.11.0 +6 |fffff |5.2.9 +7 |ggggg |5.2.9-SNAPSHOT +8 |hhhhh |1.2.3.4 +9 |iiiii |bad +10 |jjjjj |5.2.9 +11 |kkkkk |null +12 |aaaaa |1.2.3.4 +13 |lllll |null +14 |mmmmm |5.2.9 +; + +filterByVersion +FROM apps | WHERE version == to_ver("2.12.0"); + +id:i |name:k |version:v +4 |ddddd |2.12.0 +; + +projectionVersion +FROM apps | WHERE id == 3 | KEEP version; + +version:v +2.3.4 +; + +versionRange1 +FROM apps | WHERE version > to_ver("2.2") | SORT id; + +id:i |name:k |version:v +3 |ccccc |2.3.4 +4 |ddddd |2.12.0 +6 |fffff |5.2.9 +7 |ggggg |5.2.9-SNAPSHOT +9 |iiiii |bad +10 |jjjjj |5.2.9 +14 |mmmmm |5.2.9 +; + +versionRange2 +FROM apps | WHERE version >= to_ver("2.3.4") | SORT id; + +id:i |name:k |version:v +3 |ccccc |2.3.4 +4 |ddddd |2.12.0 +6 |fffff |5.2.9 +7 |ggggg |5.2.9-SNAPSHOT +9 |iiiii |bad +10 |jjjjj |5.2.9 +14 |mmmmm |5.2.9 +; + +between +FROM apps | WHERE version >= to_ver("1.10") AND version <= to_ver("5.2.9") | SORT id; + +id:i |name:k |version:v +2 |bbbbb | 2.1 +3 |ccccc | 2.3.4 +4 |ddddd | 2.12.0 +5 |eeeee | 1.11.0 +6 |fffff | 5.2.9 +7 |ggggg | 5.2.9-SNAPSHOT +10 |jjjjj | 5.2.9 +14 |mmmmm | 5.2.9 +; + +orderByVersion +FROM apps | SORT version, id; + +id:i |name:s |version:v +1 |aaaaa |1 +8 |hhhhh |1.2.3.4 +12 |aaaaa |1.2.3.4 +5 |eeeee |1.11.0 +2 |bbbbb |2.1 +3 |ccccc |2.3.4 +4 |ddddd |2.12.0 +7 |ggggg |5.2.9-SNAPSHOT +6 |fffff |5.2.9 +10 |jjjjj |5.2.9 +14 |mmmmm |5.2.9 +9 |iiiii |bad +11 |kkkkk |null +13 |lllll |null +; + +orderByVersionDesc +FROM apps | SORT version DESC, id ASC; + +id:i |name:s |version:v +11 |kkkkk |null +13 |lllll |null +9 |iiiii |bad +6 |fffff |5.2.9 +10 |jjjjj |5.2.9 +14 |mmmmm |5.2.9 +7 |ggggg |5.2.9-SNAPSHOT +4 |ddddd |2.12.0 +3 |ccccc |2.3.4 +2 |bbbbb |2.1 +5 |eeeee |1.11.0 +8 |hhhhh |1.2.3.4 +12 |aaaaa |1.2.3.4 +1 |aaaaa |1 +; + +orderByVersionNullsFirst +FROM apps | SORT version NULLS FIRST, id; + +id:i |name:s |version:v +11 |kkkkk |null +13 |lllll |null +1 |aaaaa |1 +8 |hhhhh |1.2.3.4 +12 |aaaaa |1.2.3.4 +5 |eeeee |1.11.0 +2 |bbbbb |2.1 +3 |ccccc |2.3.4 +4 |ddddd |2.12.0 +7 |ggggg |5.2.9-SNAPSHOT +6 |fffff |5.2.9 +10 |jjjjj |5.2.9 +14 |mmmmm |5.2.9 +9 |iiiii |bad +; + +orderByVersionMultipleCasts +FROM apps | EVAL o = TO_VER(CONCAT("1.", TO_STR(version))) | SORT o, id; + +id:i |name:s |version:v |o:v +1 |aaaaa |1 |1.1 +8 |hhhhh |1.2.3.4 |1.1.2.3.4 +12 |aaaaa |1.2.3.4 |1.1.2.3.4 +5 |eeeee |1.11.0 |1.1.11.0 +2 |bbbbb |2.1 |1.2.1 +3 |ccccc |2.3.4 |1.2.3.4 +4 |ddddd |2.12.0 |1.2.12.0 +7 |ggggg |5.2.9-SNAPSHOT |1.5.2.9-SNAPSHOT +6 |fffff |5.2.9 |1.5.2.9 +10 |jjjjj |5.2.9 |1.5.2.9 +14 |mmmmm |5.2.9 |1.5.2.9 +9 |iiiii |bad |1.bad +11 |kkkkk |null |null +13 |lllll |null |null +; + +countVersion +FROM apps | RENAME name AS k | STATS v = COUNT(version) BY k | SORT k; + +v:l | k:s +2 | aaaaa +1 | bbbbb +1 | ccccc +1 | ddddd +1 | eeeee +1 | fffff +1 | ggggg +1 | hhhhh +1 | iiiii +1 | jjjjj +0 | kkkkk +0 | lllll +1 | mmmmm +; + +groupByVersion +FROM apps | STATS c = COUNT(version), maxid = MAX(id) BY version | SORT version; + +c:l |maxid:i |version:v +1 |1 |1 +2 |12 |1.2.3.4 +1 |5 |1.11.0 +1 |2 |2.1 +1 |3 |2.3.4 +1 |4 |2.12.0 +1 |7 |5.2.9-SNAPSHOT +3 |14 |5.2.9 +1 |9 |bad +0 |13 |null +; + +groupOrderLimit +FROM apps | WHERE version is not null | STATS c = COUNT(version) BY version | SORT version DESC | DROP c | LIMIT 3; + +version:v +bad +5.2.9 +5.2.9-SNAPSHOT +; + +groupByVersionCast +FROM apps | EVAL g = TO_VER(CONCAT("1.", TO_STR(version))) | STATS id = MAX(id) BY g | SORT id | DROP g; + +id:i +1 +2 +3 +4 +5 +7 +9 +12 +13 +14 +; + +castConstantToVersion +// tag::to_version[] +ROW v = TO_VERSION("1.2.3") +// end::to_version[] +; + +// tag::to_version-result[] +v:version +1.2.3 +// end::to_version-result[] +; + +// AwaitFix: #1521 better plan queries that return only constants +castConstantToVersion2-Ignore +FROM apps | EVAL v = TO_VERSION("1.2.3") | KEEP v; + +v:v +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +; + +// AwaitFix: #1521 better plan queries that return only constants +multipleCast-Ignore +FROM apps | EVAL v = TO_STR(TO_VER("1.2.3")) | KEEP v; + +v:s +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +1.2.3 +; + +compareVersions +ROW v1 = TO_VER("1.2.3"), v2 = TO_VER("1.11.4") | EVAL v = v1 < v2 | KEEP v; + +v:boolean +true +; + +groupByVersionAfterStats +FROM apps | STATS idx = MAX(id) BY version | WHERE idx == 14; + +idx:i |version:v +14 | 5.2.9 +; + +case +FROM apps +| EVAL version_text = TO_STR(version) +| WHERE version IS NULL OR version_text LIKE "1*" +| EVAL v = TO_VER(CONCAT("123", TO_STR(version))) +| EVAL m = CASE(version > TO_VER("1.1"), 1, 0) +| EVAL g = CASE(version > TO_VER("1.3.0"), version, TO_VER("1.3.0")) +| EVAL i = CASE(version IS NULL, TO_VER("0.1"), version) +| EVAL c = CASE( + 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; + +v:v | version:v |version_text:s | id:i | m:i | g:v | i:v | c:s +1231.11.0 | 1.11.0 | 1.11.0 | 5 | 1 | 1.11.0 | 1.11.0 | high +1231.2.3.4 | 1.2.3.4 | 1.2.3.4 | 12 | 1 | 1.3.0 | 1.2.3.4 | high +1231.2.3.4 | 1.2.3.4 | 1.2.3.4 | 8 | 1 | 1.3.0 | 1.2.3.4 | high +1231 | 1 | 1 | 1 | 0 | 1.3.0 | 1 | low +null | null | null | 13 | 0 | 1.3.0 | 0.1 | none +null | null | null | 11 | 0 | 1.3.0 | 0.1 | none +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where-like.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where-like.csv-spec new file mode 100644 index 0000000000000..36e92c723354b --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where-like.csv-spec @@ -0,0 +1,273 @@ +likePrefix +from employees | where first_name like "Eberhar*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +likeSuffix +from employees | where first_name like "*uhito" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10016 | Kazuhito +; + + +likePrefixSuffix +from employees | where first_name like "*har*" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +10058 | Berhard +10068 | Charlene +10089 | Sudharsan +; + + +likePrefixSuffix2 +from employees | where first_name like "?berhar*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +likeAndEquals +from employees | where first_name like "Mayu*" and last_name == "Warwick" | keep emp_no, first_name, last_name; + +emp_no:integer | first_name:keyword | last_name:keyword +10020 | Mayuko | Warwick +; + + +likeAndOr +from employees | where first_name like "Eberhar*" or first_name like "*zuh*" and last_name like "*eha" | keep emp_no, first_name, last_name; + +emp_no:integer | first_name:keyword | last_name:keyword +10013 | Eberhardt | Terkki +10018 | Kazuhide | Peha +; + + +evalLike +from employees | eval x = concat(first_name, "--")| where x like "Hidefu*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10051 | Hidefumi +; + + +likeExpression +from employees | where concat(first_name, "--") like "Hidefu*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10051 | Hidefumi +; + + +likeNoWildcard +from employees | where first_name like "Eberhardt" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +likeEvalNoWildcard +from employees | eval x = concat(first_name, "X") | where x like "EberhardtX" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +likeAll +from employees | where first_name like "*" and emp_no > 10028 | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10029 | Otmar +10040 | Weiyi +; + + + +notFieldLike +from employees | where not first_name like "Geor*" | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10002 | Bezalel +10003 | Parto +; + + +fieldNotLike +from employees | where first_name not like "Geor*" | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10002 | Bezalel +10003 | Parto +; + + +notFieldNotLike +from employees | where not first_name not like "Xing*" | sort emp_no | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10087 | Xinglin +; + + +notBraceFieldNotLike +from employees | where not (first_name not like "Xing*") | sort emp_no | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10087 | Xinglin +; + + +rLikePrefix +from employees | where first_name rlike "Aleja.*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10059 | Alejandro +; + + +rLikeSuffix +from employees | where first_name rlike ".*itij" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10079 | Kshitij +; + + +rLikePrefixSuffix +from employees | where first_name rlike ".*har.*" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +10058 | Berhard +10068 | Charlene +10089 | Sudharsan +; + + +rLikePrefix2 +from employees | where first_name rlike ".leja.*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10059 | Alejandro +; + + +rLikeComplex +from employees | where first_name rlike "(Eberhar.*)|(.*arlene)" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +10068 | Charlene +; + + +rlikeAndEquals +from employees | where first_name rlike "Mayu.*" and last_name == "Warwick" | keep emp_no, first_name, last_name; + +emp_no:integer | first_name:keyword | last_name:keyword +10020 | Mayuko | Warwick +; + + +rLikeAndOr +from employees | where first_name rlike "Eberhar.*" or first_name rlike ".*zuh.*" and last_name rlike ".*eha" | keep emp_no, first_name, last_name; + +emp_no:integer | first_name:keyword | last_name:keyword +10013 | Eberhardt | Terkki +10018 | Kazuhide | Peha +; + + +evalRLike +from employees | eval x = concat(first_name, "--")| where x rlike "Hidefu.*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10051 | Hidefumi +; + + +rlikeExpression +from employees | where concat(first_name, "--") rlike "Hidefu.*" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10051 | Hidefumi +; + + +rLikeNoWildcard +from employees | where first_name rlike "Eberhardt" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +rLikeEvalNoWildcard +from employees | eval x = concat(first_name, "X") | where x rlike "EberhardtX" | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10013 | Eberhardt +; + + +rLikeAll +from employees | where first_name rlike ".*" and emp_no > 10028 | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10029 | Otmar +10040 | Weiyi +; + + +notFieldRLike +from employees | where not first_name rlike "Geor.*" | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10002 | Bezalel +10003 | Parto +; + + +fieldNotRLike +from employees | where first_name not rlike "Geor.*" | sort emp_no | keep emp_no, first_name | limit 2; + +emp_no:integer | first_name:keyword +10002 | Bezalel +10003 | Parto +; + + +notFieldNotRLike +from employees | where not first_name not rlike "Xing.*" | sort emp_no | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10087 | Xinglin +; + + +notBraceFieldNotRLike +from employees | where not (first_name not rlike "Xing.*") | sort emp_no | keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10087 | Xinglin +; + + +rLikeOrComplexExpression +from employees | keep emp_no, first_name, last_name | where first_name RLIKE ".*o{2,}.*" OR last_name RLIKE ".*o{2,}.*" | sort emp_no; + +emp_no:integer | first_name:keyword | last_name:keyword +10015 | Guoxiang | Nooteboom +10062 | Anoosh | Peyn +10086 | Somnath | Foote +10088 | Jungsoon | Syrzycki +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where.csv-spec new file mode 100644 index 0000000000000..89f329bc6dcb9 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/where.csv-spec @@ -0,0 +1,78 @@ +twoEqualsOr +from employees | where emp_no == 10010 or emp_no == 10011 | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + + +twoEqualsOrKeyword +from employees | where first_name == "Duangkaew" or first_name == "Mary" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + + +twoEqualsAndOr +from employees | where emp_no == 10010 and first_name == "Duangkaew" or emp_no == 10011 and first_name == "Mary" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + + +twoEqualsAndOr2 +from employees | where emp_no == 10010 and first_name == "Duangkaew" or emp_no == 10011 and first_name == "FooBar" | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +; + + +twoEqualsOrBraces +from employees | where (emp_no == 10010 or emp_no == 10011) | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + + +twoInequalityAnd +from employees | where emp_no >= 10010 and emp_no <= 10011 | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + +threeEqualsOr +from employees | where emp_no == 10010 or emp_no == 10011 or emp_no == 10012 | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +10012 |Patricio +; + + +evalTwoEqualsOr +from employees | eval x = emp_no + 10010 - emp_no | where emp_no == x or emp_no == 10011 | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; + + +evalTwoInequalityAnd +from employees | eval x = emp_no + 10010 - emp_no | where emp_no >= x and emp_no <= 10011 | keep emp_no, first_name | sort emp_no; + +emp_no:integer | first_name:keyword +10010 |Duangkaew +10011 |Mary +; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractEsqlIntegTestCase.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractEsqlIntegTestCase.java new file mode 100644 index 0000000000000..ad61bbfd61779 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractEsqlIntegTestCase.java @@ -0,0 +1,115 @@ +/* + * 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.action; + +import org.elasticsearch.Build; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.plugin.TransportEsqlQueryAction; +import org.junit.After; + +import java.util.Collection; +import java.util.List; + +@TestLogging(value = "org.elasticsearch.xpack.esql.session:DEBUG", reason = "to better understand planning") +public abstract class AbstractEsqlIntegTestCase extends ESIntegTestCase { + + @After + public void ensureExchangesAreReleased() throws Exception { + for (String node : internalCluster().getNodeNames()) { + TransportEsqlQueryAction esqlQueryAction = internalCluster().getInstance(TransportEsqlQueryAction.class, node); + ExchangeService exchangeService = esqlQueryAction.exchangeService(); + assertBusy(() -> assertTrue("Leftover exchanges " + exchangeService + " on node " + node, exchangeService.isEmpty())); + } + } + + public static class InternalExchangePlugin extends Plugin { + @Override + public List> getSettings() { + return List.of( + Setting.timeSetting( + ExchangeService.INACTIVE_SINKS_INTERVAL_SETTING, + TimeValue.timeValueSeconds(5), + Setting.Property.NodeScope + ) + ); + } + } + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), EsqlPlugin.class); + } + + protected EsqlQueryResponse run(String esqlCommands) { + return run(esqlCommands, randomPragmas()); + } + + protected EsqlQueryResponse run(String esqlCommands, QueryPragmas pragmas) { + return run(esqlCommands, pragmas, null); + } + + protected EsqlQueryResponse run(String esqlCommands, QueryPragmas pragmas, QueryBuilder filter) { + EsqlQueryRequest request = new EsqlQueryRequest(); + request.query(esqlCommands); + request.pragmas(pragmas); + if (filter != null) { + request.filter(filter); + } + return run(request); + } + + protected EsqlQueryResponse run(EsqlQueryRequest request) { + return client().execute(EsqlQueryAction.INSTANCE, request).actionGet(); + } + + protected static QueryPragmas randomPragmas() { + Settings.Builder settings = Settings.builder(); + // pragmas are only enabled on snapshot builds + if (Build.current().isSnapshot()) { + if (randomBoolean()) { + settings.put("task_concurrency", randomLongBetween(1, 10)); + } + if (randomBoolean()) { + final int exchangeBufferSize; + if (frequently()) { + exchangeBufferSize = randomIntBetween(1, 10); + } else { + exchangeBufferSize = randomIntBetween(5, 5000); + } + settings.put("exchange_buffer_size", exchangeBufferSize); + } + if (randomBoolean()) { + settings.put("exchange_concurrent_clients", randomIntBetween(1, 10)); + } + if (randomBoolean()) { + settings.put("data_partitioning", randomFrom("shard", "segment", "doc")); + } + if (randomBoolean()) { + final int pageSize = switch (between(0, 2)) { + case 0 -> between(1, 16); + case 1 -> between(1, 1024); + case 2 -> between(64, 10 * 1024); + default -> throw new AssertionError("unknown"); + }; + settings.put("page_size", pageSize); + } + } + return new QueryPragmas(settings.build()); + } + +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java new file mode 100644 index 0000000000000..2bffd5c64cdaf --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java @@ -0,0 +1,91 @@ +/* + * 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.action; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; + +/** + * Makes sure that the circuit breaker is "plugged in" to ESQL by configuring an + * unreasonably small breaker and tripping it. + */ +@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) +public class EsqlActionBreakerIT extends AbstractEsqlIntegTestCase { + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(super.nodePlugins()); + plugins.add(InternalExchangePlugin.class); + return plugins; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), "1kb") + /* + * Force standard settings for the request breaker or we may not break at all. + * Without this we can randomly decide to use the `noop` breaker for request + * and it won't break..... + */ + .put( + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.getKey(), + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.getDefault(Settings.EMPTY) + ) + .put( + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.getKey(), + HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.getDefault(Settings.EMPTY) + ) + .put(ExchangeService.INACTIVE_SINKS_INTERVAL_SETTING, TimeValue.timeValueMillis(between(500, 2000))) + .build(); + } + + public void testBreaker() { + for (int i = 0; i < 5000; i++) { + IndexResponse response = client().prepareIndex("test").setId(Integer.toString(i)).setSource("foo", i, "bar", i * 2).get(); + if (response.getResult() != DocWriteResponse.Result.CREATED) { + fail("failure: " + response); + } + } + client().admin().indices().prepareRefresh("test").get(); + ensureYellow("test"); + ElasticsearchException e = expectThrows( + ElasticsearchException.class, + () -> run("from test | stats avg(foo) by bar", QueryPragmas.EMPTY) + ); + logger.info("expected error", e); + if (e instanceof CircuitBreakingException) { + // The failure occurred before starting the drivers + assertThat(e.getMessage(), containsString("Data too large")); + } else { + // The failure occurred after starting the drivers + assertThat(e.getMessage(), containsString("Compute engine failure")); + assertThat(e.getMessage(), containsString("Data too large")); + assertThat(e.getCause(), instanceOf(CircuitBreakingException.class)); + assertThat(e.getCause().getMessage(), containsString("Data too large")); + } + } +} 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 new file mode 100644 index 0000000000000..c8eac41e59d9c --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java @@ -0,0 +1,1176 @@ +/* + * 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.action; + +import org.elasticsearch.Build; +import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.esql.analysis.VerificationException; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.junit.Assert; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalDouble; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.IntStream; +import java.util.stream.LongStream; + +import static java.util.Comparator.comparing; +import static java.util.Comparator.naturalOrder; +import static java.util.Comparator.reverseOrder; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; + +public class EsqlActionIT extends AbstractEsqlIntegTestCase { + + long epoch = System.currentTimeMillis(); + + @Before + public void setupIndex() { + createAndPopulateIndex("test"); + } + + public void testRow() { + long value = randomLongBetween(0, Long.MAX_VALUE); + EsqlQueryResponse response = run("row " + value); + assertEquals(List.of(List.of(value)), response.values()); + } + + public void testFromStatsGroupingAvgWithSort() { + testFromStatsGroupingAvgImpl("from test | stats avg(count) by data | sort data | limit 2", "data", "avg(count)"); + } + + public void testFromStatsGroupingAvg() { + testFromStatsGroupingAvgImpl("from test | stats avg(count) by data", "data", "avg(count)"); + } + + public void testFromStatsGroupingAvgWithAliases() { + testFromStatsGroupingAvgImpl("from test | eval g = data | stats f = avg(count) by g", "g", "f"); + } + + private void testFromStatsGroupingAvgImpl(String command, String expectedGroupName, String expectedFieldName) { + EsqlQueryResponse results = run(command); + logger.info(results); + Assert.assertEquals(2, results.columns().size()); + + // assert column metadata + ColumnInfo valuesColumn = results.columns().get(0); + assertEquals(expectedFieldName, valuesColumn.name()); + assertEquals("double", valuesColumn.type()); + ColumnInfo groupColumn = results.columns().get(1); + assertEquals(expectedGroupName, groupColumn.name()); + assertEquals("long", groupColumn.type()); + + // assert column values + List> valueValues = results.values(); + assertEquals(2, valueValues.size()); + // This is loathsome, find a declarative way to assert the expected output. + if ((long) valueValues.get(0).get(1) == 1L) { + assertEquals(42.0, (double) valueValues.get(0).get(0), 0.0); + assertEquals(2L, (long) valueValues.get(1).get(1)); + assertEquals(44.0, (double) valueValues.get(1).get(0), 0.0); + } else if ((long) valueValues.get(0).get(1) == 2L) { + assertEquals(42.0, (double) valueValues.get(1).get(0), 0.0); + assertEquals(1L, (long) valueValues.get(1).get(1)); + assertEquals(44.0, (double) valueValues.get(0).get(0), 0.0); + } else { + fail("Unexpected group value: " + valueValues.get(0).get(0)); + } + } + + public void testFromStatsGroupingCount() { + testFromStatsGroupingCountImpl("from test | stats count(count) by data", "data", "count(count)"); + } + + public void testFromStatsGroupingCountWithAliases() { + testFromStatsGroupingCountImpl("from test | eval grp = data | stats total = count(count) by grp", "grp", "total"); + } + + private void testFromStatsGroupingCountImpl(String command, String expectedFieldName, String expectedGroupName) { + EsqlQueryResponse results = run(command); + logger.info(results); + Assert.assertEquals(2, results.columns().size()); + + // assert column metadata + ColumnInfo groupColumn = results.columns().get(0); + assertEquals(expectedGroupName, groupColumn.name()); + assertEquals("long", groupColumn.type()); + ColumnInfo valuesColumn = results.columns().get(1); + assertEquals(expectedFieldName, valuesColumn.name()); + assertEquals("long", valuesColumn.type()); + + // assert column values + List> valueValues = results.values(); + assertEquals(2, valueValues.size()); + // This is loathsome, find a declarative way to assert the expected output. + if ((long) valueValues.get(0).get(1) == 1L) { + assertEquals(20L, valueValues.get(0).get(0)); + assertEquals(2L, valueValues.get(1).get(1)); + assertEquals(20L, valueValues.get(1).get(0)); + } else if ((long) valueValues.get(0).get(1) == 2L) { + assertEquals(20L, valueValues.get(1).get(0)); + assertEquals(1L, valueValues.get(1).get(1)); + assertEquals(20L, valueValues.get(0).get(0)); + } else { + fail("Unexpected group value: " + valueValues.get(0).get(1)); + } + } + + // Grouping where the groupby field is of a date type. + public void testFromStatsGroupingByDate() { + EsqlQueryResponse results = run("from test | stats avg(count) by time"); + logger.info(results); + Assert.assertEquals(2, results.columns().size()); + Assert.assertEquals(40, results.values().size()); + + // assert column metadata + assertEquals("avg(count)", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals("time", results.columns().get(1).name()); + assertEquals("long", results.columns().get(1).type()); + + // assert column values + List expectedValues = LongStream.range(0, 40).map(i -> epoch + i).sorted().boxed().toList(); + List actualValues = IntStream.range(0, 40).mapToLong(i -> (Long) results.values().get(i).get(1)).sorted().boxed().toList(); + assertEquals(expectedValues, actualValues); + } + + public void testFromGroupingByNumericFieldWithNulls() { + for (int i = 0; i < 5; i++) { + client().prepareBulk() + .add(new IndexRequest("test").id("no_count_old_" + i).source("data", between(1, 2), "data_d", 1d)) + .add(new IndexRequest("test").id("no_count_new_" + i).source("data", 99, "data_d", 1d)) + .add(new IndexRequest("test").id("no_data_" + i).source("count", 12, "count_d", 12d)) + .get(); + if (randomBoolean()) { + client().admin().indices().prepareRefresh("test").get(); + } + } + client().admin().indices().prepareRefresh("test").get(); + EsqlQueryResponse results = run("from test | stats avg(count) by data | sort data"); + logger.info(results); + + assertThat(results.columns(), hasSize(2)); + assertEquals("avg(count)", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals("data", results.columns().get(1).name()); + assertEquals("long", results.columns().get(1).type()); + + record Group(Long data, Double avg) {} + List expectedGroups = List.of(new Group(1L, 42.0), new Group(2L, 44.0), new Group(99L, null), new Group(null, 12.0)); + List actualGroups = results.values().stream().map(l -> new Group((Long) l.get(1), (Double) l.get(0))).toList(); + assertThat(actualGroups, equalTo(expectedGroups)); + } + + public void testFromStatsGroupingByKeyword() { + EsqlQueryResponse results = run("from test | stats avg(count) by color"); + logger.info(results); + Assert.assertEquals(2, results.columns().size()); + Assert.assertEquals(3, results.values().size()); + + // assert column metadata + assertEquals("avg(count)", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals("color", results.columns().get(1).name()); + assertEquals("keyword", results.columns().get(1).type()); + record Group(String color, double avg) { + + } + List expectedGroups = List.of(new Group("blue", 42.0), new Group("green", 44.0), new Group("red", 43)); + List actualGroups = results.values() + .stream() + .map(l -> new Group((String) l.get(1), (Double) l.get(0))) + .sorted(comparing(c -> c.color)) + .toList(); + assertThat(actualGroups, equalTo(expectedGroups)); + } + + public void testFromStatsGroupingByKeywordWithNulls() { + for (int i = 0; i < 5; i++) { + client().prepareBulk() + .add(new IndexRequest("test").id("no_color_" + i).source("data", 12, "count", 120, "data_d", 2d, "count_d", 120d)) + .add(new IndexRequest("test").id("no_count_red_" + i).source("data", 2, "data_d", 2d, "color", "red")) + .add(new IndexRequest("test").id("no_count_yellow_" + i).source("data", 2, "data_d", 2d, "color", "yellow")) + .get(); + if (randomBoolean()) { + client().admin().indices().prepareRefresh("test").get(); + } + } + client().admin().indices().prepareRefresh("test").get(); + for (String field : List.of("count", "count_d")) { + EsqlQueryResponse results = run("from test | stats avg = avg(" + field + ") by color"); + logger.info(results); + Assert.assertEquals(2, results.columns().size()); + Assert.assertEquals(4, results.values().size()); + + // assert column metadata + assertEquals("avg", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals("color", results.columns().get(1).name()); + assertEquals("keyword", results.columns().get(1).type()); + record Group(String color, Double avg) { + + } + List expectedGroups = List.of( + new Group("blue", 42.0), + new Group("green", 44.0), + new Group("red", 43.0), + new Group("yellow", null) + ); + List actualGroups = results.values() + .stream() + .map(l -> new Group((String) l.get(1), (Double) l.get(0))) + .sorted(comparing(c -> c.color)) + .toList(); + assertThat(actualGroups, equalTo(expectedGroups)); + } + for (int i = 0; i < 5; i++) { + client().prepareBulk() + .add(new DeleteRequest("test").id("no_color_" + i)) + .add(new DeleteRequest("test").id("no_count_red_" + i)) + .add(new DeleteRequest("test").id("no_count_yellow_" + i)) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + } + } + + public void testFromStatsMultipleAggs() { + EsqlQueryResponse results = run( + "from test | stats a=avg(count), mi=min(count), ma=max(count), s=sum(count), c=count(count) by color" + ); + logger.info(results); + Assert.assertEquals(6, results.columns().size()); + Assert.assertEquals(3, results.values().size()); + + // assert column metadata + assertEquals("a", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals("mi", results.columns().get(1).name()); + assertEquals("long", results.columns().get(1).type()); + assertEquals("ma", results.columns().get(2).name()); + assertEquals("long", results.columns().get(2).type()); + assertEquals("s", results.columns().get(3).name()); + assertEquals("long", results.columns().get(3).type()); + assertEquals("c", results.columns().get(4).name()); + assertEquals("long", results.columns().get(4).type()); + assertEquals("color", results.columns().get(5).name()); + assertEquals("keyword", results.columns().get(5).type()); + record Group(double avg, long mi, long ma, long s, long c, String color) {} + List expectedGroups = List.of( + new Group(42, 42, 42, 420, 10, "blue"), + new Group(44, 44, 44, 440, 10, "green"), + new Group(43, 40, 46, 860, 20, "red") + ); + // TODO: each aggregator returns Double now, it should in fact mirror the data type of the fields it's aggregating + List actualGroups = results.values() + .stream() + .map(l -> new Group((Double) l.get(0), (Long) l.get(1), (Long) l.get(2), (Long) l.get(3), (Long) l.get(4), (String) l.get(5))) + .sorted(comparing(c -> c.color)) + .toList(); + assertThat(actualGroups, equalTo(expectedGroups)); + } + + public void testFromSortWithTieBreakerLimit() { + EsqlQueryResponse results = run("from test | sort data, count desc, time | limit 5 | keep data, count, time"); + logger.info(results); + assertThat( + results.values(), + contains( + List.of(1L, 44L, epoch + 2), + List.of(1L, 44L, epoch + 6), + List.of(1L, 44L, epoch + 10), + List.of(1L, 44L, epoch + 14), + List.of(1L, 44L, epoch + 18) + ) + ); + } + + public void testFromStatsProjectGroup() { + EsqlQueryResponse results = run("from test | stats avg_count = avg(count) by data | keep data"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("data")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("long")); + assertThat(results.values(), containsInAnyOrder(List.of(1L), List.of(2L))); + } + + public void testRowStatsProjectGroupByInt() { + EsqlQueryResponse results = run("row a = 1, b = 2 | stats count(b) by a | keep a"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("a")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("integer")); + assertThat(results.values(), contains(List.of(1))); + } + + public void testRowStatsProjectGroupByLong() { + EsqlQueryResponse results = run("row a = 1000000000000, b = 2 | stats count(b) by a | keep a"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("a")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("long")); + assertThat(results.values(), contains(List.of(1000000000000L))); + } + + public void testRowStatsProjectGroupByDouble() { + EsqlQueryResponse results = run("row a = 1.0, b = 2 | stats count(b) by a | keep a"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("a")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double")); + assertThat(results.values(), contains(List.of(1.0))); + } + + public void testRowStatsProjectGroupByKeyword() { + EsqlQueryResponse results = run("row a = \"hello\", b = 2 | stats count(b) by a | keep a"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("a")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("keyword")); + assertThat(results.values(), contains(List.of("hello"))); + } + + public void testFromStatsProjectGroupByDouble() { + EsqlQueryResponse results = run("from test | stats count(count) by data_d | keep data_d"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("data_d")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double")); + assertThat(results.values(), containsInAnyOrder(List.of(1.0), List.of(2.0))); + } + + public void testFromStatsProjectGroupWithAlias() { + String query = "from test | stats avg_count = avg(count) by data | eval d2 = data | rename data as d | keep d, d2"; + EsqlQueryResponse results = run(query); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("d", "d2")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("long", "long")); + assertThat(results.values(), containsInAnyOrder(List.of(1L, 1L), List.of(2L, 2L))); + } + + public void testFromStatsProjectAgg() { + EsqlQueryResponse results = run("from test | stats a = avg(count) by data | keep a"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("a")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double")); + assertThat(results.values(), containsInAnyOrder(List.of(42d), List.of(44d))); + } + + public void testFromStatsProjectAggWithAlias() { + EsqlQueryResponse results = run("from test | stats a = avg(count) by data | rename a as b | keep b"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("b")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double")); + assertThat(results.values(), containsInAnyOrder(List.of(42d), List.of(44d))); + } + + public void testFromProjectStatsGroupByAlias() { + EsqlQueryResponse results = run("from test | rename data as d | keep d, count | stats avg(count) by d"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("avg(count)", "d")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double", "long")); + assertThat(results.values(), containsInAnyOrder(List.of(42d, 1L), List.of(44d, 2L))); + } + + public void testFromProjectStatsAggregateAlias() { + EsqlQueryResponse results = run("from test | rename count as c | keep c, data | stats avg(c) by data"); + logger.info(results); + assertThat(results.columns().stream().map(ColumnInfo::name).toList(), contains("avg(c)", "data")); + assertThat(results.columns().stream().map(ColumnInfo::type).toList(), contains("double", "long")); + assertThat(results.values(), containsInAnyOrder(List.of(42d, 1L), List.of(44d, 2L))); + } + + public void testFromEvalStats() { + EsqlQueryResponse results = run("from test | eval ratio = data_d / count_d | stats avg(ratio)"); + logger.info(results); + Assert.assertEquals(1, results.columns().size()); + Assert.assertEquals(1, results.values().size()); + assertEquals("avg(ratio)", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals(1, results.values().get(0).size()); + assertEquals(0.034d, (double) results.values().get(0).get(0), 0.001d); + } + + public void testFromStatsEvalWithPragma() { + assumeTrue("pragmas only enabled on snapshot builds", Build.current().isSnapshot()); + EsqlQueryResponse results = run("from test | stats avg_count = avg(count) | eval x = avg_count + 7"); + logger.info(results); + Assert.assertEquals(1, results.values().size()); + assertEquals(2, results.values().get(0).size()); + assertEquals(50, (double) results.values().get(0).get(results.columns().indexOf(new ColumnInfo("x", "double"))), 1d); + assertEquals(43, (double) results.values().get(0).get(results.columns().indexOf(new ColumnInfo("avg_count", "double"))), 1d); + } + + public void testWhere() { + EsqlQueryResponse results = run("from test | where count > 40"); + logger.info(results); + Assert.assertEquals(30, results.values().size()); + var countIndex = results.columns().indexOf(new ColumnInfo("count", "long")); + for (List values : results.values()) { + assertThat((Long) values.get(countIndex), greaterThan(40L)); + } + } + + public void testProjectWhere() { + EsqlQueryResponse results = run("from test | keep count | where count > 40"); + logger.info(results); + Assert.assertEquals(30, results.values().size()); + int countIndex = results.columns().indexOf(new ColumnInfo("count", "long")); + for (List values : results.values()) { + assertThat((Long) values.get(countIndex), greaterThan(40L)); + } + } + + public void testEvalWhere() { + EsqlQueryResponse results = run("from test | eval x = count / 2 | where x > 20"); + logger.info(results); + Assert.assertEquals(30, results.values().size()); + int countIndex = results.columns().indexOf(new ColumnInfo("x", "long")); + for (List values : results.values()) { + assertThat((Long) values.get(countIndex), greaterThan(20L)); + } + } + + public void testFilterWithNullAndEval() { + EsqlQueryResponse results = run("row a = 1 | eval b = a + null | where b > 1"); + logger.info(results); + Assert.assertEquals(0, results.values().size()); + } + + public void testStringLength() { + EsqlQueryResponse results = run("from test | eval l = length(color)"); + logger.info(results); + assertThat(results.values(), hasSize(40)); + int countIndex = results.columns().indexOf(new ColumnInfo("l", "integer")); + for (List values : results.values()) { + assertThat((Integer) values.get(countIndex), greaterThanOrEqualTo(3)); + } + } + + public void testFilterWithNullAndEvalFromIndex() { + // append entry, with an absent count, to the index + client().prepareBulk().add(new IndexRequest("test").id("no_count").source("data", 12, "data_d", 2d, "color", "red")).get(); + + client().admin().indices().prepareRefresh("test").get(); + // sanity + EsqlQueryResponse results = run("from test"); + Assert.assertEquals(41, results.values().size()); + + results = run("from test | eval newCount = count + 1 | where newCount > 1"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + assertThat(results.columns(), hasItem(equalTo(new ColumnInfo("count", "long")))); + assertThat(results.columns(), hasItem(equalTo(new ColumnInfo("count_d", "double")))); + assertThat(results.columns(), hasItem(equalTo(new ColumnInfo("data", "long")))); + assertThat(results.columns(), hasItem(equalTo(new ColumnInfo("data_d", "double")))); + assertThat(results.columns(), hasItem(equalTo(new ColumnInfo("time", "long")))); + + // restore index to original pre-test state + client().prepareBulk().add(new DeleteRequest("test").id("no_count")).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); + results = run("from test"); + Assert.assertEquals(40, results.values().size()); + } + + public void testMultiConditionalWhere() { + EsqlQueryResponse results = run( + "from test | eval abc = 1+2 | where (abc + count >= 44 or data_d == 2) and data == 1 | keep color, abc" + ); + logger.info(results); + Assert.assertEquals(10, results.values().size()); + Assert.assertEquals(2, results.columns().size()); + for (List values : results.values()) { + assertThat((String) values.get(0), equalTo("green")); + assertThat((Integer) values.get(1), equalTo(3)); + } + } + + public void testWhereNegatedCondition() { + EsqlQueryResponse results = run("from test | eval abc=1+2 | where abc + count > 45 and data != 1 | keep color, data"); + logger.info(results); + Assert.assertEquals(10, results.values().size()); + Assert.assertEquals(2, results.columns().size()); + for (List values : results.values()) { + assertThat((String) values.get(0), equalTo("red")); + assertThat((Long) values.get(1), equalTo(2L)); + } + } + + public void testEvalOverride() { + EsqlQueryResponse results = run("from test | eval count = count + 1 | eval count = count + 1"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + Assert.assertEquals(1, results.columns().stream().filter(c -> c.name().equals("count")).count()); + int countIndex = results.columns().size() - 1; + Assert.assertEquals(new ColumnInfo("count", "long"), results.columns().get(countIndex)); + for (List values : results.values()) { + assertThat((Long) values.get(countIndex), greaterThanOrEqualTo(42L)); + } + } + + public void testProjectRename() { + EsqlQueryResponse results = run("from test | eval y = count | rename count as x | keep x, y"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + assertThat(results.columns(), contains(new ColumnInfo("x", "long"), new ColumnInfo("y", "long"))); + for (List values : results.values()) { + assertThat((Long) values.get(0), greaterThanOrEqualTo(40L)); + assertThat(values.get(1), is(values.get(0))); + } + } + + public void testProjectRenameEval() { + EsqlQueryResponse results = run("from test | eval y = count | rename count as x | keep x, y | eval x2 = x + 1 | eval y2 = y + 2"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + assertThat( + results.columns(), + contains(new ColumnInfo("x", "long"), new ColumnInfo("y", "long"), new ColumnInfo("x2", "long"), new ColumnInfo("y2", "long")) + ); + for (List values : results.values()) { + assertThat((Long) values.get(0), greaterThanOrEqualTo(40L)); + assertThat(values.get(1), is(values.get(0))); + assertThat(values.get(2), is(((Long) values.get(0)) + 1)); + assertThat(values.get(3), is(((Long) values.get(0)) + 2)); + } + } + + public void testProjectRenameEvalProject() { + EsqlQueryResponse results = run("from test | eval y = count | rename count as x | keep x, y | eval z = x + y | keep x, y, z"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + assertThat(results.columns(), contains(new ColumnInfo("x", "long"), new ColumnInfo("y", "long"), new ColumnInfo("z", "long"))); + for (List values : results.values()) { + assertThat((Long) values.get(0), greaterThanOrEqualTo(40L)); + assertThat(values.get(1), is(values.get(0))); + assertThat(values.get(2), is((Long) values.get(0) * 2)); + } + } + + public void testProjectOverride() { + EsqlQueryResponse results = run("from test | eval cnt = count | rename count as data | keep cnt, data"); + logger.info(results); + Assert.assertEquals(40, results.values().size()); + assertThat(results.columns(), contains(new ColumnInfo("cnt", "long"), new ColumnInfo("data", "long"))); + for (List values : results.values()) { + assertThat(values.get(1), is(values.get(0))); + } + } + + public void testRefreshSearchIdleShards() throws Exception { + String indexName = "test_refresh"; + int numShards = between(1, 2); + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings( + Settings.builder() + .put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShards) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.rebalance.enable", "none") + ) + .get() + ); + ensureYellow(indexName); + AtomicLong totalValues = new AtomicLong(); + CountDownLatch latch = new CountDownLatch(1); + AtomicBoolean stopped = new AtomicBoolean(); + Thread indexingThread = new Thread(() -> { + try { + assertTrue(latch.await(30, TimeUnit.SECONDS)); + } catch (Exception e) { + throw new AssertionError(e); + } + int numDocs = randomIntBetween(10, 20); + while (stopped.get() == false) { + if (rarely()) { + numDocs++; + } + logger.info("--> indexing {} docs", numDocs); + long sum = 0; + for (int i = 0; i < numDocs; i++) { + long value = randomLongBetween(1, 1000); + client().prepareBulk().add(new IndexRequest(indexName).id("doc-" + i).source("data", 1, "value", value)).get(); + sum += value; + } + totalValues.set(sum); + } + }); + indexingThread.start(); + try { + logger.info("--> waiting for shards to have pending refresh"); + Index index = resolveIndex(indexName); + latch.countDown(); + assertBusy(() -> { + int pendingRefreshes = 0; + for (IndicesService indicesService : internalCluster().getInstances(IndicesService.class)) { + IndexService indexService = indicesService.indexService(index); + if (indexService != null) { + for (IndexShard shard : indexService) { + if (shard.hasRefreshPending()) { + pendingRefreshes++; + } + } + } + } + assertThat("shards don't have any pending refresh", pendingRefreshes, equalTo(numShards)); + }, 30, TimeUnit.SECONDS); + } finally { + stopped.set(true); + indexingThread.join(); + } + EsqlQueryResponse results = run("from test_refresh | stats s = sum(value)"); + logger.info(results); + assertThat(results.values().get(0), equalTo(List.of(totalValues.get()))); + } + + public void testESFilter() throws Exception { + String indexName = "test_filter"; + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))) + .get() + ); + ensureYellow(indexName); + int numDocs = randomIntBetween(1, 5000); + Map docs = new HashMap<>(); + List indexRequests = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + String id = "id-" + i; + long value = randomLongBetween(-100_000, 100_000); + docs.put(id, value); + indexRequests.add(client().prepareIndex().setIndex(indexName).setId(id).setSource(Map.of("val", value))); + } + indexRandom(true, randomBoolean(), indexRequests); + String command = "from test_filter | stats avg = avg(val)"; + long from = randomBoolean() ? Long.MIN_VALUE : randomLongBetween(-1000, 1000); + long to = randomBoolean() ? Long.MAX_VALUE : randomLongBetween(from, from + 1000); + QueryBuilder filter = new RangeQueryBuilder("val").from(from, true).to(to, true); + EsqlQueryResponse results = new EsqlQueryRequestBuilder(client(), EsqlQueryAction.INSTANCE).query(command) + .filter(filter) + .pragmas(randomPragmas()) + .get(); + logger.info(results); + OptionalDouble avg = docs.values().stream().filter(v -> from <= v && v <= to).mapToLong(n -> n).average(); + if (avg.isPresent()) { + assertEquals(avg.getAsDouble(), (double) results.values().get(0).get(0), 0.01d); + } else { + assertThat(results.values().get(0).get(0), nullValue()); + } + } + + public void testExtractFields() throws Exception { + String indexName = "test_extract_fields"; + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))) + .setMapping("val", "type=long", "tag", "type=keyword") + .get() + ); + int numDocs = randomIntBetween(1, 100); + List indexRequests = new ArrayList<>(); + record Doc(long val, String tag) { + + } + List allDocs = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + Doc d = new Doc(i, "tag-" + randomIntBetween(1, 100)); + allDocs.add(d); + indexRequests.add( + client().prepareIndex().setIndex(indexName).setId(Integer.toString(i)).setSource(Map.of("val", d.val, "tag", d.tag)) + ); + } + indexRandom(true, randomBoolean(), indexRequests); + int limit = randomIntBetween(1, 10); + String command = "from test_extract_fields | sort val | limit " + limit; + EsqlQueryResponse results = run(command); + logger.info(results); + // _doc, _segment, _shard are pruned + assertThat(results.columns().size(), equalTo(2)); + assertThat(results.values(), hasSize(Math.min(limit, numDocs))); + assertThat(results.columns().get(1).name(), equalTo("val")); + assertThat(results.columns().get(0).name(), equalTo("tag")); + List actualDocs = new ArrayList<>(); + for (int i = 0; i < results.values().size(); i++) { + List values = results.values().get(i); + actualDocs.add(new Doc((Long) values.get(1), (String) values.get(0))); + } + assertThat(actualDocs, equalTo(allDocs.stream().limit(limit).toList())); + } + + public void testEvalWithNullAndAvg() { + EsqlQueryResponse results = run("from test | eval nullsum = count_d + null | stats avg(nullsum)"); + logger.info(results); + Assert.assertEquals(1, results.columns().size()); + Assert.assertEquals(1, results.values().size()); + assertEquals("avg(nullsum)", results.columns().get(0).name()); + assertEquals("double", results.columns().get(0).type()); + assertEquals(1, results.values().get(0).size()); + assertNull(results.values().get(0).get(0)); + } + + public void testFromStatsLimit() { + EsqlQueryResponse results = run("from test | stats ac = avg(count) by data | limit 1"); + logger.info(results); + assertThat(results.columns(), contains(new ColumnInfo("ac", "double"), new ColumnInfo("data", "long"))); + assertThat(results.values(), contains(anyOf(contains(42.0, 1L), contains(44.0, 2L)))); + } + + public void testFromLimit() { + EsqlQueryResponse results = run("from test | keep data | limit 2"); + logger.info(results); + assertThat(results.columns(), contains(new ColumnInfo("data", "long"))); + assertThat(results.values(), contains(anyOf(contains(1L), contains(2L)), anyOf(contains(1L), contains(2L)))); + } + + public void testDropAllColumns() { + EsqlQueryResponse results = run("from test | keep data | drop data | eval a = 1"); + logger.info(results); + assertThat(results.columns(), hasSize(1)); + assertThat(results.columns(), contains(new ColumnInfo("a", "integer"))); + assertThat(results.values(), is(empty())); + } + + public void testDropAllColumnsWithStats() { + EsqlQueryResponse results = run("from test | stats g = count(data) | drop g"); + logger.info(results); + assertThat(results.columns(), is(empty())); + assertThat(results.values(), is(empty())); + } + + public void testIndexPatterns() throws Exception { + String[] indexNames = { "test_index_patterns_1", "test_index_patterns_2", "test_index_patterns_3" }; + int i = 0; + for (String indexName : indexNames) { + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))) + .setMapping("data", "type=long", "count", "type=long") + .get() + ); + ensureYellow(indexName); + client().prepareBulk() + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest(indexName).id("1").source("data", ++i, "count", i * 1000)) + .add(new IndexRequest(indexName).id("2").source("data", ++i, "count", i * 1000)) + .add(new IndexRequest(indexName).id("3").source("data", ++i, "count", i * 1000)) + .add(new IndexRequest(indexName).id("4").source("data", ++i, "count", i * 1000)) + .add(new IndexRequest(indexName).id("5").source("data", ++i, "count", i * 1000)) + .get(); + } + + EsqlQueryResponse results = run("from test_index_patterns* | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(15L, results.values().get(0).get(0)); + assertEquals(120000L, results.values().get(0).get(1)); + + results = run("from test_index_patterns_1,test_index_patterns_2 | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(10L, results.values().get(0).get(0)); + assertEquals(55000L, results.values().get(0).get(1)); + + results = run("from test_index_patterns_1*,test_index_patterns_2* | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(10L, results.values().get(0).get(0)); + assertEquals(55000L, results.values().get(0).get(1)); + + results = run("from test_index_patterns_*,-test_index_patterns_1 | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(10L, results.values().get(0).get(0)); + assertEquals(105000L, results.values().get(0).get(1)); + + results = run("from * | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(55L, results.values().get(0).get(0)); + assertEquals(121720L, results.values().get(0).get(1)); + + results = run("from test_index_patterns_2 | stats count(data), sum(count)"); + assertEquals(1, results.values().size()); + assertEquals(5L, results.values().get(0).get(0)); + assertEquals(40000L, results.values().get(0).get(1)); + } + + public void testOverlappingIndexPatterns() throws Exception { + String[] indexNames = { "test_overlapping_index_patterns_1", "test_overlapping_index_patterns_2" }; + + assertAcked( + client().admin() + .indices() + .prepareCreate("test_overlapping_index_patterns_1") + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))) + .setMapping("field", "type=long") + .get() + ); + ensureYellow("test_overlapping_index_patterns_1"); + client().prepareBulk() + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest("test_overlapping_index_patterns_1").id("1").source("field", 10)) + .get(); + + assertAcked( + client().admin() + .indices() + .prepareCreate("test_overlapping_index_patterns_2") + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, between(1, 5))) + .setMapping("field", "type=keyword") + .get() + ); + ensureYellow("test_overlapping_index_patterns_2"); + client().prepareBulk() + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest("test_overlapping_index_patterns_2").id("1").source("field", "foo")) + .get(); + + expectThrows(VerificationException.class, () -> run("from test_overlapping_index_patterns_* | sort field")); + } + + public void testEmptyIndex() { + assertAcked(client().admin().indices().prepareCreate("test_empty").setMapping("k", "type=keyword", "v", "type=long").get()); + EsqlQueryResponse results = run("from test_empty"); + assertThat(results.columns(), equalTo(List.of(new ColumnInfo("k", "keyword"), new ColumnInfo("v", "long")))); + assertThat(results.values(), empty()); + } + + public void testShowInfo() { + EsqlQueryResponse results = run("show info"); + assertThat( + results.columns(), + equalTo(List.of(new ColumnInfo("version", "keyword"), new ColumnInfo("date", "keyword"), new ColumnInfo("hash", "keyword"))) + ); + assertThat(results.values().size(), equalTo(1)); + assertThat(results.values().get(0).get(0), equalTo(Build.current().version())); + assertThat(results.values().get(0).get(1), equalTo(Build.current().date())); + assertThat(results.values().get(0).get(2), equalTo(Build.current().hash())); + } + + public void testShowFunctions() { + EsqlQueryResponse results = run("show functions"); + assertThat(results.columns(), equalTo(List.of(new ColumnInfo("name", "keyword"), new ColumnInfo("synopsis", "keyword")))); + assertThat(results.values().size(), equalTo(new EsqlFunctionRegistry().listFunctions().size())); + } + + public void testInWithNullValue() { + EsqlQueryResponse results = run("from test | where null in (data, 2) | keep data"); + assertThat(results.columns(), equalTo(List.of(new ColumnInfo("data", "long")))); + assertThat(results.values().size(), equalTo(0)); + } + + public void testTopNPushedToLucene() { + BulkRequestBuilder bulkDelete = client().prepareBulk(); + bulkDelete.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + + for (int i = 5; i < 11; i++) { + var yellowDocId = "yellow_" + i; + var yellowNullCountDocId = "yellow_null_count_" + i; + var yellowNullDataDocId = "yellow_null_data_" + i; + + client().prepareBulk() + .add(new IndexRequest("test").id(yellowDocId).source("data", i, "count", i * 10, "color", "yellow")) + .add(new IndexRequest("test").id(yellowNullCountDocId).source("data", i, "color", "yellow")) + .add(new IndexRequest("test").id(yellowNullDataDocId).source("count", i * 10, "color", "yellow")) + .get(); + if (randomBoolean()) { + client().admin().indices().prepareRefresh("test").get(); + } + + // build the cleanup request now, as well, not to miss anything ;-) + bulkDelete.add(new DeleteRequest("test").id(yellowDocId)) + .add(new DeleteRequest("test").id(yellowNullCountDocId)) + .add(new DeleteRequest("test").id(yellowNullDataDocId)); + } + client().admin().indices().prepareRefresh("test").get(); + + EsqlQueryResponse results = run(""" + from test + | where color == "yellow" + | sort data desc nulls first, count asc nulls first + | limit 10 + | keep data, count, color + """); + logger.info(results); + Assert.assertEquals(3, results.columns().size()); + Assert.assertEquals(10, results.values().size()); + + // assert column metadata + assertEquals("data", results.columns().get(0).name()); + assertEquals("long", results.columns().get(0).type()); + assertEquals("count", results.columns().get(1).name()); + assertEquals("long", results.columns().get(1).type()); + assertEquals("color", results.columns().get(2).name()); + assertEquals("keyword", results.columns().get(2).type()); + record Group(Long data, Long count, String color) { + Group(Long data, Long count) { + this(data, count, "yellow"); + } + } + List expectedGroups = List.of( + // data sorted descending nulls first; count sorted ascending nulls first + new Group(null, 50L), + new Group(null, 60L), + new Group(null, 70L), + new Group(null, 80L), + new Group(null, 90L), + new Group(null, 100L), + new Group(10L, null), + new Group(10L, 100L), + new Group(9L, null), + new Group(9L, 90L) + ); + List actualGroups = results.values() + .stream() + .map(l -> new Group((Long) l.get(0), (Long) l.get(1), (String) l.get(2))) + .toList(); + assertThat(actualGroups, equalTo(expectedGroups)); + + // clean-up what we created + bulkDelete.get(); + } + + /** + * This test covers the scenarios where Lucene is throwing a {@link org.apache.lucene.search.CollectionTerminatedException} when + * it's signaling that it could stop collecting hits early. For example, in the case the index is sorted in the same order as the query. + * The {@link org.elasticsearch.compute.lucene.LuceneTopNSourceOperator#getOutput()} is handling this exception by + * ignoring it (which is the right thing to do) and sort of cleaning up and moving to the next docs collection. + */ + public void testTopNPushedToLuceneOnSortedIndex() { + var sortOrder = randomFrom("asc", "desc"); + createAndPopulateIndex( + "sorted_test_index", + Settings.builder().put("index.sort.field", "time").put("index.sort.order", sortOrder).build() + ); + + int limit = randomIntBetween(1, 5); + EsqlQueryResponse results = run("from sorted_test_index | sort time " + sortOrder + " | limit " + limit + " | keep time"); + logger.info(results); + Assert.assertEquals(1, results.columns().size()); + Assert.assertEquals(limit, results.values().size()); + + // assert column metadata + assertEquals("time", results.columns().get(0).name()); + assertEquals("long", results.columns().get(0).type()); + + boolean sortedDesc = "desc".equals(sortOrder); + var expected = LongStream.range(0, 40) + .map(i -> epoch + i) + .boxed() + .sorted(sortedDesc ? reverseOrder() : naturalOrder()) + .limit(limit) + .toList(); + var actual = results.values().stream().map(l -> (Long) l.get(0)).toList(); + assertThat(actual, equalTo(expected)); + + // clean-up + client().admin().indices().delete(new DeleteIndexRequest("sorted_test_index")).actionGet(); + } + + /* + * Create two indices that both have nested documents in them. Create an alias pointing to the two indices. + * Query an individual index, then query the alias checking that no nested documents are returned. + */ + public void testReturnNoNestedDocuments() throws IOException, ExecutionException, InterruptedException { + var indexName1 = "test_nested_docs_1"; + var indexName2 = "test_nested_docs_2"; + var indices = List.of(indexName1, indexName2); + var alias = "test-alias"; + int docsCount = randomIntBetween(50, 100); + int[] countValuesGreaterThanFifty = new int[indices.size()]; + + for (int i = 0; i < indices.size(); i++) { + String indexName = indices.get(i); + createNestedMappingIndex(indexName); + countValuesGreaterThanFifty[i] = indexDocsIntoNestedMappingIndex(indexName, docsCount); + } + createAlias(indices, alias); + + var indexToTest = randomIntBetween(0, indices.size() - 1); + var indexNameToTest = indices.get(indexToTest); + // simple query + assertNoNestedDocuments("from " + indexNameToTest, docsCount, 0L, 100L); + // simple query with filter that gets pushed to ES + assertNoNestedDocuments("from " + indexNameToTest + " | where data >= 50", countValuesGreaterThanFifty[indexToTest], 50L, 100L); + // simple query against alias + assertNoNestedDocuments("from " + alias, docsCount * 2, 0L, 100L); + // simple query against alias with filter that gets pushed to ES + assertNoNestedDocuments("from " + alias + " | where data >= 50", Arrays.stream(countValuesGreaterThanFifty).sum(), 50L, 100L); + } + + private void createNestedMappingIndex(String indexName) throws IOException { + XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + { + builder.startObject("properties"); + { + builder.startObject("nested"); + { + builder.field("type", "nested"); + builder.startObject("properties"); + { + builder.startObject("foo"); + builder.field("type", "long"); + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + builder.startObject("data"); + builder.field("type", "long"); + builder.endObject(); + } + builder.endObject(); + } + builder.endObject(); + + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put("index.number_of_shards", ESTestCase.randomIntBetween(1, 3))) + .setMapping(builder) + .get() + ); + } + + private int indexDocsIntoNestedMappingIndex(String indexName, int docsCount) throws IOException { + int countValuesGreaterThanFifty = 0; + BulkRequestBuilder bulkBuilder = client().prepareBulk(); + for (int j = 0; j < docsCount; j++) { + XContentBuilder builder = JsonXContent.contentBuilder(); + int randomValue = randomIntBetween(0, 100); + countValuesGreaterThanFifty += randomValue >= 50 ? 1 : 0; + builder.startObject(); + { + builder.field("data", randomValue); + builder.startArray("nested"); + { + for (int k = 0, max = randomIntBetween(1, 5); k < max; k++) { + // nested values are all greater than any non-nested values found in the "data" long field + builder.startObject().field("foo", randomIntBetween(1000, 10000)).endObject(); + } + } + builder.endArray(); + } + builder.endObject(); + bulkBuilder.add(new IndexRequest(indexName).id(Integer.toString(j)).source(builder)); + } + bulkBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); + ensureYellow(indexName); + + return countValuesGreaterThanFifty; + } + + private void createAlias(List indices, String alias) throws InterruptedException, ExecutionException { + IndicesAliasesRequest aliasesRequest = new IndicesAliasesRequest(); + for (String indexName : indices) { + aliasesRequest.addAliasAction(IndicesAliasesRequest.AliasActions.add().index(indexName).alias(alias)); + } + assertAcked(admin().indices().aliases(aliasesRequest).get()); + } + + private void assertNoNestedDocuments(String query, int docsCount, long minValue, long maxValue) { + EsqlQueryResponse results = run(query); + assertThat(results.columns(), contains(new ColumnInfo("data", "long"))); + assertThat(results.columns().size(), is(1)); + assertThat(results.values().size(), is(docsCount)); + for (List row : results.values()) { + assertThat(row.size(), is(1)); + // check that all the values returned are the regular ones + assertThat((Long) row.get(0), allOf(greaterThanOrEqualTo(minValue), lessThanOrEqualTo(maxValue))); + } + } + + private void createAndPopulateIndex(String indexName) { + createAndPopulateIndex(indexName, Settings.EMPTY); + } + + private void createAndPopulateIndex(String indexName, Settings additionalSettings) { + assertAcked( + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put(additionalSettings).put("index.number_of_shards", ESTestCase.randomIntBetween(1, 5))) + .setMapping( + "data", + "type=long", + "data_d", + "type=double", + "count", + "type=long", + "count_d", + "type=double", + "time", + "type=long", + "color", + "type=keyword" + ) + .get() + ); + long timestamp = epoch; + for (int i = 0; i < 10; i++) { + client().prepareBulk() + .add( + new IndexRequest(indexName).id("1" + i) + .source("data", 1, "count", 40, "data_d", 1d, "count_d", 40d, "time", timestamp++, "color", "red") + ) + .add( + new IndexRequest(indexName).id("2" + i) + .source("data", 2, "count", 42, "data_d", 2d, "count_d", 42d, "time", timestamp++, "color", "blue") + ) + .add( + new IndexRequest(indexName).id("3" + i) + .source("data", 1, "count", 44, "data_d", 1d, "count_d", 44d, "time", timestamp++, "color", "green") + ) + .add( + new IndexRequest(indexName).id("4" + i) + .source("data", 2, "count", 46, "data_d", 2d, "count_d", 46d, "time", timestamp++, "color", "red") + ) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + } + ensureYellow(indexName); + } +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java new file mode 100644 index 0000000000000..6b271debcf2c3 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java @@ -0,0 +1,240 @@ +/* + * 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.action; + +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.index.mapper.OnScriptError; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.script.BooleanFieldScript; +import org.elasticsearch.script.DateFieldScript; +import org.elasticsearch.script.DoubleFieldScript; +import org.elasticsearch.script.LongFieldScript; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptEngine; +import org.elasticsearch.script.StringFieldScript; +import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE; +import static org.hamcrest.Matchers.equalTo; + +/** + * Makes sure that the circuit breaker is "plugged in" to ESQL by configuring an + * unreasonably small breaker and tripping it. + */ +@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) +// @TestLogging(value = "org.elasticsearch.xpack.esql:TRACE", reason = "debug") +public class EsqlActionRuntimeFieldIT extends AbstractEsqlIntegTestCase { + private final int SIZE = between(10, 100); + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), TestRuntimeFieldPlugin.class); + } + + public void testLong() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("long"); + EsqlQueryResponse response = run("from test | stats sum(const)"); + assertThat(response.values(), equalTo(List.of(List.of((long) SIZE)))); + } + + public void testDouble() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("double"); + EsqlQueryResponse response = run("from test | stats sum(const)"); + assertThat(response.values(), equalTo(List.of(List.of((double) SIZE)))); + } + + public void testKeyword() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("keyword"); + EsqlQueryResponse response = run("from test | keep const | limit 1"); + assertThat(response.values(), equalTo(List.of(List.of("const")))); + } + + /** + * Test grouping by runtime keyword which requires disabling the ordinals + * optimization available to more keyword fields. + */ + public void testKeywordBy() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("keyword"); + EsqlQueryResponse response = run("from test | stats max(foo) by const"); + assertThat(response.values(), equalTo(List.of(List.of(SIZE - 1L, "const")))); + } + + public void testBoolean() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("boolean"); + EsqlQueryResponse response = run("from test | sort foo | limit 3"); + assertThat(response.values(), equalTo(List.of(List.of(true, 0L), List.of(true, 1L), List.of(true, 2L)))); + } + + public void testDate() throws InterruptedException, IOException { + createIndexWithConstRuntimeField("date"); + EsqlQueryResponse response = run(""" + from test | eval d=date_format(const, "yyyy") | stats min (foo) by d"""); + assertThat(response.values(), equalTo(List.of(List.of(0L, "2023")))); + } + + private void createIndexWithConstRuntimeField(String type) throws InterruptedException, IOException { + XContentBuilder mapping = JsonXContent.contentBuilder().startObject(); + mapping.startObject("runtime"); + { + mapping.startObject("const"); + { + mapping.field("type", type); + mapping.startObject("script").field("source", "").field("lang", "dummy").endObject(); + } + mapping.endObject(); + } + mapping.endObject(); + client().admin().indices().prepareCreate("test").setMapping(mapping.endObject()).get(); + + BulkRequestBuilder bulk = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int i = 0; i < SIZE; i++) { + bulk.add(client().prepareIndex("test").setId(Integer.toString(i)).setSource("foo", i)); + } + bulk.get(); + } + + public static class TestRuntimeFieldPlugin extends Plugin implements ScriptPlugin { + @Override + public ScriptEngine getScriptEngine(Settings settings, Collection> contexts) { + return new ScriptEngine() { + @Override + public String getType() { + return "dummy"; + } + + @Override + @SuppressWarnings("unchecked") + public FactoryType compile( + String name, + String code, + ScriptContext context, + Map params + ) { + if (context == LongFieldScript.CONTEXT) { + return (FactoryType) new LongFieldScript.Factory() { + @Override + public LongFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + OnScriptError onScriptError + ) { + return ctx -> new LongFieldScript(fieldName, params, searchLookup, onScriptError, ctx) { + @Override + public void execute() { + emit(1); + } + }; + } + }; + } + if (context == DoubleFieldScript.CONTEXT) { + return (FactoryType) new DoubleFieldScript.Factory() { + @Override + public DoubleFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + OnScriptError onScriptError + ) { + return ctx -> new DoubleFieldScript(fieldName, params, searchLookup, onScriptError, ctx) { + @Override + public void execute() { + emit(1.0); + } + }; + } + }; + } + if (context == StringFieldScript.CONTEXT) { + return (FactoryType) new StringFieldScript.Factory() { + @Override + public StringFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + OnScriptError onScriptError + ) { + return ctx -> new StringFieldScript(fieldName, params, searchLookup, onScriptError, ctx) { + @Override + public void execute() { + emit("const"); + } + }; + } + }; + } + if (context == BooleanFieldScript.CONTEXT) { + return (FactoryType) new BooleanFieldScript.Factory() { + @Override + public BooleanFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + OnScriptError onScriptError + ) { + return ctx -> new BooleanFieldScript(fieldName, params, searchLookup, onScriptError, ctx) { + @Override + public void execute() { + emit(true); + } + }; + } + }; + } + if (context == DateFieldScript.CONTEXT) { + return (FactoryType) new DateFieldScript.Factory() { + @Override + public DateFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + DateFormatter dateFormatter, + OnScriptError onScriptError + ) { + return ctx -> new DateFieldScript(fieldName, params, searchLookup, dateFormatter, onScriptError, ctx) { + @Override + public void execute() { + emit(dateFormatter.parseMillis("2023-01-01T00:00:00Z")); + } + }; + } + }; + } + throw new IllegalArgumentException("unsupported context " + context); + } + + @Override + public Set> getSupportedContexts() { + return Set.of( + LongFieldScript.CONTEXT, + DoubleFieldScript.CONTEXT, + StringFieldScript.CONTEXT, + BooleanFieldScript.CONTEXT, + DateFieldScript.CONTEXT + ); + } + }; + } + } + +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java new file mode 100644 index 0000000000000..500017e73e615 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java @@ -0,0 +1,328 @@ +/* + * 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.action; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.DriverStatus; +import org.elasticsearch.compute.operator.DriverTaskRunner; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkOperator; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceOperator; +import org.elasticsearch.index.mapper.OnScriptError; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.script.LongFieldScript; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptEngine; +import org.elasticsearch.search.lookup.SearchLookup; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.either; +import static org.hamcrest.Matchers.emptyIterable; +import static org.hamcrest.Matchers.emptyOrNullString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; + +/** + * Tests that we expose a reasonable task status. + */ +public class EsqlActionTaskIT extends AbstractEsqlIntegTestCase { + private static int PAGE_SIZE; + private static int NUM_DOCS; + + private static String READ_DESCRIPTION; + private static String MERGE_DESCRIPTION; + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), PausableFieldPlugin.class); + } + + @Before + public void setupIndex() throws IOException { + PAGE_SIZE = between(10, 100); + NUM_DOCS = between(4 * PAGE_SIZE, 5 * PAGE_SIZE); + READ_DESCRIPTION = """ + \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = PAGE_SIZE, limit = 2147483647] + \\_ValuesSourceReaderOperator[field = pause_me] + \\_AggregationOperator[mode = INITIAL, aggs = sum of longs] + \\_ExchangeSinkOperator""".replace("PAGE_SIZE", Integer.toString(PAGE_SIZE)); + MERGE_DESCRIPTION = """ + \\_ExchangeSourceOperator[] + \\_AggregationOperator[mode = FINAL, aggs = sum of longs] + \\_LimitOperator[limit = 10000] + \\_OutputOperator[columns = sum(pause_me)]"""; + + XContentBuilder mapping = JsonXContent.contentBuilder().startObject(); + mapping.startObject("runtime"); + { + mapping.startObject("pause_me"); + { + mapping.field("type", "long"); + mapping.startObject("script").field("source", "").field("lang", "pause").endObject(); + } + mapping.endObject(); + } + mapping.endObject(); + client().admin().indices().prepareCreate("test").setSettings(Map.of("number_of_shards", 1)).setMapping(mapping.endObject()).get(); + + BulkRequestBuilder bulk = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int i = 0; i < NUM_DOCS; i++) { + bulk.add(client().prepareIndex("test").setId(Integer.toString(i)).setSource("foo", i)); + } + bulk.get(); + } + + @AwaitsFix(bugUrl = "the task status is only updated after max_iterations") + public void testTaskContents() throws Exception { + ActionFuture response = startEsql(); + getTasksStarting(); + List foundTasks = getTasksRunning(); + int luceneSources = 0; + int valuesSourceReaders = 0; + int exchangeSources = 0; + int exchangeSinks = 0; + for (TaskInfo task : foundTasks) { + DriverStatus status = (DriverStatus) task.status(); + assertThat(status.sessionId(), not(emptyOrNullString())); + for (DriverStatus.OperatorStatus o : status.activeOperators()) { + if (o.operator().equals("LuceneSourceOperator[shardId=0, maxPageSize=" + PAGE_SIZE + "]")) { + LuceneSourceOperator.Status oStatus = (LuceneSourceOperator.Status) o.status(); + assertThat(oStatus.currentLeaf(), lessThanOrEqualTo(oStatus.totalLeaves())); + assertThat(oStatus.leafPosition(), lessThanOrEqualTo(oStatus.leafSize())); + luceneSources++; + continue; + } + if (o.operator().equals("ValuesSourceReaderOperator[field = pause_me]")) { + ValuesSourceReaderOperator.Status oStatus = (ValuesSourceReaderOperator.Status) o.status(); + assertThat(oStatus.readersBuilt(), equalTo(Map.of("LongValuesReader", 1))); + assertThat(oStatus.pagesProcessed(), greaterThanOrEqualTo(1)); + valuesSourceReaders++; + continue; + } + if (o.operator().equals("ExchangeSourceOperator")) { + ExchangeSourceOperator.Status oStatus = (ExchangeSourceOperator.Status) o.status(); + assertThat(oStatus.pagesWaiting(), greaterThanOrEqualTo(0)); + assertThat(oStatus.pagesEmitted(), greaterThanOrEqualTo(0)); + exchangeSources++; + continue; + } + if (o.operator().equals("ExchangeSinkOperator")) { + ExchangeSinkOperator.Status oStatus = (ExchangeSinkOperator.Status) o.status(); + assertThat(oStatus.pagesAccepted(), greaterThanOrEqualTo(0)); + exchangeSinks++; + } + } + } + assertThat(luceneSources, greaterThanOrEqualTo(1)); + assertThat(valuesSourceReaders, equalTo(1)); + assertThat(exchangeSinks, greaterThanOrEqualTo(1)); + assertThat(exchangeSources, equalTo(1)); + + scriptPermits.release(Integer.MAX_VALUE); + assertThat(response.get().values(), equalTo(List.of(List.of((long) NUM_DOCS)))); + } + + public void testCancelRead() throws Exception { + ActionFuture response = startEsql(); + List infos = getTasksStarting(); + TaskInfo running = infos.stream().filter(t -> t.description().equals(READ_DESCRIPTION)).findFirst().get(); + cancelTask(running.taskId()); + assertCancelled(response); + } + + public void testCancelMerge() throws Exception { + ActionFuture response = startEsql(); + List infos = getTasksStarting(); + TaskInfo running = infos.stream().filter(t -> t.description().equals(MERGE_DESCRIPTION)).findFirst().get(); + cancelTask(running.taskId()); + assertCancelled(response); + } + + public void testCancelEsqlTask() throws Exception { + ActionFuture response = startEsql(); + getTasksStarting(); + List tasks = client().admin() + .cluster() + .prepareListTasks() + .setActions(EsqlQueryAction.NAME) + .setDetailed(true) + .get() + .getTasks(); + cancelTask(tasks.get(0).taskId()); + assertCancelled(response); + } + + private ActionFuture startEsql() { + scriptPermits.drainPermits(); + scriptPermits.release(between(1, 10)); + var pragmas = new QueryPragmas(Settings.builder().put("data_partitioning", "shard").put("page_size", PAGE_SIZE).build()); + return new EsqlQueryRequestBuilder(client(), EsqlQueryAction.INSTANCE).query("from test | stats sum(pause_me)") + .pragmas(pragmas) + .execute(); + } + + private void cancelTask(TaskId taskId) { + CancelTasksRequest request = new CancelTasksRequest().setTargetTaskId(taskId).setReason("test cancel"); + request.setWaitForCompletion(false); + client().admin().cluster().execute(CancelTasksAction.INSTANCE, request).actionGet(); + scriptPermits.release(Integer.MAX_VALUE); + request = new CancelTasksRequest().setTargetTaskId(taskId).setReason("test cancel"); + request.setWaitForCompletion(true); + client().admin().cluster().execute(CancelTasksAction.INSTANCE, request).actionGet(); + } + + /** + * Fetches tasks until it finds all of them are "starting". + */ + private List getTasksStarting() throws Exception { + List foundTasks = new ArrayList<>(); + assertBusy(() -> { + List tasks = client().admin() + .cluster() + .prepareListTasks() + .setActions(DriverTaskRunner.ACTION_NAME) + .setDetailed(true) + .get() + .getTasks(); + assertThat(tasks, hasSize(equalTo(2))); + for (TaskInfo task : tasks) { + assertThat(task.action(), equalTo(DriverTaskRunner.ACTION_NAME)); + logger.info("{}", task.description()); + assertThat(task.description(), either(equalTo(READ_DESCRIPTION)).or(equalTo(MERGE_DESCRIPTION))); + DriverStatus status = (DriverStatus) task.status(); + logger.info("{}", status.status()); + assertThat(status.status(), equalTo(DriverStatus.Status.STARTING)); + } + foundTasks.addAll(tasks); + }); + return foundTasks; + } + + /** + * Fetches tasks until it finds at least one running. + */ + private List getTasksRunning() throws Exception { + List foundTasks = new ArrayList<>(); + assertBusy(() -> { + List tasks = client().admin() + .cluster() + .prepareListTasks() + .setActions(DriverTaskRunner.ACTION_NAME) + .setDetailed(true) + .get() + .getTasks(); + assertThat(tasks, hasSize(equalTo(2))); + for (TaskInfo task : tasks) { + assertThat(task.action(), equalTo(DriverTaskRunner.ACTION_NAME)); + assertThat(task.description(), either(equalTo(READ_DESCRIPTION)).or(equalTo(MERGE_DESCRIPTION))); + DriverStatus status = (DriverStatus) task.status(); + // TODO: Running is not after one iteration? + assertThat(status.status(), equalTo(DriverStatus.Status.STARTING)); + } + foundTasks.addAll(tasks); + }); + return foundTasks; + } + + private void assertCancelled(ActionFuture response) throws Exception { + Exception e = expectThrows(Exception.class, response::actionGet); + Throwable cancelException = ExceptionsHelper.unwrap(e, TaskCancelledException.class); + assertNotNull(cancelException); + assertThat(cancelException.getMessage(), equalTo("test cancel")); + assertBusy( + () -> assertThat( + client().admin() + .cluster() + .prepareListTasks() + .setActions(EsqlQueryAction.NAME, DriverTaskRunner.ACTION_NAME) + .setDetailed(true) + .get() + .getTasks(), + emptyIterable() + ) + ); + } + + private static final Semaphore scriptPermits = new Semaphore(0); + + public static class PausableFieldPlugin extends Plugin implements ScriptPlugin { + @Override + public ScriptEngine getScriptEngine(Settings settings, Collection> contexts) { + return new ScriptEngine() { + @Override + public String getType() { + return "pause"; + } + + @Override + @SuppressWarnings("unchecked") + public FactoryType compile( + String name, + String code, + ScriptContext context, + Map params + ) { + return (FactoryType) new LongFieldScript.Factory() { + @Override + public LongFieldScript.LeafFactory newFactory( + String fieldName, + Map params, + SearchLookup searchLookup, + OnScriptError onScriptError + ) { + return ctx -> new LongFieldScript(fieldName, params, searchLookup, onScriptError, ctx) { + @Override + public void execute() { + try { + assertTrue(scriptPermits.tryAcquire(1, TimeUnit.MINUTES)); + } catch (Exception e) { + throw new AssertionError(e); + } + emit(1); + } + }; + } + }; + } + + @Override + public Set> getSupportedContexts() { + return Set.of(LongFieldScript.CONTEXT); + } + }; + } + } + +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java new file mode 100644 index 0000000000000..3aaf06ead7ee5 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java @@ -0,0 +1,129 @@ +/* + * 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.action; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.cluster.coordination.Coordinator; +import org.elasticsearch.cluster.coordination.FollowersChecker; +import org.elasticsearch.cluster.coordination.LeaderChecker; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.disruption.NetworkDisruption; +import org.elasticsearch.test.disruption.ServiceDisruptionScheme; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportSettings; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; + +@ESIntegTestCase.ClusterScope(scope = TEST, minNumDataNodes = 2, maxNumDataNodes = 4) +public class EsqlDisruptionIT extends EsqlActionIT { + + // copied from AbstractDisruptionTestCase + public static final Settings DEFAULT_SETTINGS = Settings.builder() + .put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "5s") // for hitting simulated network failures quickly + .put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly + .put(FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING.getKey(), "5s") // for hitting simulated network failures quickly + .put(FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly + .put(Coordinator.PUBLISH_TIMEOUT_SETTING.getKey(), "5s") // <-- for hitting simulated network failures quickly + .put(TransportSettings.CONNECT_TIMEOUT.getKey(), "10s") // Network delay disruption waits for the min between this + // value and the time of disruption and does not recover immediately + // when disruption is stop. We should make sure we recover faster + // then the default of 30s, causing ensureGreen and friends to time out + .build(); + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(DEFAULT_SETTINGS) + .put(ExchangeService.INACTIVE_SINKS_INTERVAL_SETTING, TimeValue.timeValueMillis(between(1000, 2000))) + .build(); + } + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(super.nodePlugins()); + plugins.add(MockTransportService.TestPlugin.class); + plugins.add(InternalExchangePlugin.class); + return plugins; + } + + @Override + protected EsqlQueryResponse run(EsqlQueryRequest request) { + // IndexResolver currently ignores failures from field-caps responses and can resolve to a smaller set of concrete indices. + boolean singleIndex = request.query().startsWith("from test |"); + if (singleIndex && randomIntBetween(0, 100) <= 20) { + return runQueryWithDisruption(request); + } else { + return super.run(request); + } + } + + private EsqlQueryResponse runQueryWithDisruption(EsqlQueryRequest request) { + final ServiceDisruptionScheme disruptionScheme = addRandomDisruptionScheme(); + logger.info("--> start disruption scheme [{}]", disruptionScheme); + disruptionScheme.startDisrupting(); + logger.info("--> executing esql query with disruption {} ", request.query()); + ActionFuture future = client().execute(EsqlQueryAction.INSTANCE, request); + try { + return future.actionGet(2, TimeUnit.MINUTES); + } catch (Exception ignored) { + + } finally { + clearDisruption(); + } + try { + return future.actionGet(2, TimeUnit.MINUTES); + } catch (Exception e) { + assertTrue("request must be failed or completed after clearing disruption", future.isDone()); + logger.info("--> failed to execute esql query with disruption; retrying...", e); + return client().execute(EsqlQueryAction.INSTANCE, request).actionGet(2, TimeUnit.MINUTES); + } + } + + private ServiceDisruptionScheme addRandomDisruptionScheme() { + try { + ensureClusterStateConsistency(); + ensureClusterSizeConsistency(); + var disruptedLinks = NetworkDisruption.TwoPartitions.random(random(), internalCluster().getNodeNames()); + final NetworkDisruption.NetworkLinkDisruptionType disruptionType = switch (randomInt(2)) { + case 0 -> NetworkDisruption.UNRESPONSIVE; + case 1 -> NetworkDisruption.DISCONNECT; + case 2 -> NetworkDisruption.NetworkDelay.random(random(), TimeValue.timeValueMillis(2000), TimeValue.timeValueMillis(5000)); + default -> throw new IllegalArgumentException(); + }; + final ServiceDisruptionScheme scheme = new NetworkDisruption(disruptedLinks, disruptionType); + setDisruptionScheme(scheme); + return scheme; + } catch (Exception e) { + throw new AssertionError(e); + } + + } + + private void clearDisruption() { + logger.info("--> clear disruption scheme"); + try { + internalCluster().clearDisruptionScheme(false); + ensureFullyConnectedCluster(); + assertBusy(() -> assertAcked(clusterAdmin().prepareReroute().setRetryFailed(true)), 1, TimeUnit.MINUTES); + ensureYellow(); + } catch (Exception e) { + throw new AssertionError(e); + } + } +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/lookup/EnrichLookupIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/lookup/EnrichLookupIT.java new file mode 100644 index 0000000000000..8f15c6b07f5e3 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/lookup/EnrichLookupIT.java @@ -0,0 +1,215 @@ +/* + * 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.lookup; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.DriverRunner; +import org.elasticsearch.compute.operator.OutputOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupOperator; +import org.elasticsearch.xpack.esql.plugin.TransportEsqlQueryAction; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +import static org.hamcrest.Matchers.equalTo; + +public class EnrichLookupIT extends AbstractEsqlIntegTestCase { + + public void testSimple() { + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareCreate("users") + .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)) + .setMapping( + "uid", + "type=keyword,doc_values=false", + "name", + "type=keyword,index=false", + "city", + "type=keyword,index=false", + "joined", + "type=date,index=false,format=yyyy-MM-dd" + ) + ); + List> users = List.of( + Map.of("uid", "j1", "name", "John", "city", "New York/NY", "joined", "2020-03-01"), + Map.of("uid", "m4", "name", "Mike", "city", "Boston/MA", "joined", "2010-06-20"), + Map.of("uid", "j2", "name", "Jack", "city", "Austin/TX", "joined", "1999-11-03") + ); + for (Map user : users) { + client().prepareIndex("users").setSource(user).get(); + if (randomBoolean()) { + client().admin().indices().prepareRefresh("users").get(); + } + } + if (randomBoolean()) { + client().admin().indices().prepareForceMerge("users").setMaxNumSegments(1).get(); + } + client().admin().indices().prepareRefresh("users").get(); + List enrichAttributes = List.of( + new FieldAttribute(Source.EMPTY, "name", new EsField("name", DataTypes.KEYWORD, Map.of(), true)), + new FieldAttribute(Source.EMPTY, "city", new EsField("city", DataTypes.KEYWORD, Map.of(), true)), + new FieldAttribute(Source.EMPTY, "joined", new EsField("joined", DataTypes.DATETIME, Map.of(), true)) + ); + + DiscoveryNode clientNode = randomFrom(clusterService().state().nodes().stream().toList()); + var lookupService = internalCluster().getInstance(TransportEsqlQueryAction.class, clientNode.getName()).enrichLookupService(); + TransportService transportService = internalCluster().getInstance(TransportService.class, clientNode.getName()); + + EsqlQueryRequest parentRequest = new EsqlQueryRequest(); + parentRequest.query("FROM index"); + CancellableTask parentTask = (CancellableTask) transportService.getTaskManager().register("test", "test-action", parentRequest); + EnrichLookupOperator enrichOperator = new EnrichLookupOperator( + "test-session", + parentTask, + randomIntBetween(1, 3), + 0, + lookupService, + "users", + "match", + "uid", + enrichAttributes + ); + BytesRefBlock userBlock = BytesRefBlock.newBlockBuilder(5) + .appendBytesRef(new BytesRef("j1")) + .appendNull() + .appendBytesRef(new BytesRef("j2")) + .appendBytesRef(new BytesRef("j1")) + .appendBytesRef(new BytesRef("m3")) + .build(); + SourceOperator sourceOperator = sourceOperator(userBlock); + + AtomicReference outputPage = new AtomicReference<>(); + OutputOperator outputOperator = new OutputOperator(List.of(), Function.identity(), page -> { + outputPage.getAndUpdate(current -> { + if (current == null) { + return page; + } + Block.Builder[] builders = new Block.Builder[current.getBlockCount()]; + for (int i = 0; i < current.getBlockCount(); i++) { + ElementType elementType = current.getBlock(i).elementType(); + if (elementType == ElementType.NULL) { + elementType = page.getBlock(i).elementType(); + } + builders[i] = elementType.newBlockBuilder(1); + builders[i].copyFrom(current.getBlock(i), 0, current.getPositionCount()); + builders[i].copyFrom(page.getBlock(i), 0, page.getPositionCount()); + } + return new Page(Arrays.stream(builders).map(Block.Builder::build).toArray(Block[]::new)); + }); + }); + + DateFormatter dateFmt = DateFormatter.forPattern("yyyy-MM-dd"); + + DriverRunner.runToCompletion( + internalCluster().getInstance(TransportService.class).getThreadPool(), + between(1, 10_000), + List.of(new Driver(new DriverContext(), sourceOperator, List.of(enrichOperator), outputOperator, () -> {})) + ); + transportService.getTaskManager().unregister(parentTask); + Page output = outputPage.get(); + assertThat(output.getBlockCount(), equalTo(4)); + assertThat(output.getPositionCount(), equalTo(5)); + BytesRef scratch = new BytesRef(); + BytesRefBlock names = output.getBlock(1); + BytesRefBlock cities = output.getBlock(2); + LongBlock dates = output.getBlock(3); + + assertThat(names.getBytesRef(0, scratch), equalTo(new BytesRef("John"))); + assertThat(cities.getBytesRef(0, scratch), equalTo(new BytesRef("New York/NY"))); + assertThat(dateFmt.formatMillis(dates.getLong(0)), equalTo("2020-03-01")); + + assertTrue(names.isNull(1)); + assertTrue(cities.isNull(1)); + assertTrue(dates.isNull(1)); + + assertThat(names.getBytesRef(2, scratch), equalTo(new BytesRef("Jack"))); + assertThat(cities.getBytesRef(2, scratch), equalTo(new BytesRef("Austin/TX"))); + assertThat(dateFmt.formatMillis(dates.getLong(2)), equalTo("1999-11-03")); + + assertThat(names.getBytesRef(3, scratch), equalTo(new BytesRef("John"))); + assertThat(cities.getBytesRef(3, scratch), equalTo(new BytesRef("New York/NY"))); + assertThat(dateFmt.formatMillis(dates.getLong(3)), equalTo("2020-03-01")); + + assertTrue(names.isNull(4)); + assertTrue(cities.isNull(4)); + assertTrue(dates.isNull(4)); + } + + private static SourceOperator sourceOperator(BytesRefBlock input) { + return new SourceOperator() { + int position = 0; + + @Override + public void finish() { + + } + + @Override + public boolean isFinished() { + return position >= input.getPositionCount(); + } + + @Override + public Page getOutput() { + if (isFinished()) { + return null; + } + int remaining = input.getPositionCount() - position; + int size = between(1, remaining); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(size); + builder.copyFrom(input, position, position + size); + position += size; + Block block = builder.build(); + if (block.areAllValuesNull() && randomBoolean()) { + block = Block.constantNullBlock(block.getPositionCount()); + } + return new Page(block); + } + + @Override + public void close() { + + } + }; + } + + public void testRandom() { + + } + + public void testMultipleMatches() { + + } +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/CanMatchIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/CanMatchIT.java new file mode 100644 index 0000000000000..b88ba60b47b3b --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/CanMatchIT.java @@ -0,0 +1,220 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.query.MatchQueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; + +public class CanMatchIT extends AbstractEsqlIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), MockTransportService.TestPlugin.class); + } + + /** + * Make sure that we don't send data-node requests to the target shards which won't match the query + */ + public void testCanMatch() { + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareCreate("events_2022") + .setMapping("@timestamp", "type=date,format=yyyy-MM-dd", "uid", "type=keyword") + ); + client().prepareBulk("events_2022") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source("@timestamp", "2022-02-15", "uid", "u1")) + .add(new IndexRequest().source("@timestamp", "2022-05-02", "uid", "u1")) + .add(new IndexRequest().source("@timestamp", "2022-12-15", "uid", "u1")) + .get(); + ElasticsearchAssertions.assertAcked( + client().admin().indices().prepareCreate("events_2023").setMapping("@timestamp", "type=date", "uid", "type=keyword") + ); + client().prepareBulk("events_2023") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source("@timestamp", "2023-01-15", "uid", "u2")) + .add(new IndexRequest().source("@timestamp", "2023-02-01", "uid", "u2")) + .add(new IndexRequest().source("@timestamp", "2023-02-11", "uid", "u1")) + .add(new IndexRequest().source("@timestamp", "2023-03-25", "uid", "u1")) + .get(); + try { + Set queriedIndices = ConcurrentCollections.newConcurrentSet(); + for (TransportService ts : internalCluster().getInstances(TransportService.class)) { + MockTransportService transportService = (MockTransportService) ts; + transportService.addRequestHandlingBehavior(ComputeService.DATA_ACTION_NAME, (handler, request, channel, task) -> { + DataNodeRequest dataNodeRequest = (DataNodeRequest) request; + for (ShardId shardId : dataNodeRequest.shardIds()) { + queriedIndices.add(shardId.getIndexName()); + } + handler.messageReceived(request, channel, task); + }); + } + EsqlQueryResponse resp = run("from events_*", randomPragmas(), new RangeQueryBuilder("@timestamp").gte("2023-01-01")); + assertThat(resp.values(), hasSize(4)); + assertThat(queriedIndices, equalTo(Set.of("events_2023"))); + queriedIndices.clear(); + + resp = run("from events_*", randomPragmas(), new RangeQueryBuilder("@timestamp").lt("2023-01-01")); + assertThat(resp.values(), hasSize(3)); + assertThat(queriedIndices, equalTo(Set.of("events_2022"))); + queriedIndices.clear(); + + resp = run("from events_*", randomPragmas(), new RangeQueryBuilder("@timestamp").gt("2022-01-01").lt("2023-12-31")); + assertThat(resp.values(), hasSize(7)); + assertThat(queriedIndices, equalTo(Set.of("events_2022", "events_2023"))); + queriedIndices.clear(); + + resp = run("from events_*", randomPragmas(), new RangeQueryBuilder("@timestamp").gt("2021-01-01").lt("2021-12-31")); + assertThat(resp.values(), hasSize(0)); + assertThat(queriedIndices, empty()); + queriedIndices.clear(); + + } finally { + for (TransportService ts : internalCluster().getInstances(TransportService.class)) { + ((MockTransportService) ts).clearAllRules(); + } + } + } + + public void testAliasFilters() { + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareCreate("employees") + .setMapping("emp_no", "type=long", "dept", "type=keyword", "hired", "type=date,format=yyyy-MM-dd", "salary", "type=double") + ); + client().prepareBulk("employees") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source("emp_no", 101, "dept", "engineering", "hired", "2012-02-05", "salary", 20)) + .add(new IndexRequest().source("emp_no", 102, "dept", "sales", "hired", "2012-03-15", "salary", 25)) + .add(new IndexRequest().source("emp_no", 103, "dept", "engineering", "hired", "2012-03-27", "salary", 22)) + .add(new IndexRequest().source("emp_no", 104, "dept", "engineering", "hired", "2012-04-20", "salary", 39.6)) + .add(new IndexRequest().source("emp_no", 105, "dept", "engineering", "hired", "2012-06-30", "salary", 25)) + .add(new IndexRequest().source("emp_no", 106, "dept", "sales", "hired", "2012-08-09", "salary", 30.1)) + .get(); + + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareAliases() + .addAlias("employees", "engineers", new MatchQueryBuilder("dept", "engineering")) + .addAlias("employees", "sales", new MatchQueryBuilder("dept", "sales")) + ); + EsqlQueryResponse resp; + // employees index + resp = run("from employees | stats count(emp_no)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(6L))); + resp = run("from employees | stats avg(salary)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(26.95d))); + + resp = run("from employees | stats count(emp_no)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(4L))); + resp = run("from employees | stats avg(salary)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(26.65d))); + + // match both employees index and engineers alias -> employees + resp = run("from e* | stats count(emp_no)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(6L))); + resp = run("from employees | stats avg(salary)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(26.95d))); + + resp = run("from e* | stats count(emp_no)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(4L))); + resp = run("from e* | stats avg(salary)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(26.65d))); + + // engineers alias + resp = run("from engineer* | stats count(emp_no)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(4L))); + resp = run("from engineer* | stats avg(salary)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(26.65d))); + + resp = run("from engineer* | stats count(emp_no)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(3L))); + resp = run("from engineer* | stats avg(salary)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(27.2d))); + + // sales alias + resp = run("from sales | stats count(emp_no)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(2L))); + resp = run("from sales | stats avg(salary)", randomPragmas()); + assertThat(resp.values().get(0), equalTo(List.of(27.55d))); + + resp = run("from sales | stats count(emp_no)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(1L))); + resp = run("from sales | stats avg(salary)", randomPragmas(), new RangeQueryBuilder("hired").lt("2012-04-30")); + assertThat(resp.values().get(0), equalTo(List.of(25.0d))); + } + + public void testFailOnUnavailableShards() throws Exception { + internalCluster().ensureAtLeastNumDataNodes(2); + String logsOnlyNode = internalCluster().startDataOnlyNode(); + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareCreate("events") + .setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.allocation.exclude._name", logsOnlyNode) + ) + .setMapping("timestamp", "type=long", "message", "type=keyword") + ); + client().prepareBulk("events") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source("timestamp", 1, "message", "a")) + .add(new IndexRequest().source("timestamp", 2, "message", "b")) + .add(new IndexRequest().source("timestamp", 3, "message", "c")) + .get(); + ElasticsearchAssertions.assertAcked( + client().admin() + .indices() + .prepareCreate("logs") + .setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.allocation.include._name", logsOnlyNode) + ) + .setMapping("timestamp", "type=long", "message", "type=keyword") + ); + client().prepareBulk("logs") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .add(new IndexRequest().source("timestamp", 10, "message", "aa")) + .add(new IndexRequest().source("timestamp", 11, "message", "bb")) + .get(); + EsqlQueryResponse resp = run("from events,logs | KEEP timestamp,message"); + assertThat(resp.values(), hasSize(5)); + internalCluster().stopNode(logsOnlyNode); + ensureClusterSizeConsistency(); + Exception error = expectThrows(Exception.class, () -> run("from events,logs | KEEP timestamp,message")); + assertThat(error.getMessage(), containsString("no shard copies found")); + } +} diff --git a/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.g4 b/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.g4 new file mode 100644 index 0000000000000..747c1fdcd1921 --- /dev/null +++ b/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.g4 @@ -0,0 +1,191 @@ +lexer grammar EsqlBaseLexer; + +DISSECT : 'dissect' -> pushMode(EXPRESSION); +DROP : 'drop' -> pushMode(SOURCE_IDENTIFIERS); +ENRICH : 'enrich' -> pushMode(SOURCE_IDENTIFIERS); +EVAL : 'eval' -> pushMode(EXPRESSION); +EXPLAIN : 'explain' -> pushMode(EXPLAIN_MODE); +FROM : 'from' -> pushMode(SOURCE_IDENTIFIERS); +GROK : 'grok' -> pushMode(EXPRESSION); +INLINESTATS : 'inlinestats' -> pushMode(EXPRESSION); +KEEP : 'keep' -> pushMode(SOURCE_IDENTIFIERS); +LIMIT : 'limit' -> pushMode(EXPRESSION); +MV_EXPAND : 'mv_expand' -> pushMode(SOURCE_IDENTIFIERS); +PROJECT : 'project' -> pushMode(SOURCE_IDENTIFIERS); +RENAME : 'rename' -> pushMode(SOURCE_IDENTIFIERS); +ROW : 'row' -> pushMode(EXPRESSION); +SHOW : 'show' -> pushMode(EXPRESSION); +SORT : 'sort' -> pushMode(EXPRESSION); +STATS : 'stats' -> pushMode(EXPRESSION); +WHERE : 'where' -> pushMode(EXPRESSION); +UNKNOWN_CMD : ~[ \r\n\t[\]/]+ -> pushMode(EXPRESSION); + +LINE_COMMENT + : '//' ~[\r\n]* '\r'? '\n'? -> channel(HIDDEN) + ; + +MULTILINE_COMMENT + : '/*' (MULTILINE_COMMENT|.)*? '*/' -> channel(HIDDEN) + ; + +WS + : [ \r\n\t]+ -> channel(HIDDEN) + ; + + +mode EXPLAIN_MODE; +EXPLAIN_OPENING_BRACKET : '[' -> type(OPENING_BRACKET), pushMode(DEFAULT_MODE); +EXPLAIN_PIPE : '|' -> type(PIPE), popMode; +EXPLAIN_WS : WS -> channel(HIDDEN); +EXPLAIN_LINE_COMMENT : LINE_COMMENT -> channel(HIDDEN); +EXPLAIN_MULTILINE_COMMENT : MULTILINE_COMMENT -> channel(HIDDEN); + +mode EXPRESSION; + +PIPE : '|' -> popMode; + +fragment DIGIT + : [0-9] + ; + +fragment LETTER + : [A-Za-z] + ; + +fragment ESCAPE_SEQUENCE + : '\\' [tnr"\\] + ; + +fragment UNESCAPED_CHARS + : ~[\r\n"\\] + ; + +fragment EXPONENT + : [Ee] [+-]? DIGIT+ + ; + +STRING + : '"' (ESCAPE_SEQUENCE | UNESCAPED_CHARS)* '"' + | '"""' (~[\r\n])*? '"""' '"'? '"'? + ; + +INTEGER_LITERAL + : DIGIT+ + ; + +DECIMAL_LITERAL + : DIGIT+ DOT DIGIT* + | DOT DIGIT+ + | DIGIT+ (DOT DIGIT*)? EXPONENT + | DOT DIGIT+ EXPONENT + ; + +BY : 'by'; + +AND : 'and'; +ASC : 'asc'; +ASSIGN : '='; +COMMA : ','; +DESC : 'desc'; +DOT : '.'; +FALSE : 'false'; +FIRST : 'first'; +LAST : 'last'; +LP : '('; +IN: 'in'; +IS: 'is'; +LIKE: 'like'; +NOT : 'not'; +NULL : 'null'; +NULLS : 'nulls'; +OR : 'or'; +PARAM: '?'; +RLIKE: 'rlike'; +RP : ')'; +TRUE : 'true'; +INFO : 'info'; +FUNCTIONS : 'functions'; + +EQ : '=='; +NEQ : '!='; +LT : '<'; +LTE : '<='; +GT : '>'; +GTE : '>='; + +PLUS : '+'; +MINUS : '-'; +ASTERISK : '*'; +SLASH : '/'; +PERCENT : '%'; + +// Brackets are funny. We can happen upon a CLOSING_BRACKET in two ways - one +// way is to start in an explain command which then shifts us to expression +// mode. Thus, the two popModes on CLOSING_BRACKET. The other way could as +// the start of a multivalued field constant. To line up with the double pop +// the explain mode needs, we double push when we see that. +OPENING_BRACKET : '[' -> pushMode(EXPRESSION), pushMode(EXPRESSION); +CLOSING_BRACKET : ']' -> popMode, popMode; + + +UNQUOTED_IDENTIFIER + : LETTER (LETTER | DIGIT | '_')* + // only allow @ at beginning of identifier to keep the option to allow @ as infix operator in the future + // also, single `_` and `@` characters are not valid identifiers + | ('_' | '@') (LETTER | DIGIT | '_')+ + ; + +QUOTED_IDENTIFIER + : '`' ( ~'`' | '``' )* '`' + ; + +EXPR_LINE_COMMENT + : LINE_COMMENT -> channel(HIDDEN) + ; + +EXPR_MULTILINE_COMMENT + : MULTILINE_COMMENT -> channel(HIDDEN) + ; + +EXPR_WS + : WS -> channel(HIDDEN) + ; + + + +mode SOURCE_IDENTIFIERS; + +SRC_PIPE : '|' -> type(PIPE), popMode; +SRC_OPENING_BRACKET : '[' -> type(OPENING_BRACKET), pushMode(SOURCE_IDENTIFIERS), pushMode(SOURCE_IDENTIFIERS); +SRC_CLOSING_BRACKET : ']' -> popMode, popMode, type(CLOSING_BRACKET); +SRC_COMMA : ',' -> type(COMMA); +SRC_ASSIGN : '=' -> type(ASSIGN); +AS : 'as'; +METADATA: 'metadata'; +ON : 'on'; +WITH : 'with'; + +SRC_UNQUOTED_IDENTIFIER + : SRC_UNQUOTED_IDENTIFIER_PART+ + ; + +fragment SRC_UNQUOTED_IDENTIFIER_PART + : ~[=`|,[\]/ \t\r\n]+ + | '/' ~[*/] // allow single / but not followed by another / or * which would start a comment + ; + +SRC_QUOTED_IDENTIFIER + : QUOTED_IDENTIFIER + ; + +SRC_LINE_COMMENT + : LINE_COMMENT -> channel(HIDDEN) + ; + +SRC_MULTILINE_COMMENT + : MULTILINE_COMMENT -> channel(HIDDEN) + ; + +SRC_WS + : WS -> channel(HIDDEN) + ; diff --git a/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens b/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens new file mode 100644 index 0000000000000..d8761f5eb0d73 --- /dev/null +++ b/x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens @@ -0,0 +1,137 @@ +DISSECT=1 +DROP=2 +ENRICH=3 +EVAL=4 +EXPLAIN=5 +FROM=6 +GROK=7 +INLINESTATS=8 +KEEP=9 +LIMIT=10 +MV_EXPAND=11 +PROJECT=12 +RENAME=13 +ROW=14 +SHOW=15 +SORT=16 +STATS=17 +WHERE=18 +UNKNOWN_CMD=19 +LINE_COMMENT=20 +MULTILINE_COMMENT=21 +WS=22 +EXPLAIN_WS=23 +EXPLAIN_LINE_COMMENT=24 +EXPLAIN_MULTILINE_COMMENT=25 +PIPE=26 +STRING=27 +INTEGER_LITERAL=28 +DECIMAL_LITERAL=29 +BY=30 +AND=31 +ASC=32 +ASSIGN=33 +COMMA=34 +DESC=35 +DOT=36 +FALSE=37 +FIRST=38 +LAST=39 +LP=40 +IN=41 +IS=42 +LIKE=43 +NOT=44 +NULL=45 +NULLS=46 +OR=47 +PARAM=48 +RLIKE=49 +RP=50 +TRUE=51 +INFO=52 +FUNCTIONS=53 +EQ=54 +NEQ=55 +LT=56 +LTE=57 +GT=58 +GTE=59 +PLUS=60 +MINUS=61 +ASTERISK=62 +SLASH=63 +PERCENT=64 +OPENING_BRACKET=65 +CLOSING_BRACKET=66 +UNQUOTED_IDENTIFIER=67 +QUOTED_IDENTIFIER=68 +EXPR_LINE_COMMENT=69 +EXPR_MULTILINE_COMMENT=70 +EXPR_WS=71 +AS=72 +METADATA=73 +ON=74 +WITH=75 +SRC_UNQUOTED_IDENTIFIER=76 +SRC_QUOTED_IDENTIFIER=77 +SRC_LINE_COMMENT=78 +SRC_MULTILINE_COMMENT=79 +SRC_WS=80 +EXPLAIN_PIPE=81 +'dissect'=1 +'drop'=2 +'enrich'=3 +'eval'=4 +'explain'=5 +'from'=6 +'grok'=7 +'inlinestats'=8 +'keep'=9 +'limit'=10 +'mv_expand'=11 +'project'=12 +'rename'=13 +'row'=14 +'show'=15 +'sort'=16 +'stats'=17 +'where'=18 +'by'=30 +'and'=31 +'asc'=32 +'desc'=35 +'.'=36 +'false'=37 +'first'=38 +'last'=39 +'('=40 +'in'=41 +'is'=42 +'like'=43 +'not'=44 +'null'=45 +'nulls'=46 +'or'=47 +'?'=48 +'rlike'=49 +')'=50 +'true'=51 +'info'=52 +'functions'=53 +'=='=54 +'!='=55 +'<'=56 +'<='=57 +'>'=58 +'>='=59 +'+'=60 +'-'=61 +'*'=62 +'/'=63 +'%'=64 +']'=66 +'as'=72 +'metadata'=73 +'on'=74 +'with'=75 diff --git a/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4 b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4 new file mode 100644 index 0000000000000..8f07a8a5dcdea --- /dev/null +++ b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4 @@ -0,0 +1,242 @@ + +/* + * 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. + */ + +parser grammar EsqlBaseParser; + +options {tokenVocab=EsqlBaseLexer;} + +singleStatement + : query EOF + ; + +query + : sourceCommand #singleCommandQuery + | query PIPE processingCommand #compositeQuery + ; + +sourceCommand + : explainCommand + | fromCommand + | rowCommand + | showCommand + ; + +processingCommand + : evalCommand + | inlinestatsCommand + | limitCommand + | keepCommand + | sortCommand + | statsCommand + | whereCommand + | dropCommand + | renameCommand + | dissectCommand + | grokCommand + | enrichCommand + | mvExpandCommand + ; + +whereCommand + : WHERE booleanExpression + ; + +booleanExpression + : NOT booleanExpression #logicalNot + | valueExpression #booleanDefault + | regexBooleanExpression #regexExpression + | left=booleanExpression operator=AND right=booleanExpression #logicalBinary + | left=booleanExpression operator=OR right=booleanExpression #logicalBinary + | valueExpression (NOT)? IN LP valueExpression (COMMA valueExpression)* RP #logicalIn + | valueExpression IS NOT? NULL #isNull + ; + +regexBooleanExpression + : valueExpression (NOT)? kind=LIKE pattern=string + | valueExpression (NOT)? kind=RLIKE pattern=string + ; + +valueExpression + : operatorExpression #valueExpressionDefault + | left=operatorExpression comparisonOperator right=operatorExpression #comparison + ; + +operatorExpression + : primaryExpression #operatorExpressionDefault + | operator=(MINUS | PLUS) operatorExpression #arithmeticUnary + | left=operatorExpression operator=(ASTERISK | SLASH | PERCENT) right=operatorExpression #arithmeticBinary + | left=operatorExpression operator=(PLUS | MINUS) right=operatorExpression #arithmeticBinary + ; + +primaryExpression + : constant #constantDefault + | qualifiedName #dereference + | LP booleanExpression RP #parenthesizedExpression + | identifier LP (booleanExpression (COMMA booleanExpression)*)? RP #functionExpression + ; + +rowCommand + : ROW fields + ; + +fields + : field (COMMA field)* + ; + +field + : booleanExpression + | qualifiedName ASSIGN booleanExpression + ; + +fromCommand + : FROM sourceIdentifier (COMMA sourceIdentifier)* metadata? + ; + +metadata + : OPENING_BRACKET METADATA sourceIdentifier (COMMA sourceIdentifier)* CLOSING_BRACKET + ; + + +evalCommand + : EVAL fields + ; + +statsCommand + : STATS fields? (BY grouping)? + ; + +inlinestatsCommand + : INLINESTATS fields (BY grouping)? + ; + +grouping + : qualifiedName (COMMA qualifiedName)* + ; + +sourceIdentifier + : SRC_UNQUOTED_IDENTIFIER + | SRC_QUOTED_IDENTIFIER + ; + +qualifiedName + : identifier (DOT identifier)* + ; + + +identifier + : UNQUOTED_IDENTIFIER + | QUOTED_IDENTIFIER + ; + +constant + : NULL #nullLiteral + | integerValue UNQUOTED_IDENTIFIER #qualifiedIntegerLiteral + | decimalValue #decimalLiteral + | integerValue #integerLiteral + | booleanValue #booleanLiteral + | PARAM #inputParam + | string #stringLiteral + | OPENING_BRACKET numericValue (COMMA numericValue)* CLOSING_BRACKET #numericArrayLiteral + | OPENING_BRACKET booleanValue (COMMA booleanValue)* CLOSING_BRACKET #booleanArrayLiteral + | OPENING_BRACKET string (COMMA string)* CLOSING_BRACKET #stringArrayLiteral + ; + +limitCommand + : LIMIT INTEGER_LITERAL + ; + +sortCommand + : SORT orderExpression (COMMA orderExpression)* + ; + +orderExpression + : booleanExpression ordering=(ASC | DESC)? (NULLS nullOrdering=(FIRST | LAST))? + ; + +keepCommand + : KEEP sourceIdentifier (COMMA sourceIdentifier)* + | PROJECT sourceIdentifier (COMMA sourceIdentifier)* + ; + +dropCommand + : DROP sourceIdentifier (COMMA sourceIdentifier)* + ; + +renameCommand + : RENAME renameClause (COMMA renameClause)* + ; + +renameClause: + oldName=sourceIdentifier AS newName=sourceIdentifier + ; + +dissectCommand + : DISSECT primaryExpression string commandOptions? + ; + +grokCommand + : GROK primaryExpression string + ; + +mvExpandCommand + : MV_EXPAND sourceIdentifier + ; + +commandOptions + : commandOption (COMMA commandOption)* + ; + +commandOption + : identifier ASSIGN constant + ; + +booleanValue + : TRUE | FALSE + ; + +numericValue + : decimalValue + | integerValue + ; + +decimalValue + : (PLUS | MINUS)? DECIMAL_LITERAL + ; + +integerValue + : (PLUS | MINUS)? INTEGER_LITERAL + ; + +string + : STRING + ; + +comparisonOperator + : EQ | NEQ | LT | LTE | GT | GTE + ; + +explainCommand + : EXPLAIN subqueryExpression + ; + +subqueryExpression + : OPENING_BRACKET query CLOSING_BRACKET + ; + +showCommand + : SHOW INFO #showInfo + | SHOW FUNCTIONS #showFunctions + ; + +enrichCommand + : ENRICH policyName=sourceIdentifier (ON matchField=sourceIdentifier)? (WITH enrichWithClause (COMMA enrichWithClause)*)? + ; + +enrichWithClause + : (newName=sourceIdentifier ASSIGN)? enrichField=sourceIdentifier + ; diff --git a/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens new file mode 100644 index 0000000000000..d8761f5eb0d73 --- /dev/null +++ b/x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens @@ -0,0 +1,137 @@ +DISSECT=1 +DROP=2 +ENRICH=3 +EVAL=4 +EXPLAIN=5 +FROM=6 +GROK=7 +INLINESTATS=8 +KEEP=9 +LIMIT=10 +MV_EXPAND=11 +PROJECT=12 +RENAME=13 +ROW=14 +SHOW=15 +SORT=16 +STATS=17 +WHERE=18 +UNKNOWN_CMD=19 +LINE_COMMENT=20 +MULTILINE_COMMENT=21 +WS=22 +EXPLAIN_WS=23 +EXPLAIN_LINE_COMMENT=24 +EXPLAIN_MULTILINE_COMMENT=25 +PIPE=26 +STRING=27 +INTEGER_LITERAL=28 +DECIMAL_LITERAL=29 +BY=30 +AND=31 +ASC=32 +ASSIGN=33 +COMMA=34 +DESC=35 +DOT=36 +FALSE=37 +FIRST=38 +LAST=39 +LP=40 +IN=41 +IS=42 +LIKE=43 +NOT=44 +NULL=45 +NULLS=46 +OR=47 +PARAM=48 +RLIKE=49 +RP=50 +TRUE=51 +INFO=52 +FUNCTIONS=53 +EQ=54 +NEQ=55 +LT=56 +LTE=57 +GT=58 +GTE=59 +PLUS=60 +MINUS=61 +ASTERISK=62 +SLASH=63 +PERCENT=64 +OPENING_BRACKET=65 +CLOSING_BRACKET=66 +UNQUOTED_IDENTIFIER=67 +QUOTED_IDENTIFIER=68 +EXPR_LINE_COMMENT=69 +EXPR_MULTILINE_COMMENT=70 +EXPR_WS=71 +AS=72 +METADATA=73 +ON=74 +WITH=75 +SRC_UNQUOTED_IDENTIFIER=76 +SRC_QUOTED_IDENTIFIER=77 +SRC_LINE_COMMENT=78 +SRC_MULTILINE_COMMENT=79 +SRC_WS=80 +EXPLAIN_PIPE=81 +'dissect'=1 +'drop'=2 +'enrich'=3 +'eval'=4 +'explain'=5 +'from'=6 +'grok'=7 +'inlinestats'=8 +'keep'=9 +'limit'=10 +'mv_expand'=11 +'project'=12 +'rename'=13 +'row'=14 +'show'=15 +'sort'=16 +'stats'=17 +'where'=18 +'by'=30 +'and'=31 +'asc'=32 +'desc'=35 +'.'=36 +'false'=37 +'first'=38 +'last'=39 +'('=40 +'in'=41 +'is'=42 +'like'=43 +'not'=44 +'null'=45 +'nulls'=46 +'or'=47 +'?'=48 +'rlike'=49 +')'=50 +'true'=51 +'info'=52 +'functions'=53 +'=='=54 +'!='=55 +'<'=56 +'<='=57 +'>'=58 +'>='=59 +'+'=60 +'-'=61 +'*'=62 +'/'=63 +'%'=64 +']'=66 +'as'=72 +'metadata'=73 +'on'=74 +'with'=75 diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromDoubleEvaluator.java new file mode 100644 index 0000000000000..ee95635e52b0b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromDoubleEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToBoolean}. + * This class is generated. Do not edit it. + */ +public final class ToBooleanFromDoubleEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToBooleanFromDoubleEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToBoolean"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBooleanVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BooleanArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BooleanArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static boolean evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToBoolean.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + boolean value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBoolean(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static boolean evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToBoolean.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromIntEvaluator.java new file mode 100644 index 0000000000000..6005ffe9be356 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromIntEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToBoolean}. + * This class is generated. Do not edit it. + */ +public final class ToBooleanFromIntEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToBooleanFromIntEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToBoolean"; + } + + @Override + public Block evalVector(Vector v) { + IntVector vector = (IntVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBooleanVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BooleanArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BooleanArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static boolean evalValue(IntVector container, int index) { + int value = container.getInt(index); + return ToBoolean.fromInt(value); + } + + @Override + public Block evalBlock(Block b) { + IntBlock block = (IntBlock) b; + int positionCount = block.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + boolean value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBoolean(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static boolean evalValue(IntBlock container, int index) { + int value = container.getInt(index); + return ToBoolean.fromInt(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromLongEvaluator.java new file mode 100644 index 0000000000000..06306a9a2acab --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToBoolean}. + * This class is generated. Do not edit it. + */ +public final class ToBooleanFromLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToBooleanFromLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToBoolean"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBooleanVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BooleanArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BooleanArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static boolean evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToBoolean.fromLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + boolean value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBoolean(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static boolean evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToBoolean.fromLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromStringEvaluator.java new file mode 100644 index 0000000000000..b71ee10036822 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToBoolean}. + * This class is generated. Do not edit it. + */ +public final class ToBooleanFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToBooleanFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToBoolean"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantBooleanVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BooleanArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BooleanArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static boolean evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToBoolean.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + boolean value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBoolean(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static boolean evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToBoolean.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..9d391b5730d05 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBooleanFromUnsignedLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ConstantBooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToBoolean}. + * This class is generated. Do not edit it. + */ +public final class ToBooleanFromUnsignedLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToBooleanFromUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToBoolean"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBooleanVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BooleanArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BooleanArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static boolean evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToBoolean.fromUnsignedLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + boolean value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBoolean(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static boolean evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToBoolean.fromUnsignedLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromStringEvaluator.java new file mode 100644 index 0000000000000..56e3f7c52fe39 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetimeFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDatetime}. + * This class is generated. Do not edit it. + */ +public final class ToDatetimeFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDatetimeFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDatetime"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToDatetime.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToDatetime.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegreesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegreesEvaluator.java new file mode 100644 index 0000000000000..a168d93e73ba3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegreesEvaluator.java @@ -0,0 +1,107 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDegrees}. + * This class is generated. Do not edit it. + */ +public final class ToDegreesEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDegreesEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDegrees"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToDegrees.process(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToDegrees.process(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromBooleanEvaluator.java new file mode 100644 index 0000000000000..c3ab6f87db2a1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromBooleanEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDouble}. + * This class is generated. Do not edit it. + */ +public final class ToDoubleFromBooleanEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDoubleFromBooleanEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDouble"; + } + + @Override + public Block evalVector(Vector v) { + BooleanVector vector = (BooleanVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(BooleanVector container, int index) { + boolean value = container.getBoolean(index); + return ToDouble.fromBoolean(value); + } + + @Override + public Block evalBlock(Block b) { + BooleanBlock block = (BooleanBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(BooleanBlock container, int index) { + boolean value = container.getBoolean(index); + return ToDouble.fromBoolean(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromIntEvaluator.java new file mode 100644 index 0000000000000..57bb11132d1bc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromIntEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDouble}. + * This class is generated. Do not edit it. + */ +public final class ToDoubleFromIntEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDoubleFromIntEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDouble"; + } + + @Override + public Block evalVector(Vector v) { + IntVector vector = (IntVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(IntVector container, int index) { + int value = container.getInt(index); + return ToDouble.fromInt(value); + } + + @Override + public Block evalBlock(Block b) { + IntBlock block = (IntBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(IntBlock container, int index) { + int value = container.getInt(index); + return ToDouble.fromInt(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromLongEvaluator.java new file mode 100644 index 0000000000000..ee5defac00278 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDouble}. + * This class is generated. Do not edit it. + */ +public final class ToDoubleFromLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDoubleFromLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDouble"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToDouble.fromLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToDouble.fromLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromStringEvaluator.java new file mode 100644 index 0000000000000..3c3dc8f7eb298 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDouble}. + * This class is generated. Do not edit it. + */ +public final class ToDoubleFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDoubleFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDouble"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToDouble.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToDouble.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..3d0cb711e9dde --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDoubleFromUnsignedLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToDouble}. + * This class is generated. Do not edit it. + */ +public final class ToDoubleFromUnsignedLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToDoubleFromUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToDouble"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToDouble.fromUnsignedLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToDouble.fromUnsignedLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIPFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIPFromStringEvaluator.java new file mode 100644 index 0000000000000..6277d89a7f918 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIPFromStringEvaluator.java @@ -0,0 +1,112 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToIP}. + * This class is generated. Do not edit it. + */ +public final class ToIPFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIPFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToIP"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p, scratchPad)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToIP.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToIP.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromBooleanEvaluator.java new file mode 100644 index 0000000000000..997ba90fc8043 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromBooleanEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToInteger}. + * This class is generated. Do not edit it. + */ +public final class ToIntegerFromBooleanEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIntegerFromBooleanEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToInteger"; + } + + @Override + public Block evalVector(Vector v) { + BooleanVector vector = (BooleanVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantIntVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new IntArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new IntArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static int evalValue(BooleanVector container, int index) { + boolean value = container.getBoolean(index); + return ToInteger.fromBoolean(value); + } + + @Override + public Block evalBlock(Block b) { + BooleanBlock block = (BooleanBlock) b; + int positionCount = block.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + int value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendInt(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static int evalValue(BooleanBlock container, int index) { + boolean value = container.getBoolean(index); + return ToInteger.fromBoolean(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromDoubleEvaluator.java new file mode 100644 index 0000000000000..2589c116dc106 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromDoubleEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToInteger}. + * This class is generated. Do not edit it. + */ +public final class ToIntegerFromDoubleEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIntegerFromDoubleEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToInteger"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantIntVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new IntArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new IntArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static int evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToInteger.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + int value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendInt(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static int evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToInteger.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromLongEvaluator.java new file mode 100644 index 0000000000000..f79fd470ae76c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToInteger}. + * This class is generated. Do not edit it. + */ +public final class ToIntegerFromLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIntegerFromLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToInteger"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantIntVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new IntArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new IntArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static int evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToInteger.fromLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + int value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendInt(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static int evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToInteger.fromLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromStringEvaluator.java new file mode 100644 index 0000000000000..feb0c565290b2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToInteger}. + * This class is generated. Do not edit it. + */ +public final class ToIntegerFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIntegerFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToInteger"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantIntVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new IntArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new IntArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static int evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToInteger.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + int value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendInt(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static int evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToInteger.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..d5b02c0d87889 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIntegerFromUnsignedLongEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntArrayBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToInteger}. + * This class is generated. Do not edit it. + */ +public final class ToIntegerFromUnsignedLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToIntegerFromUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToInteger"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantIntVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new IntArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new IntArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static int evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToInteger.fromUnsignedLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + int value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendInt(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static int evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToInteger.fromUnsignedLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromBooleanEvaluator.java new file mode 100644 index 0000000000000..87879b26c7d23 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromBooleanEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToLong}. + * This class is generated. Do not edit it. + */ +public final class ToLongFromBooleanEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToLongFromBooleanEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToLong"; + } + + @Override + public Block evalVector(Vector v) { + BooleanVector vector = (BooleanVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(BooleanVector container, int index) { + boolean value = container.getBoolean(index); + return ToLong.fromBoolean(value); + } + + @Override + public Block evalBlock(Block b) { + BooleanBlock block = (BooleanBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(BooleanBlock container, int index) { + boolean value = container.getBoolean(index); + return ToLong.fromBoolean(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromDoubleEvaluator.java new file mode 100644 index 0000000000000..23900011be8ae --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromDoubleEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToLong}. + * This class is generated. Do not edit it. + */ +public final class ToLongFromDoubleEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToLongFromDoubleEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToLong"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToLong.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToLong.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromIntEvaluator.java new file mode 100644 index 0000000000000..a043af09a47cf --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromIntEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToLong}. + * This class is generated. Do not edit it. + */ +public final class ToLongFromIntEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToLongFromIntEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToLong"; + } + + @Override + public Block evalVector(Vector v) { + IntVector vector = (IntVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(IntVector container, int index) { + int value = container.getInt(index); + return ToLong.fromInt(value); + } + + @Override + public Block evalBlock(Block b) { + IntBlock block = (IntBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(IntBlock container, int index) { + int value = container.getInt(index); + return ToLong.fromInt(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromStringEvaluator.java new file mode 100644 index 0000000000000..3629c0d0d046f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToLong}. + * This class is generated. Do not edit it. + */ +public final class ToLongFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToLongFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToLong"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToLong.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToLong.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..4f6e2363a676b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLongFromUnsignedLongEvaluator.java @@ -0,0 +1,107 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToLong}. + * This class is generated. Do not edit it. + */ +public final class ToLongFromUnsignedLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToLongFromUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToLong"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToLong.fromUnsignedLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToLong.fromUnsignedLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadiansEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadiansEvaluator.java new file mode 100644 index 0000000000000..33ae94093dd85 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadiansEvaluator.java @@ -0,0 +1,107 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantDoubleVector; +import org.elasticsearch.compute.data.DoubleArrayBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToRadians}. + * This class is generated. Do not edit it. + */ +public final class ToRadiansEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToRadiansEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToRadians"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantDoubleVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new DoubleArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new DoubleArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static double evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToRadians.process(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + double value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendDouble(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static double evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToRadians.process(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromBooleanEvaluator.java new file mode 100644 index 0000000000000..7f8244391ba90 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromBooleanEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromBooleanEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromBooleanEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + BooleanVector vector = (BooleanVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(BooleanVector container, int index) { + boolean value = container.getBoolean(index); + return ToString.fromBoolean(value); + } + + @Override + public Block evalBlock(Block b) { + BooleanBlock block = (BooleanBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(BooleanBlock container, int index) { + boolean value = container.getBoolean(index); + return ToString.fromBoolean(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDatetimeEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDatetimeEvaluator.java new file mode 100644 index 0000000000000..a681fa818cf8f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDatetimeEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromDatetimeEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromDatetimeEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToString.fromDatetime(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToString.fromDatetime(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDoubleEvaluator.java new file mode 100644 index 0000000000000..2eb67c1739d9c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDoubleEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromDoubleEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromDoubleEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToString.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToString.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIPEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIPEvaluator.java new file mode 100644 index 0000000000000..d774668f7c654 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIPEvaluator.java @@ -0,0 +1,112 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromIPEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromIPEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p, scratchPad)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToString.fromIP(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToString.fromIP(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIntEvaluator.java new file mode 100644 index 0000000000000..b004b9974b18e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromIntEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromIntEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromIntEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + IntVector vector = (IntVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(IntVector container, int index) { + int value = container.getInt(index); + return ToString.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + IntBlock block = (IntBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(IntBlock container, int index) { + int value = container.getInt(index); + return ToString.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromLongEvaluator.java new file mode 100644 index 0000000000000..4ea62a864f894 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromLongEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToString.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToString.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..057ff28f0c32d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromUnsignedLongEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromUnsignedLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToString.fromUnsignedLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToString.fromUnsignedLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromVersionEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromVersionEvaluator.java new file mode 100644 index 0000000000000..832984e467fa3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromVersionEvaluator.java @@ -0,0 +1,112 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToString}. + * This class is generated. Do not edit it. + */ +public final class ToStringFromVersionEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToStringFromVersionEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToString"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p, scratchPad)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToString.fromVersion(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToString.fromVersion(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromBooleanEvaluator.java new file mode 100644 index 0000000000000..379cf1b80f4d6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromBooleanEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToUnsignedLong}. + * This class is generated. Do not edit it. + */ +public final class ToUnsignedLongFromBooleanEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToUnsignedLongFromBooleanEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToUnsignedLong"; + } + + @Override + public Block evalVector(Vector v) { + BooleanVector vector = (BooleanVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(BooleanVector container, int index) { + boolean value = container.getBoolean(index); + return ToUnsignedLong.fromBoolean(value); + } + + @Override + public Block evalBlock(Block b) { + BooleanBlock block = (BooleanBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(BooleanBlock container, int index) { + boolean value = container.getBoolean(index); + return ToUnsignedLong.fromBoolean(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromDoubleEvaluator.java new file mode 100644 index 0000000000000..80ec90c02d2d4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromDoubleEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToUnsignedLong}. + * This class is generated. Do not edit it. + */ +public final class ToUnsignedLongFromDoubleEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToUnsignedLongFromDoubleEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToUnsignedLong"; + } + + @Override + public Block evalVector(Vector v) { + DoubleVector vector = (DoubleVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(DoubleVector container, int index) { + double value = container.getDouble(index); + return ToUnsignedLong.fromDouble(value); + } + + @Override + public Block evalBlock(Block b) { + DoubleBlock block = (DoubleBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(DoubleBlock container, int index) { + double value = container.getDouble(index); + return ToUnsignedLong.fromDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromIntEvaluator.java new file mode 100644 index 0000000000000..c8e370296aa9f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromIntEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToUnsignedLong}. + * This class is generated. Do not edit it. + */ +public final class ToUnsignedLongFromIntEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToUnsignedLongFromIntEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToUnsignedLong"; + } + + @Override + public Block evalVector(Vector v) { + IntVector vector = (IntVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(IntVector container, int index) { + int value = container.getInt(index); + return ToUnsignedLong.fromInt(value); + } + + @Override + public Block evalBlock(Block b) { + IntBlock block = (IntBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(IntBlock container, int index) { + int value = container.getInt(index); + return ToUnsignedLong.fromInt(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromLongEvaluator.java new file mode 100644 index 0000000000000..af2b02fb3d73a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromLongEvaluator.java @@ -0,0 +1,107 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToUnsignedLong}. + * This class is generated. Do not edit it. + */ +public final class ToUnsignedLongFromLongEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToUnsignedLongFromLongEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToUnsignedLong"; + } + + @Override + public Block evalVector(Vector v) { + LongVector vector = (LongVector) v; + int positionCount = v.getPositionCount(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(LongVector container, int index) { + long value = container.getLong(index); + return ToUnsignedLong.fromLong(value); + } + + @Override + public Block evalBlock(Block b) { + LongBlock block = (LongBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(LongBlock container, int index) { + long value = container.getLong(index); + return ToUnsignedLong.fromLong(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromStringEvaluator.java new file mode 100644 index 0000000000000..201c1fbc4157e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLongFromStringEvaluator.java @@ -0,0 +1,111 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantLongVector; +import org.elasticsearch.compute.data.LongArrayBlock; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToUnsignedLong}. + * This class is generated. Do not edit it. + */ +public final class ToUnsignedLongFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToUnsignedLongFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToUnsignedLong"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantLongVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + try { + values[p] = evalValue(vector, p, scratchPad); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new LongArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new LongArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static long evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToUnsignedLong.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + long value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendLong(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static long evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToUnsignedLong.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersionFromStringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersionFromStringEvaluator.java new file mode 100644 index 0000000000000..8630c6fa98d17 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersionFromStringEvaluator.java @@ -0,0 +1,112 @@ +// 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.expression.function.scalar.convert; + +import java.lang.Override; +import java.lang.String; +import java.util.BitSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayBlock; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.ConstantBytesRefVector; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link ToVersion}. + * This class is generated. Do not edit it. + */ +public final class ToVersionFromStringEvaluator extends AbstractConvertFunction.AbstractEvaluator { + public ToVersionFromStringEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + super(field, source); + } + + @Override + public String name() { + return "ToVersion"; + } + + @Override + public Block evalVector(Vector v) { + BytesRefVector vector = (BytesRefVector) v; + int positionCount = v.getPositionCount(); + BytesRef scratchPad = new BytesRef(); + if (vector.isConstant()) { + try { + return new ConstantBytesRefVector(evalValue(vector, 0, scratchPad), positionCount).asBlock(); + } catch (Exception e) { + registerException(e); + return Block.constantNullBlock(positionCount); + } + } + BitSet nullsMask = null; + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + for (int p = 0; p < positionCount; p++) { + try { + values.append(evalValue(vector, p, scratchPad)); + } catch (Exception e) { + registerException(e); + if (nullsMask == null) { + nullsMask = new BitSet(positionCount); + } + nullsMask.set(p); + } + } + return nullsMask == null + ? new BytesRefArrayVector(values, positionCount).asBlock() + // UNORDERED, since whatever ordering there is, it isn't necessarily preserved + : new BytesRefArrayBlock(values, positionCount, null, nullsMask, Block.MvOrdering.UNORDERED); + } + + private static BytesRef evalValue(BytesRefVector container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToVersion.fromKeyword(value); + } + + @Override + public Block evalBlock(Block b) { + BytesRefBlock block = (BytesRefBlock) b; + int positionCount = block.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef scratchPad = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + int end = start + valueCount; + boolean positionOpened = false; + boolean valuesAppended = false; + for (int i = start; i < end; i++) { + try { + BytesRef value = evalValue(block, i, scratchPad); + if (positionOpened == false && valueCount > 1) { + builder.beginPositionEntry(); + positionOpened = true; + } + builder.appendBytesRef(value); + valuesAppended = true; + } catch (Exception e) { + registerException(e); + } + } + if (valuesAppended == false) { + builder.appendNull(); + } else if (positionOpened) { + builder.endPositionEntry(); + } + } + return builder.build(); + } + + private static BytesRef evalValue(BytesRefBlock container, int index, BytesRef scratchPad) { + BytesRef value = container.getBytesRef(index, scratchPad); + return ToVersion.fromKeyword(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractConstantEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractConstantEvaluator.java new file mode 100644 index 0000000000000..920581cdeaf80 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractConstantEvaluator.java @@ -0,0 +1,73 @@ +// 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.expression.function.scalar.date; + +import java.lang.Override; +import java.lang.String; +import java.time.ZoneId; +import java.time.temporal.ChronoField; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateExtract}. + * This class is generated. Do not edit it. + */ +public final class DateExtractConstantEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator value; + + private final ChronoField chronoField; + + private final ZoneId zone; + + public DateExtractConstantEvaluator(EvalOperator.ExpressionEvaluator value, + ChronoField chronoField, ZoneId zone) { + this.value = value; + this.chronoField = chronoField; + this.zone = zone; + } + + @Override + public Block eval(Page page) { + Block valueUncastBlock = value.eval(page); + if (valueUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valueBlock = (LongBlock) valueUncastBlock; + LongVector valueVector = valueBlock.asVector(); + if (valueVector == null) { + return eval(page.getPositionCount(), valueBlock); + } + return eval(page.getPositionCount(), valueVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock valueBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valueBlock.isNull(p) || valueBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(DateExtract.process(valueBlock.getLong(valueBlock.getFirstValueIndex(p)), chronoField, zone)); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector valueVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(DateExtract.process(valueVector.getLong(p), chronoField, zone)); + } + return result.build(); + } + + @Override + public String toString() { + return "DateExtractConstantEvaluator[" + "value=" + value + ", chronoField=" + chronoField + ", zone=" + zone + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractEvaluator.java new file mode 100644 index 0000000000000..f4893d59a9f2d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractEvaluator.java @@ -0,0 +1,107 @@ +// 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.expression.function.scalar.date; + +import java.lang.IllegalArgumentException; +import java.lang.Override; +import java.lang.String; +import java.time.ZoneId; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateExtract}. + * This class is generated. Do not edit it. + */ +public final class DateExtractEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator value; + + private final EvalOperator.ExpressionEvaluator chronoField; + + private final ZoneId zone; + + public DateExtractEvaluator(Source source, EvalOperator.ExpressionEvaluator value, + EvalOperator.ExpressionEvaluator chronoField, ZoneId zone) { + this.warnings = new Warnings(source); + this.value = value; + this.chronoField = chronoField; + this.zone = zone; + } + + @Override + public Block eval(Page page) { + Block valueUncastBlock = value.eval(page); + if (valueUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valueBlock = (LongBlock) valueUncastBlock; + Block chronoFieldUncastBlock = chronoField.eval(page); + if (chronoFieldUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock chronoFieldBlock = (BytesRefBlock) chronoFieldUncastBlock; + LongVector valueVector = valueBlock.asVector(); + if (valueVector == null) { + return eval(page.getPositionCount(), valueBlock, chronoFieldBlock); + } + BytesRefVector chronoFieldVector = chronoFieldBlock.asVector(); + if (chronoFieldVector == null) { + return eval(page.getPositionCount(), valueBlock, chronoFieldBlock); + } + return eval(page.getPositionCount(), valueVector, chronoFieldVector); + } + + public LongBlock eval(int positionCount, LongBlock valueBlock, BytesRefBlock chronoFieldBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef chronoFieldScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valueBlock.isNull(p) || valueBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (chronoFieldBlock.isNull(p) || chronoFieldBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(DateExtract.process(valueBlock.getLong(valueBlock.getFirstValueIndex(p)), chronoFieldBlock.getBytesRef(chronoFieldBlock.getFirstValueIndex(p), chronoFieldScratch), zone)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector valueVector, + BytesRefVector chronoFieldVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef chronoFieldScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(DateExtract.process(valueVector.getLong(p), chronoFieldVector.getBytesRef(p, chronoFieldScratch), zone)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DateExtractEvaluator[" + "value=" + value + ", chronoField=" + chronoField + ", zone=" + zone + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatConstantEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatConstantEvaluator.java new file mode 100644 index 0000000000000..dc2c041532bb8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatConstantEvaluator.java @@ -0,0 +1,71 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License +// 2.0; you may not use this file except in compliance with the Elastic License +// 2.0. +package org.elasticsearch.xpack.esql.expression.function.scalar.date; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateFormat}. + * This class is generated. Do not edit it. + */ +public final class DateFormatConstantEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final DateFormatter formatter; + + public DateFormatConstantEvaluator(EvalOperator.ExpressionEvaluator val, + DateFormatter formatter) { + this.val = val; + this.formatter = formatter; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public BytesRefBlock eval(int positionCount, LongBlock valBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBytesRef(DateFormat.process(valBlock.getLong(valBlock.getFirstValueIndex(p)), formatter)); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, LongVector valVector) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBytesRef(DateFormat.process(valVector.getLong(p), formatter)); + } + return result.build(); + } + + @Override + public String toString() { + return "DateFormatConstantEvaluator[" + "val=" + val + ", formatter=" + formatter + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatEvaluator.java new file mode 100644 index 0000000000000..847cbc011f8ad --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatEvaluator.java @@ -0,0 +1,91 @@ +// 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.expression.function.scalar.date; + +import java.lang.Override; +import java.lang.String; +import java.util.Locale; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateFormat}. + * This class is generated. Do not edit it. + */ +public final class DateFormatEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator formatter; + + private final Locale locale; + + public DateFormatEvaluator(EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator formatter, Locale locale) { + this.val = val; + this.formatter = formatter; + this.locale = locale; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + Block formatterUncastBlock = formatter.eval(page); + if (formatterUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock formatterBlock = (BytesRefBlock) formatterUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, formatterBlock); + } + BytesRefVector formatterVector = formatterBlock.asVector(); + if (formatterVector == null) { + return eval(page.getPositionCount(), valBlock, formatterBlock); + } + return eval(page.getPositionCount(), valVector, formatterVector).asBlock(); + } + + public BytesRefBlock eval(int positionCount, LongBlock valBlock, BytesRefBlock formatterBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef formatterScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (formatterBlock.isNull(p) || formatterBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBytesRef(DateFormat.process(valBlock.getLong(valBlock.getFirstValueIndex(p)), formatterBlock.getBytesRef(formatterBlock.getFirstValueIndex(p), formatterScratch), locale)); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, LongVector valVector, + BytesRefVector formatterVector) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + BytesRef formatterScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBytesRef(DateFormat.process(valVector.getLong(p), formatterVector.getBytesRef(p, formatterScratch), locale)); + } + return result.build(); + } + + @Override + public String toString() { + return "DateFormatEvaluator[" + "val=" + val + ", formatter=" + formatter + ", locale=" + locale + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseConstantEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseConstantEvaluator.java new file mode 100644 index 0000000000000..7130476b96e23 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseConstantEvaluator.java @@ -0,0 +1,89 @@ +// 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.expression.function.scalar.date; + +import java.lang.IllegalArgumentException; +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateParse}. + * This class is generated. Do not edit it. + */ +public final class DateParseConstantEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator val; + + private final DateFormatter formatter; + + public DateParseConstantEvaluator(Source source, EvalOperator.ExpressionEvaluator val, + DateFormatter formatter) { + this.warnings = new Warnings(source); + this.val = val; + this.formatter = formatter; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock valBlock = (BytesRefBlock) valUncastBlock; + BytesRefVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector); + } + + public LongBlock eval(int positionCount, BytesRefBlock valBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(DateParse.process(valBlock.getBytesRef(valBlock.getFirstValueIndex(p), valScratch), formatter)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, BytesRefVector valVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(DateParse.process(valVector.getBytesRef(p, valScratch), formatter)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DateParseConstantEvaluator[" + "val=" + val + ", formatter=" + formatter + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseEvaluator.java new file mode 100644 index 0000000000000..a4f79021c1e70 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseEvaluator.java @@ -0,0 +1,108 @@ +// 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.expression.function.scalar.date; + +import java.lang.IllegalArgumentException; +import java.lang.Override; +import java.lang.String; +import java.time.ZoneId; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateParse}. + * This class is generated. Do not edit it. + */ +public final class DateParseEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator formatter; + + private final ZoneId zoneId; + + public DateParseEvaluator(Source source, EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator formatter, ZoneId zoneId) { + this.warnings = new Warnings(source); + this.val = val; + this.formatter = formatter; + this.zoneId = zoneId; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock valBlock = (BytesRefBlock) valUncastBlock; + Block formatterUncastBlock = formatter.eval(page); + if (formatterUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock formatterBlock = (BytesRefBlock) formatterUncastBlock; + BytesRefVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, formatterBlock); + } + BytesRefVector formatterVector = formatterBlock.asVector(); + if (formatterVector == null) { + return eval(page.getPositionCount(), valBlock, formatterBlock); + } + return eval(page.getPositionCount(), valVector, formatterVector); + } + + public LongBlock eval(int positionCount, BytesRefBlock valBlock, BytesRefBlock formatterBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + BytesRef formatterScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (formatterBlock.isNull(p) || formatterBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(DateParse.process(valBlock.getBytesRef(valBlock.getFirstValueIndex(p), valScratch), formatterBlock.getBytesRef(formatterBlock.getFirstValueIndex(p), formatterScratch), zoneId)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, BytesRefVector valVector, + BytesRefVector formatterVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + BytesRef formatterScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(DateParse.process(valVector.getBytesRef(p, valScratch), formatterVector.getBytesRef(p, formatterScratch), zoneId)); + } catch (IllegalArgumentException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DateParseEvaluator[" + "val=" + val + ", formatter=" + formatter + ", zoneId=" + zoneId + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java new file mode 100644 index 0000000000000..42d9fc3250919 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java @@ -0,0 +1,68 @@ +// 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.expression.function.scalar.date; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateTrunc}. + * This class is generated. Do not edit it. + */ +public final class DateTruncEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator fieldVal; + + private final Rounding.Prepared rounding; + + public DateTruncEvaluator(EvalOperator.ExpressionEvaluator fieldVal, Rounding.Prepared rounding) { + this.fieldVal = fieldVal; + this.rounding = rounding; + } + + @Override + public Block eval(Page page) { + Block fieldValUncastBlock = fieldVal.eval(page); + if (fieldValUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock fieldValBlock = (LongBlock) fieldValUncastBlock; + LongVector fieldValVector = fieldValBlock.asVector(); + if (fieldValVector == null) { + return eval(page.getPositionCount(), fieldValBlock); + } + return eval(page.getPositionCount(), fieldValVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock fieldValBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (fieldValBlock.isNull(p) || fieldValBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(DateTrunc.process(fieldValBlock.getLong(fieldValBlock.getFirstValueIndex(p)), rounding)); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector fieldValVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(DateTrunc.process(fieldValVector.getLong(p), rounding)); + } + return result.build(); + } + + @Override + public String toString() { + return "DateTruncEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/NowEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/NowEvaluator.java new file mode 100644 index 0000000000000..290662cfc33ba --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/NowEvaluator.java @@ -0,0 +1,42 @@ +// 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.expression.function.scalar.date; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Now}. + * This class is generated. Do not edit it. + */ +public final class NowEvaluator implements EvalOperator.ExpressionEvaluator { + private final long now; + + public NowEvaluator(long now) { + this.now = now; + } + + @Override + public Block eval(Page page) { + return eval(page.getPositionCount()).asBlock(); + } + + public LongVector eval(int positionCount) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Now.process(now)); + } + return result.build(); + } + + @Override + public String toString() { + return "NowEvaluator[" + "now=" + now + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatchEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatchEvaluator.java new file mode 100644 index 0000000000000..d87502789de97 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatchEvaluator.java @@ -0,0 +1,115 @@ +// 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.expression.function.scalar.ip; + +import java.lang.Override; +import java.lang.String; +import java.util.Arrays; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link CIDRMatch}. + * This class is generated. Do not edit it. + */ +public final class CIDRMatchEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator ip; + + private final EvalOperator.ExpressionEvaluator[] cidrs; + + public CIDRMatchEvaluator(EvalOperator.ExpressionEvaluator ip, + EvalOperator.ExpressionEvaluator[] cidrs) { + this.ip = ip; + this.cidrs = cidrs; + } + + @Override + public Block eval(Page page) { + Block ipUncastBlock = ip.eval(page); + if (ipUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock ipBlock = (BytesRefBlock) ipUncastBlock; + BytesRefBlock[] cidrsBlocks = new BytesRefBlock[cidrs.length]; + for (int i = 0; i < cidrsBlocks.length; i++) { + Block block = cidrs[i].eval(page); + if (block.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + cidrsBlocks[i] = (BytesRefBlock) block; + } + BytesRefVector ipVector = ipBlock.asVector(); + if (ipVector == null) { + return eval(page.getPositionCount(), ipBlock, cidrsBlocks); + } + BytesRefVector[] cidrsVectors = new BytesRefVector[cidrs.length]; + for (int i = 0; i < cidrsBlocks.length; i++) { + cidrsVectors[i] = cidrsBlocks[i].asVector(); + if (cidrsVectors[i] == null) { + return eval(page.getPositionCount(), ipBlock, cidrsBlocks); + } + } + return eval(page.getPositionCount(), ipVector, cidrsVectors).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock ipBlock, BytesRefBlock[] cidrsBlocks) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef ipScratch = new BytesRef(); + BytesRef[] cidrsValues = new BytesRef[cidrs.length]; + BytesRef[] cidrsScratch = new BytesRef[cidrs.length]; + for (int i = 0; i < cidrs.length; i++) { + cidrsScratch[i] = new BytesRef(); + } + position: for (int p = 0; p < positionCount; p++) { + if (ipBlock.isNull(p) || ipBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + for (int i = 0; i < cidrsBlocks.length; i++) { + if (cidrsBlocks[i].isNull(p) || cidrsBlocks[i].getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + } + // unpack cidrsBlocks into cidrsValues + for (int i = 0; i < cidrsBlocks.length; i++) { + int o = cidrsBlocks[i].getFirstValueIndex(p); + cidrsValues[i] = cidrsBlocks[i].getBytesRef(o, cidrsScratch[i]); + } + result.appendBoolean(CIDRMatch.process(ipBlock.getBytesRef(ipBlock.getFirstValueIndex(p), ipScratch), cidrsValues)); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector ipVector, + BytesRefVector[] cidrsVectors) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef ipScratch = new BytesRef(); + BytesRef[] cidrsValues = new BytesRef[cidrs.length]; + BytesRef[] cidrsScratch = new BytesRef[cidrs.length]; + for (int i = 0; i < cidrs.length; i++) { + cidrsScratch[i] = new BytesRef(); + } + position: for (int p = 0; p < positionCount; p++) { + // unpack cidrsVectors into cidrsValues + for (int i = 0; i < cidrsVectors.length; i++) { + cidrsValues[i] = cidrsVectors[i].getBytesRef(p, cidrsScratch[i]); + } + result.appendBoolean(CIDRMatch.process(ipVector.getBytesRef(p, ipScratch), cidrsValues)); + } + return result.build(); + } + + @Override + public String toString() { + return "CIDRMatchEvaluator[" + "ip=" + ip + ", cidrs=" + Arrays.toString(cidrs) + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsDoubleEvaluator.java new file mode 100644 index 0000000000000..8250081a5ddd8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsDoubleEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}. + * This class is generated. Do not edit it. + */ +public final class AbsDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator fieldVal; + + public AbsDoubleEvaluator(EvalOperator.ExpressionEvaluator fieldVal) { + this.fieldVal = fieldVal; + } + + @Override + public Block eval(Page page) { + Block fieldValUncastBlock = fieldVal.eval(page); + if (fieldValUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock fieldValBlock = (DoubleBlock) fieldValUncastBlock; + DoubleVector fieldValVector = fieldValBlock.asVector(); + if (fieldValVector == null) { + return eval(page.getPositionCount(), fieldValBlock); + } + return eval(page.getPositionCount(), fieldValVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock fieldValBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (fieldValBlock.isNull(p) || fieldValBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Abs.process(fieldValBlock.getDouble(fieldValBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector fieldValVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Abs.process(fieldValVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AbsDoubleEvaluator[" + "fieldVal=" + fieldVal + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsIntEvaluator.java new file mode 100644 index 0000000000000..1282d3f7401d6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsIntEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}. + * This class is generated. Do not edit it. + */ +public final class AbsIntEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator fieldVal; + + public AbsIntEvaluator(EvalOperator.ExpressionEvaluator fieldVal) { + this.fieldVal = fieldVal; + } + + @Override + public Block eval(Page page) { + Block fieldValUncastBlock = fieldVal.eval(page); + if (fieldValUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock fieldValBlock = (IntBlock) fieldValUncastBlock; + IntVector fieldValVector = fieldValBlock.asVector(); + if (fieldValVector == null) { + return eval(page.getPositionCount(), fieldValBlock); + } + return eval(page.getPositionCount(), fieldValVector).asBlock(); + } + + public IntBlock eval(int positionCount, IntBlock fieldValBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (fieldValBlock.isNull(p) || fieldValBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendInt(Abs.process(fieldValBlock.getInt(fieldValBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public IntVector eval(int positionCount, IntVector fieldValVector) { + IntVector.Builder result = IntVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendInt(Abs.process(fieldValVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AbsIntEvaluator[" + "fieldVal=" + fieldVal + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsLongEvaluator.java new file mode 100644 index 0000000000000..3d87f8007d4ba --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsLongEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}. + * This class is generated. Do not edit it. + */ +public final class AbsLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator fieldVal; + + public AbsLongEvaluator(EvalOperator.ExpressionEvaluator fieldVal) { + this.fieldVal = fieldVal; + } + + @Override + public Block eval(Page page) { + Block fieldValUncastBlock = fieldVal.eval(page); + if (fieldValUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock fieldValBlock = (LongBlock) fieldValUncastBlock; + LongVector fieldValVector = fieldValBlock.asVector(); + if (fieldValVector == null) { + return eval(page.getPositionCount(), fieldValBlock); + } + return eval(page.getPositionCount(), fieldValVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock fieldValBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (fieldValBlock.isNull(p) || fieldValBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Abs.process(fieldValBlock.getLong(fieldValBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector fieldValVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Abs.process(fieldValVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AbsLongEvaluator[" + "fieldVal=" + fieldVal + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosEvaluator.java new file mode 100644 index 0000000000000..db47de5027f07 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Acos}. + * This class is generated. Do not edit it. + */ +public final class AcosEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public AcosEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Acos.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Acos.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AcosEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinEvaluator.java new file mode 100644 index 0000000000000..0c7cae266b348 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Asin}. + * This class is generated. Do not edit it. + */ +public final class AsinEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public AsinEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Asin.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Asin.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AsinEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Evaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Evaluator.java new file mode 100644 index 0000000000000..4ea7cb15c625c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Evaluator.java @@ -0,0 +1,80 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Atan2}. + * This class is generated. Do not edit it. + */ +public final class Atan2Evaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator y; + + private final EvalOperator.ExpressionEvaluator x; + + public Atan2Evaluator(EvalOperator.ExpressionEvaluator y, EvalOperator.ExpressionEvaluator x) { + this.y = y; + this.x = x; + } + + @Override + public Block eval(Page page) { + Block yUncastBlock = y.eval(page); + if (yUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock yBlock = (DoubleBlock) yUncastBlock; + Block xUncastBlock = x.eval(page); + if (xUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock xBlock = (DoubleBlock) xUncastBlock; + DoubleVector yVector = yBlock.asVector(); + if (yVector == null) { + return eval(page.getPositionCount(), yBlock, xBlock); + } + DoubleVector xVector = xBlock.asVector(); + if (xVector == null) { + return eval(page.getPositionCount(), yBlock, xBlock); + } + return eval(page.getPositionCount(), yVector, xVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock yBlock, DoubleBlock xBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (yBlock.isNull(p) || yBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (xBlock.isNull(p) || xBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Atan2.process(yBlock.getDouble(yBlock.getFirstValueIndex(p)), xBlock.getDouble(xBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector yVector, DoubleVector xVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Atan2.process(yVector.getDouble(p), xVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "Atan2Evaluator[" + "y=" + y + ", x=" + x + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanEvaluator.java new file mode 100644 index 0000000000000..fac99b790d262 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Atan}. + * This class is generated. Do not edit it. + */ +public final class AtanEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public AtanEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Atan.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Atan.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AtanEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToDoubleEvaluator.java new file mode 100644 index 0000000000000..95105fce34831 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToDoubleEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastIntToDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastIntToDoubleEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock vBlock = (IntBlock) vUncastBlock; + IntVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, IntBlock vBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Cast.castIntToDouble(vBlock.getInt(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, IntVector vVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Cast.castIntToDouble(vVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastIntToDoubleEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToLongEvaluator.java new file mode 100644 index 0000000000000..76f4bc3a89cb3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToLongEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastIntToLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastIntToLongEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock vBlock = (IntBlock) vUncastBlock; + IntVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public LongBlock eval(int positionCount, IntBlock vBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Cast.castIntToLong(vBlock.getInt(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, IntVector vVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Cast.castIntToLong(vVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastIntToLongEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..7b8bfc2b8d199 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToUnsignedLongEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastIntToUnsignedLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastIntToUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock vBlock = (IntBlock) vUncastBlock; + IntVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public LongBlock eval(int positionCount, IntBlock vBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Cast.castIntToUnsignedLong(vBlock.getInt(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, IntVector vVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Cast.castIntToUnsignedLong(vVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastIntToUnsignedLongEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToDoubleEvaluator.java new file mode 100644 index 0000000000000..22d3f4f5b8c48 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToDoubleEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastLongToDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastLongToDoubleEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock vBlock = (LongBlock) vUncastBlock; + LongVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, LongBlock vBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Cast.castLongToDouble(vBlock.getLong(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, LongVector vVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Cast.castLongToDouble(vVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastLongToDoubleEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..de215d3e22373 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToUnsignedLongEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastLongToUnsignedLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastLongToUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock vBlock = (LongBlock) vUncastBlock; + LongVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock vBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Cast.castLongToUnsignedLong(vBlock.getLong(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector vVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Cast.castLongToUnsignedLong(vVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastLongToUnsignedLongEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastUnsignedLongToDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastUnsignedLongToDoubleEvaluator.java new file mode 100644 index 0000000000000..d1e009c2a0b2e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastUnsignedLongToDoubleEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}. + * This class is generated. Do not edit it. + */ +public final class CastUnsignedLongToDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public CastUnsignedLongToDoubleEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock vBlock = (LongBlock) vUncastBlock; + LongVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, LongBlock vBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Cast.castUnsignedLongToDouble(vBlock.getLong(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, LongVector vVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Cast.castUnsignedLongToDouble(vVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CastUnsignedLongToDoubleEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosEvaluator.java new file mode 100644 index 0000000000000..226bca608e01e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cos}. + * This class is generated. Do not edit it. + */ +public final class CosEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public CosEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Cos.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Cos.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CosEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshEvaluator.java new file mode 100644 index 0000000000000..c0e3a1937bb26 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cosh}. + * This class is generated. Do not edit it. + */ +public final class CoshEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public CoshEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Cosh.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Cosh.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "CoshEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorDoubleEvaluator.java new file mode 100644 index 0000000000000..d7b5a1263e85d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorDoubleEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Floor}. + * This class is generated. Do not edit it. + */ +public final class FloorDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public FloorDoubleEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Floor.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Floor.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "FloorDoubleEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteEvaluator.java new file mode 100644 index 0000000000000..233c95aea3cfd --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsFinite}. + * This class is generated. Do not edit it. + */ +public final class IsFiniteEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public IsFiniteEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock valBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(IsFinite.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector valVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(IsFinite.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "IsFiniteEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteEvaluator.java new file mode 100644 index 0000000000000..b53623bc48514 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsInfinite}. + * This class is generated. Do not edit it. + */ +public final class IsInfiniteEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public IsInfiniteEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock valBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(IsInfinite.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector valVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(IsInfinite.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "IsInfiniteEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNEvaluator.java new file mode 100644 index 0000000000000..c947eb5126c45 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsNaN}. + * This class is generated. Do not edit it. + */ +public final class IsNaNEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public IsNaNEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock valBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(IsNaN.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector valVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(IsNaN.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "IsNaNEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10DoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10DoubleEvaluator.java new file mode 100644 index 0000000000000..5303aaed79e0a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10DoubleEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Log10}. + * This class is generated. Do not edit it. + */ +public final class Log10DoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public Log10DoubleEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Log10.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Log10.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "Log10DoubleEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10IntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10IntEvaluator.java new file mode 100644 index 0000000000000..86dcad75c1c9d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10IntEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Log10}. + * This class is generated. Do not edit it. + */ +public final class Log10IntEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public Log10IntEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock valBlock = (IntBlock) valUncastBlock; + IntVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, IntBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Log10.process(valBlock.getInt(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, IntVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Log10.process(valVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "Log10IntEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10LongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10LongEvaluator.java new file mode 100644 index 0000000000000..fe74f2d34b776 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10LongEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Log10}. + * This class is generated. Do not edit it. + */ +public final class Log10LongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public Log10LongEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, LongBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Log10.process(valBlock.getLong(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, LongVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Log10.process(valVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "Log10LongEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowDoubleEvaluator.java new file mode 100644 index 0000000000000..0ecd7f5455942 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowDoubleEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.function.scalar.math; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Pow}. + * This class is generated. Do not edit it. + */ +public final class PowDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator base; + + private final EvalOperator.ExpressionEvaluator exponent; + + public PowDoubleEvaluator(Source source, EvalOperator.ExpressionEvaluator base, + EvalOperator.ExpressionEvaluator exponent) { + this.warnings = new Warnings(source); + this.base = base; + this.exponent = exponent; + } + + @Override + public Block eval(Page page) { + Block baseUncastBlock = base.eval(page); + if (baseUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock baseBlock = (DoubleBlock) baseUncastBlock; + Block exponentUncastBlock = exponent.eval(page); + if (exponentUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock exponentBlock = (DoubleBlock) exponentUncastBlock; + DoubleVector baseVector = baseBlock.asVector(); + if (baseVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + DoubleVector exponentVector = exponentBlock.asVector(); + if (exponentVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + return eval(page.getPositionCount(), baseVector, exponentVector); + } + + public DoubleBlock eval(int positionCount, DoubleBlock baseBlock, DoubleBlock exponentBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (baseBlock.isNull(p) || baseBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (exponentBlock.isNull(p) || exponentBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendDouble(Pow.process(baseBlock.getDouble(baseBlock.getFirstValueIndex(p)), exponentBlock.getDouble(exponentBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public DoubleBlock eval(int positionCount, DoubleVector baseVector, DoubleVector exponentVector) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendDouble(Pow.process(baseVector.getDouble(p), exponentVector.getDouble(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "PowDoubleEvaluator[" + "base=" + base + ", exponent=" + exponent + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java new file mode 100644 index 0000000000000..65a20463c26e1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java @@ -0,0 +1,98 @@ +// 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.expression.function.scalar.math; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Pow}. + * This class is generated. Do not edit it. + */ +public final class PowIntEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator base; + + private final EvalOperator.ExpressionEvaluator exponent; + + public PowIntEvaluator(Source source, EvalOperator.ExpressionEvaluator base, + EvalOperator.ExpressionEvaluator exponent) { + this.warnings = new Warnings(source); + this.base = base; + this.exponent = exponent; + } + + @Override + public Block eval(Page page) { + Block baseUncastBlock = base.eval(page); + if (baseUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock baseBlock = (DoubleBlock) baseUncastBlock; + Block exponentUncastBlock = exponent.eval(page); + if (exponentUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock exponentBlock = (DoubleBlock) exponentUncastBlock; + DoubleVector baseVector = baseBlock.asVector(); + if (baseVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + DoubleVector exponentVector = exponentBlock.asVector(); + if (exponentVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + return eval(page.getPositionCount(), baseVector, exponentVector); + } + + public IntBlock eval(int positionCount, DoubleBlock baseBlock, DoubleBlock exponentBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (baseBlock.isNull(p) || baseBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (exponentBlock.isNull(p) || exponentBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Pow.processInt(baseBlock.getDouble(baseBlock.getFirstValueIndex(p)), exponentBlock.getDouble(exponentBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, DoubleVector baseVector, DoubleVector exponentVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Pow.processInt(baseVector.getDouble(p), exponentVector.getDouble(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "PowIntEvaluator[" + "base=" + base + ", exponent=" + exponent + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java new file mode 100644 index 0000000000000..99ee34f7ee770 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java @@ -0,0 +1,98 @@ +// 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.expression.function.scalar.math; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Pow}. + * This class is generated. Do not edit it. + */ +public final class PowLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator base; + + private final EvalOperator.ExpressionEvaluator exponent; + + public PowLongEvaluator(Source source, EvalOperator.ExpressionEvaluator base, + EvalOperator.ExpressionEvaluator exponent) { + this.warnings = new Warnings(source); + this.base = base; + this.exponent = exponent; + } + + @Override + public Block eval(Page page) { + Block baseUncastBlock = base.eval(page); + if (baseUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock baseBlock = (DoubleBlock) baseUncastBlock; + Block exponentUncastBlock = exponent.eval(page); + if (exponentUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock exponentBlock = (DoubleBlock) exponentUncastBlock; + DoubleVector baseVector = baseBlock.asVector(); + if (baseVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + DoubleVector exponentVector = exponentBlock.asVector(); + if (exponentVector == null) { + return eval(page.getPositionCount(), baseBlock, exponentBlock); + } + return eval(page.getPositionCount(), baseVector, exponentVector); + } + + public LongBlock eval(int positionCount, DoubleBlock baseBlock, DoubleBlock exponentBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (baseBlock.isNull(p) || baseBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (exponentBlock.isNull(p) || exponentBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Pow.processLong(baseBlock.getDouble(baseBlock.getFirstValueIndex(p)), exponentBlock.getDouble(exponentBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, DoubleVector baseVector, DoubleVector exponentVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Pow.processLong(baseVector.getDouble(p), exponentVector.getDouble(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "PowLongEvaluator[" + "base=" + base + ", exponent=" + exponent + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleEvaluator.java new file mode 100644 index 0000000000000..52a51ba610d38 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}. + * This class is generated. Do not edit it. + */ +public final class RoundDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator decimals; + + public RoundDoubleEvaluator(EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator decimals) { + this.val = val; + this.decimals = decimals; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + Block decimalsUncastBlock = decimals.eval(page); + if (decimalsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock decimalsBlock = (LongBlock) decimalsUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + LongVector decimalsVector = decimalsBlock.asVector(); + if (decimalsVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + return eval(page.getPositionCount(), valVector, decimalsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock, LongBlock decimalsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (decimalsBlock.isNull(p) || decimalsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Round.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)), decimalsBlock.getLong(decimalsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector, LongVector decimalsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Round.process(valVector.getDouble(p), decimalsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "RoundDoubleEvaluator[" + "val=" + val + ", decimals=" + decimals + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleNoDecimalsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleNoDecimalsEvaluator.java new file mode 100644 index 0000000000000..671aaf5f3d029 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleNoDecimalsEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}. + * This class is generated. Do not edit it. + */ +public final class RoundDoubleNoDecimalsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public RoundDoubleNoDecimalsEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Round.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Round.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "RoundDoubleNoDecimalsEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundIntEvaluator.java new file mode 100644 index 0000000000000..f178a571b7e9d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundIntEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}. + * This class is generated. Do not edit it. + */ +public final class RoundIntEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator decimals; + + public RoundIntEvaluator(EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator decimals) { + this.val = val; + this.decimals = decimals; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock valBlock = (IntBlock) valUncastBlock; + Block decimalsUncastBlock = decimals.eval(page); + if (decimalsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock decimalsBlock = (LongBlock) decimalsUncastBlock; + IntVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + LongVector decimalsVector = decimalsBlock.asVector(); + if (decimalsVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + return eval(page.getPositionCount(), valVector, decimalsVector).asBlock(); + } + + public IntBlock eval(int positionCount, IntBlock valBlock, LongBlock decimalsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (decimalsBlock.isNull(p) || decimalsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendInt(Round.process(valBlock.getInt(valBlock.getFirstValueIndex(p)), decimalsBlock.getLong(decimalsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public IntVector eval(int positionCount, IntVector valVector, LongVector decimalsVector) { + IntVector.Builder result = IntVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendInt(Round.process(valVector.getInt(p), decimalsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "RoundIntEvaluator[" + "val=" + val + ", decimals=" + decimals + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundLongEvaluator.java new file mode 100644 index 0000000000000..12f193fe216f3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundLongEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}. + * This class is generated. Do not edit it. + */ +public final class RoundLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator decimals; + + public RoundLongEvaluator(EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator decimals) { + this.val = val; + this.decimals = decimals; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + Block decimalsUncastBlock = decimals.eval(page); + if (decimalsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock decimalsBlock = (LongBlock) decimalsUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + LongVector decimalsVector = decimalsBlock.asVector(); + if (decimalsVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + return eval(page.getPositionCount(), valVector, decimalsVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock valBlock, LongBlock decimalsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (decimalsBlock.isNull(p) || decimalsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Round.process(valBlock.getLong(valBlock.getFirstValueIndex(p)), decimalsBlock.getLong(decimalsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector valVector, LongVector decimalsVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Round.process(valVector.getLong(p), decimalsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "RoundLongEvaluator[" + "val=" + val + ", decimals=" + decimals + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..1a8247ba34c80 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundUnsignedLongEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}. + * This class is generated. Do not edit it. + */ +public final class RoundUnsignedLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + private final EvalOperator.ExpressionEvaluator decimals; + + public RoundUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator val, + EvalOperator.ExpressionEvaluator decimals) { + this.val = val; + this.decimals = decimals; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + Block decimalsUncastBlock = decimals.eval(page); + if (decimalsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock decimalsBlock = (LongBlock) decimalsUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + LongVector decimalsVector = decimalsBlock.asVector(); + if (decimalsVector == null) { + return eval(page.getPositionCount(), valBlock, decimalsBlock); + } + return eval(page.getPositionCount(), valVector, decimalsVector).asBlock(); + } + + public LongBlock eval(int positionCount, LongBlock valBlock, LongBlock decimalsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (decimalsBlock.isNull(p) || decimalsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendLong(Round.processUnsignedLong(valBlock.getLong(valBlock.getFirstValueIndex(p)), decimalsBlock.getLong(decimalsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public LongVector eval(int positionCount, LongVector valVector, LongVector decimalsVector) { + LongVector.Builder result = LongVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(Round.processUnsignedLong(valVector.getLong(p), decimalsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "RoundUnsignedLongEvaluator[" + "val=" + val + ", decimals=" + decimals + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinEvaluator.java new file mode 100644 index 0000000000000..aa1649021be09 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sin}. + * This class is generated. Do not edit it. + */ +public final class SinEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public SinEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sin.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sin.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SinEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhEvaluator.java new file mode 100644 index 0000000000000..da5ce241e645c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sinh}. + * This class is generated. Do not edit it. + */ +public final class SinhEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public SinhEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sinh.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sinh.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SinhEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtDoubleEvaluator.java new file mode 100644 index 0000000000000..3efce9f4f0f98 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtDoubleEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sqrt}. + * This class is generated. Do not edit it. + */ +public final class SqrtDoubleEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public SqrtDoubleEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sqrt.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sqrt.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SqrtDoubleEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtIntEvaluator.java new file mode 100644 index 0000000000000..a4ab65c58f151 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtIntEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sqrt}. + * This class is generated. Do not edit it. + */ +public final class SqrtIntEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public SqrtIntEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock valBlock = (IntBlock) valUncastBlock; + IntVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, IntBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sqrt.process(valBlock.getInt(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, IntVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sqrt.process(valVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SqrtIntEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtLongEvaluator.java new file mode 100644 index 0000000000000..28939040d0dfc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtLongEvaluator.java @@ -0,0 +1,66 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sqrt}. + * This class is generated. Do not edit it. + */ +public final class SqrtLongEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public SqrtLongEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock valBlock = (LongBlock) valUncastBlock; + LongVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, LongBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sqrt.process(valBlock.getLong(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, LongVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sqrt.process(valVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SqrtLongEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanEvaluator.java new file mode 100644 index 0000000000000..0c8de1fe98abc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Tan}. + * This class is generated. Do not edit it. + */ +public final class TanEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public TanEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Tan.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Tan.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "TanEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhEvaluator.java new file mode 100644 index 0000000000000..207ae56fb227d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.math; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Tanh}. + * This class is generated. Do not edit it. + */ +public final class TanhEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public TanhEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock valBlock = (DoubleBlock) valUncastBlock; + DoubleVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock valBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Tanh.process(valBlock.getDouble(valBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector valVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Tanh.process(valVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "TanhEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java new file mode 100644 index 0000000000000..775723ee66c0b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgDoubleEvaluator.java @@ -0,0 +1,79 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvAvg}. + * This class is generated. Do not edit it. + */ +public final class MvAvgDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvAvgDoubleEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvAvg"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java new file mode 100644 index 0000000000000..655d9fdbe97a4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgIntEvaluator.java @@ -0,0 +1,136 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvAvg}. + * This class is generated. Do not edit it. + */ +public final class MvAvgIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvAvgIntEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvAvg"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + int value = v.getInt(first); + double result = MvAvg.single(value); + builder.appendDouble(result); + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + int value = v.getInt(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + int value = v.getInt(first); + double result = MvAvg.single(value); + values[p] = result; + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + int value = v.getInt(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Block evalSingleValuedNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + int value = v.getInt(first); + double result = MvAvg.single(value); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Vector evalSingleValuedNotNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + int value = v.getInt(first); + double result = MvAvg.single(value); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java new file mode 100644 index 0000000000000..95dee758eaa32 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgLongEvaluator.java @@ -0,0 +1,136 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvAvg}. + * This class is generated. Do not edit it. + */ +public final class MvAvgLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvAvgLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvAvg"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + long value = v.getLong(first); + double result = MvAvg.single(value); + builder.appendDouble(result); + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + long value = v.getLong(first); + double result = MvAvg.single(value); + values[p] = result; + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvAvg.process(work, value); + } + double result = MvAvg.finish(work, valueCount); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Block evalSingleValuedNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + long value = v.getLong(first); + double result = MvAvg.single(value); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Vector evalSingleValuedNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + long value = v.getLong(first); + double result = MvAvg.single(value); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..97845cd82e105 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgUnsignedLongEvaluator.java @@ -0,0 +1,136 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvAvg}. + * This class is generated. Do not edit it. + */ +public final class MvAvgUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvAvgUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvAvg"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + long value = v.getLong(first); + double result = MvAvg.singleUnsignedLong(value); + builder.appendDouble(result); + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvAvg.processUnsignedLong(work, value); + } + double result = MvAvg.finish(work, valueCount); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + if (valueCount == 1) { + long value = v.getLong(first); + double result = MvAvg.singleUnsignedLong(value); + values[p] = result; + continue; + } + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvAvg.processUnsignedLong(work, value); + } + double result = MvAvg.finish(work, valueCount); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Block evalSingleValuedNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + long value = v.getLong(first); + double result = MvAvg.singleUnsignedLong(value); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing only single valued fields. + */ + @Override + public Vector evalSingleValuedNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + assert valueCount == 1; + int first = v.getFirstValueIndex(p); + long value = v.getLong(first); + double result = MvAvg.singleUnsignedLong(value); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java new file mode 100644 index 0000000000000..a2cf3af0bd9e2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBooleanEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMax}. + * This class is generated. Do not edit it. + */ +public final class MvMaxBooleanEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMaxBooleanEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMax"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + boolean value = v.getBoolean(first); + for (int i = first + 1; i < end; i++) { + boolean next = v.getBoolean(i); + value = MvMax.process(value, next); + } + boolean result = value; + builder.appendBoolean(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + boolean value = v.getBoolean(first); + for (int i = first + 1; i < end; i++) { + boolean next = v.getBoolean(i); + value = MvMax.process(value, next); + } + boolean result = value; + values[p] = result; + } + return new BooleanArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + boolean result = v.getBoolean(first + idx); + builder.appendBoolean(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + boolean result = v.getBoolean(first + idx); + values[p] = result; + } + return new BooleanArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java new file mode 100644 index 0000000000000..b96d9830b9cdc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxBytesRefEvaluator.java @@ -0,0 +1,133 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMax}. + * This class is generated. Do not edit it. + */ +public final class MvMaxBytesRefEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMaxBytesRefEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMax"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + BytesRef value = v.getBytesRef(first, firstScratch); + for (int i = first + 1; i < end; i++) { + BytesRef next = v.getBytesRef(i, nextScratch); + MvMax.process(value, next); + } + BytesRef result = value; + builder.appendBytesRef(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + BytesRef value = v.getBytesRef(first, firstScratch); + for (int i = first + 1; i < end; i++) { + BytesRef next = v.getBytesRef(i, nextScratch); + MvMax.process(value, next); + } + BytesRef result = value; + values.append(result); + } + return new BytesRefArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + BytesRef result = v.getBytesRef(first + idx, firstScratch); + builder.appendBytesRef(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + BytesRef result = v.getBytesRef(first + idx, firstScratch); + values.append(result); + } + return new BytesRefArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java new file mode 100644 index 0000000000000..0465808883020 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxDoubleEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMax}. + * This class is generated. Do not edit it. + */ +public final class MvMaxDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMaxDoubleEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMax"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + double value = v.getDouble(first); + for (int i = first + 1; i < end; i++) { + double next = v.getDouble(i); + value = MvMax.process(value, next); + } + double result = value; + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + double value = v.getDouble(first); + for (int i = first + 1; i < end; i++) { + double next = v.getDouble(i); + value = MvMax.process(value, next); + } + double result = value; + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + double result = v.getDouble(first + idx); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + double result = v.getDouble(first + idx); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java new file mode 100644 index 0000000000000..e166fa38a1eae --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxIntEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMax}. + * This class is generated. Do not edit it. + */ +public final class MvMaxIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMaxIntEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMax"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + int value = v.getInt(first); + for (int i = first + 1; i < end; i++) { + int next = v.getInt(i); + value = MvMax.process(value, next); + } + int result = value; + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + int value = v.getInt(first); + for (int i = first + 1; i < end; i++) { + int next = v.getInt(i); + value = MvMax.process(value, next); + } + int result = value; + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + int result = v.getInt(first + idx); + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + int result = v.getInt(first + idx); + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java new file mode 100644 index 0000000000000..fe72bdd726c20 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxLongEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMax}. + * This class is generated. Do not edit it. + */ +public final class MvMaxLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMaxLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMax"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvMax.process(value, next); + } + long result = value; + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvMax.process(value, next); + } + long result = value; + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + long result = v.getLong(first + idx); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMax.ascendingIndex(valueCount); + long result = v.getLong(first + idx); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java new file mode 100644 index 0000000000000..266b2b8e3d4f9 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianDoubleEvaluator.java @@ -0,0 +1,78 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMedian}. + * This class is generated. Do not edit it. + */ +public final class MvMedianDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMedianDoubleEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMedian"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + MvMedian.Doubles work = new MvMedian.Doubles(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvMedian.process(work, value); + } + double result = MvMedian.finish(work); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + MvMedian.Doubles work = new MvMedian.Doubles(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvMedian.process(work, value); + } + double result = MvMedian.finish(work); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java new file mode 100644 index 0000000000000..d20f953a6d34c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianIntEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMedian}. + * This class is generated. Do not edit it. + */ +public final class MvMedianIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMedianIntEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMedian"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + MvMedian.Ints work = new MvMedian.Ints(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + int value = v.getInt(i); + MvMedian.process(work, value); + } + int result = MvMedian.finish(work); + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + MvMedian.Ints work = new MvMedian.Ints(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + int value = v.getInt(i); + MvMedian.process(work, value); + } + int result = MvMedian.finish(work); + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + MvMedian.Ints work = new MvMedian.Ints(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int result = MvMedian.ascending(v, first, valueCount); + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + MvMedian.Ints work = new MvMedian.Ints(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int result = MvMedian.ascending(v, first, valueCount); + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java new file mode 100644 index 0000000000000..710d79e8aa6f6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianLongEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMedian}. + * This class is generated. Do not edit it. + */ +public final class MvMedianLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMedianLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMedian"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvMedian.process(work, value); + } + long result = MvMedian.finish(work); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvMedian.process(work, value); + } + long result = MvMedian.finish(work); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + long result = MvMedian.ascending(v, first, valueCount); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + long result = MvMedian.ascending(v, first, valueCount); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..cfefbb492d53e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianUnsignedLongEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMedian}. + * This class is generated. Do not edit it. + */ +public final class MvMedianUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMedianUnsignedLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMedian"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvMedian.processUnsignedLong(work, value); + } + long result = MvMedian.finishUnsignedLong(work); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + long value = v.getLong(i); + MvMedian.processUnsignedLong(work, value); + } + long result = MvMedian.finishUnsignedLong(work); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + long result = MvMedian.ascendingUnsignedLong(v, first, valueCount); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + MvMedian.Longs work = new MvMedian.Longs(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + long result = MvMedian.ascendingUnsignedLong(v, first, valueCount); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java new file mode 100644 index 0000000000000..5e3697243d9cb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBooleanEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMin}. + * This class is generated. Do not edit it. + */ +public final class MvMinBooleanEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMinBooleanEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMin"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + boolean value = v.getBoolean(first); + for (int i = first + 1; i < end; i++) { + boolean next = v.getBoolean(i); + value = MvMin.process(value, next); + } + boolean result = value; + builder.appendBoolean(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + boolean value = v.getBoolean(first); + for (int i = first + 1; i < end; i++) { + boolean next = v.getBoolean(i); + value = MvMin.process(value, next); + } + boolean result = value; + values[p] = result; + } + return new BooleanArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + BooleanBlock.Builder builder = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + boolean result = v.getBoolean(first + idx); + builder.appendBoolean(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + BooleanBlock v = (BooleanBlock) fieldVal; + int positionCount = v.getPositionCount(); + boolean[] values = new boolean[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + boolean result = v.getBoolean(first + idx); + values[p] = result; + } + return new BooleanArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java new file mode 100644 index 0000000000000..74173a3d18e5b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinBytesRefEvaluator.java @@ -0,0 +1,133 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BytesRefArray; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefArrayVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMin}. + * This class is generated. Do not edit it. + */ +public final class MvMinBytesRefEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMinBytesRefEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMin"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + BytesRef value = v.getBytesRef(first, firstScratch); + for (int i = first + 1; i < end; i++) { + BytesRef next = v.getBytesRef(i, nextScratch); + MvMin.process(value, next); + } + BytesRef result = value; + builder.appendBytesRef(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + BytesRef value = v.getBytesRef(first, firstScratch); + for (int i = first + 1; i < end; i++) { + BytesRef next = v.getBytesRef(i, nextScratch); + MvMin.process(value, next); + } + BytesRef result = value; + values.append(result); + } + return new BytesRefArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + BytesRef result = v.getBytesRef(first + idx, firstScratch); + builder.appendBytesRef(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + BytesRefBlock v = (BytesRefBlock) fieldVal; + int positionCount = v.getPositionCount(); + BytesRefArray values = new BytesRefArray(positionCount, BigArrays.NON_RECYCLING_INSTANCE); + BytesRef firstScratch = new BytesRef(); + BytesRef nextScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + BytesRef result = v.getBytesRef(first + idx, firstScratch); + values.append(result); + } + return new BytesRefArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java new file mode 100644 index 0000000000000..5fd2d66a2afce --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinDoubleEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMin}. + * This class is generated. Do not edit it. + */ +public final class MvMinDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMinDoubleEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMin"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + double value = v.getDouble(first); + for (int i = first + 1; i < end; i++) { + double next = v.getDouble(i); + value = MvMin.process(value, next); + } + double result = value; + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + double value = v.getDouble(first); + for (int i = first + 1; i < end; i++) { + double next = v.getDouble(i); + value = MvMin.process(value, next); + } + double result = value; + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + double result = v.getDouble(first + idx); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + double result = v.getDouble(first + idx); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java new file mode 100644 index 0000000000000..37d3b5c98778b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinIntEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMin}. + * This class is generated. Do not edit it. + */ +public final class MvMinIntEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMinIntEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMin"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + int value = v.getInt(first); + for (int i = first + 1; i < end; i++) { + int next = v.getInt(i); + value = MvMin.process(value, next); + } + int result = value; + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + int value = v.getInt(first); + for (int i = first + 1; i < end; i++) { + int next = v.getInt(i); + value = MvMin.process(value, next); + } + int result = value; + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + int result = v.getInt(first + idx); + builder.appendInt(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + int[] values = new int[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + int result = v.getInt(first + idx); + values[p] = result; + } + return new IntArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java new file mode 100644 index 0000000000000..f0f0734e8d176 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinLongEvaluator.java @@ -0,0 +1,122 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvMin}. + * This class is generated. Do not edit it. + */ +public final class MvMinLongEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvMinLongEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvMin"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvMin.process(value, next); + } + long result = value; + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + if (fieldVal.mvOrdering() == Block.MvOrdering.ASCENDING) { + return evalAscendingNotNullable(fieldVal); + } + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvMin.process(value, next); + } + long result = value; + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Block evalAscendingNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + long result = v.getLong(first + idx); + builder.appendLong(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field and all multivalued fields are in ascending order. + */ + private Vector evalAscendingNotNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + long[] values = new long[positionCount]; + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int idx = MvMin.ascendingIndex(valueCount); + long result = v.getLong(first + idx); + values[p] = result; + } + return new LongArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java new file mode 100644 index 0000000000000..e945863404fa7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumDoubleEvaluator.java @@ -0,0 +1,79 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvSum}. + * This class is generated. Do not edit it. + */ +public final class MvSumDoubleEvaluator extends AbstractMultivalueFunction.AbstractEvaluator { + public MvSumDoubleEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + public String name() { + return "MvSum"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + DoubleBlock.Builder builder = DoubleBlock.newBlockBuilder(positionCount); + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvSum.process(work, value); + } + double result = MvSum.finish(work); + builder.appendDouble(result); + } + return builder.build(); + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Vector evalNotNullable(Block fieldVal) { + DoubleBlock v = (DoubleBlock) fieldVal; + int positionCount = v.getPositionCount(); + double[] values = new double[positionCount]; + CompensatedSum work = new CompensatedSum(); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + for (int i = first; i < end; i++) { + double value = v.getDouble(i); + MvSum.process(work, value); + } + double result = MvSum.finish(work); + values[p] = result; + } + return new DoubleArrayVector(values, positionCount); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java new file mode 100644 index 0000000000000..e1217cae07ec3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumIntEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvSum}. + * This class is generated. Do not edit it. + */ +public final class MvSumIntEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { + private final Warnings warnings; + + public MvSumIntEvaluator(Source source, EvalOperator.ExpressionEvaluator field) { + super(field); + this.warnings = new Warnings(source); + } + + @Override + public String name() { + return "MvSum"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + IntBlock v = (IntBlock) fieldVal; + int positionCount = v.getPositionCount(); + IntBlock.Builder builder = IntBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + try { + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + int value = v.getInt(first); + for (int i = first + 1; i < end; i++) { + int next = v.getInt(i); + value = MvSum.process(value, next); + } + int result = value; + builder.appendInt(result); + } catch (ArithmeticException e) { + warnings.registerException(e); + builder.appendNull(); + } + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java new file mode 100644 index 0000000000000..4f5c301448b43 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumLongEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvSum}. + * This class is generated. Do not edit it. + */ +public final class MvSumLongEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { + private final Warnings warnings; + + public MvSumLongEvaluator(Source source, EvalOperator.ExpressionEvaluator field) { + super(field); + this.warnings = new Warnings(source); + } + + @Override + public String name() { + return "MvSum"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + try { + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvSum.process(value, next); + } + long result = value; + builder.appendLong(result); + } catch (ArithmeticException e) { + warnings.registerException(e); + builder.appendNull(); + } + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java new file mode 100644 index 0000000000000..6e78f1e851921 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumUnsignedLongEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.function.scalar.multivalue; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link MvSum}. + * This class is generated. Do not edit it. + */ +public final class MvSumUnsignedLongEvaluator extends AbstractMultivalueFunction.AbstractNullableEvaluator { + private final Warnings warnings; + + public MvSumUnsignedLongEvaluator(Source source, EvalOperator.ExpressionEvaluator field) { + super(field); + this.warnings = new Warnings(source); + } + + @Override + public String name() { + return "MvSum"; + } + + /** + * Evaluate blocks containing at least one multivalued field. + */ + @Override + public Block evalNullable(Block fieldVal) { + LongBlock v = (LongBlock) fieldVal; + int positionCount = v.getPositionCount(); + LongBlock.Builder builder = LongBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + int valueCount = v.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + try { + int first = v.getFirstValueIndex(p); + int end = first + valueCount; + long value = v.getLong(first); + for (int i = first + 1; i < end; i++) { + long next = v.getLong(i); + value = MvSum.processUnsignedLong(value, next); + } + long result = value; + builder.appendLong(result); + } catch (ArithmeticException e) { + warnings.registerException(e); + builder.appendNull(); + } + } + return builder.build(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatEvaluator.java new file mode 100644 index 0000000000000..0d0d9dd23091e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import java.util.Arrays; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Concat}. + * This class is generated. Do not edit it. + */ +public final class ConcatEvaluator implements EvalOperator.ExpressionEvaluator { + private final BytesRefBuilder scratch; + + private final EvalOperator.ExpressionEvaluator[] values; + + public ConcatEvaluator(BytesRefBuilder scratch, EvalOperator.ExpressionEvaluator[] values) { + this.scratch = scratch; + this.values = values; + } + + @Override + public Block eval(Page page) { + BytesRefBlock[] valuesBlocks = new BytesRefBlock[values.length]; + for (int i = 0; i < valuesBlocks.length; i++) { + Block block = values[i].eval(page); + if (block.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + valuesBlocks[i] = (BytesRefBlock) block; + } + BytesRefVector[] valuesVectors = new BytesRefVector[values.length]; + for (int i = 0; i < valuesBlocks.length; i++) { + valuesVectors[i] = valuesBlocks[i].asVector(); + if (valuesVectors[i] == null) { + return eval(page.getPositionCount(), valuesBlocks); + } + } + return eval(page.getPositionCount(), valuesVectors).asBlock(); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock[] valuesBlocks) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef[] valuesValues = new BytesRef[values.length]; + BytesRef[] valuesScratch = new BytesRef[values.length]; + for (int i = 0; i < values.length; i++) { + valuesScratch[i] = new BytesRef(); + } + position: for (int p = 0; p < positionCount; p++) { + for (int i = 0; i < valuesBlocks.length; i++) { + if (valuesBlocks[i].isNull(p) || valuesBlocks[i].getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + } + // unpack valuesBlocks into valuesValues + for (int i = 0; i < valuesBlocks.length; i++) { + int o = valuesBlocks[i].getFirstValueIndex(p); + valuesValues[i] = valuesBlocks[i].getBytesRef(o, valuesScratch[i]); + } + result.appendBytesRef(Concat.process(scratch, valuesValues)); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, BytesRefVector[] valuesVectors) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + BytesRef[] valuesValues = new BytesRef[values.length]; + BytesRef[] valuesScratch = new BytesRef[values.length]; + for (int i = 0; i < values.length; i++) { + valuesScratch[i] = new BytesRef(); + } + position: for (int p = 0; p < positionCount; p++) { + // unpack valuesVectors into valuesValues + for (int i = 0; i < valuesVectors.length; i++) { + valuesValues[i] = valuesVectors[i].getBytesRef(p, valuesScratch[i]); + } + result.appendBytesRef(Concat.process(scratch, valuesValues)); + } + return result.build(); + } + + @Override + public String toString() { + return "ConcatEvaluator[" + "values=" + Arrays.toString(values) + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthEvaluator.java new file mode 100644 index 0000000000000..c32ebf511dc99 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthEvaluator.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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Length}. + * This class is generated. Do not edit it. + */ +public final class LengthEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public LengthEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock valBlock = (BytesRefBlock) valUncastBlock; + BytesRefVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public IntBlock eval(int positionCount, BytesRefBlock valBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendInt(Length.process(valBlock.getBytesRef(valBlock.getFirstValueIndex(p), valScratch))); + } + return result.build(); + } + + public IntVector eval(int positionCount, BytesRefVector valVector) { + IntVector.Builder result = IntVector.newVectorBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendInt(Length.process(valVector.getBytesRef(p, valScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "LengthEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitSingleByteEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitSingleByteEvaluator.java new file mode 100644 index 0000000000000..5f721c3d8ad88 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitSingleByteEvaluator.java @@ -0,0 +1,74 @@ +// 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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Split}. + * This class is generated. Do not edit it. + */ +public final class SplitSingleByteEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator str; + + private final byte delim; + + private final BytesRef scratch; + + public SplitSingleByteEvaluator(EvalOperator.ExpressionEvaluator str, byte delim, + BytesRef scratch) { + this.str = str; + this.delim = delim; + this.scratch = scratch; + } + + @Override + public Block eval(Page page) { + Block strUncastBlock = str.eval(page); + if (strUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock strBlock = (BytesRefBlock) strUncastBlock; + BytesRefVector strVector = strBlock.asVector(); + if (strVector == null) { + return eval(page.getPositionCount(), strBlock); + } + return eval(page.getPositionCount(), strVector); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (strBlock.isNull(p) || strBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + Split.process(result, strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), delim, scratch); + } + return result.build(); + } + + public BytesRefBlock eval(int positionCount, BytesRefVector strVector) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + Split.process(result, strVector.getBytesRef(p, strScratch), delim, scratch); + } + return result.build(); + } + + @Override + public String toString() { + return "SplitSingleByteEvaluator[" + "str=" + str + ", delim=" + delim + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitVariableEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitVariableEvaluator.java new file mode 100644 index 0000000000000..090f580b8ce06 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitVariableEvaluator.java @@ -0,0 +1,90 @@ +// 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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Split}. + * This class is generated. Do not edit it. + */ +public final class SplitVariableEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator str; + + private final EvalOperator.ExpressionEvaluator delim; + + private final BytesRef scratch; + + public SplitVariableEvaluator(EvalOperator.ExpressionEvaluator str, + EvalOperator.ExpressionEvaluator delim, BytesRef scratch) { + this.str = str; + this.delim = delim; + this.scratch = scratch; + } + + @Override + public Block eval(Page page) { + Block strUncastBlock = str.eval(page); + if (strUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock strBlock = (BytesRefBlock) strUncastBlock; + Block delimUncastBlock = delim.eval(page); + if (delimUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock delimBlock = (BytesRefBlock) delimUncastBlock; + BytesRefVector strVector = strBlock.asVector(); + if (strVector == null) { + return eval(page.getPositionCount(), strBlock, delimBlock); + } + BytesRefVector delimVector = delimBlock.asVector(); + if (delimVector == null) { + return eval(page.getPositionCount(), strBlock, delimBlock); + } + return eval(page.getPositionCount(), strVector, delimVector); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock, BytesRefBlock delimBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + BytesRef delimScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (strBlock.isNull(p) || strBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (delimBlock.isNull(p) || delimBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + Split.process(result, strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), delimBlock.getBytesRef(delimBlock.getFirstValueIndex(p), delimScratch), scratch); + } + return result.build(); + } + + public BytesRefBlock eval(int positionCount, BytesRefVector strVector, + BytesRefVector delimVector) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + BytesRef delimScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + Split.process(result, strVector.getBytesRef(p, strScratch), delimVector.getBytesRef(p, delimScratch), scratch); + } + return result.build(); + } + + @Override + public String toString() { + return "SplitVariableEvaluator[" + "str=" + str + ", delim=" + delim + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithEvaluator.java new file mode 100644 index 0000000000000..2fe2c13ca0659 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithEvaluator.java @@ -0,0 +1,89 @@ +// 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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link StartsWith}. + * This class is generated. Do not edit it. + */ +public final class StartsWithEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator str; + + private final EvalOperator.ExpressionEvaluator prefix; + + public StartsWithEvaluator(EvalOperator.ExpressionEvaluator str, + EvalOperator.ExpressionEvaluator prefix) { + this.str = str; + this.prefix = prefix; + } + + @Override + public Block eval(Page page) { + Block strUncastBlock = str.eval(page); + if (strUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock strBlock = (BytesRefBlock) strUncastBlock; + Block prefixUncastBlock = prefix.eval(page); + if (prefixUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock prefixBlock = (BytesRefBlock) prefixUncastBlock; + BytesRefVector strVector = strBlock.asVector(); + if (strVector == null) { + return eval(page.getPositionCount(), strBlock, prefixBlock); + } + BytesRefVector prefixVector = prefixBlock.asVector(); + if (prefixVector == null) { + return eval(page.getPositionCount(), strBlock, prefixBlock); + } + return eval(page.getPositionCount(), strVector, prefixVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock strBlock, BytesRefBlock prefixBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + BytesRef prefixScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (strBlock.isNull(p) || strBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (prefixBlock.isNull(p) || prefixBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(StartsWith.process(strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), prefixBlock.getBytesRef(prefixBlock.getFirstValueIndex(p), prefixScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector strVector, + BytesRefVector prefixVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + BytesRef prefixScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(StartsWith.process(strVector.getBytesRef(p, strScratch), prefixVector.getBytesRef(p, prefixScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "StartsWithEvaluator[" + "str=" + str + ", prefix=" + prefix + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringEvaluator.java new file mode 100644 index 0000000000000..75a35bee93db8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringEvaluator.java @@ -0,0 +1,104 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License +// 2.0; you may not use this file except in compliance with the Elastic License +// 2.0. +package org.elasticsearch.xpack.esql.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Substring}. + * This class is generated. Do not edit it. + */ +public final class SubstringEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator str; + + private final EvalOperator.ExpressionEvaluator start; + + private final EvalOperator.ExpressionEvaluator length; + + public SubstringEvaluator(EvalOperator.ExpressionEvaluator str, + EvalOperator.ExpressionEvaluator start, EvalOperator.ExpressionEvaluator length) { + this.str = str; + this.start = start; + this.length = length; + } + + @Override + public Block eval(Page page) { + Block strUncastBlock = str.eval(page); + if (strUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock strBlock = (BytesRefBlock) strUncastBlock; + Block startUncastBlock = start.eval(page); + if (startUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock startBlock = (IntBlock) startUncastBlock; + Block lengthUncastBlock = length.eval(page); + if (lengthUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lengthBlock = (IntBlock) lengthUncastBlock; + BytesRefVector strVector = strBlock.asVector(); + if (strVector == null) { + return eval(page.getPositionCount(), strBlock, startBlock, lengthBlock); + } + IntVector startVector = startBlock.asVector(); + if (startVector == null) { + return eval(page.getPositionCount(), strBlock, startBlock, lengthBlock); + } + IntVector lengthVector = lengthBlock.asVector(); + if (lengthVector == null) { + return eval(page.getPositionCount(), strBlock, startBlock, lengthBlock); + } + return eval(page.getPositionCount(), strVector, startVector, lengthVector).asBlock(); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock, IntBlock startBlock, + IntBlock lengthBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (strBlock.isNull(p) || strBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (startBlock.isNull(p) || startBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (lengthBlock.isNull(p) || lengthBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBytesRef(Substring.process(strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), startBlock.getInt(startBlock.getFirstValueIndex(p)), lengthBlock.getInt(lengthBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, BytesRefVector strVector, IntVector startVector, + IntVector lengthVector) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBytesRef(Substring.process(strVector.getBytesRef(p, strScratch), startVector.getInt(p), lengthVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SubstringEvaluator[" + "str=" + str + ", start=" + start + ", length=" + length + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringNoLengthEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringNoLengthEvaluator.java new file mode 100644 index 0000000000000..71fb35a06dfa3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringNoLengthEvaluator.java @@ -0,0 +1,86 @@ +// 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.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Substring}. + * This class is generated. Do not edit it. + */ +public final class SubstringNoLengthEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator str; + + private final EvalOperator.ExpressionEvaluator start; + + public SubstringNoLengthEvaluator(EvalOperator.ExpressionEvaluator str, + EvalOperator.ExpressionEvaluator start) { + this.str = str; + this.start = start; + } + + @Override + public Block eval(Page page) { + Block strUncastBlock = str.eval(page); + if (strUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock strBlock = (BytesRefBlock) strUncastBlock; + Block startUncastBlock = start.eval(page); + if (startUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock startBlock = (IntBlock) startUncastBlock; + BytesRefVector strVector = strBlock.asVector(); + if (strVector == null) { + return eval(page.getPositionCount(), strBlock, startBlock); + } + IntVector startVector = startBlock.asVector(); + if (startVector == null) { + return eval(page.getPositionCount(), strBlock, startBlock); + } + return eval(page.getPositionCount(), strVector, startVector).asBlock(); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock, IntBlock startBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (strBlock.isNull(p) || strBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (startBlock.isNull(p) || startBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBytesRef(Substring.process(strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), startBlock.getInt(startBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, BytesRefVector strVector, IntVector startVector) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + BytesRef strScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBytesRef(Substring.process(strVector.getBytesRef(p, strScratch), startVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SubstringNoLengthEvaluator[" + "str=" + str + ", start=" + start + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimEvaluator.java new file mode 100644 index 0000000000000..d276116afbf55 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimEvaluator.java @@ -0,0 +1,67 @@ +// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one +// or more contributor license agreements. Licensed under the Elastic License +// 2.0; you may not use this file except in compliance with the Elastic License +// 2.0. +package org.elasticsearch.xpack.esql.expression.function.scalar.string; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Trim}. + * This class is generated. Do not edit it. + */ +public final class TrimEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator val; + + public TrimEvaluator(EvalOperator.ExpressionEvaluator val) { + this.val = val; + } + + @Override + public Block eval(Page page) { + Block valUncastBlock = val.eval(page); + if (valUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock valBlock = (BytesRefBlock) valUncastBlock; + BytesRefVector valVector = valBlock.asVector(); + if (valVector == null) { + return eval(page.getPositionCount(), valBlock); + } + return eval(page.getPositionCount(), valVector).asBlock(); + } + + public BytesRefBlock eval(int positionCount, BytesRefBlock valBlock) { + BytesRefBlock.Builder result = BytesRefBlock.newBlockBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (valBlock.isNull(p) || valBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBytesRef(Trim.process(valBlock.getBytesRef(valBlock.getFirstValueIndex(p), valScratch))); + } + return result.build(); + } + + public BytesRefVector eval(int positionCount, BytesRefVector valVector) { + BytesRefVector.Builder result = BytesRefVector.newVectorBuilder(positionCount); + BytesRef valScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBytesRef(Trim.process(valVector.getBytesRef(p, valScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "TrimEvaluator[" + "val=" + val + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/logical/NotEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/logical/NotEvaluator.java new file mode 100644 index 0000000000000..ab533cdb01299 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/logical/NotEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.predicate.logical; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Not}. + * This class is generated. Do not edit it. + */ +public final class NotEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public NotEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BooleanBlock vBlock = (BooleanBlock) vUncastBlock; + BooleanVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BooleanBlock vBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Not.process(vBlock.getBoolean(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BooleanVector vVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Not.process(vVector.getBoolean(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddDoublesEvaluator.java new file mode 100644 index 0000000000000..af04a4c68e021 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddDoublesEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Add}. + * This class is generated. Do not edit it. + */ +public final class AddDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public AddDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Add.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Add.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "AddDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddIntsEvaluator.java new file mode 100644 index 0000000000000..35d80b4604c74 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddIntsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Add}. + * This class is generated. Do not edit it. + */ +public final class AddIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public AddIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public IntBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Add.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Add.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "AddIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddLongsEvaluator.java new file mode 100644 index 0000000000000..0a66f66d8da7e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Add}. + * This class is generated. Do not edit it. + */ +public final class AddLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public AddLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Add.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Add.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "AddLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddUnsignedLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddUnsignedLongsEvaluator.java new file mode 100644 index 0000000000000..1bf3b8fa3ddb7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddUnsignedLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Add}. + * This class is generated. Do not edit it. + */ +public final class AddUnsignedLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public AddUnsignedLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Add.processUnsignedLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Add.processUnsignedLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "AddUnsignedLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivDoublesEvaluator.java new file mode 100644 index 0000000000000..4b13bc1c5c072 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivDoublesEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Div}. + * This class is generated. Do not edit it. + */ +public final class DivDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public DivDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Div.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Div.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "DivDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivIntsEvaluator.java new file mode 100644 index 0000000000000..5cdc73fbd99bb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivIntsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Div}. + * This class is generated. Do not edit it. + */ +public final class DivIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public DivIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public IntBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Div.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Div.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DivIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivLongsEvaluator.java new file mode 100644 index 0000000000000..7a7311152f924 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Div}. + * This class is generated. Do not edit it. + */ +public final class DivLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public DivLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Div.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Div.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DivLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivUnsignedLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivUnsignedLongsEvaluator.java new file mode 100644 index 0000000000000..de0ce3aafb46f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivUnsignedLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Div}. + * This class is generated. Do not edit it. + */ +public final class DivUnsignedLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public DivUnsignedLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Div.processUnsignedLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Div.processUnsignedLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "DivUnsignedLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModDoublesEvaluator.java new file mode 100644 index 0000000000000..0698e816d8a86 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModDoublesEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mod}. + * This class is generated. Do not edit it. + */ +public final class ModDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public ModDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Mod.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Mod.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "ModDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModIntsEvaluator.java new file mode 100644 index 0000000000000..c1af534a07da4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModIntsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mod}. + * This class is generated. Do not edit it. + */ +public final class ModIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public ModIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public IntBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Mod.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Mod.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "ModIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModLongsEvaluator.java new file mode 100644 index 0000000000000..295724e041211 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mod}. + * This class is generated. Do not edit it. + */ +public final class ModLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public ModLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Mod.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Mod.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "ModLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModUnsignedLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModUnsignedLongsEvaluator.java new file mode 100644 index 0000000000000..95621fb9ef61b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModUnsignedLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mod}. + * This class is generated. Do not edit it. + */ +public final class ModUnsignedLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public ModUnsignedLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Mod.processUnsignedLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Mod.processUnsignedLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "ModUnsignedLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulDoublesEvaluator.java new file mode 100644 index 0000000000000..0a0fbebbe18d6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulDoublesEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mul}. + * This class is generated. Do not edit it. + */ +public final class MulDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public MulDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Mul.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Mul.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "MulDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulIntsEvaluator.java new file mode 100644 index 0000000000000..e94d174f8249c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulIntsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mul}. + * This class is generated. Do not edit it. + */ +public final class MulIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public MulIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public IntBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Mul.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Mul.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "MulIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulLongsEvaluator.java new file mode 100644 index 0000000000000..ffa437390fead --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mul}. + * This class is generated. Do not edit it. + */ +public final class MulLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public MulLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Mul.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Mul.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "MulLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulUnsignedLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulUnsignedLongsEvaluator.java new file mode 100644 index 0000000000000..c9d1b95851ed0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulUnsignedLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Mul}. + * This class is generated. Do not edit it. + */ +public final class MulUnsignedLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public MulUnsignedLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Mul.processUnsignedLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Mul.processUnsignedLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "MulUnsignedLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegDoublesEvaluator.java new file mode 100644 index 0000000000000..3a54d490bb003 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegDoublesEvaluator.java @@ -0,0 +1,64 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Neg}. + * This class is generated. Do not edit it. + */ +public final class NegDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator v; + + public NegDoublesEvaluator(EvalOperator.ExpressionEvaluator v) { + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock vBlock = (DoubleBlock) vUncastBlock; + DoubleVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock vBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Neg.processDoubles(vBlock.getDouble(vBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector vVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Neg.processDoubles(vVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NegDoublesEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegIntsEvaluator.java new file mode 100644 index 0000000000000..aefa05097cb96 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegIntsEvaluator.java @@ -0,0 +1,80 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Neg}. + * This class is generated. Do not edit it. + */ +public final class NegIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator v; + + public NegIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator v) { + this.warnings = new Warnings(source); + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock vBlock = (IntBlock) vUncastBlock; + IntVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector); + } + + public IntBlock eval(int positionCount, IntBlock vBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Neg.processInts(vBlock.getInt(vBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector vVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Neg.processInts(vVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "NegIntsEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegLongsEvaluator.java new file mode 100644 index 0000000000000..5bd301b8f76fc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegLongsEvaluator.java @@ -0,0 +1,80 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Neg}. + * This class is generated. Do not edit it. + */ +public final class NegLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator v; + + public NegLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator v) { + this.warnings = new Warnings(source); + this.v = v; + } + + @Override + public Block eval(Page page) { + Block vUncastBlock = v.eval(page); + if (vUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock vBlock = (LongBlock) vUncastBlock; + LongVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(page.getPositionCount(), vBlock); + } + return eval(page.getPositionCount(), vVector); + } + + public LongBlock eval(int positionCount, LongBlock vBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p) || vBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Neg.processLongs(vBlock.getLong(vBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector vVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Neg.processLongs(vVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "NegLongsEvaluator[" + "v=" + v + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubDoublesEvaluator.java new file mode 100644 index 0000000000000..c245ad03a0cea --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubDoublesEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sub}. + * This class is generated. Do not edit it. + */ +public final class SubDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public SubDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public DoubleBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + DoubleBlock.Builder result = DoubleBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendDouble(Sub.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public DoubleVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + DoubleVector.Builder result = DoubleVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendDouble(Sub.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "SubDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubIntsEvaluator.java new file mode 100644 index 0000000000000..bc942ca7522fb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubIntsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sub}. + * This class is generated. Do not edit it. + */ +public final class SubIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public SubIntsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public IntBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendInt(Sub.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public IntBlock eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + IntBlock.Builder result = IntBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendInt(Sub.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "SubIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubLongsEvaluator.java new file mode 100644 index 0000000000000..4b22842c74d8a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sub}. + * This class is generated. Do not edit it. + */ +public final class SubLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public SubLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Sub.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Sub.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "SubLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubUnsignedLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubUnsignedLongsEvaluator.java new file mode 100644 index 0000000000000..76ae796c5205a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubUnsignedLongsEvaluator.java @@ -0,0 +1,97 @@ +// 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.expression.predicate.operator.arithmetic; + +import java.lang.ArithmeticException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.ql.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sub}. + * This class is generated. Do not edit it. + */ +public final class SubUnsignedLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final Warnings warnings; + + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public SubUnsignedLongsEvaluator(Source source, EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.warnings = new Warnings(source); + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector); + } + + public LongBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + try { + result.appendLong(Sub.processUnsignedLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + public LongBlock eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + LongBlock.Builder result = LongBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + try { + result.appendLong(Sub.processUnsignedLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } catch (ArithmeticException e) { + warnings.registerException(e); + result.appendNull(); + } + } + return result.build(); + } + + @Override + public String toString() { + return "SubUnsignedLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsBoolsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsBoolsEvaluator.java new file mode 100644 index 0000000000000..a3d003dfc1372 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsBoolsEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}. + * This class is generated. Do not edit it. + */ +public final class EqualsBoolsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public EqualsBoolsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BooleanBlock lhsBlock = (BooleanBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BooleanBlock rhsBlock = (BooleanBlock) rhsUncastBlock; + BooleanVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BooleanVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BooleanBlock lhsBlock, BooleanBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Equals.processBools(lhsBlock.getBoolean(lhsBlock.getFirstValueIndex(p)), rhsBlock.getBoolean(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BooleanVector lhsVector, BooleanVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Equals.processBools(lhsVector.getBoolean(p), rhsVector.getBoolean(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "EqualsBoolsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsDoublesEvaluator.java new file mode 100644 index 0000000000000..8805e47dc2bec --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}. + * This class is generated. Do not edit it. + */ +public final class EqualsDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public EqualsDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Equals.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Equals.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "EqualsDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsIntsEvaluator.java new file mode 100644 index 0000000000000..51b074d6783f8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}. + * This class is generated. Do not edit it. + */ +public final class EqualsIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public EqualsIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Equals.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Equals.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "EqualsIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsKeywordsEvaluator.java new file mode 100644 index 0000000000000..efbd8469a2bf3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}. + * This class is generated. Do not edit it. + */ +public final class EqualsKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public EqualsKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Equals.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Equals.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "EqualsKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsLongsEvaluator.java new file mode 100644 index 0000000000000..cc1621be501c3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}. + * This class is generated. Do not edit it. + */ +public final class EqualsLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public EqualsLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(Equals.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(Equals.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "EqualsLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanDoublesEvaluator.java new file mode 100644 index 0000000000000..f06729d6ced6e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThan.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThan.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanIntsEvaluator.java new file mode 100644 index 0000000000000..3be64b9d4ea95 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThan.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThan.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanKeywordsEvaluator.java new file mode 100644 index 0000000000000..d688c0c4b0b5e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThan.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThan.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanLongsEvaluator.java new file mode 100644 index 0000000000000..5732d88709fb5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThan.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThan.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualDoublesEvaluator.java new file mode 100644 index 0000000000000..937dcb1cb8b9d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanOrEqualDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanOrEqualDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThanOrEqual.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThanOrEqual.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanOrEqualDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualIntsEvaluator.java new file mode 100644 index 0000000000000..baf8a28230bbc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanOrEqualIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanOrEqualIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThanOrEqual.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThanOrEqual.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanOrEqualIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualKeywordsEvaluator.java new file mode 100644 index 0000000000000..d45fd6a9d1062 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanOrEqualKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanOrEqualKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThanOrEqual.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThanOrEqual.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanOrEqualKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualLongsEvaluator.java new file mode 100644 index 0000000000000..7fc3517365cfe --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class GreaterThanOrEqualLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public GreaterThanOrEqualLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(GreaterThanOrEqual.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(GreaterThanOrEqual.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "GreaterThanOrEqualLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanDoublesEvaluator.java new file mode 100644 index 0000000000000..661385fb25465 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}. + * This class is generated. Do not edit it. + */ +public final class LessThanDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThan.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThan.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanIntsEvaluator.java new file mode 100644 index 0000000000000..0aa1baea78aa2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}. + * This class is generated. Do not edit it. + */ +public final class LessThanIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThan.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThan.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanKeywordsEvaluator.java new file mode 100644 index 0000000000000..f7c7876ace4d8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}. + * This class is generated. Do not edit it. + */ +public final class LessThanKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThan.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThan.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanLongsEvaluator.java new file mode 100644 index 0000000000000..d73bbe10deae8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}. + * This class is generated. Do not edit it. + */ +public final class LessThanLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThan.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThan.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualDoublesEvaluator.java new file mode 100644 index 0000000000000..cf1f43ca6a91f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class LessThanOrEqualDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanOrEqualDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThanOrEqual.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThanOrEqual.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanOrEqualDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualIntsEvaluator.java new file mode 100644 index 0000000000000..281f3701166ae --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class LessThanOrEqualIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanOrEqualIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThanOrEqual.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThanOrEqual.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanOrEqualIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualKeywordsEvaluator.java new file mode 100644 index 0000000000000..e5fca1e688bcf --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class LessThanOrEqualKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanOrEqualKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThanOrEqual.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThanOrEqual.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanOrEqualKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualLongsEvaluator.java new file mode 100644 index 0000000000000..c961e276ba1ed --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}. + * This class is generated. Do not edit it. + */ +public final class LessThanOrEqualLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public LessThanOrEqualLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(LessThanOrEqual.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(LessThanOrEqual.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "LessThanOrEqualLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsBoolsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsBoolsEvaluator.java new file mode 100644 index 0000000000000..db950c8c9c808 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsBoolsEvaluator.java @@ -0,0 +1,81 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}. + * This class is generated. Do not edit it. + */ +public final class NotEqualsBoolsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public NotEqualsBoolsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BooleanBlock lhsBlock = (BooleanBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BooleanBlock rhsBlock = (BooleanBlock) rhsUncastBlock; + BooleanVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BooleanVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BooleanBlock lhsBlock, BooleanBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(NotEquals.processBools(lhsBlock.getBoolean(lhsBlock.getFirstValueIndex(p)), rhsBlock.getBoolean(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BooleanVector lhsVector, BooleanVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(NotEquals.processBools(lhsVector.getBoolean(p), rhsVector.getBoolean(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEqualsBoolsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsDoublesEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsDoublesEvaluator.java new file mode 100644 index 0000000000000..d90b6efee264e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsDoublesEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}. + * This class is generated. Do not edit it. + */ +public final class NotEqualsDoublesEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public NotEqualsDoublesEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock lhsBlock = (DoubleBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + DoubleBlock rhsBlock = (DoubleBlock) rhsUncastBlock; + DoubleVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + DoubleVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, DoubleBlock lhsBlock, DoubleBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(NotEquals.processDoubles(lhsBlock.getDouble(lhsBlock.getFirstValueIndex(p)), rhsBlock.getDouble(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, DoubleVector lhsVector, DoubleVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(NotEquals.processDoubles(lhsVector.getDouble(p), rhsVector.getDouble(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEqualsDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsIntsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsIntsEvaluator.java new file mode 100644 index 0000000000000..818bd32f2118a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsIntsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}. + * This class is generated. Do not edit it. + */ +public final class NotEqualsIntsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public NotEqualsIntsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock lhsBlock = (IntBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + IntBlock rhsBlock = (IntBlock) rhsUncastBlock; + IntVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + IntVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, IntBlock lhsBlock, IntBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(NotEquals.processInts(lhsBlock.getInt(lhsBlock.getFirstValueIndex(p)), rhsBlock.getInt(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, IntVector lhsVector, IntVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(NotEquals.processInts(lhsVector.getInt(p), rhsVector.getInt(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEqualsIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsKeywordsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsKeywordsEvaluator.java new file mode 100644 index 0000000000000..fb8b10ca9953b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsKeywordsEvaluator.java @@ -0,0 +1,88 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}. + * This class is generated. Do not edit it. + */ +public final class NotEqualsKeywordsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public NotEqualsKeywordsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock lhsBlock = (BytesRefBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock rhsBlock = (BytesRefBlock) rhsUncastBlock; + BytesRefVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + BytesRefVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock lhsBlock, BytesRefBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(NotEquals.processKeywords(lhsBlock.getBytesRef(lhsBlock.getFirstValueIndex(p), lhsScratch), rhsBlock.getBytesRef(rhsBlock.getFirstValueIndex(p), rhsScratch))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector lhsVector, BytesRefVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef lhsScratch = new BytesRef(); + BytesRef rhsScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(NotEquals.processKeywords(lhsVector.getBytesRef(p, lhsScratch), rhsVector.getBytesRef(p, rhsScratch))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEqualsKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsLongsEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsLongsEvaluator.java new file mode 100644 index 0000000000000..4de21bbcbde66 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsLongsEvaluator.java @@ -0,0 +1,83 @@ +// 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.expression.predicate.operator.comparison; + +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}. + * This class is generated. Do not edit it. + */ +public final class NotEqualsLongsEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator lhs; + + private final EvalOperator.ExpressionEvaluator rhs; + + public NotEqualsLongsEvaluator(EvalOperator.ExpressionEvaluator lhs, + EvalOperator.ExpressionEvaluator rhs) { + this.lhs = lhs; + this.rhs = rhs; + } + + @Override + public Block eval(Page page) { + Block lhsUncastBlock = lhs.eval(page); + if (lhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock lhsBlock = (LongBlock) lhsUncastBlock; + Block rhsUncastBlock = rhs.eval(page); + if (rhsUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + LongBlock rhsBlock = (LongBlock) rhsUncastBlock; + LongVector lhsVector = lhsBlock.asVector(); + if (lhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + LongVector rhsVector = rhsBlock.asVector(); + if (rhsVector == null) { + return eval(page.getPositionCount(), lhsBlock, rhsBlock); + } + return eval(page.getPositionCount(), lhsVector, rhsVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, LongBlock lhsBlock, LongBlock rhsBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + if (lhsBlock.isNull(p) || lhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + if (rhsBlock.isNull(p) || rhsBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(NotEquals.processLongs(lhsBlock.getLong(lhsBlock.getFirstValueIndex(p)), rhsBlock.getLong(rhsBlock.getFirstValueIndex(p)))); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, LongVector lhsVector, LongVector rhsVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(NotEquals.processLongs(lhsVector.getLong(p), rhsVector.getLong(p))); + } + return result.build(); + } + + @Override + public String toString() { + return "NotEqualsLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatchEvaluator.java b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatchEvaluator.java new file mode 100644 index 0000000000000..a694148b4e7d0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatchEvaluator.java @@ -0,0 +1,74 @@ +// 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.expression.predicate.operator.regex; + +import java.lang.Override; +import java.lang.String; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link RegexMatch}. + * This class is generated. Do not edit it. + */ +public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator input; + + private final CharacterRunAutomaton pattern; + + public RegexMatchEvaluator(EvalOperator.ExpressionEvaluator input, + CharacterRunAutomaton pattern) { + this.input = input; + this.pattern = pattern; + } + + @Override + public Block eval(Page page) { + Block inputUncastBlock = input.eval(page); + if (inputUncastBlock.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock inputBlock = (BytesRefBlock) inputUncastBlock; + BytesRefVector inputVector = inputBlock.asVector(); + if (inputVector == null) { + return eval(page.getPositionCount(), inputBlock); + } + return eval(page.getPositionCount(), inputVector).asBlock(); + } + + public BooleanBlock eval(int positionCount, BytesRefBlock inputBlock) { + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + BytesRef inputScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + if (inputBlock.isNull(p) || inputBlock.getValueCount(p) != 1) { + result.appendNull(); + continue position; + } + result.appendBoolean(RegexMatch.process(inputBlock.getBytesRef(inputBlock.getFirstValueIndex(p), inputScratch), pattern)); + } + return result.build(); + } + + public BooleanVector eval(int positionCount, BytesRefVector inputVector) { + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + BytesRef inputScratch = new BytesRef(); + position: for (int p = 0; p < positionCount; p++) { + result.appendBoolean(RegexMatch.process(inputVector.getBytesRef(p, inputScratch), pattern)); + } + return result.build(); + } + + @Override + public String toString() { + return "RegexMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlClientException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlClientException.java new file mode 100644 index 0000000000000..48f03e2df911e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlClientException.java @@ -0,0 +1,25 @@ +/* + * 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; + +import org.elasticsearch.xpack.ql.QlClientException; + +public abstract class EsqlClientException extends QlClientException { + + protected EsqlClientException(String message, Object... args) { + super(message, args); + } + + protected EsqlClientException(String message, Throwable cause) { + super(message, cause); + } + + protected EsqlClientException(Throwable cause, String message, Object... args) { + super(cause, message, args); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlIllegalArgumentException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlIllegalArgumentException.java new file mode 100644 index 0000000000000..93b33b25ba454 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlIllegalArgumentException.java @@ -0,0 +1,36 @@ +/* + * 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; + +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; + +public class EsqlIllegalArgumentException extends QlIllegalArgumentException { + public EsqlIllegalArgumentException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public EsqlIllegalArgumentException(String message, Throwable cause) { + super(message, cause); + } + + public EsqlIllegalArgumentException(String message, Object... args) { + super(message, args); + } + + public EsqlIllegalArgumentException(Throwable cause, String message, Object... args) { + super(cause, message, args); + } + + public EsqlIllegalArgumentException(String message) { + super(message); + } + + public EsqlIllegalArgumentException(Throwable cause) { + super(cause); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlInfoTransportAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlInfoTransportAction.java new file mode 100644 index 0000000000000..b454bc197c90e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlInfoTransportAction.java @@ -0,0 +1,38 @@ +/* + * 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; + +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.XPackField; +import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction; +import org.elasticsearch.xpack.core.action.XPackInfoFeatureTransportAction; + +public class EsqlInfoTransportAction extends XPackInfoFeatureTransportAction { + + @Inject + public EsqlInfoTransportAction(TransportService transportService, ActionFilters actionFilters) { + super(XPackInfoFeatureAction.ESQL.name(), transportService, actionFilters); + } + + @Override + public String name() { + return XPackField.ESQL; + } + + @Override + public boolean available() { + return true; + } + + @Override + public boolean enabled() { + return true; + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlUsageTransportAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlUsageTransportAction.java new file mode 100644 index 0000000000000..a96911fba6037 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/EsqlUsageTransportAction.java @@ -0,0 +1,79 @@ +/* + * 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; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.protocol.xpack.XPackUsageRequest; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureTransportAction; +import org.elasticsearch.xpack.core.esql.EsqlFeatureSetUsage; +import org.elasticsearch.xpack.core.watcher.common.stats.Counters; +import org.elasticsearch.xpack.esql.plugin.EsqlStatsAction; +import org.elasticsearch.xpack.esql.plugin.EsqlStatsRequest; +import org.elasticsearch.xpack.esql.plugin.EsqlStatsResponse; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +public class EsqlUsageTransportAction extends XPackUsageFeatureTransportAction { + + private final Client client; + + @Inject + public EsqlUsageTransportAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, + Client client + ) { + super( + XPackUsageFeatureAction.ESQL.name(), + transportService, + clusterService, + threadPool, + actionFilters, + indexNameExpressionResolver + ); + this.client = client; + } + + @Override + protected void masterOperation( + Task task, + XPackUsageRequest request, + ClusterState state, + ActionListener listener + ) { + + EsqlStatsRequest esqlRequest = new EsqlStatsRequest(); + esqlRequest.includeStats(true); + esqlRequest.setParentTask(clusterService.localNode().getId(), task.getId()); + client.execute(EsqlStatsAction.INSTANCE, esqlRequest, ActionListener.wrap(r -> { + List countersPerNode = r.getNodes() + .stream() + .map(EsqlStatsResponse.NodeStatsResponse::getStats) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + Counters mergedCounters = Counters.merge(countersPerNode); + EsqlFeatureSetUsage usage = new EsqlFeatureSetUsage(mergedCounters.toNestedMap()); + listener.onResponse(new XPackUsageFeatureResponse(usage)); + }, listener::onFailure)); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ColumnInfo.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ColumnInfo.java new file mode 100644 index 0000000000000..bc648678984d5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ColumnInfo.java @@ -0,0 +1,192 @@ +/* + * 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.action; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.lucene.UnsupportedValueSource; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.xcontent.InstantiatingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.versionfield.Version; + +import java.io.IOException; + +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public record ColumnInfo(String name, String type) implements Writeable { + + private static final InstantiatingObjectParser PARSER; + static { + InstantiatingObjectParser.Builder parser = InstantiatingObjectParser.builder( + "esql/column_info", + true, + ColumnInfo.class + ); + parser.declareString(constructorArg(), new ParseField("name")); + parser.declareString(constructorArg(), new ParseField("type")); + PARSER = parser.build(); + } + + public static ColumnInfo fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + public ColumnInfo(StreamInput in) throws IOException { + this(in.readString(), in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeString(type); + } + + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field("name", name); + builder.field("type", type); + builder.endObject(); + return builder; + } + + public abstract class PositionToXContent { + private final Block block; + + PositionToXContent(Block block) { + this.block = block; + } + + public XContentBuilder positionToXContent(XContentBuilder builder, ToXContent.Params params, int position) throws IOException { + if (block.isNull(position)) { + return builder.nullValue(); + } + int count = block.getValueCount(position); + int start = block.getFirstValueIndex(position); + if (count == 1) { + return valueToXContent(builder, params, start); + } + builder.startArray(); + int end = start + count; + for (int i = start; i < end; i++) { + valueToXContent(builder, params, i); + } + return builder.endArray(); + } + + protected abstract XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException; + } + + public PositionToXContent positionToXContent(Block block, BytesRef scratch) { + return switch (type) { + case "long" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.value(((LongBlock) block).getLong(valueIndex)); + } + }; + case "integer" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.value(((IntBlock) block).getInt(valueIndex)); + } + }; + case "double" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.value(((DoubleBlock) block).getDouble(valueIndex)); + } + }; + case "unsigned_long" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + long l = ((LongBlock) block).getLong(valueIndex); + return builder.value(unsignedLongAsNumber(l)); + } + }; + case "keyword", "text" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + BytesRef val = ((BytesRefBlock) block).getBytesRef(valueIndex, scratch); + if (builder.contentType() == XContentType.CBOR && val.offset != 0) { + // cbor needs a zero offset because of a bug in jackson + // https://github.com/FasterXML/jackson-dataformats-binary/issues/366 + val = BytesRef.deepCopyOf(scratch); + } + return builder.utf8Value(val.bytes, val.offset, val.length); + } + }; + case "ip" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + BytesRef val = ((BytesRefBlock) block).getBytesRef(valueIndex, scratch); + return builder.value(DocValueFormat.IP.format(val)); + } + }; + case "date" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + long longVal = ((LongBlock) block).getLong(valueIndex); + return builder.value(UTC_DATE_TIME_FORMATTER.formatMillis(longVal)); + } + }; + case "boolean" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.value(((BooleanBlock) block).getBoolean(valueIndex)); + } + }; + case "version" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + BytesRef val = ((BytesRefBlock) block).getBytesRef(valueIndex, scratch); + return builder.value(new Version(val).toString()); + } + }; + case "null" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.nullValue(); + } + }; + case "unsupported" -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + return builder.value(UnsupportedValueSource.UNSUPPORTED_OUTPUT); + } + }; + default -> throw new IllegalArgumentException("can't convert values of type [" + type + "]"); + }; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryAction.java new file mode 100644 index 0000000000000..b16b7b78f2eb0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryAction.java @@ -0,0 +1,20 @@ +/* + * 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.action; + +import org.elasticsearch.action.ActionType; + +public class EsqlQueryAction extends ActionType { + + public static final EsqlQueryAction INSTANCE = new EsqlQueryAction(); + public static final String NAME = "indices:data/read/esql"; + + private EsqlQueryAction() { + super(NAME, EsqlQueryResponse::new); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequest.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequest.java new file mode 100644 index 0000000000000..812b22a9857dc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequest.java @@ -0,0 +1,265 @@ +/* + * 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.action; + +import org.elasticsearch.Build; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.CompositeIndicesRequest; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentLocation; +import org.elasticsearch.xcontent.XContentParseException; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.esql.parser.ContentLocation; +import org.elasticsearch.xpack.esql.parser.TypedParamValue; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Supplier; + +import static org.elasticsearch.action.ValidateActions.addValidationError; +import static org.elasticsearch.xcontent.ObjectParser.ValueType.VALUE_ARRAY; + +public class EsqlQueryRequest extends ActionRequest implements CompositeIndicesRequest { + + private static final ConstructingObjectParser PARAM_PARSER = new ConstructingObjectParser<>( + "params", + true, + objects -> new TypedParamValue((String) objects[1], objects[0]) + ); + + private static final ParseField QUERY_FIELD = new ParseField("query"); + private static final ParseField COLUMNAR_FIELD = new ParseField("columnar"); + private static final ParseField TIME_ZONE_FIELD = new ParseField("time_zone"); + private static final ParseField FILTER_FIELD = new ParseField("filter"); + private static final ParseField PRAGMA_FIELD = new ParseField("pragma"); + private static final ParseField PARAMS_FIELD = new ParseField("params"); + private static final ParseField LOCALE_FIELD = new ParseField("locale"); + + private static final ObjectParser PARSER = objectParser(EsqlQueryRequest::new); + + private String query; + private boolean columnar; + private ZoneId zoneId; + private Locale locale; + private QueryBuilder filter; + private QueryPragmas pragmas = new QueryPragmas(Settings.EMPTY); + private List params = List.of(); + + public EsqlQueryRequest(StreamInput in) throws IOException { + super(in); + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (Strings.hasText(query) == false) { + validationException = addValidationError("[query] is required", validationException); + } + if (Build.current().isSnapshot() == false && pragmas.isEmpty() == false) { + validationException = addValidationError("[pragma] only allowed in snapshot builds", validationException); + } + return validationException; + } + + public EsqlQueryRequest() {} + + public void query(String query) { + this.query = query; + } + + public String query() { + return query; + } + + public void columnar(boolean columnar) { + this.columnar = columnar; + } + + public boolean columnar() { + return columnar; + } + + public void zoneId(ZoneId zoneId) { + this.zoneId = zoneId; + } + + public ZoneId zoneId() { + return zoneId; + } + + public void locale(Locale locale) { + this.locale = locale; + } + + public Locale locale() { + return locale; + } + + public void filter(QueryBuilder filter) { + this.filter = filter; + } + + public QueryBuilder filter() { + return filter; + } + + public void pragmas(QueryPragmas pragmas) { + this.pragmas = pragmas; + } + + public QueryPragmas pragmas() { + return pragmas; + } + + public List params() { + return params; + } + + public void params(List params) { + this.params = params; + } + + public static EsqlQueryRequest fromXContent(XContentParser parser) { + EsqlQueryRequest result = PARSER.apply(parser, null); + validateParams(result.params); + return result; + } + + private static ObjectParser objectParser(Supplier supplier) { + ObjectParser parser = new ObjectParser<>("esql/query", false, supplier); + parser.declareString(EsqlQueryRequest::query, QUERY_FIELD); + parser.declareBoolean(EsqlQueryRequest::columnar, COLUMNAR_FIELD); + parser.declareString((request, zoneId) -> request.zoneId(ZoneId.of(zoneId)), TIME_ZONE_FIELD); + parser.declareObject(EsqlQueryRequest::filter, (p, c) -> AbstractQueryBuilder.parseTopLevelQuery(p), FILTER_FIELD); + parser.declareObject( + EsqlQueryRequest::pragmas, + (p, c) -> new QueryPragmas(Settings.builder().loadFromMap(p.map()).build()), + PRAGMA_FIELD + ); + parser.declareField(EsqlQueryRequest::params, EsqlQueryRequest::parseParams, PARAMS_FIELD, VALUE_ARRAY); + parser.declareString((request, localeTag) -> request.locale(Locale.forLanguageTag(localeTag)), LOCALE_FIELD); + + return parser; + } + + private static List parseParams(XContentParser p) throws IOException { + List result = new ArrayList<>(); + XContentParser.Token token = p.currentToken(); + + if (token == XContentParser.Token.START_ARRAY) { + Object value = null; + String type = null; + TypedParamValue previousParam = null; + TypedParamValue currentParam = null; + + while ((token = p.nextToken()) != XContentParser.Token.END_ARRAY) { + XContentLocation loc = p.getTokenLocation(); + + if (token == XContentParser.Token.START_OBJECT) { + // we are at the start of a value/type pair... hopefully + currentParam = PARAM_PARSER.apply(p, null); + /* + * Always set the xcontentlocation for the first param just in case the first one happens to not meet the parsing rules + * that are checked later in validateParams method. + * Also, set the xcontentlocation of the param that is different from the previous param in list when it comes to + * its type being explicitly set or inferred. + */ + if ((previousParam != null && previousParam.hasExplicitType() == false) || result.isEmpty()) { + currentParam.tokenLocation(toProto(loc)); + } + } else { + if (token == XContentParser.Token.VALUE_STRING) { + value = p.text(); + type = "keyword"; + } else if (token == XContentParser.Token.VALUE_NUMBER) { + XContentParser.NumberType numberType = p.numberType(); + if (numberType == XContentParser.NumberType.INT) { + value = p.intValue(); + type = "integer"; + } else if (numberType == XContentParser.NumberType.LONG) { + value = p.longValue(); + type = "long"; + } else if (numberType == XContentParser.NumberType.FLOAT) { + value = p.floatValue(); + type = "float"; + } else if (numberType == XContentParser.NumberType.DOUBLE) { + value = p.doubleValue(); + type = "double"; + } + } else if (token == XContentParser.Token.VALUE_BOOLEAN) { + value = p.booleanValue(); + type = "boolean"; + } else if (token == XContentParser.Token.VALUE_NULL) { + value = null; + type = "null"; + } else { + throw new XContentParseException(loc, "Failed to parse object: unexpected token [" + token + "] found"); + } + + currentParam = new TypedParamValue(type, value, false); + if ((previousParam != null && previousParam.hasExplicitType()) || result.isEmpty()) { + currentParam.tokenLocation(toProto(loc)); + } + } + + result.add(currentParam); + previousParam = currentParam; + } + } + + return result; + } + + static ContentLocation toProto(org.elasticsearch.xcontent.XContentLocation toProto) { + if (toProto == null) { + return null; + } + return new ContentLocation(toProto.lineNumber(), toProto.columnNumber()); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + // Pass the query as the description + return new CancellableTask(id, type, action, query, parentTaskId, headers); + } + + protected static void validateParams(List params) { + for (TypedParamValue param : params) { + if (param.hasExplicitType()) { + throw new XContentParseException( + fromProto(param.tokenLocation()), + "[params] must be an array where each entry is a single field (no " + "objects supported)" + ); + } + } + } + + static org.elasticsearch.xcontent.XContentLocation fromProto(ContentLocation fromProto) { + if (fromProto == null) { + return null; + } + return new org.elasticsearch.xcontent.XContentLocation(fromProto.lineNumber, fromProto.columnNumber); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestBuilder.java new file mode 100644 index 0000000000000..8d57e606e5b91 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestBuilder.java @@ -0,0 +1,51 @@ +/* + * 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.action; + +import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.client.internal.ElasticsearchClient; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; + +import java.time.ZoneId; + +public class EsqlQueryRequestBuilder extends ActionRequestBuilder { + + public EsqlQueryRequestBuilder(ElasticsearchClient client, EsqlQueryAction action, EsqlQueryRequest request) { + super(client, action, request); + } + + public EsqlQueryRequestBuilder(ElasticsearchClient client, EsqlQueryAction action) { + this(client, action, new EsqlQueryRequest()); + } + + public EsqlQueryRequestBuilder query(String query) { + request.query(query); + return this; + } + + public EsqlQueryRequestBuilder columnar(boolean columnar) { + request.columnar(columnar); + return this; + } + + public EsqlQueryRequestBuilder timeZone(ZoneId zoneId) { + request.zoneId(zoneId); + return this; + } + + public EsqlQueryRequestBuilder filter(QueryBuilder filter) { + request.filter(filter); + return this; + } + + public EsqlQueryRequestBuilder pragmas(QueryPragmas pragmas) { + request.pragmas(pragmas); + return this; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java new file mode 100644 index 0000000000000..239756487e5a9 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java @@ -0,0 +1,279 @@ +/* + * 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.action; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.common.xcontent.ChunkedToXContentHelper; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +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.UnsupportedValueSource; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.xcontent.InstantiatingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.versionfield.Version; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; +import static org.elasticsearch.xpack.ql.util.StringUtils.parseIP; + +public class EsqlQueryResponse extends ActionResponse implements ChunkedToXContent { + + private final List columns; + private final List pages; + private final boolean columnar; + + private static final InstantiatingObjectParser PARSER; + static { + InstantiatingObjectParser.Builder parser = InstantiatingObjectParser.builder( + "esql/query_response", + true, + EsqlQueryResponse.class + ); + parser.declareObjectArray(constructorArg(), (p, c) -> ColumnInfo.fromXContent(p), new ParseField("columns")); + parser.declareField(constructorArg(), (p, c) -> p.list(), new ParseField("values"), ObjectParser.ValueType.OBJECT_ARRAY); + PARSER = parser.build(); + } + + public EsqlQueryResponse(List columns, List pages, boolean columnar) { + this.columns = columns; + this.pages = pages; + this.columnar = columnar; + } + + public EsqlQueryResponse(List columns, List> values) { + this.columns = columns; + this.pages = List.of(valuesToPage(columns.stream().map(ColumnInfo::type).toList(), values)); + this.columnar = false; + } + + public EsqlQueryResponse(StreamInput in) throws IOException { + super(in); + this.columns = in.readList(ColumnInfo::new); + this.pages = in.readList(Page::new); + this.columnar = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeList(columns); + out.writeList(pages); + out.writeBoolean(columnar); + } + + public List columns() { + return columns; + } + + List pages() { + return pages; + } + + public List> values() { + return pagesToValues(columns.stream().map(ColumnInfo::type).toList(), pages); + } + + public boolean columnar() { + return columnar; + } + + @Override + public Iterator toXContentChunked(ToXContent.Params unused) { + BytesRef scratch = new BytesRef(); + final Iterator valuesIt; + if (pages.isEmpty()) { + valuesIt = Collections.emptyIterator(); + } else if (columnar) { + valuesIt = IntStream.range(0, columns().size()).mapToObj(column -> { + Stream values = pages.stream().flatMap(page -> { + ColumnInfo.PositionToXContent toXContent = columns.get(column).positionToXContent(page.getBlock(column), scratch); + return IntStream.range(0, page.getPositionCount()) + .mapToObj(position -> (builder, params) -> toXContent.positionToXContent(builder, params, position)); + }); + return Stream.concat( + Stream.of((builder, params) -> builder.startArray()), + Stream.concat(values, Stream.of((builder, params) -> builder.endArray())) + ); + }).flatMap(Function.identity()).iterator(); + } else { + valuesIt = pages.stream().flatMap(page -> { + List toXContents = IntStream.range(0, page.getBlockCount()) + .mapToObj(column -> columns.get(column).positionToXContent(page.getBlock(column), scratch)) + .toList(); + return IntStream.range(0, page.getPositionCount()).mapToObj(position -> (ToXContent) (builder, params) -> { + builder.startArray(); + for (int c = 0; c < columns.size(); c++) { + toXContents.get(c).positionToXContent(builder, params, position); + } + return builder.endArray(); + }); + }).iterator(); + } + return Iterators.concat( + ChunkedToXContentHelper.startObject(), // + ChunkedToXContentHelper.singleChunk((builder, params) -> { + builder.startArray("columns"); + for (ColumnInfo col : columns) { + col.toXContent(builder, params); + } + builder.endArray(); + return builder; + }),// + ChunkedToXContentHelper.array("values", valuesIt),// + ChunkedToXContentHelper.endObject() + ); + } + + @Override + public boolean isFragment() { + return false; + } + + public static EsqlQueryResponse fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + EsqlQueryResponse that = (EsqlQueryResponse) o; + return Objects.equals(columns, that.columns) && Objects.equals(values(), that.values()) && columnar == that.columnar; + } + + @Override + public int hashCode() { + return Objects.hash(columns, values(), columnar); + } + + @Override + public String toString() { + return Strings.toString(ChunkedToXContent.wrapAsToXContent(this)); + } + + public static List> pagesToValues(List dataTypes, List pages) { + BytesRef scratch = new BytesRef(); + List> result = new ArrayList<>(); + for (Page page : pages) { + for (int p = 0; p < page.getPositionCount(); p++) { + List row = new ArrayList<>(page.getBlockCount()); + for (int b = 0; b < page.getBlockCount(); b++) { + Block block = page.getBlock(b); + if (block.isNull(p)) { + row.add(null); + continue; + } + /* + * Use the ESQL data type to map to the output to make sure compute engine + * respects its types. See the INTEGER clause where is doesn't always + * respect it. + */ + int count = block.getValueCount(p); + int start = block.getFirstValueIndex(p); + if (count == 1) { + row.add(valueAt(dataTypes.get(b), block, start, scratch)); + continue; + } + List thisResult = new ArrayList<>(count); + int end = count + start; + for (int i = start; i < end; i++) { + thisResult.add(valueAt(dataTypes.get(b), block, i, scratch)); + } + row.add(thisResult); + } + result.add(row); + } + } + return result; + } + + private static Object valueAt(String dataType, Block block, int offset, BytesRef scratch) { + return switch (dataType) { + case "unsigned_long" -> unsignedLongAsNumber(((LongBlock) block).getLong(offset)); + case "long" -> ((LongBlock) block).getLong(offset); + case "integer" -> ((IntBlock) block).getInt(offset); + case "double" -> ((DoubleBlock) block).getDouble(offset); + case "keyword", "text" -> ((BytesRefBlock) block).getBytesRef(offset, scratch).utf8ToString(); + case "ip" -> { + BytesRef val = ((BytesRefBlock) block).getBytesRef(offset, scratch); + yield DocValueFormat.IP.format(val); + } + case "date" -> { + long longVal = ((LongBlock) block).getLong(offset); + yield UTC_DATE_TIME_FORMATTER.formatMillis(longVal); + } + case "boolean" -> ((BooleanBlock) block).getBoolean(offset); + case "version" -> new Version(((BytesRefBlock) block).getBytesRef(offset, scratch)).toString(); + case "unsupported" -> UnsupportedValueSource.UNSUPPORTED_OUTPUT; + default -> throw new UnsupportedOperationException("unsupported data type [" + dataType + "]"); + }; + } + + /** + * Convert a list of values to Pages so we can parse from xcontent. It's not + * super efficient but it doesn't really have to be. + */ + private static Page valuesToPage(List dataTypes, List> values) { + List results = dataTypes.stream() + .map(c -> LocalExecutionPlanner.toElementType(EsqlDataTypes.fromEs(c)).newBlockBuilder(values.size())) + .toList(); + + for (List row : values) { + for (int c = 0; c < row.size(); c++) { + var builder = results.get(c); + var value = row.get(c); + switch (dataTypes.get(c)) { + case "unsigned_long" -> ((LongBlock.Builder) builder).appendLong(asLongUnsigned(((Number) value).longValue())); + case "long" -> ((LongBlock.Builder) builder).appendLong(((Number) value).longValue()); + case "integer" -> ((IntBlock.Builder) builder).appendInt(((Number) value).intValue()); + case "double" -> ((DoubleBlock.Builder) builder).appendDouble(((Number) value).doubleValue()); + case "keyword", "text", "unsupported" -> ((BytesRefBlock.Builder) builder).appendBytesRef( + new BytesRef(value.toString()) + ); + case "ip" -> ((BytesRefBlock.Builder) builder).appendBytesRef(parseIP(value.toString())); + case "date" -> { + long longVal = UTC_DATE_TIME_FORMATTER.parseMillis(value.toString()); + ((LongBlock.Builder) builder).appendLong(longVal); + } + case "boolean" -> ((BooleanBlock.Builder) builder).appendBoolean(((Boolean) value)); + case "null" -> builder.appendNull(); + case "version" -> ((BytesRefBlock.Builder) builder).appendBytesRef(new Version(value.toString()).toBytesRef()); + default -> throw new UnsupportedOperationException("unsupported data type [" + dataTypes.get(c) + "]"); + } + } + } + return new Page(results.stream().map(Block.Builder::build).toArray(Block[]::new)); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResponseListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResponseListener.java new file mode 100644 index 0000000000000..b45155f0a95f7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResponseListener.java @@ -0,0 +1,70 @@ +/* + * 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.action; + +import org.elasticsearch.rest.ChunkedRestResponseBody; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestResponseListener; +import org.elasticsearch.xcontent.MediaType; +import org.elasticsearch.xpack.esql.formatter.TextFormat; +import org.elasticsearch.xpack.esql.plugin.EsqlMediaTypeParser; + +import java.util.Locale; + +import static org.elasticsearch.xpack.esql.formatter.TextFormat.CSV; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.URL_PARAM_DELIMITER; + +public class EsqlResponseListener extends RestResponseListener { + + private final RestChannel channel; + private final RestRequest restRequest; + private final MediaType mediaType; + private final long startNanos = System.nanoTime(); + private static final String HEADER_NAME_TOOK_NANOS = "Took-nanos"; + + public EsqlResponseListener(RestChannel channel, RestRequest restRequest, EsqlQueryRequest esqlRequest) { + super(channel); + + this.channel = channel; + this.restRequest = restRequest; + mediaType = EsqlMediaTypeParser.getResponseMediaType(restRequest, esqlRequest); + + /* + * Special handling for the "delimiter" parameter which should only be + * checked for being present or not in the case of CSV format. We cannot + * override {@link BaseRestHandler#responseParams()} because this + * parameter should only be checked for CSV, not other formats. + */ + if (mediaType != CSV && restRequest.hasParam(URL_PARAM_DELIMITER)) { + String message = String.format( + Locale.ROOT, + "parameter: [%s] can only be used with the format [%s] for request [%s]", + URL_PARAM_DELIMITER, + CSV.queryParameter(), + restRequest.path() + ); + throw new IllegalArgumentException(message); + } + } + + @Override + public RestResponse buildResponse(EsqlQueryResponse esqlResponse) throws Exception { + RestResponse restResponse; + if (mediaType instanceof TextFormat format) { + restResponse = new RestResponse(RestStatus.OK, format.contentType(restRequest), format.format(restRequest, esqlResponse)); + } else { + restResponse = new RestResponse(RestStatus.OK, ChunkedRestResponseBody.fromXContent(esqlResponse, channel.request(), channel)); + } + restResponse.addHeader(HEADER_NAME_TOOK_NANOS, Long.toString(System.nanoTime() - startNanos)); + + return restResponse; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlQueryAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlQueryAction.java new file mode 100644 index 0000000000000..ba173bb3bcd34 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlQueryAction.java @@ -0,0 +1,58 @@ +/* + * 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.action; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.core.RestApiVersion; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestCancellableNodeClient; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.URL_PARAM_DELIMITER; + +public class RestEsqlQueryAction extends BaseRestHandler { + + @Override + public String getName() { + return "esql_query"; + } + + @Override + public List routes() { + return List.of( + new Route(POST, "/_query"), + // TODO: remove before release + Route.builder(POST, "/_esql").deprecated("_esql endpoint has been deprecated in favour of _query", RestApiVersion.V_8).build() + ); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + EsqlQueryRequest esqlRequest; + try (XContentParser parser = request.contentOrSourceParamParser()) { + esqlRequest = EsqlQueryRequest.fromXContent(parser); + } + + return channel -> { + RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel()); + cancellableClient.execute(EsqlQueryAction.INSTANCE, esqlRequest, new EsqlResponseListener(channel, request, esqlRequest)); + }; + } + + @Override + protected Set responseParams() { + return Collections.singleton(URL_PARAM_DELIMITER); + } +} 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 new file mode 100644 index 0000000000000..7eafd35b596e2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java @@ -0,0 +1,674 @@ +/* + * 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.analysis; + +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.esql.plan.logical.Drop; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.EsqlUnresolvedRelation; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Keep; +import org.elasticsearch.xpack.esql.plan.logical.Rename; +import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules; +import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.BaseAnalyzerRule; +import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.ParameterizedAnalyzerRule; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.EmptyAttribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedStar; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.TableIdentifier; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.rule.ParameterizedRule; +import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor; +import org.elasticsearch.xpack.ql.rule.Rule; +import org.elasticsearch.xpack.ql.rule.RuleExecutor; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.elasticsearch.xpack.ql.type.InvalidMappedField; +import org.elasticsearch.xpack.ql.type.UnsupportedEsField; +import org.elasticsearch.xpack.ql.util.CollectionUtils; +import org.elasticsearch.xpack.ql.util.StringUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static java.util.Collections.singletonList; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.resolveFunction; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.NESTED; + +public class Analyzer extends ParameterizedRuleExecutor { + private static final Iterable> rules; + + static { + var resolution = new Batch<>( + "Resolution", + new ResolveTable(), + new ResolveEnrich(), + new ResolveRefs(), + new ResolveFunctions(), + new RemoveDuplicateProjections() + ); + var finish = new Batch<>("Finish Analysis", Limiter.ONCE, new AddImplicitLimit(), new PromoteStringsInDateComparisons()); + rules = List.of(resolution, finish); + } + + private final Verifier verifier; + + public Analyzer(AnalyzerContext context, Verifier verifier) { + super(context); + this.verifier = verifier; + } + + public LogicalPlan analyze(LogicalPlan plan) { + return verify(execute(plan)); + } + + public LogicalPlan verify(LogicalPlan plan) { + Collection failures = verifier.verify(plan); + if (failures.isEmpty() == false) { + throw new VerificationException(failures); + } + return plan; + } + + @Override + protected Iterable> batches() { + return rules; + } + + private static class ResolveTable extends ParameterizedAnalyzerRule { + + @Override + protected LogicalPlan rule(EsqlUnresolvedRelation plan, AnalyzerContext context) { + if (context.indexResolution().isValid() == false) { + return plan.unresolvedMessage().equals(context.indexResolution().toString()) + ? plan + : new EsqlUnresolvedRelation(plan.source(), plan.table(), plan.metadataFields(), context.indexResolution().toString()); + } + TableIdentifier table = plan.table(); + if (context.indexResolution().matches(table.index()) == false) { + new EsqlUnresolvedRelation( + plan.source(), + plan.table(), + plan.metadataFields(), + "invalid [" + table + "] resolution to [" + context.indexResolution() + "]" + ); + } + + EsIndex esIndex = context.indexResolution().get(); + var attributes = mappingAsAttributes(plan.source(), esIndex.mapping()); + attributes.addAll(plan.metadataFields()); + return new EsRelation(plan.source(), esIndex, attributes); + } + + } + + /** + * Specific flattening method, different from the default EsRelation that: + * 1. takes care of data type widening (for certain types) + * 2. drops the object and keyword hierarchy + */ + private static List mappingAsAttributes(Source source, Map mapping) { + var list = new ArrayList(); + mappingAsAttributes(list, source, null, mapping); + list.sort(Comparator.comparing(Attribute::qualifiedName)); + return list; + } + + private static void mappingAsAttributes(List list, Source source, String parentName, Map mapping) { + for (Map.Entry entry : mapping.entrySet()) { + String name = entry.getKey(); + EsField t = entry.getValue(); + + if (t != null) { + name = parentName == null ? name : parentName + "." + name; + var fieldProperties = t.getProperties(); + // widen the data type + var type = EsqlDataTypes.widenSmallNumericTypes(t.getDataType()); + // due to a bug also copy the field since the Attribute hierarchy extracts the data type + // directly even if the data type is passed explicitly + if (type != t.getDataType()) { + t = new EsField(t.getName(), type, t.getProperties(), t.isAggregatable(), t.isAlias()); + } + + // primitive branch + if (EsqlDataTypes.isPrimitive(type)) { + Attribute attribute; + if (t instanceof UnsupportedEsField uef) { + attribute = new UnsupportedAttribute(source, name, uef); + } else { + attribute = new FieldAttribute(source, null, name, t); + } + list.add(attribute); + } + // allow compound object even if they are unknown (but not NESTED) + if (type != NESTED && fieldProperties.isEmpty() == false) { + mappingAsAttributes(list, source, name, fieldProperties); + } + } + } + } + + private static class ResolveEnrich extends ParameterizedAnalyzerRule { + + @Override + protected LogicalPlan rule(Enrich plan, AnalyzerContext context) { + if (plan.policyName().resolved() == false) { + // the policy does not exist + return plan; + } + String policyName = (String) plan.policyName().fold(); + EnrichPolicyResolution policyRes = context.enrichResolution() + .resolvedPolicies() + .stream() + .filter(x -> x.policyName().equals(policyName)) + .findFirst() + .orElse(new EnrichPolicyResolution(policyName, null, null)); + + IndexResolution idx = policyRes.index(); + EnrichPolicy policy = policyRes.policy(); + + var policyNameExp = policy == null || idx == null + ? new UnresolvedAttribute( + plan.policyName().source(), + policyName, + null, + unresolvedPolicyError(policyName, context.enrichResolution()) + ) + : plan.policyName(); + + var matchField = plan.matchField() == null || plan.matchField() instanceof EmptyAttribute + ? new UnresolvedAttribute(plan.source(), policy.getMatchField()) + : plan.matchField(); + + List enrichFields = policy == null || idx == null + ? (plan.enrichFields() == null ? List.of() : plan.enrichFields()) + : calculateEnrichFields( + plan.source(), + policyName, + mappingAsAttributes(plan.source(), idx.get().mapping()), + plan.enrichFields(), + policy + ); + + return new Enrich(plan.source(), plan.child(), policyNameExp, matchField, policyRes, enrichFields); + } + + private String unresolvedPolicyError(String policyName, EnrichResolution enrichResolution) { + List potentialMatches = StringUtils.findSimilar(policyName, enrichResolution.existingPolicies()); + String msg = "unresolved enrich policy [" + policyName + "]"; + if (CollectionUtils.isEmpty(potentialMatches) == false) { + msg += ", did you mean " + + (potentialMatches.size() == 1 ? "[" + potentialMatches.get(0) + "]" : "any of " + potentialMatches) + + "?"; + } + return msg; + } + + public static List calculateEnrichFields( + Source source, + String policyName, + List mapping, + List enrichFields, + EnrichPolicy policy + ) { + Map fieldMap = mapping.stream().collect(Collectors.toMap(NamedExpression::name, Function.identity())); + fieldMap.remove(policy.getMatchField()); + List result = new ArrayList<>(); + if (enrichFields == null || enrichFields.isEmpty()) { + // use the policy to infer the enrich fields + for (String enrichFieldName : policy.getEnrichFields()) { + result.add(createEnrichFieldExpression(source, policyName, fieldMap, enrichFieldName)); + } + } else { + for (NamedExpression enrichField : enrichFields) { + String enrichFieldName = Expressions.name(enrichField instanceof Alias a ? a.child() : enrichField); + NamedExpression field = createEnrichFieldExpression(source, policyName, fieldMap, enrichFieldName); + result.add(enrichField instanceof Alias a ? new Alias(a.source(), a.name(), field) : field); + } + } + return result; + } + + private static NamedExpression createEnrichFieldExpression( + Source source, + String policyName, + Map fieldMap, + String enrichFieldName + ) { + Attribute mappedField = fieldMap.get(enrichFieldName); + if (mappedField == null) { + String msg = "Enrich field [" + enrichFieldName + "] not found in enrich policy [" + policyName + "]"; + List similar = StringUtils.findSimilar(enrichFieldName, fieldMap.keySet()); + if (CollectionUtils.isEmpty(similar) == false) { + msg += ", did you mean " + (similar.size() == 1 ? "[" + similar.get(0) + "]" : "any of " + similar) + "?"; + } + return new UnresolvedAttribute(source, enrichFieldName, null, msg); + } else { + return new ReferenceAttribute(source, enrichFieldName, mappedField.dataType(), null, Nullability.TRUE, null, false); + } + } + } + + private static class ResolveRefs extends BaseAnalyzerRule { + + @Override + protected LogicalPlan doRule(LogicalPlan plan) { + final List childrenOutput = new ArrayList<>(); + + for (LogicalPlan child : plan.children()) { + var output = child.output(); + childrenOutput.addAll(output); + } + + if (plan instanceof Drop d) { + return resolveDrop(d, childrenOutput); + } + + if (plan instanceof Rename r) { + return resolveRename(r, childrenOutput); + } + + if (plan instanceof Keep p) { + return resolveKeep(p, childrenOutput); + } + + if (plan instanceof Eval p) { + return resolveEval(p, childrenOutput); + } + + if (plan instanceof Enrich p) { + return resolveEnrich(p, childrenOutput); + } + + return plan.transformExpressionsUp(UnresolvedAttribute.class, ua -> resolveAttribute(ua, childrenOutput)); + } + + private Attribute resolveAttribute(UnresolvedAttribute ua, List childrenOutput) { + if (ua.customMessage()) { + return ua; + } + Attribute resolved = ua; + var named = resolveAgainstList(ua, childrenOutput); + // if resolved, return it; otherwise keep it in place to be resolved later + if (named.size() == 1) { + resolved = named.get(0); + if (log.isTraceEnabled() && resolved.resolved()) { + log.trace("Resolved {} to {}", ua, resolved); + } + } else { + if (named.size() > 0) { + resolved = ua.withUnresolvedMessage("Resolved [" + ua + "] unexpectedly to multiple attributes " + named); + } + } + return resolved; + } + + private LogicalPlan resolveEval(Eval eval, List childOutput) { + List allResolvedInputs = new ArrayList<>(childOutput); + List newFields = new ArrayList<>(); + boolean changed = false; + for (NamedExpression field : eval.fields()) { + NamedExpression result = (NamedExpression) field.transformUp( + UnresolvedAttribute.class, + ua -> resolveAttribute(ua, allResolvedInputs) + ); + + changed |= result != field; + newFields.add(result); + + if (result.resolved()) { + // for proper resolution, duplicate attribute names are problematic, only last occurrence matters + Attribute existing = allResolvedInputs.stream() + .filter(attr -> attr.name().equals(result.name())) + .findFirst() + .orElse(null); + if (existing != null) { + allResolvedInputs.remove(existing); + } + allResolvedInputs.add(result.toAttribute()); + } + } + return changed ? new Eval(eval.source(), eval.child(), newFields) : eval; + } + + private LogicalPlan resolveKeep(Project p, List childOutput) { + List resolvedProjections = new ArrayList<>(); + var projections = p.projections(); + // start with projections + + // no projection specified or just * + if (projections.isEmpty() || (projections.size() == 1 && projections.get(0) instanceof UnresolvedStar)) { + resolvedProjections.addAll(childOutput); + } + // otherwise resolve them + else { + var starPosition = -1; // no star + // resolve each item manually while paying attention to: + // 1. name patterns a*, *b, a*b + // 2. star * - which can only appear once and signifies "everything else" - this will be added at the end + for (var ne : projections) { + if (ne instanceof UnresolvedStar) { + starPosition = resolvedProjections.size(); + } else if (ne instanceof UnresolvedAttribute ua) { + resolvedProjections.addAll(resolveAgainstList(ua, childOutput)); + } else { + // if this gets here it means it was already resolved + resolvedProjections.add(ne); + } + } + // compute star if specified and add it to the list + if (starPosition >= 0) { + var remainingProjections = new ArrayList<>(childOutput); + remainingProjections.removeAll(resolvedProjections); + resolvedProjections.addAll(starPosition, remainingProjections); + } + } + + return new EsqlProject(p.source(), p.child(), resolvedProjections); + } + + private LogicalPlan resolveDrop(Drop drop, List childOutput) { + List resolvedProjections = new ArrayList<>(childOutput); + + for (var ne : drop.removals()) { + var resolved = ne instanceof UnresolvedAttribute ua ? resolveAgainstList(ua, childOutput) : singletonList(ne); + // the return list might contain either resolved elements or unresolved ones. + // if things are resolved, remove them - if not add them to the list to trip the Verifier; + // thus make sure to remove the intersection but add the unresolved difference (if any). + // so, remove things that are in common + resolvedProjections.removeIf(resolved::contains); + // but add non-projected, unresolved extras to later trip the Verifier. + resolved.forEach(r -> { + if (r.resolved() == false && r instanceof UnsupportedAttribute == false) { + resolvedProjections.add(r); + } + }); + } + + return new EsqlProject(drop.source(), drop.child(), resolvedProjections); + } + + private LogicalPlan resolveRename(Rename rename, List childrenOutput) { + List projections = new ArrayList<>(childrenOutput); + + int renamingsCount = rename.renamings().size(); + List unresolved = new ArrayList<>(renamingsCount); + Map reverseAliasing = new HashMap<>(renamingsCount); // `| rename a as x` => map(a: x) + + rename.renamings().forEach(alias -> { + // skip NOPs: `| rename a as a` + if (alias.child() instanceof UnresolvedAttribute ua && alias.name().equals(ua.name()) == false) { + // remove attributes overwritten by a renaming: `| keep a, b, c | rename a as b` + projections.removeIf(x -> x.name().equals(alias.name())); + + var resolved = resolveAttribute(ua, childrenOutput); + if (resolved instanceof UnsupportedAttribute || resolved.resolved()) { + var realiased = (NamedExpression) alias.replaceChildren(List.of(resolved)); + projections.replaceAll(x -> x.equals(resolved) ? realiased : x); + childrenOutput.removeIf(x -> x.equals(resolved)); + reverseAliasing.put(resolved.name(), alias.name()); + } else { // remained UnresolvedAttribute + // is the current alias referencing a previously declared alias? + boolean updated = false; + if (reverseAliasing.containsValue(resolved.name())) { + for (var li = projections.listIterator(); li.hasNext();) { + // does alias still exist? i.e. it hasn't been renamed again (`| rename a as b, b as c, b as d`) + if (li.next() instanceof Alias a && a.name().equals(resolved.name())) { + reverseAliasing.put(resolved.name(), alias.name()); + // update aliased projection in place + li.set((NamedExpression) alias.replaceChildren(a.children())); + updated = true; + break; + } + } + } + if (updated == false) { + var u = resolved; + var previousAliasName = reverseAliasing.get(resolved.name()); + if (previousAliasName != null) { + String message = format( + null, + "Column [{}] renamed to [{}] and is no longer available [{}]", + resolved.name(), + previousAliasName, + alias.sourceText() + ); + u = ua.withUnresolvedMessage(message); + } + unresolved.add(u); + } + } + } + }); + + // add unresolved renamings to later trip the Verifier. + projections.addAll(unresolved); + + return new EsqlProject(rename.source(), rename.child(), projections); + } + + private LogicalPlan resolveEnrich(Enrich enrich, List childrenOutput) { + + if (enrich.matchField().toAttribute() instanceof UnresolvedAttribute ua) { + Attribute resolved = resolveAttribute(ua, childrenOutput); + if (resolved.equals(ua)) { + return enrich; + } + if (resolved.resolved() && resolved.dataType() != KEYWORD) { + resolved = ua.withUnresolvedMessage( + "Unsupported type [" + + resolved.dataType() + + "] for enrich matching field [" + + ua.name() + + "]; only KEYWORD allowed" + ); + } + return new Enrich(enrich.source(), enrich.child(), enrich.policyName(), resolved, enrich.policy(), enrich.enrichFields()); + } + return enrich; + } + } + + private static List resolveAgainstList(UnresolvedAttribute u, Collection attrList) { + var matches = AnalyzerRules.maybeResolveAgainstList(u, attrList, false, true, Analyzer::handleSpecialFields); + + // none found - add error message + if (matches.isEmpty()) { + UnresolvedAttribute unresolved; + var name = u.name(); + if (Regex.isSimpleMatchPattern(name)) { + unresolved = u.withUnresolvedMessage(format(null, "No match found for [{}]", name)); + } else { + Set names = new HashSet<>(attrList.size()); + for (var a : attrList) { + String nameCandidate = a.name(); + if (EsqlDataTypes.isPrimitive(a.dataType())) { + names.add(nameCandidate); + } + } + unresolved = u.withUnresolvedMessage(UnresolvedAttribute.errorMessage(name, StringUtils.findSimilar(name, names))); + } + return singletonList(unresolved); + } + + return matches; + } + + private static Attribute handleSpecialFields(UnresolvedAttribute u, Attribute named) { + if (named instanceof FieldAttribute fa) { + // incompatible mappings + var field = fa.field(); + if (field instanceof InvalidMappedField imf) { + named = u.withUnresolvedMessage("Cannot use field [" + fa.name() + "] due to ambiguities being " + imf.errorMessage()); + } + } + + return named; + } + + private static class ResolveFunctions extends ParameterizedAnalyzerRule { + + @Override + protected LogicalPlan rule(LogicalPlan plan, AnalyzerContext context) { + return plan.transformExpressionsUp( + UnresolvedFunction.class, + uf -> resolveFunction(uf, context.configuration(), context.functionRegistry()) + ); + } + } + + /** + * Rule that removes duplicate projects - this is done as a separate rule to allow + * full validation of the node before looking at the duplication. + * The duplication needs to be addressed to avoid ambiguity errors from commands further down + * the line. + */ + private static class RemoveDuplicateProjections extends BaseAnalyzerRule { + + @Override + protected boolean skipResolved() { + return false; + } + + @Override + protected LogicalPlan doRule(LogicalPlan plan) { + if (plan.resolved()) { + if (plan instanceof Aggregate agg) { + plan = removeAggDuplicates(agg); + } + } + return plan; + } + + private static LogicalPlan removeAggDuplicates(Aggregate agg) { + var groupings = agg.groupings(); + var newGroupings = new LinkedHashSet<>(groupings); + // reuse existing objects + groupings = newGroupings.size() == groupings.size() ? groupings : new ArrayList<>(newGroupings); + + var aggregates = agg.aggregates(); + var newAggregates = new ArrayList<>(aggregates); + var nameSet = Sets.newHashSetWithExpectedSize(newAggregates.size()); + // remove duplicates in reverse to preserve the last one appearing + for (int i = newAggregates.size() - 1; i >= 0; i--) { + var aggregate = newAggregates.get(i); + if (nameSet.add(aggregate.name()) == false) { + newAggregates.remove(i); + } + } + // reuse existing objects + aggregates = newAggregates.size() == aggregates.size() ? aggregates : newAggregates; + // replace aggregate if needed + agg = (groupings == agg.groupings() && newAggregates == agg.aggregates()) + ? agg + : new Aggregate(agg.source(), agg.child(), groupings, aggregates); + return agg; + } + } + + private static class AddImplicitLimit extends ParameterizedRule { + @Override + public LogicalPlan apply(LogicalPlan logicalPlan, AnalyzerContext context) { + return new Limit( + Source.EMPTY, + new Literal(Source.EMPTY, context.configuration().resultTruncationMaxSize(), DataTypes.INTEGER), + logicalPlan + ); + } + } + + private static class PromoteStringsInDateComparisons extends Rule { + + @Override + public LogicalPlan apply(LogicalPlan plan) { + return plan.transformExpressionsUp(BinaryComparison.class, PromoteStringsInDateComparisons::promote); + } + + private static Expression promote(BinaryComparison cmp) { + if (cmp.resolved() == false) { + return cmp; + } + var left = cmp.left(); + var right = cmp.right(); + boolean modified = false; + if (left.dataType() == DATETIME) { + if (right.dataType() == KEYWORD && right.foldable()) { + right = stringToDate(right); + modified = true; + } + } else { + if (right.dataType() == DATETIME) { + if (left.dataType() == KEYWORD && left.foldable()) { + left = stringToDate(left); + modified = true; + } + } + } + return modified ? cmp.replaceChildren(List.of(left, right)) : cmp; + } + + private static Expression stringToDate(Expression stringExpression) { + var str = stringExpression.fold().toString(); + + Long millis = null; + // TODO: better control over this string format - do we want this to be flexible or always redirect folks to use date parsing + try { + millis = str == null ? null : DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis(str); + } catch (Exception ex) { // in case of exception, millis will be null which will trigger an error + } + + var source = stringExpression.source(); + Expression result; + if (millis == null) { + var errorMessage = format(null, "Invalid date [{}]", str); + result = new UnresolvedAttribute(source, source.text(), null, errorMessage); + } else { + result = new Literal(source, millis, DATETIME); + } + return result; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java new file mode 100644 index 0000000000000..34acd2ac1b541 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/AnalyzerContext.java @@ -0,0 +1,19 @@ +/* + * 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.analysis; + +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.IndexResolution; + +public record AnalyzerContext( + EsqlConfiguration configuration, + FunctionRegistry functionRegistry, + IndexResolution indexResolution, + EnrichResolution enrichResolution +) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/EnrichResolution.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/EnrichResolution.java new file mode 100644 index 0000000000000..332e5e60565b6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/EnrichResolution.java @@ -0,0 +1,14 @@ +/* + * 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.analysis; + +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; + +import java.util.Set; + +public record EnrichResolution(Set resolvedPolicies, Set existingPolicies) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java new file mode 100644 index 0000000000000..f34a4f0f37a70 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java @@ -0,0 +1,54 @@ +/* + * 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.analysis; + +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.EsqlUnresolvedRelation; +import org.elasticsearch.xpack.ql.analyzer.TableInfo; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; + +import java.util.ArrayList; +import java.util.List; + +import static java.util.Collections.emptyList; + +public class PreAnalyzer { + + public static class PreAnalysis { + public static final PreAnalysis EMPTY = new PreAnalysis(emptyList(), emptyList()); + + public final List indices; + public final List policyNames; + + public PreAnalysis(List indices, List policyNames) { + this.indices = indices; + this.policyNames = policyNames; + } + } + + public PreAnalysis preAnalyze(LogicalPlan plan) { + if (plan.analyzed()) { + return PreAnalysis.EMPTY; + } + + return doPreAnalyze(plan); + } + + protected PreAnalysis doPreAnalyze(LogicalPlan plan) { + List indices = new ArrayList<>(); + List policyNames = new ArrayList<>(); + + plan.forEachUp(EsqlUnresolvedRelation.class, p -> indices.add(new TableInfo(p.table(), p.frozen()))); + plan.forEachUp(Enrich.class, p -> policyNames.add((String) p.policyName().fold())); + + // mark plan as preAnalyzed (if it were marked, there would be no analysis) + plan.forEachUp(LogicalPlan::setPreAnalyzed); + + return new PreAnalysis(indices, policyNames); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/VerificationException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/VerificationException.java new file mode 100644 index 0000000000000..972ebb40cb8bc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/VerificationException.java @@ -0,0 +1,26 @@ +/* + * 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.analysis; + +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.esql.EsqlClientException; +import org.elasticsearch.xpack.ql.common.Failure; + +import java.util.Collection; + +public class VerificationException extends EsqlClientException { + + protected VerificationException(Collection sources) { + super(Failure.failMessage(sources)); + } + + @Override + public RestStatus status() { + return RestStatus.BAD_REQUEST; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java new file mode 100644 index 0000000000000..7604c5e839f63 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java @@ -0,0 +1,327 @@ +/* + * 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.analysis; + +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.RegexExtract; +import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.esql.stats.FeatureMetric; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.capabilities.Unresolvable; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Stream; + +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.DISSECT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.EVAL; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.GROK; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.LIMIT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.SORT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.STATS; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.WHERE; +import static org.elasticsearch.xpack.ql.common.Failure.fail; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; + +public class Verifier { + + private final Metrics metrics; + + public Verifier(Metrics metrics) { + this.metrics = metrics; + } + + /** + * Verify that a {@link LogicalPlan} can be executed. + * + * @param plan The logical plan to be verified + * @return a collection of verification failures; empty if and only if the plan is valid + */ + Collection verify(LogicalPlan plan) { + Set failures = new LinkedHashSet<>(); + + // quick verification for unresolved attributes + plan.forEachUp(p -> { + // if the children are unresolved, so will this node; counting it will only add noise + if (p.childrenResolved() == false) { + return; + } + + if (p instanceof Unresolvable u) { + failures.add(fail(p, u.unresolvedMessage())); + } + // p is resolved, skip + else if (p.resolved()) { + return; + } + p.forEachExpression(e -> { + // everything is fine, skip expression + if (e.resolved()) { + return; + } + + e.forEachUp(ae -> { + // we're only interested in the children + if (ae.childrenResolved() == false) { + return; + } + + if (ae instanceof Unresolvable u) { + // special handling for Project and unsupported types + if (p instanceof Project == false || u instanceof UnsupportedAttribute == false) { + failures.add(fail(ae, u.unresolvedMessage())); + } + } + if (ae.typeResolved().unresolved()) { + failures.add(fail(ae, ae.typeResolved().message())); + } + }); + }); + }); + + // in case of failures bail-out as all other checks will be redundant + if (failures.isEmpty() == false) { + return failures; + } + + // Concrete verifications + plan.forEachDown(p -> { + if (p instanceof Aggregate agg) { + agg.aggregates().forEach(e -> { + var exp = e instanceof Alias ? ((Alias) e).child() : e; + if (exp instanceof AggregateFunction aggFunc) { + aggFunc.arguments().forEach(a -> { + // TODO: allow an expression? + if ((a instanceof FieldAttribute + || a instanceof MetadataAttribute + || a instanceof ReferenceAttribute + || a instanceof Literal) == false) { + failures.add( + fail( + e, + "aggregate function's parameters must be an attribute or literal; found [" + + a.sourceText() + + "] of type [" + + a.nodeName() + + "]" + ) + ); + } + }); + } else if (agg.groupings().contains(exp) == false) { // TODO: allow an expression? + failures.add( + fail( + exp, + "expected an aggregate function or group but got [" + + exp.sourceText() + + "] of type [" + + exp.nodeName() + + "]" + ) + ); + } + }); + } else if (p instanceof RegexExtract re) { + Expression expr = re.input(); + DataType type = expr.dataType(); + if (EsqlDataTypes.isString(type) == false) { + failures.add( + fail( + expr, + "{} only supports KEYWORD or TEXT values, found expression [{}] type [{}]", + re.getClass().getSimpleName(), + expr.sourceText(), + type + ) + ); + } + } else if (p instanceof Row row) { + failures.addAll(validateRow(row)); + } + + p.forEachExpression(BinaryOperator.class, bo -> { + Failure f = validateUnsignedLongOperator(bo); + if (f != null) { + failures.add(f); + } + }); + p.forEachExpression(BinaryComparison.class, bc -> { + Failure f = validateBinaryComparison(bc); + if (f != null) { + failures.add(f); + } + }); + p.forEachExpression(Neg.class, neg -> { + Failure f = validateUnsignedLongNegation(neg); + if (f != null) { + failures.add(f); + } + }); + }); + + // gather metrics + if (failures.isEmpty()) { + gatherMetrics(plan); + } + + return failures; + } + + private void gatherMetrics(LogicalPlan plan) { + BitSet b = new BitSet(FeatureMetric.values().length); + plan.forEachDown(p -> { + if (p instanceof Dissect) { + b.set(DISSECT.ordinal()); + } else if (p instanceof Eval) { + b.set(EVAL.ordinal()); + } else if (p instanceof Grok) { + b.set(GROK.ordinal()); + } else if (p instanceof Limit) { + b.set(LIMIT.ordinal()); + } else if (p instanceof OrderBy) { + b.set(SORT.ordinal()); + } else if (p instanceof Aggregate) { + b.set(STATS.ordinal()); + } else if (p instanceof Filter) { + b.set(WHERE.ordinal()); + } + }); + for (int i = b.nextSetBit(0); i >= 0; i = b.nextSetBit(i + 1)) { + metrics.inc(FeatureMetric.values()[i]); + } + } + + private static Collection validateRow(Row row) { + List failures = new ArrayList<>(row.fields().size()); + row.fields().forEach(o -> { + if (EsqlDataTypes.isRepresentable(o.dataType()) == false && o instanceof Alias a) { + failures.add(fail(o, "cannot use [{}] directly in a row assignment", a.child().sourceText())); + } + }); + return failures; + } + + /** + * Limit QL's comparisons to types we support. + */ + public static Failure validateBinaryComparison(BinaryComparison bc) { + if (bc.left().dataType().isNumeric()) { + if (false == bc.right().dataType().isNumeric()) { + return fail( + bc, + "first argument of [{}] is [numeric] so second argument must also be [numeric] but was [{}]", + bc.sourceText(), + bc.right().dataType().typeName() + ); + } + return null; + } + + List allowed = new ArrayList<>(); + allowed.add(DataTypes.KEYWORD); + allowed.add(DataTypes.TEXT); + allowed.add(DataTypes.IP); + allowed.add(DataTypes.DATETIME); + allowed.add(DataTypes.VERSION); + if (bc instanceof Equals || bc instanceof NotEquals) { + allowed.add(DataTypes.BOOLEAN); + } + Expression.TypeResolution r = TypeResolutions.isType( + bc.left(), + allowed::contains, + bc.sourceText(), + FIRST, + Stream.concat(Stream.of("numeric"), allowed.stream().map(DataType::typeName)).toArray(String[]::new) + ); + if (false == r.resolved()) { + return fail(bc, r.message()); + } + if (bc.left().dataType() != bc.right().dataType()) { + return fail( + bc, + "first argument of [{}] is [{}] so second argument must also be [{}] but was [{}]", + bc.sourceText(), + bc.left().dataType().typeName(), + bc.left().dataType().typeName(), + bc.right().dataType().typeName() + ); + } + return null; + } + + /** Ensure that UNSIGNED_LONG types are not implicitly converted when used in arithmetic binary operator, as this cannot be done since: + * - unsigned longs are passed through the engine as longs, so/and + * - negative values cannot be represented (i.e. range [Long.MIN_VALUE, "abs"(Long.MIN_VALUE) + Long.MAX_VALUE] won't fit on 64 bits); + * - a conversion to double isn't possible, since upper range UL values can no longer be distinguished + * ex: (double) 18446744073709551615 == (double) 18446744073709551614 + * - the implicit ESQL's Cast doesn't currently catch Exception and nullify the result. + * Let the user handle the operation explicitly. + */ + public static Failure validateUnsignedLongOperator(BinaryOperator bo) { + DataType leftType = bo.left().dataType(); + DataType rightType = bo.right().dataType(); + if ((leftType == DataTypes.UNSIGNED_LONG || rightType == DataTypes.UNSIGNED_LONG) && leftType != rightType) { + return fail( + bo, + "first argument of [{}] is [{}] and second is [{}]. [{}] can only be operated on together with another [{}]", + bo.sourceText(), + leftType.typeName(), + rightType.typeName(), + DataTypes.UNSIGNED_LONG.typeName(), + DataTypes.UNSIGNED_LONG.typeName() + ); + } + return null; + } + + /** + * Negating an unsigned long is invalid. + */ + private static Failure validateUnsignedLongNegation(Neg neg) { + DataType childExpressionType = neg.field().dataType(); + if (childExpressionType.equals(DataTypes.UNSIGNED_LONG)) { + return fail( + neg, + "negation unsupported for arguments of type [{}] in expression [{}]", + childExpressionType.typeName(), + neg.sourceText() + ); + } + return null; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupOperator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupOperator.java new file mode 100644 index 0000000000000..5ac551103f338 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupOperator.java @@ -0,0 +1,113 @@ +/* + * 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.enrich; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.AsyncOperator; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.xpack.ql.expression.NamedExpression; + +import java.util.List; + +public final class EnrichLookupOperator extends AsyncOperator { + private final EnrichLookupService enrichLookupService; + private final String sessionId; + private final CancellableTask parentTask; + private final int inputChannel; + private final String enrichIndex; + private final String matchType; + private final String matchField; + private final List enrichFields; + + public record Factory( + String sessionId, + CancellableTask parentTask, + int maxOutstandingRequests, + int inputChannel, + EnrichLookupService enrichLookupService, + String enrichIndex, + String matchType, + String matchField, + List enrichFields + ) implements OperatorFactory { + @Override + public String describe() { + return "EnrichOperator[index=" + + enrichIndex + + " match_field=" + + matchField + + " enrich_fields=" + + enrichFields + + " inputChannel=" + + inputChannel + + "]"; + } + + @Override + public Operator get(DriverContext driverContext) { + return new EnrichLookupOperator( + sessionId, + parentTask, + maxOutstandingRequests, + inputChannel, + enrichLookupService, + enrichIndex, + matchType, + matchField, + enrichFields + ); + } + } + + public EnrichLookupOperator( + String sessionId, + CancellableTask parentTask, + int maxOutstandingRequests, + int inputChannel, + EnrichLookupService enrichLookupService, + String enrichIndex, + String matchType, + String matchField, + List enrichFields + ) { + super(maxOutstandingRequests); + this.sessionId = sessionId; + this.parentTask = parentTask; + this.inputChannel = inputChannel; + this.enrichLookupService = enrichLookupService; + this.enrichIndex = enrichIndex; + this.matchType = matchType; + this.matchField = matchField; + this.enrichFields = enrichFields; + } + + @Override + protected void performAsync(Page inputPage, ActionListener listener) { + final Block inputBlock = inputPage.getBlock(inputChannel); + enrichLookupService.lookupAsync( + sessionId, + parentTask, + enrichIndex, + matchType, + matchField, + enrichFields, + new Page(inputBlock), + listener.map(inputPage::appendPage) + ); + } + + @Override + public void close() { + // TODO: Maybe create a sub-task as the parent task of all the lookup tasks + // then cancel it when this operator terminates early (e.g., have enough result). + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java new file mode 100644 index 0000000000000..78f12e56d64f7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java @@ -0,0 +1,376 @@ +/* + * 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.enrich; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.UnavailableShardsException; +import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.ContextPreservingActionListener; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.ValueSources; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OutputOperator; +import org.elasticsearch.compute.operator.ProjectOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.action.EsqlQueryAction; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.NamedExpression; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.IntStream; + +import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanReader.readerFromPlanReader; +import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanWriter.writerFromPlanWriter; + +/** + * {@link EnrichLookupService} performs enrich lookup for a given input page. The lookup process consists of three stages: + * - Stage 1: Finding matching document IDs for the input page. This stage is done by the {@link EnrichQuerySourceOperator} or its variants. + * The output page of this stage is represented as [DocVector, IntBlock: positions of the input terms]. + *

+ * - Stage 2: Extracting field values for the matched document IDs. The output page is represented as + * [DocVector, IntBlock: positions, Block: field1, Block: field2,...]. + *

+ * - Stage 3: Combining the extracted values based on positions and filling nulls for positions without matches. + * This is done by {@link MergePositionsOperator}. The output page is represented as [Block: field1, Block: field2,...]. + *

+ * The positionCount of the output page must be equal to the positionCount of the input page. + */ +public class EnrichLookupService { + public static final String LOOKUP_ACTION_NAME = EsqlQueryAction.NAME + "/lookup"; + + private final ClusterService clusterService; + private final SearchService searchService; + private final TransportService transportService; + private final Executor executor; + + public EnrichLookupService(ClusterService clusterService, SearchService searchService, TransportService transportService) { + this.clusterService = clusterService; + this.searchService = searchService; + this.transportService = transportService; + this.executor = transportService.getThreadPool().executor(EsqlPlugin.ESQL_THREAD_POOL_NAME); + transportService.registerRequestHandler( + LOOKUP_ACTION_NAME, + EsqlPlugin.ESQL_THREAD_POOL_NAME, + LookupRequest::new, + new TransportHandler() + ); + } + + public void lookupAsync( + String sessionId, + CancellableTask parentTask, + String index, + String matchType, + String matchField, + List extractFields, + Page inputPage, + ActionListener listener + ) { + ClusterState clusterState = clusterService.state(); + GroupShardsIterator shardIterators = clusterService.operationRouting() + .searchShards(clusterState, new String[] { index }, Map.of(), "_local"); + if (shardIterators.size() != 1) { + listener.onFailure(new EsqlIllegalArgumentException("target index {} has more than one shard", index)); + return; + } + ShardIterator shardIt = shardIterators.get(0); + ShardRouting shardRouting = shardIt.nextOrNull(); + if (shardRouting == null) { + listener.onFailure(new UnavailableShardsException(shardIt.shardId(), "enrich index is not available")); + return; + } + DiscoveryNode targetNode = clusterState.nodes().get(shardRouting.currentNodeId()); + LookupRequest lookupRequest = new LookupRequest(sessionId, shardIt.shardId(), matchType, matchField, inputPage, extractFields); + ThreadContext threadContext = transportService.getThreadPool().getThreadContext(); + listener = ContextPreservingActionListener.wrapPreservingContext(listener, threadContext); + try (ThreadContext.StoredContext ignored = threadContext.stashWithOrigin(ClientHelper.ENRICH_ORIGIN)) { + // TODO: handle retry and avoid forking for the local lookup + transportService.sendChildRequest( + targetNode, + LOOKUP_ACTION_NAME, + lookupRequest, + parentTask, + TransportRequestOptions.EMPTY, + new ActionListenerResponseHandler<>( + listener.map(r -> r.page), + LookupResponse::new, + TransportResponseHandler.TRANSPORT_WORKER + ) + ); + } + } + + private void doLookup( + String sessionId, + CancellableTask task, + ShardId shardId, + String matchType, + String matchField, + Page inputPage, + List extractFields, + ActionListener listener + ) { + Block inputBlock = inputPage.getBlock(0); + if (inputBlock.areAllValuesNull()) { + listener.onResponse(createNullResponse(inputPage.getPositionCount(), extractFields)); + return; + } + try { + ShardSearchRequest shardSearchRequest = new ShardSearchRequest(shardId, 0, AliasFilter.EMPTY); + SearchContext searchContext = searchService.createSearchContext(shardSearchRequest, SearchService.NO_TIMEOUT); + listener = ActionListener.runBefore(listener, searchContext::close); + SearchExecutionContext searchExecutionContext = searchContext.getSearchExecutionContext(); + MappedFieldType fieldType = searchExecutionContext.getFieldType(matchField); + final SourceOperator queryOperator = switch (matchType) { + case "match", "range" -> { + QueryList queryList = QueryList.termQueryList(fieldType, searchExecutionContext, inputBlock); + yield new EnrichQuerySourceOperator(queryList, searchExecutionContext.getIndexReader()); + } + default -> throw new UnsupportedOperationException("unsupported match type " + matchType); + }; + List intermediateOperators = new ArrayList<>(extractFields.size() + 2); + final ElementType[] mergingTypes = new ElementType[extractFields.size()]; + // extract-field operators + for (int i = 0; i < extractFields.size(); i++) { + NamedExpression extractField = extractFields.get(i); + final ElementType elementType = LocalExecutionPlanner.toElementType(extractField.dataType()); + mergingTypes[i] = elementType; + var sources = ValueSources.sources( + List.of(searchContext), + extractField instanceof Alias a ? ((NamedExpression) a.child()).name() : extractField.name(), + EsqlDataTypes.isUnsupported(extractField.dataType()), + elementType + ); + intermediateOperators.add(new ValuesSourceReaderOperator(sources, 0, extractField.name())); + } + // drop docs block + intermediateOperators.add(droppingBlockOperator(extractFields.size() + 2, 0)); + boolean singleLeaf = searchContext.searcher().getLeafContexts().size() == 1; + // merging field-values by position + final int[] mergingChannels = IntStream.range(0, extractFields.size()).map(i -> i + 1).toArray(); + intermediateOperators.add( + new MergePositionsOperator(singleLeaf, inputPage.getPositionCount(), 0, mergingChannels, mergingTypes) + ); + AtomicReference result = new AtomicReference<>(); + OutputOperator outputOperator = new OutputOperator(List.of(), Function.identity(), result::set); + Driver driver = new Driver( + "enrich-lookup:" + sessionId, + new DriverContext(), + () -> lookupDescription(sessionId, shardId, matchType, matchField, extractFields, inputPage.getPositionCount()), + queryOperator, + intermediateOperators, + outputOperator, + searchContext + ); + task.addListener(() -> { + String reason = Objects.requireNonNullElse(task.getReasonCancelled(), "task was cancelled"); + driver.cancel(reason); + }); + Driver.start(executor, driver, Driver.DEFAULT_MAX_ITERATIONS, listener.map(ignored -> { + Page out = result.get(); + if (out == null) { + out = createNullResponse(inputPage.getPositionCount(), extractFields); + } + return out; + })); + } catch (Exception e) { + listener.onFailure(e); + } + } + + private static Page createNullResponse(int positionCount, List extractFields) { + final Block[] blocks = new Block[extractFields.size()]; + for (int i = 0; i < extractFields.size(); i++) { + blocks[i] = Block.constantNullBlock(positionCount); + } + return new Page(blocks); + } + + private static Operator droppingBlockOperator(int totalBlocks, int droppingPosition) { + BitSet bitSet = new BitSet(totalBlocks); + bitSet.set(0, totalBlocks); + bitSet.clear(droppingPosition); + return new ProjectOperator(bitSet); + } + + private class TransportHandler implements TransportRequestHandler { + @Override + public void messageReceived(LookupRequest request, TransportChannel channel, Task task) { + ActionListener listener = new ChannelActionListener<>(channel); + doLookup( + request.sessionId, + (CancellableTask) task, + request.shardId, + request.matchType, + request.matchField, + request.inputPage, + request.extractFields, + listener.map(LookupResponse::new) + ); + } + } + + private static class LookupRequest extends TransportRequest implements IndicesRequest { + private final String sessionId; + private final ShardId shardId; + private final String matchType; + private final String matchField; + private final Page inputPage; + private final List extractFields; + + LookupRequest( + String sessionId, + ShardId shardId, + String matchType, + String matchField, + Page inputPage, + List extractFields + ) { + this.sessionId = sessionId; + this.shardId = shardId; + this.matchType = matchType; + this.matchField = matchField; + this.inputPage = inputPage; + this.extractFields = extractFields; + } + + LookupRequest(StreamInput in) throws IOException { + super(in); + this.sessionId = in.readString(); + this.shardId = new ShardId(in); + this.matchType = in.readString(); + this.matchField = in.readString(); + this.inputPage = new Page(in); + PlanStreamInput planIn = new PlanStreamInput(in, PlanNameRegistry.INSTANCE, in.namedWriteableRegistry(), null); + this.extractFields = planIn.readList(readerFromPlanReader(PlanStreamInput::readNamedExpression)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sessionId); + out.writeWriteable(shardId); + out.writeString(matchType); + out.writeString(matchField); + out.writeWriteable(inputPage); + PlanStreamOutput planOut = new PlanStreamOutput(out, PlanNameRegistry.INSTANCE); + planOut.writeCollection(extractFields, writerFromPlanWriter(PlanStreamOutput::writeNamedExpression)); + } + + @Override + public String[] indices() { + return new String[] { shardId.getIndexName() }; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, "", parentTaskId, headers) { + @Override + public String getDescription() { + return lookupDescription(sessionId, shardId, matchType, matchField, extractFields, inputPage.getPositionCount()); + } + }; + } + } + + private static String lookupDescription( + String sessionId, + ShardId shardId, + String matchType, + String matchField, + List extractFields, + int positionCount + ) { + return "ENRICH_LOOKUP(" + + " session=" + + sessionId + + " ,shard=" + + shardId + + " ,match_type=" + + matchType + + " ,match_field=" + + matchField + + " ,extract_fields=" + + extractFields + + " ,positions=" + + positionCount + + ")"; + } + + private static class LookupResponse extends TransportResponse { + private final Page page; + + LookupResponse(Page page) { + this.page = page; + } + + LookupResponse(StreamInput in) throws IOException { + this.page = new Page(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + page.writeTo(out); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolution.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolution.java new file mode 100644 index 0000000000000..5014fe1fcd1df --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolution.java @@ -0,0 +1,13 @@ +/* + * 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.enrich; + +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.ql.index.IndexResolution; + +public record EnrichPolicyResolution(String policyName, EnrichPolicy policy, IndexResolution index) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java new file mode 100644 index 0000000000000..69234ffc7834e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java @@ -0,0 +1,61 @@ +/* + * 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.enrich; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ContextPreservingActionListener; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ClientHelper; +import org.elasticsearch.xpack.core.enrich.EnrichMetadata; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.ql.index.IndexResolver; + +import java.util.Map; +import java.util.Set; + +public class EnrichPolicyResolver { + + private final ClusterService clusterService; + private final IndexResolver indexResolver; + private final ThreadPool threadPool; + + public EnrichPolicyResolver(ClusterService clusterService, IndexResolver indexResolver, ThreadPool threadPool) { + this.clusterService = clusterService; + this.indexResolver = indexResolver; + this.threadPool = threadPool; + } + + public void resolvePolicy(String policyName, ActionListener listener) { + EnrichPolicy policy = policies().get(policyName); + ThreadContext threadContext = threadPool.getThreadContext(); + listener = ContextPreservingActionListener.wrapPreservingContext(listener, threadContext); + try (ThreadContext.StoredContext ignored = threadContext.stashWithOrigin(ClientHelper.ENRICH_ORIGIN)) { + indexResolver.resolveAsMergedMapping( + EnrichPolicy.getBaseName(policyName), + false, + Map.of(), + listener.map(indexResult -> new EnrichPolicyResolution(policyName, policy, indexResult)) + ); + } + } + + public Set allPolicyNames() { + return policies().keySet(); + } + + private Map policies() { + if (clusterService == null || clusterService.state() == null) { + return Map.of(); + } + EnrichMetadata metadata = clusterService.state().metadata().custom(EnrichMetadata.TYPE); + return metadata == null ? Map.of() : metadata.getPolicies(); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperator.java new file mode 100644 index 0000000000000..de785e161d527 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperator.java @@ -0,0 +1,128 @@ +/* + * 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.enrich; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.ConstantScoreQuery; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorable; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Weight; +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.SourceOperator; + +import java.io.IOException; +import java.io.UncheckedIOException; + +/** + * Lookup document IDs for the input queries. + * This operator will emit Pages consisting of a {@link DocVector} and {@link IntBlock} of positions for each query of the input queries. + * The position block will be used as keys to combine the extracted values by {@link MergePositionsOperator}. + */ +final class EnrichQuerySourceOperator extends SourceOperator { + + private final QueryList queryList; + private int queryPosition; + private Weight weight = null; + private final IndexReader indexReader; + private int leafIndex = 0; + private final IndexSearcher searcher; + + EnrichQuerySourceOperator(QueryList queryList, IndexReader indexReader) { + this.queryList = queryList; + this.indexReader = indexReader; + this.searcher = new IndexSearcher(indexReader); + } + + @Override + public void finish() {} + + @Override + public boolean isFinished() { + return queryPosition >= queryList.getPositionCount(); + } + + @Override + public Page getOutput() { + if (leafIndex == indexReader.leaves().size()) { + queryPosition++; + leafIndex = 0; + weight = null; + } + if (isFinished()) { + return null; + } + if (weight == null) { + Query query = queryList.getQuery(queryPosition); + if (query != null) { + try { + query = searcher.rewrite(new ConstantScoreQuery(query)); + weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + try { + return queryOneLeaf(weight, leafIndex++); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + private Page queryOneLeaf(Weight weight, int leafIndex) throws IOException { + if (weight == null) { + return null; + } + LeafReaderContext leafReaderContext = indexReader.leaves().get(leafIndex); + var scorer = weight.bulkScorer(leafReaderContext); + if (scorer == null) { + return null; + } + DocCollector collector = new DocCollector(); + scorer.score(collector, leafReaderContext.reader().getLiveDocs()); + final int matches = collector.matches; + DocVector docVector = new DocVector( + new ConstantIntVector(0, matches), + new ConstantIntVector(leafIndex, matches), + new IntArrayVector(collector.docs, matches), + true + ); + IntBlock positionBlock = new ConstantIntVector(queryPosition, matches).asBlock(); + return new Page(docVector.asBlock(), positionBlock); + } + + private static class DocCollector implements LeafCollector { + int matches = 0; + int[] docs = new int[0]; + + @Override + public void setScorer(Scorable scorer) { + + } + + @Override + public void collect(int doc) throws IOException { + docs = ArrayUtil.grow(docs, matches + 1); + docs[matches++] = doc; + } + } + + @Override + public void close() { + + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperator.java new file mode 100644 index 0000000000000..e840838a1d848 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperator.java @@ -0,0 +1,172 @@ +/* + * 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.enrich; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Operator; + +import java.util.Arrays; + +/** + * Combines values at the given blocks with the same positions into a single position for the blocks at the given channels + * Example, input pages consisting of three blocks: + * positions | field-1 | field-2 | + * ----------------------------------- + * Page 1: + * 1 | a,b | 2020 | + * 1 | c | 2021 | + * --------------------------------- + * Page 2: + * 2 | a,e | 2021 | + * --------------------------------- + * Page 3: + * 4 | d | null | + * --------------------------------- + * Output: + * | field-1 | field-2 | + * --------------------------- + * | null | null | + * | a,b,c | 2020,2021 | + * | a,e | 2021 | + * | null | null | + * | d | 2023 | + */ +final class MergePositionsOperator implements Operator { + private boolean finished = false; + private int filledPositions = 0; + private final boolean singleMode; + private final int positionCount; + private final int positionChannel; + + private final Block.Builder[] outputBuilders; + private final int[] mergingChannels; + private final ElementType[] mergingTypes; + private PositionBuilder positionBuilder = null; + + private Page outputPage; + + MergePositionsOperator(boolean singleMode, int positionCount, int positionChannel, int[] mergingChannels, ElementType[] mergingTypes) { + if (mergingChannels.length != mergingTypes.length) { + throw new IllegalArgumentException( + "Merging channels don't match merging types; channels=" + + Arrays.toString(mergingChannels) + + ",types=" + + Arrays.toString(mergingTypes) + ); + } + this.singleMode = singleMode; + this.positionCount = positionCount; + this.positionChannel = positionChannel; + this.mergingChannels = mergingChannels; + this.mergingTypes = mergingTypes; + this.outputBuilders = new Block.Builder[mergingTypes.length]; + for (int i = 0; i < mergingTypes.length; i++) { + outputBuilders[i] = mergingTypes[i].newBlockBuilder(positionCount); + } + } + + @Override + public boolean needsInput() { + return true; + } + + @Override + public void addInput(Page page) { + final IntBlock positions = page.getBlock(positionChannel); + final int currentPosition = positions.getInt(0); + if (singleMode) { + fillNullUpToPosition(currentPosition); + for (int i = 0; i < mergingChannels.length; i++) { + int channel = mergingChannels[i]; + outputBuilders[i].appendAllValuesToCurrentPosition(page.getBlock(channel)); + } + filledPositions++; + } else { + if (positionBuilder != null && positionBuilder.position != currentPosition) { + flushPositionBuilder(); + } + if (positionBuilder == null) { + positionBuilder = new PositionBuilder(currentPosition, mergingTypes); + } + positionBuilder.combine(page, mergingChannels); + } + } + + static final class PositionBuilder { + private final int position; + private final Block.Builder[] builders; + + PositionBuilder(int position, ElementType[] elementTypes) { + this.position = position; + this.builders = new Block.Builder[elementTypes.length]; + for (int i = 0; i < builders.length; i++) { + builders[i] = elementTypes[i].newBlockBuilder(1); + } + } + + void combine(Page page, int[] channels) { + for (int i = 0; i < channels.length; i++) { + builders[i].appendAllValuesToCurrentPosition(page.getBlock(channels[i])); + } + } + + void buildTo(Block.Builder[] output) { + for (int i = 0; i < output.length; i++) { + output[i].appendAllValuesToCurrentPosition(builders[i].build()); + } + } + } + + private void flushPositionBuilder() { + fillNullUpToPosition(positionBuilder.position); + filledPositions++; + positionBuilder.buildTo(outputBuilders); + positionBuilder = null; + } + + private void fillNullUpToPosition(int position) { + while (filledPositions < position) { + for (Block.Builder builder : outputBuilders) { + builder.appendNull(); + } + filledPositions++; + } + } + + @Override + public void finish() { + if (positionBuilder != null) { + flushPositionBuilder(); + } + fillNullUpToPosition(positionCount); + Block[] blocks = Arrays.stream(outputBuilders).map(Block.Builder::build).toArray(Block[]::new); + outputPage = new Page(blocks); + finished = true; + assert outputPage.getPositionCount() == positionCount; + } + + @Override + public boolean isFinished() { + return finished && outputPage == null; + } + + @Override + public Page getOutput() { + Page page = this.outputPage; + this.outputPage = null; + return page; + } + + @Override + public void close() { + + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/QueryList.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/QueryList.java new file mode 100644 index 0000000000000..61ffb9c0dcb11 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/QueryList.java @@ -0,0 +1,103 @@ +/* + * 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.enrich; + +import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.query.SearchExecutionContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.IntFunction; + +/** + * Generates a list of Lucene queries based on the input block. + */ +abstract class QueryList { + protected final Block block; + + protected QueryList(Block block) { + this.block = block; + } + + /** + * Returns the number of positions in this query list + */ + int getPositionCount() { + return block.getPositionCount(); + } + + /** + * Returns the query at the given position. + */ + @Nullable + abstract Query getQuery(int position); + + /** + * Returns a list of term queries for the given field and the input block. + */ + static QueryList termQueryList(MappedFieldType field, SearchExecutionContext searchExecutionContext, Block block) { + return new QueryList(block) { + private final IntFunction blockValueReader = QueryList.blockToJavaObject(block); + + @Override + Query getQuery(int position) { + final int first = block.getFirstValueIndex(position); + final int count = block.getValueCount(position); + return switch (count) { + case 0 -> null; + case 1 -> field.termQuery(blockValueReader.apply(first), searchExecutionContext); + default -> { + final List terms = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + final Object value = blockValueReader.apply(first + i); + terms.add(value); + } + yield field.termsQuery(terms, searchExecutionContext); + } + }; + } + }; + } + + private static IntFunction blockToJavaObject(Block block) { + return switch (block.elementType()) { + case BOOLEAN -> { + BooleanBlock booleanBlock = (BooleanBlock) block; + yield booleanBlock::getBoolean; + } + case BYTES_REF -> { + BytesRefBlock bytesRefBlock = (BytesRefBlock) block; + yield offset -> bytesRefBlock.getBytesRef(offset, new BytesRef()); + } + case DOUBLE -> { + DoubleBlock doubleBlock = ((DoubleBlock) block); + yield doubleBlock::getDouble; + } + case INT -> { + IntBlock intBlock = (IntBlock) block; + yield intBlock::getInt; + } + case LONG -> { + LongBlock longBlock = (LongBlock) block; + yield longBlock::getLong; + } + case NULL -> offset -> null; + case DOC -> throw new UnsupportedOperationException("can't read values from doc block"); + case UNKNOWN -> throw new IllegalArgumentException("can't read values from [" + block + "]"); + }; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java new file mode 100644 index 0000000000000..798927d2c9329 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java @@ -0,0 +1,77 @@ +/* + * 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.execution; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.analysis.PreAnalyzer; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolver; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.session.EsqlSession; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.stats.QueryMetric; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.IndexResolver; + +import static org.elasticsearch.action.ActionListener.wrap; + +public class PlanExecutor { + + private final IndexResolver indexResolver; + private final EnrichPolicyResolver enrichPolicyResolver; + private final PreAnalyzer preAnalyzer; + private final FunctionRegistry functionRegistry; + private final LogicalPlanOptimizer logicalPlanOptimizer; + private final Mapper mapper; + private final Metrics metrics; + private final Verifier verifier; + + public PlanExecutor(IndexResolver indexResolver, EnrichPolicyResolver enrichPolicyResolver) { + this.indexResolver = indexResolver; + this.enrichPolicyResolver = enrichPolicyResolver; + this.preAnalyzer = new PreAnalyzer(); + this.functionRegistry = new EsqlFunctionRegistry(); + this.logicalPlanOptimizer = new LogicalPlanOptimizer(); + this.mapper = new Mapper(functionRegistry); + this.metrics = new Metrics(); + this.verifier = new Verifier(metrics); + } + + public void esql(EsqlQueryRequest request, String sessionId, EsqlConfiguration cfg, ActionListener listener) { + QueryMetric clientId = QueryMetric.fromString("rest"); + metrics.total(clientId); + newSession(sessionId, cfg).execute(request, wrap(listener::onResponse, ex -> { + // TODO when we decide if we will differentiate Kibana from REST, this String value will likely come from the request + metrics.failed(clientId); + listener.onFailure(ex); + })); + } + + private EsqlSession newSession(String sessionId, EsqlConfiguration cfg) { + return new EsqlSession( + sessionId, + cfg, + indexResolver, + enrichPolicyResolver, + preAnalyzer, + functionRegistry, + logicalPlanOptimizer, + mapper, + verifier + ); + } + + public Metrics metrics() { + return this.metrics; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/NamedExpressions.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/NamedExpressions.java new file mode 100644 index 0000000000000..8c3e32d971d5b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/NamedExpressions.java @@ -0,0 +1,60 @@ +/* + * 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.expression; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; + +import java.util.ArrayList; +import java.util.List; + +public class NamedExpressions { + + /** + * Calculates the actual output of a command given the new attributes plus the existing inputs that are emitted as outputs + * @param fields the fields added by the command + * @param childOutput the command input that has to be propagated as output + * @return + */ + public static List mergeOutputAttributes( + List fields, + List childOutput + ) { + return Expressions.asAttributes(mergeOutputExpressions(fields, childOutput)); + } + + /** + * Merges output expressions of a command given the new attributes plus the existing inputs that are emitted as outputs. + * As a general rule, child output will come first in the list, followed by the new fields. + * In case of name collisions, only last entry is preserved (previous expressions with the same name are discarded) + * @param fields the fields added by the command + * @param childOutput the command input that has to be propagated as output + * @return + */ + public static List mergeOutputExpressions( + List fields, + List childOutput + ) { + List fieldNames = Expressions.names(fields); + List output = new ArrayList<>(childOutput.size() + fields.size()); + for (NamedExpression childAttr : childOutput) { + if (fieldNames.contains(childAttr.name()) == false) { + output.add(childAttr); + } + } + // do not add duplicate fields multiple times, only last one matters as output + for (int i = 0; i < fields.size(); i++) { + NamedExpression field = fields.get(i); + if (fieldNames.lastIndexOf(field.name()) == i) { + output.add(field); + } + } + return output; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/SurrogateExpression.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/SurrogateExpression.java new file mode 100644 index 0000000000000..e7f507e3983a7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/SurrogateExpression.java @@ -0,0 +1,19 @@ +/* + * 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.expression; + +import org.elasticsearch.xpack.ql.expression.Expression; + +/** + * Interface signaling to the planner that the declaring expression + * has to be replaced by a different form: e.g. avg = sum / count + */ +public interface SurrogateExpression { + + Expression surrogate(); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java new file mode 100644 index 0000000000000..9a16435684648 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java @@ -0,0 +1,175 @@ +/* + * 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.expression.function; + +import org.elasticsearch.xpack.esql.expression.function.aggregate.Avg; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.function.aggregate.CountDistinct; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Max; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Median; +import org.elasticsearch.xpack.esql.expression.function.aggregate.MedianAbsoluteDeviation; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Min; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Percentile; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.esql.expression.function.scalar.conditional.Case; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToBoolean; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDatetime; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDegrees; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDouble; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToIP; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToInteger; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToLong; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToRadians; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToString; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToUnsignedLong; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToVersion; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateExtract; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateFormat; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateParse; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.Now; +import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Abs; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Acos; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Asin; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Atan; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Atan2; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.AutoBucket; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Cos; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Cosh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.E; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Floor; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsFinite; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsInfinite; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsNaN; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Log10; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pi; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sin; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sinh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sqrt; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tan; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tanh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tau; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvAvg; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvConcat; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvCount; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvDedupe; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMax; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMedian; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMin; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvSum; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Concat; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Length; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Split; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.StartsWith; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Trim; +import org.elasticsearch.xpack.ql.expression.function.FunctionDefinition; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; + +import java.util.Locale; + +public class EsqlFunctionRegistry extends FunctionRegistry { + + public EsqlFunctionRegistry() { + register(functions()); + } + + EsqlFunctionRegistry(FunctionDefinition... functions) { + register(functions); + } + + private FunctionDefinition[][] functions() { + return new FunctionDefinition[][] { + // aggregate functions + new FunctionDefinition[] { + def(Avg.class, Avg::new, "avg"), + def(Count.class, Count::new, "count"), + def(CountDistinct.class, CountDistinct::new, "count_distinct"), + def(Max.class, Max::new, "max"), + def(Median.class, Median::new, "median"), + def(MedianAbsoluteDeviation.class, MedianAbsoluteDeviation::new, "median_absolute_deviation"), + def(Min.class, Min::new, "min"), + def(Percentile.class, Percentile::new, "percentile"), + def(Sum.class, Sum::new, "sum") }, + // math + new FunctionDefinition[] { + def(Abs.class, Abs::new, "abs"), + def(Acos.class, Acos::new, "acos"), + def(Asin.class, Asin::new, "asin"), + def(Atan.class, Atan::new, "atan"), + def(Atan2.class, Atan2::new, "atan2"), + def(AutoBucket.class, AutoBucket::new, "auto_bucket"), + def(Cos.class, Cos::new, "cos"), + def(Cosh.class, Cosh::new, "cosh"), + def(E.class, E::new, "e"), + def(Floor.class, Floor::new, "floor"), + def(IsFinite.class, IsFinite::new, "is_finite"), + def(IsInfinite.class, IsInfinite::new, "is_infinite"), + def(IsNaN.class, IsNaN::new, "is_nan"), + def(Log10.class, Log10::new, "log10"), + def(Pi.class, Pi::new, "pi"), + def(Pow.class, Pow::new, "pow"), + def(Round.class, Round::new, "round"), + def(Sin.class, Sin::new, "sin"), + def(Sinh.class, Sinh::new, "sinh"), + def(Sqrt.class, Sqrt::new, "sqrt"), + def(Tan.class, Tan::new, "tan"), + def(Tanh.class, Tanh::new, "tanh"), + def(Tau.class, Tau::new, "tau") }, + // string + new FunctionDefinition[] { + def(Length.class, Length::new, "length"), + def(Substring.class, Substring::new, "substring"), + def(Concat.class, Concat::new, "concat"), + def(Trim.class, Trim::new, "trim"), + def(StartsWith.class, StartsWith::new, "starts_with") }, + // date + new FunctionDefinition[] { + def(DateExtract.class, DateExtract::new, "date_extract"), + def(DateFormat.class, DateFormat::new, "date_format"), + def(DateParse.class, DateParse::new, "date_parse"), + def(DateTrunc.class, DateTrunc::new, "date_trunc"), + def(Now.class, Now::new, "now") }, + // conditional + new FunctionDefinition[] { def(Case.class, Case::new, "case") }, + // IP + new FunctionDefinition[] { def(CIDRMatch.class, CIDRMatch::new, "cidr_match") }, + // conversion functions + new FunctionDefinition[] { + def(ToBoolean.class, ToBoolean::new, "to_boolean", "to_bool"), + def(ToDatetime.class, ToDatetime::new, "to_datetime", "to_dt"), + def(ToDegrees.class, ToDegrees::new, "to_degrees"), + def(ToDouble.class, ToDouble::new, "to_double", "to_dbl"), + def(ToIP.class, ToIP::new, "to_ip"), + def(ToInteger.class, ToInteger::new, "to_integer", "to_int"), + def(ToLong.class, ToLong::new, "to_long"), + def(ToRadians.class, ToRadians::new, "to_radians"), + def(ToString.class, ToString::new, "to_string", "to_str"), + def(ToUnsignedLong.class, ToUnsignedLong::new, "to_unsigned_long", "to_ulong", "to_ul"), + def(ToVersion.class, ToVersion::new, "to_version", "to_ver"), }, + // multivalue functions + new FunctionDefinition[] { + def(MvAvg.class, MvAvg::new, "mv_avg"), + def(MvConcat.class, MvConcat::new, "mv_concat"), + def(MvCount.class, MvCount::new, "mv_count"), + def(MvDedupe.class, MvDedupe::new, "mv_dedupe"), + def(MvMax.class, MvMax::new, "mv_max"), + def(MvMedian.class, MvMedian::new, "mv_median"), + def(MvMin.class, MvMin::new, "mv_min"), + def(MvSum.class, MvSum::new, "mv_sum"), + def(Split.class, Split::new, "split") } }; + } + + @Override + protected String normalize(String name) { + return name.toLowerCase(Locale.ROOT); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/FunctionDoc.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/FunctionDoc.java new file mode 100644 index 0000000000000..f6514c2a44ecd --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/FunctionDoc.java @@ -0,0 +1,37 @@ +/* + * 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.expression.function; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Retention(RetentionPolicy.RUNTIME) +@Target({ ElementType.TYPE }) +public @interface FunctionDoc { + enum FunctionType { + AGGREGATE, + DATE_TIME, + MATH, + STRING, + } + + FunctionType type(); + + String description(); + + String synopsis(); + + String[] arguments() default {}; + + String output(); + + String examples(); + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/UnsupportedAttribute.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/UnsupportedAttribute.java new file mode 100644 index 0000000000000..b8353b8789786 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/UnsupportedAttribute.java @@ -0,0 +1,115 @@ +/* + * 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.expression.function; + +import org.elasticsearch.xpack.ql.capabilities.Unresolvable; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.UnsupportedEsField; + +import java.util.Objects; + +/** + * Unsupported attribute meaning an attribute that has been found yet cannot be used (hence why UnresolvedAttribute + * cannot be used) expect in special conditions (currently only in projections to allow it to flow through + * the engine). + * As such the field is marked as unresolved (so the verifier can pick up its usage outside project). + */ +public class UnsupportedAttribute extends FieldAttribute implements Unresolvable { + + private final String message; + private final boolean hasCustomMessage; + + private static String errorMessage(String name, UnsupportedEsField field) { + return "Cannot use field [" + name + "] with unsupported type [" + field.getOriginalType() + "]"; + } + + public UnsupportedAttribute(Source source, String name, UnsupportedEsField field) { + this(source, name, field, null); + } + + public UnsupportedAttribute(Source source, String name, UnsupportedEsField field, String customMessage) { + this(source, name, field, customMessage, null); + } + + public UnsupportedAttribute(Source source, String name, UnsupportedEsField field, String customMessage, NameId id) { + super(source, null, name, field, null, Nullability.TRUE, id, false); + this.hasCustomMessage = customMessage != null; + this.message = customMessage == null ? errorMessage(qualifiedName(), field) : customMessage; + } + + @Override + public boolean resolved() { + return false; + } + + @Override + public UnsupportedEsField field() { + return (UnsupportedEsField) super.field(); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, UnsupportedAttribute::new, name(), field(), hasCustomMessage ? message : null, id()); + } + + @Override + protected Attribute clone( + Source source, + String name, + DataType type, + String qualifier, + Nullability nullability, + NameId id, + boolean synthetic + ) { + return new UnsupportedAttribute(source, name, field(), hasCustomMessage ? message : null, id); + } + + protected String label() { + return "!"; + } + + @Override + public String toString() { + return "!" + qualifiedName(); + } + + @Override + public String nodeString() { + return toString(); + } + + @Override + public String unresolvedMessage() { + return message; + } + + public boolean hasCustomMessage() { + return hasCustomMessage; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), hasCustomMessage, message); + } + + @Override + public boolean equals(Object obj) { + if (super.equals(obj)) { + var ua = (UnsupportedAttribute) obj; + return Objects.equals(hasCustomMessage, ua.hasCustomMessage) && Objects.equals(message, ua.message); + } + return false; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/Warnings.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/Warnings.java new file mode 100644 index 0000000000000..5cca86294ac53 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/Warnings.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function; + +import org.elasticsearch.xpack.ql.tree.Source; + +import static org.elasticsearch.common.logging.HeaderWarning.addWarning; + +/** + * Utilities to collect warnings for running an executor. + */ +public class Warnings { + static final int MAX_ADDED_WARNINGS = 20; + + private final Source source; + + private int addedWarnings; + + public Warnings(Source source) { + this.source = source; + } + + public void registerException(Exception exception) { + if (addedWarnings < MAX_ADDED_WARNINGS) { + if (addedWarnings == 0) { + addWarning( + "Line {}:{}: evaluation of [{}] failed, treating result as null. Only first {} failures recorded.", + source.source().getLineNumber(), + source.source().getColumnNumber(), + source.text(), + MAX_ADDED_WARNINGS + ); + } + addWarning(exception.getClass().getName() + ": " + exception.getMessage()); + addedWarnings++; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Avg.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Avg.java new file mode 100644 index 0000000000000..c4b8ef4d43273 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Avg.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.xpack.esql.expression.SurrogateExpression; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Avg extends AggregateFunction implements SurrogateExpression { + + public Avg(Source source, Expression field) { + super(source, field); + } + + @Override + protected Expression.TypeResolution resolveType() { + return isNumeric(field(), sourceText(), DEFAULT); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Avg::new, field()); + } + + @Override + public Avg replaceChildren(List newChildren) { + return new Avg(source(), newChildren.get(0)); + } + + @Override + public Expression surrogate() { + var s = source(); + var field = field(); + return new Div(s, new Sum(s, field), new Count(s, field), dataType()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java new file mode 100644 index 0000000000000..fce52374bfab0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java @@ -0,0 +1,60 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountAggregatorFunction; +import org.elasticsearch.xpack.esql.planner.ToAggregator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.function.aggregate.EnclosedAgg; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +public class Count extends AggregateFunction implements EnclosedAgg, ToAggregator { + + public Count(Source source, Expression field) { + super(source, field); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Count::new, field()); + } + + @Override + public Count replaceChildren(List newChildren) { + return new Count(source(), newChildren.get(0)); + } + + @Override + public String innerName() { + return "count"; + } + + @Override + public DataType dataType() { + return DataTypes.LONG; + } + + @Override + public AggregatorFunctionSupplier supplier(BigArrays bigArrays, List inputChannels) { + return CountAggregatorFunction.supplier(bigArrays, inputChannels); + } + + @Override + public Nullability nullable() { + return Nullability.FALSE; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountDistinct.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountDistinct.java new file mode 100644 index 0000000000000..4e2dbc98bdafc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountDistinct.java @@ -0,0 +1,91 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctBooleanAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctBytesRefAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.CountDistinctLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.esql.planner.ToAggregator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isInteger; + +public class CountDistinct extends AggregateFunction implements OptionalArgument, ToAggregator { + private static final int DEFAULT_PRECISION = 3000; + private final Expression precision; + + public CountDistinct(Source source, Expression field, Expression precision) { + super(source, field, precision != null ? List.of(precision) : List.of()); + this.precision = precision; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, CountDistinct::new, field(), precision); + } + + @Override + public CountDistinct replaceChildren(List newChildren) { + return new CountDistinct(source(), newChildren.get(0), newChildren.size() > 1 ? newChildren.get(1) : null); + } + + @Override + public DataType dataType() { + return DataTypes.LONG; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = super.resolveType(); + if (resolution.unresolved() || precision == null) { + return resolution; + } + + return isInteger(precision, sourceText(), SECOND); + } + + @Override + public AggregatorFunctionSupplier supplier(BigArrays bigArrays, List inputChannels) { + DataType type = field().dataType(); + int precision = this.precision == null ? DEFAULT_PRECISION : ((Number) this.precision.fold()).intValue(); + if (type == DataTypes.BOOLEAN) { + // Booleans ignore the precision because there are only two possible values anyway + return new CountDistinctBooleanAggregatorFunctionSupplier(bigArrays, inputChannels); + } + if (type == DataTypes.DATETIME || type == DataTypes.LONG) { + return new CountDistinctLongAggregatorFunctionSupplier(bigArrays, inputChannels, precision); + } + if (type == DataTypes.INTEGER) { + return new CountDistinctIntAggregatorFunctionSupplier(bigArrays, inputChannels, precision); + } + if (type == DataTypes.DOUBLE) { + return new CountDistinctDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, precision); + } + if (type == DataTypes.KEYWORD || type == DataTypes.IP) { + return new CountDistinctBytesRefAggregatorFunctionSupplier(bigArrays, inputChannels, precision); + } + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Max.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Max.java new file mode 100644 index 0000000000000..7b65d4ba40b1e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Max.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MaxLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; + +public class Max extends NumericAggregate { + + public Max(Source source, Expression field) { + super(source, field); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Max::new, field()); + } + + @Override + public Max replaceChildren(List newChildren) { + return new Max(source(), newChildren.get(0)); + } + + @Override + protected boolean supportsDates() { + return true; + } + + @Override + public DataType dataType() { + return field().dataType(); + } + + @Override + protected AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels) { + return new MaxLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels) { + return new MaxIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels) { + return new MaxDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Median.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Median.java new file mode 100644 index 0000000000000..cffeb925d5e2b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Median.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.aggregate; + +import org.elasticsearch.compute.aggregation.QuantileStates; +import org.elasticsearch.xpack.esql.expression.SurrogateExpression; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Median extends AggregateFunction implements SurrogateExpression { + // TODO: Add the compression parameter + public Median(Source source, Expression field) { + super(source, field); + } + + @Override + protected Expression.TypeResolution resolveType() { + return isNumeric(field(), sourceText(), DEFAULT); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Median::new, field()); + } + + @Override + public Median replaceChildren(List newChildren) { + return new Median(source(), newChildren.get(0)); + } + + @Override + public Expression surrogate() { + return new Percentile(source(), field(), new Literal(source(), (int) QuantileStates.MEDIAN, DataTypes.INTEGER)); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/MedianAbsoluteDeviation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/MedianAbsoluteDeviation.java new file mode 100644 index 0000000000000..cb7bac2c2f66e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/MedianAbsoluteDeviation.java @@ -0,0 +1,52 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MedianAbsoluteDeviationIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MedianAbsoluteDeviationLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +public class MedianAbsoluteDeviation extends NumericAggregate { + + // TODO: Add parameter + public MedianAbsoluteDeviation(Source source, Expression field) { + super(source, field); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MedianAbsoluteDeviation::new, field()); + } + + @Override + public MedianAbsoluteDeviation replaceChildren(List newChildren) { + return new MedianAbsoluteDeviation(source(), newChildren.get(0)); + } + + @Override + protected AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels) { + return new MedianAbsoluteDeviationDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Min.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Min.java new file mode 100644 index 0000000000000..6a0e4aa52e721 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Min.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MinDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MinIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.MinLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; + +public class Min extends NumericAggregate { + + public Min(Source source, Expression field) { + super(source, field); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Min::new, field()); + } + + @Override + public Min replaceChildren(List newChildren) { + return new Min(source(), newChildren.get(0)); + } + + @Override + public DataType dataType() { + return field().dataType(); + } + + @Override + protected boolean supportsDates() { + return true; + } + + @Override + protected AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels) { + return new MinLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels) { + return new MinIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels) { + return new MinDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/NumericAggregate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/NumericAggregate.java new file mode 100644 index 0000000000000..50ce22f9d3283 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/NumericAggregate.java @@ -0,0 +1,81 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.xpack.esql.planner.ToAggregator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public abstract class NumericAggregate extends AggregateFunction implements ToAggregator { + + NumericAggregate(Source source, Expression field, List parameters) { + super(source, field, parameters); + } + + NumericAggregate(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveType() { + if (supportsDates()) { + return TypeResolutions.isType( + this, + e -> e.isNumeric() || e == DataTypes.DATETIME, + sourceText(), + DEFAULT, + "numeric", + "datetime" + ); + } + return isNumeric(field(), sourceText(), DEFAULT); + } + + protected boolean supportsDates() { + return false; + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + public final AggregatorFunctionSupplier supplier(BigArrays bigArrays, List inputChannels) { + DataType type = field().dataType(); + if (supportsDates() && type == DataTypes.DATETIME) { + return longSupplier(bigArrays, inputChannels); + } + if (type == DataTypes.LONG) { + return longSupplier(bigArrays, inputChannels); + } + if (type == DataTypes.INTEGER) { + return intSupplier(bigArrays, inputChannels); + } + if (type == DataTypes.DOUBLE) { + return doubleSupplier(bigArrays, inputChannels); + } + throw new UnsupportedOperationException(); + } + + protected abstract AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels); + + protected abstract AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels); + + protected abstract AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Percentile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Percentile.java new file mode 100644 index 0000000000000..db560ff7043df --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Percentile.java @@ -0,0 +1,84 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.PercentileDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.PercentileIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.PercentileLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isFoldable; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Percentile extends NumericAggregate { + private final Expression percentile; + + public Percentile(Source source, Expression field, Expression percentile) { + super(source, field, List.of(percentile)); + this.percentile = percentile; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Percentile::new, field(), percentile); + } + + @Override + public Percentile replaceChildren(List newChildren) { + return new Percentile(source(), newChildren.get(0), newChildren.get(1)); + } + + public Expression percentile() { + return percentile; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isNumeric(field(), sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + resolution = isNumeric(percentile, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + return isFoldable(percentile, sourceText(), SECOND); + } + + @Override + protected AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels) { + return new PercentileLongAggregatorFunctionSupplier(bigArrays, inputChannels, percentileValue()); + } + + @Override + protected AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels) { + return new PercentileIntAggregatorFunctionSupplier(bigArrays, inputChannels, percentileValue()); + } + + @Override + protected AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels) { + return new PercentileDoubleAggregatorFunctionSupplier(bigArrays, inputChannels, percentileValue()); + } + + private int percentileValue() { + return ((Number) percentile.fold()).intValue(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Sum.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Sum.java new file mode 100644 index 0000000000000..1f4c493613353 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Sum.java @@ -0,0 +1,64 @@ +/* + * 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.expression.function.aggregate; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumDoubleAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumIntAggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.SumLongAggregatorFunctionSupplier; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; + +/** + * Sum all values of a field in matching documents. + */ +public class Sum extends NumericAggregate { + + public Sum(Source source, Expression field) { + super(source, field); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Sum::new, field()); + } + + @Override + public Sum replaceChildren(List newChildren) { + return new Sum(source(), newChildren.get(0)); + } + + @Override + public DataType dataType() { + DataType dt = field().dataType(); + return dt.isInteger() == false || dt == UNSIGNED_LONG ? DOUBLE : LONG; + } + + @Override + protected AggregatorFunctionSupplier longSupplier(BigArrays bigArrays, List inputChannels) { + return new SumLongAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier intSupplier(BigArrays bigArrays, List inputChannels) { + return new SumIntAggregatorFunctionSupplier(bigArrays, inputChannels); + } + + @Override + protected AggregatorFunctionSupplier doubleSupplier(BigArrays bigArrays, List inputChannels) { + return new SumDoubleAggregatorFunctionSupplier(bigArrays, inputChannels); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/UnaryScalarFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/UnaryScalarFunction.java new file mode 100644 index 0000000000000..f6636241caa72 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/UnaryScalarFunction.java @@ -0,0 +1,71 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.Arrays; +import java.util.Objects; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public abstract class UnaryScalarFunction extends ScalarFunction { + protected final Expression field; + + public UnaryScalarFunction(Source source, Expression field) { + super(source, Arrays.asList(field)); + this.field = field; + } + + @Override + protected Expression.TypeResolution resolveType() { + if (childrenResolved() == false) { + return new Expression.TypeResolution("Unresolved children"); + } + + return isNumeric(field, sourceText(), TypeResolutions.ParamOrdinal.DEFAULT); + } + + @Override + public boolean foldable() { + return field.foldable(); + } + + public final Expression field() { + return field; + } + + @Override + public DataType dataType() { + return field.dataType(); + } + + @Override + public final int hashCode() { + return Objects.hash(field); + } + + @Override + public final boolean equals(Object obj) { + if (obj == null || obj.getClass() != getClass()) { + return false; + } + UnaryScalarFunction other = (UnaryScalarFunction) obj; + return Objects.equals(other.field, field); + } + + @Override + public final ScriptTemplate asScript() { + throw new UnsupportedOperationException("functions do not support scripting"); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/Case.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/Case.java new file mode 100644 index 0000000000000..691c49dea7506 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/Case.java @@ -0,0 +1,212 @@ +/* + * 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.expression.function.scalar.conditional; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.IntStream; + +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.ql.type.DataTypes.NULL; + +public class Case extends ScalarFunction implements Mappable { + record Condition(Expression condition, Expression value) {} + + private final List conditions; + private final Expression elseValue; + private DataType dataType; + + public Case(Source source, List fields) { + super(source, fields); + int conditionCount = fields.size() / 2; + conditions = new ArrayList<>(conditionCount); + for (int c = 0; c < conditionCount; c++) { + conditions.add(new Condition(fields.get(c * 2), fields.get(c * 2 + 1))); + } + elseValue = fields.size() % 2 == 0 ? new Literal(source, null, NULL) : fields.get(fields.size() - 1); + } + + @Override + public DataType dataType() { + if (dataType == null) { + resolveType(); + } + return dataType; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + if (children().size() < 2) { + return new TypeResolution(format(null, "expected at least two arguments in [{}] but got {}", sourceText(), children().size())); + } + + for (int c = 0; c < conditions.size(); c++) { + Condition condition = conditions.get(c); + + TypeResolution resolution = TypeResolutions.isBoolean( + condition.condition, + sourceText(), + TypeResolutions.ParamOrdinal.fromIndex(c * 2) + ); + if (resolution.unresolved()) { + return resolution; + } + + resolution = resolveValueType(condition.value, c * 2 + 1); + if (resolution.unresolved()) { + return resolution; + } + } + + return resolveValueType(elseValue, conditions.size() * 2); + } + + private TypeResolution resolveValueType(Expression value, int position) { + if (dataType == null || dataType == NULL) { + dataType = value.dataType(); + return TypeResolution.TYPE_RESOLVED; + } + return TypeResolutions.isType( + value, + t -> t == dataType, + sourceText(), + TypeResolutions.ParamOrdinal.fromIndex(position), + dataType.typeName() + ); + } + + @Override + public Nullability nullable() { + return Nullability.UNKNOWN; + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Case(source(), newChildren); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Case::new, children()); + } + + @Override + public boolean foldable() { + for (Condition condition : conditions) { + if (condition.condition.foldable() == false) { + return false; + } + Boolean b = (Boolean) condition.condition.fold(); + if (b != null && b) { + return condition.value.foldable(); + } + } + return elseValue.foldable(); + } + + @Override + public Object fold() { + // TODO can we partially fold? like CASE(false, foo, bar) -> bar + for (Condition condition : conditions) { + Boolean b = (Boolean) condition.condition.fold(); + if (b != null && b) { + return condition.value.fold(); + } + } + return elseValue.fold(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + List conditionsEval = conditions.stream() + .map(c -> new ConditionEvaluatorSupplier(toEvaluator.apply(c.condition), toEvaluator.apply(c.value))) + .toList(); + Supplier elseValueEval = toEvaluator.apply(elseValue); + return () -> new CaseEvaluator( + LocalExecutionPlanner.toElementType(dataType()), + conditionsEval.stream().map(Supplier::get).toList(), + elseValueEval.get() + ); + } + + record ConditionEvaluatorSupplier( + Supplier condition, + Supplier value + ) implements Supplier { + @Override + public ConditionEvaluator get() { + return new ConditionEvaluator(condition.get(), value.get()); + } + } + + record ConditionEvaluator(EvalOperator.ExpressionEvaluator condition, EvalOperator.ExpressionEvaluator value) {} + + private record CaseEvaluator(ElementType resultType, List conditions, EvalOperator.ExpressionEvaluator elseVal) + implements + EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + // Evaluate row at a time for now because its simpler. Much slower. But simpler. + int positionCount = page.getPositionCount(); + Block.Builder result = resultType.newBlockBuilder(positionCount); + position: for (int p = 0; p < positionCount; p++) { + int[] positions = new int[] { p }; + Page limited = new Page( + IntStream.range(0, page.getBlockCount()).mapToObj(b -> page.getBlock(b).filter(positions)).toArray(Block[]::new) + ); + for (ConditionEvaluator condition : conditions) { + Block e = condition.condition.eval(limited); + if (e.areAllValuesNull()) { + continue; + } + BooleanBlock b = (BooleanBlock) e; + if (b.isNull(0)) { + continue; + } + if (false == b.getBoolean(b.getFirstValueIndex(0))) { + continue; + } + result.copyFrom(condition.value.eval(limited), 0, 1); + continue position; + } + result.copyFrom(elseVal.eval(limited), 0, 1); + } + return result.build(); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/AbstractConvertFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/AbstractConvertFunction.java new file mode 100644 index 0000000000000..1b0412dab45e5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/AbstractConvertFunction.java @@ -0,0 +1,122 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.Warnings; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.Locale; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Base class for functions that converts a field into a function-specific type. + */ +public abstract class AbstractConvertFunction extends UnaryScalarFunction implements Mappable { + + protected AbstractConvertFunction(Source source, Expression field) { + super(source, field); + } + + /** + * Build the evaluator given the evaluator a multivalued field. + */ + protected Supplier evaluator(Supplier fieldEval) { + DataType sourceType = field().dataType(); + var evaluator = evaluators().get(sourceType); + if (evaluator == null) { + throw new AssertionError("unsupported type [" + sourceType + "]"); + } + return () -> evaluator.apply(fieldEval.get(), source()); + } + + @Override + protected final TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + return isType( + field(), + evaluators()::containsKey, + sourceText(), + null, + evaluators().keySet().stream().map(dt -> dt.name().toLowerCase(Locale.ROOT)).sorted().toArray(String[]::new) + ); + } + + protected abstract Map> evaluators(); + + @Override + public final Object fold() { + return Mappable.super.fold(); + } + + @Override + public final Supplier toEvaluator( + java.util.function.Function> toEvaluator + ) { + return evaluator(toEvaluator.apply(field())); + } + + public abstract static class AbstractEvaluator implements EvalOperator.ExpressionEvaluator { + + private static final Log logger = LogFactory.getLog(AbstractEvaluator.class); + + private final EvalOperator.ExpressionEvaluator fieldEvaluator; + private final Warnings warnings; + + protected AbstractEvaluator(EvalOperator.ExpressionEvaluator field, Source source) { + this.fieldEvaluator = field; + this.warnings = new Warnings(source); + } + + protected abstract String name(); + + /** + * Called when evaluating a {@link Block} that contains null values. + */ + protected abstract Block evalBlock(Block b); + + /** + * Called when evaluating a {@link Block} that does not contain null values. + */ + protected abstract Block evalVector(Vector v); + + public Block eval(Page page) { + Block block = fieldEvaluator.eval(page); + if (block.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + Vector vector = block.asVector(); + return vector == null ? evalBlock(block) : evalVector(vector); + } + + protected final void registerException(Exception exception) { + logger.trace("conversion failure", exception); + warnings.registerException(exception); + } + + @Override + public final String toString() { + return name() + "[field=" + fieldEvaluator + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBoolean.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBoolean.java new file mode 100644 index 0000000000000..3ec6492ef0d8c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBoolean.java @@ -0,0 +1,98 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.math.BigInteger; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public class ToBoolean extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + BOOLEAN, + (fieldEval, source) -> fieldEval, + KEYWORD, + ToBooleanFromStringEvaluator::new, + DOUBLE, + ToBooleanFromDoubleEvaluator::new, + LONG, + ToBooleanFromLongEvaluator::new, + UNSIGNED_LONG, + ToBooleanFromUnsignedLongEvaluator::new, + INTEGER, + ToBooleanFromIntEvaluator::new + ); + + public ToBoolean(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return BOOLEAN; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToBoolean(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToBoolean::new, field()); + } + + @ConvertEvaluator(extraName = "FromString") + static boolean fromKeyword(BytesRef keyword) { + return Boolean.parseBoolean(keyword.utf8ToString()); + } + + @ConvertEvaluator(extraName = "FromDouble") + static boolean fromDouble(double d) { + return d != 0; + } + + @ConvertEvaluator(extraName = "FromLong") + static boolean fromLong(long l) { + return l != 0; + } + + @ConvertEvaluator(extraName = "FromUnsignedLong") + static boolean fromUnsignedLong(long ul) { + Number n = unsignedLongAsNumber(ul); + return n instanceof BigInteger || n.longValue() != 0; + } + + @ConvertEvaluator(extraName = "FromInt") + static boolean fromInt(int i) { + return fromLong(i); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetime.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetime.java new file mode 100644 index 0000000000000..5049a80d075f9 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDatetime.java @@ -0,0 +1,76 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateParse; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; + +public class ToDatetime extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + DATETIME, + (fieldEval, source) -> fieldEval, + LONG, + (fieldEval, source) -> fieldEval, + KEYWORD, + ToDatetimeFromStringEvaluator::new, + DOUBLE, + ToLongFromDoubleEvaluator::new, + UNSIGNED_LONG, + ToLongFromUnsignedLongEvaluator::new, + INTEGER, + ToLongFromIntEvaluator::new // CastIntToLongEvaluator would be a candidate, but not MV'd + ); + + public ToDatetime(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return DATETIME; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToDatetime(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToDatetime::new, field()); + } + + @ConvertEvaluator(extraName = "FromString") + static long fromKeyword(BytesRef in) { + return DateParse.process(in, DateParse.DEFAULT_FORMATTER); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegrees.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegrees.java new file mode 100644 index 0000000000000..4c00b60653459 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDegrees.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.convert; + +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; + +/** + * Converts from radians + * to degrees. + */ +public class ToDegrees extends AbstractConvertFunction implements Mappable { + private static final Map> EVALUATORS = + Map.of( + DOUBLE, + ToDegreesEvaluator::new, + INTEGER, + (field, source) -> new ToDegreesEvaluator(new ToDoubleFromIntEvaluator(field, source), source), + LONG, + (field, source) -> new ToDegreesEvaluator(new ToDoubleFromLongEvaluator(field, source), source), + UNSIGNED_LONG, + (field, source) -> new ToDegreesEvaluator(new ToDoubleFromUnsignedLongEvaluator(field, source), source) + ); + + public ToDegrees(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToDegrees(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToDegrees::new, field()); + } + + @Override + public DataType dataType() { + return DOUBLE; + } + + @ConvertEvaluator + static double process(double deg) { + return Math.toDegrees(deg); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDouble.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDouble.java new file mode 100644 index 0000000000000..dc8527637c7a3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDouble.java @@ -0,0 +1,99 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public class ToDouble extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + DOUBLE, + (fieldEval, source) -> fieldEval, + BOOLEAN, + ToDoubleFromBooleanEvaluator::new, + DATETIME, + ToDoubleFromLongEvaluator::new, // CastLongToDoubleEvaluator would be a candidate, but not MV'd + KEYWORD, + ToDoubleFromStringEvaluator::new, + UNSIGNED_LONG, + ToDoubleFromUnsignedLongEvaluator::new, + LONG, + ToDoubleFromLongEvaluator::new, // CastLongToDoubleEvaluator would be a candidate, but not MV'd + INTEGER, + ToDoubleFromIntEvaluator::new // CastIntToDoubleEvaluator would be a candidate, but not MV'd + ); + + public ToDouble(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return DOUBLE; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToDouble(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToDouble::new, field()); + } + + @ConvertEvaluator(extraName = "FromBoolean") + static double fromBoolean(boolean bool) { + return bool ? 1d : 0d; + } + + @ConvertEvaluator(extraName = "FromString") + static double fromKeyword(BytesRef in) { + return Double.parseDouble(in.utf8ToString()); + } + + @ConvertEvaluator(extraName = "FromUnsignedLong") + static double fromUnsignedLong(long l) { + return unsignedLongAsNumber(l).doubleValue(); + } + + @ConvertEvaluator(extraName = "FromLong") + static double fromLong(long l) { + return l; + } + + @ConvertEvaluator(extraName = "FromInt") + static double fromInt(int i) { + return i; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIP.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIP.java new file mode 100644 index 0000000000000..0931033758dbb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToIP.java @@ -0,0 +1,59 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.IP; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.util.StringUtils.parseIP; + +public class ToIP extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of(IP, (fieldEval, source) -> fieldEval, KEYWORD, ToIPFromStringEvaluator::new); + + public ToIP(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return IP; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToIP(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToIP::new, field()); + } + + @ConvertEvaluator(extraName = "FromString") + static BytesRef fromKeyword(BytesRef asString) { + return parseIP(asString.utf8ToString()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToInteger.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToInteger.java new file mode 100644 index 0000000000000..1d26c4724a423 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToInteger.java @@ -0,0 +1,109 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeDoubleToLong; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToInt; +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; + +public class ToInteger extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + INTEGER, + (fieldEval, source) -> fieldEval, + BOOLEAN, + ToIntegerFromBooleanEvaluator::new, + DATETIME, + ToIntegerFromLongEvaluator::new, + KEYWORD, + ToIntegerFromStringEvaluator::new, + DOUBLE, + ToIntegerFromDoubleEvaluator::new, + UNSIGNED_LONG, + ToIntegerFromUnsignedLongEvaluator::new, + LONG, + ToIntegerFromLongEvaluator::new + ); + + public ToInteger(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return INTEGER; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToInteger(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToInteger::new, field()); + } + + @ConvertEvaluator(extraName = "FromBoolean") + static int fromBoolean(boolean bool) { + return bool ? 1 : 0; + } + + @ConvertEvaluator(extraName = "FromString") + static int fromKeyword(BytesRef in) { + String asString = in.utf8ToString(); + try { + return Integer.parseInt(asString); + } catch (NumberFormatException nfe) { + try { + return fromDouble(Double.parseDouble(asString)); + } catch (Exception e) { + throw nfe; + } + } + } + + @ConvertEvaluator(extraName = "FromDouble") + static int fromDouble(double dbl) { + return fromLong(safeDoubleToLong(dbl)); + } + + @ConvertEvaluator(extraName = "FromUnsignedLong") + static int fromUnsignedLong(long lng) { + return fromLong(ToLong.fromUnsignedLong(lng)); + } + + @ConvertEvaluator(extraName = "FromLong") + static int fromLong(long lng) { + return safeToInt(lng); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLong.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLong.java new file mode 100644 index 0000000000000..ffb31a77cb1fc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToLong.java @@ -0,0 +1,110 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeDoubleToLong; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToLong; +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public class ToLong extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + LONG, + (fieldEval, source) -> fieldEval, + DATETIME, + (fieldEval, source) -> fieldEval, + BOOLEAN, + ToLongFromBooleanEvaluator::new, + KEYWORD, + ToLongFromStringEvaluator::new, + DOUBLE, + ToLongFromDoubleEvaluator::new, + UNSIGNED_LONG, + ToLongFromUnsignedLongEvaluator::new, + INTEGER, + ToLongFromIntEvaluator::new // CastIntToLongEvaluator would be a candidate, but not MV'd + ); + + public ToLong(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return LONG; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToLong(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToLong::new, field()); + } + + @ConvertEvaluator(extraName = "FromBoolean") + static long fromBoolean(boolean bool) { + return bool ? 1L : 0L; + } + + @ConvertEvaluator(extraName = "FromString") + static long fromKeyword(BytesRef in) { + String asString = in.utf8ToString(); + try { + return Long.parseLong(asString); + } catch (NumberFormatException nfe) { + try { + return fromDouble(Double.parseDouble(asString)); + } catch (Exception e) { + throw nfe; + } + } + } + + @ConvertEvaluator(extraName = "FromDouble") + static long fromDouble(double dbl) { + return safeDoubleToLong(dbl); + } + + @ConvertEvaluator(extraName = "FromUnsignedLong") + static long fromUnsignedLong(long ul) { + return safeToLong(unsignedLongAsNumber(ul)); + } + + @ConvertEvaluator(extraName = "FromInt") + static long fromInt(int i) { + return i; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadians.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadians.java new file mode 100644 index 0000000000000..e003bf296522e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToRadians.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.convert; + +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; + +/** + * Converts from degrees + * to radians. + */ +public class ToRadians extends AbstractConvertFunction implements Mappable { + private static final Map> EVALUATORS = + Map.of( + DOUBLE, + ToRadiansEvaluator::new, + INTEGER, + (field, source) -> new ToRadiansEvaluator(new ToDoubleFromIntEvaluator(field, source), source), + LONG, + (field, source) -> new ToRadiansEvaluator(new ToDoubleFromLongEvaluator(field, source), source), + UNSIGNED_LONG, + (field, source) -> new ToRadiansEvaluator(new ToDoubleFromUnsignedLongEvaluator(field, source), source) + ); + + public ToRadians(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToRadians(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToRadians::new, field()); + } + + @Override + public DataType dataType() { + return DOUBLE; + } + + @ConvertEvaluator + static double process(double deg) { + return Math.toRadians(deg); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java new file mode 100644 index 0000000000000..e6d811144c562 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java @@ -0,0 +1,124 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.versionfield.Version; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.IP; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.VERSION; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public class ToString extends AbstractConvertFunction implements Mappable { + + private static final Map> EVALUATORS = + Map.of( + KEYWORD, + (fieldEval, source) -> fieldEval, + BOOLEAN, + ToStringFromBooleanEvaluator::new, + DATETIME, + ToStringFromDatetimeEvaluator::new, + IP, + ToStringFromIPEvaluator::new, + DOUBLE, + ToStringFromDoubleEvaluator::new, + LONG, + ToStringFromLongEvaluator::new, + INTEGER, + ToStringFromIntEvaluator::new, + VERSION, + ToStringFromVersionEvaluator::new, + UNSIGNED_LONG, + ToStringFromUnsignedLongEvaluator::new + ); + + public ToString(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return KEYWORD; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToString(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToString::new, field()); + } + + @ConvertEvaluator(extraName = "FromBoolean") + static BytesRef fromBoolean(boolean bool) { + return new BytesRef(String.valueOf(bool)); + } + + @ConvertEvaluator(extraName = "FromIP") + static BytesRef fromIP(BytesRef ip) { + return new BytesRef(DocValueFormat.IP.format(ip)); + } + + @ConvertEvaluator(extraName = "FromDatetime") + static BytesRef fromDatetime(long datetime) { + return new BytesRef(UTC_DATE_TIME_FORMATTER.formatMillis(datetime)); + } + + @ConvertEvaluator(extraName = "FromDouble") + static BytesRef fromDouble(double dbl) { + return new BytesRef(String.valueOf(dbl)); + } + + @ConvertEvaluator(extraName = "FromLong") + static BytesRef fromDouble(long lng) { + return new BytesRef(String.valueOf(lng)); + } + + @ConvertEvaluator(extraName = "FromInt") + static BytesRef fromDouble(int integer) { + return new BytesRef(String.valueOf(integer)); + } + + @ConvertEvaluator(extraName = "FromVersion") + static BytesRef fromVersion(BytesRef version) { + return new BytesRef(new Version(version).toString()); + } + + @ConvertEvaluator(extraName = "FromUnsignedLong") + static BytesRef fromUnsignedLong(long lng) { + return new BytesRef(unsignedLongAsNumber(lng).toString()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLong.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLong.java new file mode 100644 index 0000000000000..83deed6b18490 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToUnsignedLong.java @@ -0,0 +1,103 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToUnsignedLong; +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.ONE_AS_UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.ZERO_AS_UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; + +public class ToUnsignedLong extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of( + UNSIGNED_LONG, + (fieldEval, source) -> fieldEval, + DATETIME, + ToUnsignedLongFromLongEvaluator::new, + BOOLEAN, + ToUnsignedLongFromBooleanEvaluator::new, + KEYWORD, + ToUnsignedLongFromStringEvaluator::new, + DOUBLE, + ToUnsignedLongFromDoubleEvaluator::new, + LONG, + ToUnsignedLongFromLongEvaluator::new, + INTEGER, + ToUnsignedLongFromIntEvaluator::new + ); + + public ToUnsignedLong(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return UNSIGNED_LONG; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToUnsignedLong(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToUnsignedLong::new, field()); + } + + @ConvertEvaluator(extraName = "FromBoolean") + static long fromBoolean(boolean bool) { + return bool ? ONE_AS_UNSIGNED_LONG : ZERO_AS_UNSIGNED_LONG; + } + + @ConvertEvaluator(extraName = "FromString") + static long fromKeyword(BytesRef in) { + String asString = in.utf8ToString(); + return asLongUnsigned(safeToUnsignedLong(asString)); + } + + @ConvertEvaluator(extraName = "FromDouble") + static long fromDouble(double dbl) { + return asLongUnsigned(safeToUnsignedLong(dbl)); + } + + @ConvertEvaluator(extraName = "FromLong") + static long fromLong(long lng) { + return asLongUnsigned(safeToUnsignedLong(lng)); + } + + @ConvertEvaluator(extraName = "FromInt") + static long fromInt(int i) { + return fromLong(i); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersion.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersion.java new file mode 100644 index 0000000000000..c22a773542a94 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToVersion.java @@ -0,0 +1,59 @@ +/* + * 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.expression.function.scalar.convert; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.ConvertEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.versionfield.Version; + +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; + +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.VERSION; + +public class ToVersion extends AbstractConvertFunction { + + private static final Map> EVALUATORS = + Map.of(VERSION, (fieldEval, source) -> fieldEval, KEYWORD, ToVersionFromStringEvaluator::new); + + public ToVersion(Source source, Expression field) { + super(source, field); + } + + @Override + protected Map> evaluators() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return VERSION; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToVersion(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToVersion::new, field()); + } + + @ConvertEvaluator(extraName = "FromString") + static BytesRef fromKeyword(BytesRef asString) { + return new Version(asString.utf8ToString()).toBytesRef(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/BinaryDateTimeFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/BinaryDateTimeFunction.java new file mode 100644 index 0000000000000..0f4b52eaacb59 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/BinaryDateTimeFunction.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.expression.function.scalar.date; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.BinaryScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.Objects; + +public abstract class BinaryDateTimeFunction extends BinaryScalarFunction { + + protected static final ZoneId DEFAULT_TZ = ZoneOffset.UTC; + + private final ZoneId zoneId; + + protected BinaryDateTimeFunction(Source source, Expression timestamp, Expression argument) { + super(source, timestamp, argument); + zoneId = DEFAULT_TZ; + } + + @Override + public DataType dataType() { + return DataTypes.DATETIME; + } + + public Expression timestampField() { + return left(); + } + + public ZoneId zoneId() { + return zoneId; + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), zoneId()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (super.equals(o) == false) { + return false; + } + BinaryDateTimeFunction that = (BinaryDateTimeFunction) o; + return zoneId().equals(that.zoneId()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtract.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtract.java new file mode 100644 index 0000000000000..40657185ca830 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtract.java @@ -0,0 +1,136 @@ +/* + * 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.expression.function.scalar.date; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.BytesRefs; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ConfigurationFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.session.Configuration; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.time.Instant; +import java.time.ZoneId; +import java.time.temporal.ChronoField; +import java.util.List; +import java.util.Locale; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isDate; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isStringAndExact; + +public class DateExtract extends ConfigurationFunction implements Mappable { + + private ChronoField chronoField; + + public DateExtract(Source source, Expression field, Expression chronoFieldExp, Configuration configuration) { + super(source, List.of(field, chronoFieldExp), configuration); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier fieldEvaluator = toEvaluator.apply(children().get(0)); + if (children().get(1).foldable()) { + ChronoField chrono = chronoField(); + if (chrono == null) { + BytesRef field = (BytesRef) children().get(1).fold(); + throw new EsqlIllegalArgumentException("invalid date field for [{}]: {}", sourceText(), field.utf8ToString()); + } + return () -> new DateExtractConstantEvaluator(fieldEvaluator.get(), chrono, configuration().zoneId()); + } + Supplier chronoEvaluator = toEvaluator.apply(children().get(1)); + return () -> new DateExtractEvaluator(source(), fieldEvaluator.get(), chronoEvaluator.get(), configuration().zoneId()); + } + + private ChronoField chronoField() { + if (chronoField == null) { + Expression field = children().get(1); + if (field.foldable() && field.dataType() == DataTypes.KEYWORD) { + try { + BytesRef br = BytesRefs.toBytesRef(field.fold()); + chronoField = ChronoField.valueOf(br.utf8ToString().toUpperCase(Locale.ROOT)); + } catch (Exception e) { + return null; + } + } + } + return chronoField; + } + + @Evaluator(warnExceptions = { IllegalArgumentException.class }) + static long process(long value, BytesRef chronoField, @Fixed ZoneId zone) { + ChronoField chrono = ChronoField.valueOf(chronoField.utf8ToString().toUpperCase(Locale.ROOT)); + return Instant.ofEpochMilli(value).atZone(zone).getLong(chrono); + } + + @Evaluator(extraName = "Constant") + static long process(long value, @Fixed ChronoField chronoField, @Fixed ZoneId zone) { + return Instant.ofEpochMilli(value).atZone(zone).getLong(chronoField); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new DateExtract(source(), newChildren.get(0), newChildren.get(1), configuration()); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, DateExtract::new, children().get(0), children().get(1), configuration()); + } + + @Override + public DataType dataType() { + return DataTypes.LONG; + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + TypeResolution resolution = isDate(children().get(0), sourceText(), TypeResolutions.ParamOrdinal.FIRST); + if (resolution.unresolved()) { + return resolution; + } + resolution = isStringAndExact(children().get(1), sourceText(), TypeResolutions.ParamOrdinal.SECOND); + if (resolution.unresolved()) { + return resolution; + } + + return TypeResolution.TYPE_RESOLVED; + } + + @Override + public boolean foldable() { + return children().get(0).foldable() && children().get(1).foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormat.java new file mode 100644 index 0000000000000..30800f536c685 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormat.java @@ -0,0 +1,133 @@ +/* + * 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.expression.function.scalar.date; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.scalar.ConfigurationFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.session.Configuration; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isDate; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isStringAndExact; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; + +public class DateFormat extends ConfigurationFunction implements OptionalArgument, Mappable { + + private final Expression field; + private final Expression format; + + public DateFormat(Source source, Expression field, Expression format, Configuration configuration) { + super(source, format != null ? Arrays.asList(field, format) : Arrays.asList(field), configuration); + this.field = field; + this.format = format; + } + + @Override + public DataType dataType() { + return DataTypes.KEYWORD; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isDate(field, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + if (format != null) { + resolution = isStringAndExact(format, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + } + + return TypeResolution.TYPE_RESOLVED; + } + + @Override + public boolean foldable() { + return field.foldable() && (format == null || format.foldable()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "Constant") + static BytesRef process(long val, @Fixed DateFormatter formatter) { + return new BytesRef(formatter.formatMillis(val)); + } + + @Evaluator + static BytesRef process(long val, BytesRef formatter, @Fixed Locale locale) { + return process(val, toFormatter(formatter, locale)); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier fieldEvaluator = toEvaluator.apply(field); + if (format == null) { + return () -> new DateFormatConstantEvaluator(fieldEvaluator.get(), UTC_DATE_TIME_FORMATTER); + } + if (format.dataType() != DataTypes.KEYWORD) { + throw new IllegalArgumentException("unsupported data type for format [" + format.dataType() + "]"); + } + if (format.foldable()) { + DateFormatter formatter = toFormatter(format.fold(), ((EsqlConfiguration) configuration()).locale()); + return () -> new DateFormatConstantEvaluator(fieldEvaluator.get(), formatter); + } + Supplier formatEvaluator = toEvaluator.apply(format); + return () -> new DateFormatEvaluator(fieldEvaluator.get(), formatEvaluator.get(), ((EsqlConfiguration) configuration()).locale()); + } + + private static DateFormatter toFormatter(Object format, Locale locale) { + DateFormatter result = format == null ? UTC_DATE_TIME_FORMATTER : DateFormatter.forPattern(((BytesRef) format).utf8ToString()); + return result.withLocale(locale); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new DateFormat(source(), newChildren.get(0), newChildren.size() > 1 ? newChildren.get(1) : null, configuration()); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, DateFormat::new, field, format, configuration()); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParse.java new file mode 100644 index 0000000000000..2ad8eba2f9027 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParse.java @@ -0,0 +1,139 @@ +/* + * 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.expression.function.scalar.date; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.time.ZoneId; +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.common.time.DateFormatter.forPattern; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isStringAndExact; +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC; + +public class DateParse extends ScalarFunction implements OptionalArgument, Mappable { + + public static final DateFormatter DEFAULT_FORMATTER = toFormatter(new BytesRef("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"), UTC); + private final Expression field; + private final Expression format; + + public DateParse(Source source, Expression field, Expression format) { + super(source, format != null ? Arrays.asList(field, format) : Arrays.asList(field)); + this.field = field; + this.format = format; + } + + @Override + public DataType dataType() { + return DataTypes.DATETIME; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isString(field, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + if (format != null) { + resolution = isStringAndExact(format, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + } + + return TypeResolution.TYPE_RESOLVED; + } + + @Override + public boolean foldable() { + return field.foldable() && (format == null || format.foldable()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "Constant", warnExceptions = { IllegalArgumentException.class }) + public static long process(BytesRef val, @Fixed DateFormatter formatter) throws IllegalArgumentException { + String dateString = val.utf8ToString(); + return formatter.parseMillis(dateString); + } + + @Evaluator(warnExceptions = { IllegalArgumentException.class }) + static long process(BytesRef val, BytesRef formatter, @Fixed ZoneId zoneId) throws IllegalArgumentException { + return process(val, toFormatter(formatter, zoneId)); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + ZoneId zone = UTC; // TODO session timezone? + Supplier fieldEvaluator = toEvaluator.apply(field); + if (format == null) { + return () -> new DateParseConstantEvaluator(source(), fieldEvaluator.get(), DEFAULT_FORMATTER); + } + if (format.dataType() != DataTypes.KEYWORD) { + throw new IllegalArgumentException("unsupported data type for date_parse [" + format.dataType() + "]"); + } + if (format.foldable()) { + try { + DateFormatter formatter = toFormatter(format.fold(), zone); + return () -> new DateParseConstantEvaluator(source(), fieldEvaluator.get(), formatter); + } catch (IllegalArgumentException e) { + throw new EsqlIllegalArgumentException(e, "invalid date patter for [{}]: {}", sourceText(), e.getMessage()); + } + } + Supplier formatEvaluator = toEvaluator.apply(format); + return () -> new DateParseEvaluator(source(), fieldEvaluator.get(), formatEvaluator.get(), zone); + } + + private static DateFormatter toFormatter(Object format, ZoneId zone) { + return forPattern(((BytesRef) format).utf8ToString()).withZone(zone); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new DateParse(source(), newChildren.get(0), newChildren.size() > 1 ? newChildren.get(1) : null); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, DateParse::new, field, format); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java new file mode 100644 index 0000000000000..83a8d08185978 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java @@ -0,0 +1,179 @@ +/* + * 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.expression.function.scalar.date; + +import org.elasticsearch.common.Rounding; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.BinaryScalarFunction; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.time.Duration; +import java.time.Period; +import java.time.ZoneId; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isDate; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +public class DateTrunc extends BinaryDateTimeFunction implements Mappable { + + public DateTrunc(Source source, Expression field, Expression interval) { + super(source, field, interval); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isDate(timestampField(), sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + return isInterval(interval(), sourceText(), SECOND); + } + + private static TypeResolution isInterval(Expression e, String operationName, TypeResolutions.ParamOrdinal paramOrd) { + return isType( + e, + dt -> dt == EsqlDataTypes.DATE_PERIOD || dt == EsqlDataTypes.TIME_DURATION, + operationName, + paramOrd, + "dateperiod", + "timeduration" + ); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator + static long process(long fieldVal, @Fixed Rounding.Prepared rounding) { + return rounding.round(fieldVal); + } + + @Override + protected BinaryScalarFunction replaceChildren(Expression newLeft, Expression newRight) { + return new DateTrunc(source(), newLeft, newRight); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, DateTrunc::new, timestampField(), interval()); + } + + public Expression interval() { + return right(); + } + + static Rounding.Prepared createRounding(final Object interval) { + return createRounding(interval, DEFAULT_TZ); + } + + public static Rounding.Prepared createRounding(final Object interval, final ZoneId timeZone) { + if (interval instanceof Period period) { + return createRounding(period, timeZone); + } else if (interval instanceof Duration duration) { + return createRounding(duration, timeZone); + } + throw new IllegalArgumentException("Time interval is not supported"); + } + + private static Rounding.Prepared createRounding(final Period period, final ZoneId timeZone) { + // Zero or negative intervals are not supported + if (period == null || period.isNegative() || period.isZero()) { + throw new IllegalArgumentException("Zero or negative time interval is not supported"); + } + + long periods = period.getUnits().stream().filter(unit -> period.get(unit) != 0).count(); + if (periods != 1) { + throw new IllegalArgumentException("Time interval is not supported"); + } + + final Rounding.Builder rounding; + if (period.getDays() == 1) { + rounding = new Rounding.Builder(Rounding.DateTimeUnit.DAY_OF_MONTH); + } else if (period.getDays() == 7) { + // java.time.Period does not have a WEEKLY period, so a period of 7 days + // returns a weekly rounding + rounding = new Rounding.Builder(Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR); + } else if (period.getDays() > 1) { + rounding = new Rounding.Builder(new TimeValue(period.getDays(), TimeUnit.DAYS)); + } else if (period.getMonths() == 1) { + rounding = new Rounding.Builder(Rounding.DateTimeUnit.MONTH_OF_YEAR); + } else if (period.getMonths() == 3) { + // java.time.Period does not have a QUATERLY period, so a period of 3 months + // returns a quarterly rounding + rounding = new Rounding.Builder(Rounding.DateTimeUnit.QUARTER_OF_YEAR); + } else if (period.getYears() == 1) { + rounding = new Rounding.Builder(Rounding.DateTimeUnit.YEAR_OF_CENTURY); + } else { + throw new IllegalArgumentException("Time interval is not supported"); + } + + rounding.timeZone(timeZone); + return rounding.build().prepareForUnknown(); + } + + private static Rounding.Prepared createRounding(final Duration duration, final ZoneId timeZone) { + // Zero or negative intervals are not supported + if (duration == null || duration.isNegative() || duration.isZero()) { + throw new IllegalArgumentException("Zero or negative time interval is not supported"); + } + + final Rounding.Builder rounding = new Rounding.Builder(TimeValue.timeValueMillis(duration.toMillis())); + rounding.timeZone(timeZone); + return rounding.build().prepareForUnknown(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier fieldEvaluator = toEvaluator.apply(timestampField()); + Expression interval = interval(); + if (interval.foldable() == false) { + throw new IllegalArgumentException("Function [" + sourceText() + "] has invalid interval [" + interval().sourceText() + "]."); + } + Object foldedInterval; + try { + foldedInterval = interval.fold(); + if (foldedInterval == null) { + throw new IllegalArgumentException("Interval cannot not be null"); + } + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Function [" + sourceText() + "] has invalid interval [" + interval().sourceText() + "]. " + e.getMessage() + ); + } + return evaluator(fieldEvaluator, DateTrunc.createRounding(foldedInterval, zoneId())); + } + + public static Supplier evaluator( + Supplier fieldEvaluator, + Rounding.Prepared rounding + ) { + return () -> new DateTruncEvaluator(fieldEvaluator.get(), rounding); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/Now.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/Now.java new file mode 100644 index 0000000000000..3549baef7cb68 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/Now.java @@ -0,0 +1,86 @@ +/* + * 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.expression.function.scalar.date; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.ConfigurationFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.session.Configuration; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +public class Now extends ConfigurationFunction implements Mappable { + + private final long now; + + public Now(Source source, Configuration configuration) { + super(source, List.of(), configuration); + this.now = configuration.now() == null ? System.currentTimeMillis() : configuration.now().toInstant().toEpochMilli(); + } + + private Now(Source source, long now) { + super(source, List.of(), null); + this.now = now; + } + + public static Now newInstance(Source source, long now) { + return new Now(source, now); + } + + @Override + public Object fold() { + return now; + } + + @Override + public boolean foldable() { + return true; + } + + @Override + public DataType dataType() { + return DataTypes.DATETIME; + } + + @Evaluator + static long process(@Fixed long now) { + return now; + } + + @Override + public Expression replaceChildren(List newChildren) { + return this; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Now::new, configuration()); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + return () -> new NowEvaluator(now); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatch.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatch.java new file mode 100644 index 0000000000000..ba4ab3a657e9d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatch.java @@ -0,0 +1,120 @@ +/* + * 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.expression.function.scalar.ip; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.network.CIDRUtils; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.util.CollectionUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static java.util.Collections.singletonList; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.fromIndex; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isIPAndExact; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isStringAndExact; + +/** + * This function takes a first parameter of type IP, followed by one or more parameters evaluated to a CIDR specification: + *
    + *
  • a string literal;
  • + *
  • a field of type keyword;
  • + *
  • a function outputting a keyword.
  • + *

+ * The function will match if the IP parameter is within any (not all) of the ranges defined by the provided CIDR specs. + *

+ * Example: `| eval cidr="10.0.0.0/8" | where cidr_match(ip_field, "127.0.0.1/30", cidr)` + */ +public class CIDRMatch extends ScalarFunction implements Mappable { + + private final Expression ipField; + private final List matches; + + public CIDRMatch(Source source, Expression ipField, List matches) { + super(source, CollectionUtils.combine(singletonList(ipField), matches)); + this.ipField = ipField; + this.matches = matches; + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier ipEvaluatorSupplier = toEvaluator.apply(ipField); + return () -> new CIDRMatchEvaluator( + ipEvaluatorSupplier.get(), + matches.stream().map(x -> toEvaluator.apply(x).get()).toArray(EvalOperator.ExpressionEvaluator[]::new) + ); + } + + @Evaluator + static boolean process(BytesRef ip, BytesRef[] cidrs) { + for (var cidr : cidrs) { + // simple copy is safe, Java uses big-endian, same as network order + if (CIDRUtils.isInRange(Arrays.copyOfRange(ip.bytes, ip.offset, ip.offset + ip.length), cidr.utf8ToString())) { + return true; + } + } + return false; + } + + @Override + public DataType dataType() { + return DataTypes.BOOLEAN; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isIPAndExact(ipField, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + int i = 1; + for (var m : matches) { + resolution = isStringAndExact(m, sourceText(), fromIndex(i++)); + if (resolution.unresolved()) { + return resolution; + } + } + + return resolution; + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException("functions do not support scripting"); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new CIDRMatch(source(), newChildren.get(0), newChildren.subList(1, newChildren.size())); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, CIDRMatch::new, children().get(0), children().subList(1, children().size())); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Abs.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Abs.java new file mode 100644 index 0000000000000..90c0ece840a01 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Abs.java @@ -0,0 +1,77 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +public class Abs extends UnaryScalarFunction implements Mappable { + public Abs(Source source, Expression field) { + super(source, field); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "Double") + static double process(double fieldVal) { + return Math.abs(fieldVal); + } + + @Evaluator(extraName = "Long") + static long process(long fieldVal) { + return Math.absExact(fieldVal); + } + + @Evaluator(extraName = "Int") + static int process(int fieldVal) { + return Math.absExact(fieldVal); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + if (dataType() == DataTypes.DOUBLE) { + return () -> new AbsDoubleEvaluator(field.get()); + } + if (dataType() == DataTypes.UNSIGNED_LONG) { + return field; + } + if (dataType() == DataTypes.LONG) { + return () -> new AbsLongEvaluator(field.get()); + } + if (dataType() == DataTypes.INTEGER) { + return () -> new AbsIntEvaluator(field.get()); + } + throw new UnsupportedOperationException("unsupported data type [" + dataType() + "]"); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new Abs(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Abs::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractTrigonometricFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractTrigonometricFunction.java new file mode 100644 index 0000000000000..37be68c8e1155 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractTrigonometricFunction.java @@ -0,0 +1,60 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +/** + * Common base for trigonometric functions. + */ +abstract class AbstractTrigonometricFunction extends UnaryScalarFunction implements Mappable { + AbstractTrigonometricFunction(Source source, Expression field) { + super(source, field); + } + + protected abstract EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field); + + @Override + public final Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier fieldEval = Cast.cast(field().dataType(), DataTypes.DOUBLE, toEvaluator.apply(field())); + return () -> doubleEvaluator(fieldEval.get()); + } + + @Override + public final Object fold() { + return Mappable.super.fold(); + } + + @Override + protected final TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + return isNumeric(field, sourceText(), DEFAULT); + } + + @Override + public final DataType dataType() { + return DataTypes.DOUBLE; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Acos.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Acos.java new file mode 100644 index 0000000000000..50629f5ae0c32 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Acos.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Inverse cosine trigonometric function. + */ +public class Acos extends AbstractTrigonometricFunction { + public Acos(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new AcosEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Acos(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Acos::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.acos(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Asin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Asin.java new file mode 100644 index 0000000000000..80b382c591695 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Asin.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Inverse cosine trigonometric function. + */ +public class Asin extends AbstractTrigonometricFunction { + public Asin(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new AsinEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Asin(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Asin::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.asin(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan.java new file mode 100644 index 0000000000000..f3a74a415ee00 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Inverse cosine trigonometric function. + */ +public class Atan extends AbstractTrigonometricFunction { + public Atan(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new AtanEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Atan(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Atan::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.atan(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2.java new file mode 100644 index 0000000000000..aa2d3c224a943 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2.java @@ -0,0 +1,106 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +/** + * Inverse cosine trigonometric function. + */ +public class Atan2 extends ScalarFunction implements Mappable { + private final Expression y; + private final Expression x; + + public Atan2(Source source, Expression y, Expression x) { + super(source, List.of(y, x)); + this.y = y; + this.x = x; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Atan2(source(), newChildren.get(0), newChildren.get(1)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Atan2::new, y, x); + } + + @Evaluator + static double process(double y, double x) { + return Math.atan2(y, x); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isNumeric(y, sourceText(), TypeResolutions.ParamOrdinal.FIRST); + if (resolution.unresolved()) { + return resolution; + } + return isNumeric(x, sourceText(), TypeResolutions.ParamOrdinal.SECOND); + } + + @Override + public boolean foldable() { + return Expressions.foldable(children()); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier yEval = Cast.cast(y.dataType(), DataTypes.DOUBLE, toEvaluator.apply(y)); + Supplier xEval = Cast.cast(x.dataType(), DataTypes.DOUBLE, toEvaluator.apply(x)); + return () -> new Atan2Evaluator(yEval.get(), xEval.get()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + public Expression y() { + return y; + } + + public Expression x() { + return x; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucket.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucket.java new file mode 100644 index 0000000000000..efa926e95de7f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucket.java @@ -0,0 +1,249 @@ +/* + * 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.expression.function.scalar.math; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FOURTH; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.THIRD; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isFoldable; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isInteger; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Buckets dates into a given number of buckets. + *

+ * Takes a date field and three constants and picks a bucket size based on the + * constants. The constants are "target bucket count", "from", and "to". It looks like: + * {@code auto_bucket(hire_date, 20, "1985-01-01T00:00:00Z", "1986-01-01T00:00:00Z")}. + * We have a list of "human" bucket sizes like "one month" and "four hours". We pick + * the largest range that covers the range in fewer than the target bucket count. So + * in the above case we'll pick month long buckets, yielding 12 buckets. + *

+ */ +public class AutoBucket extends ScalarFunction implements Mappable { + // TODO maybe we should just cover the whole of representable dates here - like ten years, 100 years, 1000 years, all the way up. + // That way you never end up with more than the target number of buckets. + private static final Rounding LARGEST_HUMAN_DATE_ROUNDING = Rounding.builder(Rounding.DateTimeUnit.YEAR_OF_CENTURY).build(); + private static final Rounding[] HUMAN_DATE_ROUNDINGS = new Rounding[] { + Rounding.builder(Rounding.DateTimeUnit.MONTH_OF_YEAR).build(), + Rounding.builder(Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR).build(), + Rounding.builder(Rounding.DateTimeUnit.DAY_OF_MONTH).build(), + Rounding.builder(TimeValue.timeValueHours(12)).build(), + Rounding.builder(TimeValue.timeValueHours(3)).build(), + Rounding.builder(TimeValue.timeValueHours(1)).build(), + Rounding.builder(TimeValue.timeValueMinutes(30)).build(), + Rounding.builder(TimeValue.timeValueMinutes(10)).build(), + Rounding.builder(TimeValue.timeValueMinutes(5)).build(), + Rounding.builder(TimeValue.timeValueMinutes(1)).build(), + Rounding.builder(TimeValue.timeValueSeconds(30)).build(), + Rounding.builder(TimeValue.timeValueSeconds(10)).build(), + Rounding.builder(TimeValue.timeValueSeconds(5)).build(), + Rounding.builder(TimeValue.timeValueSeconds(1)).build(), + Rounding.builder(TimeValue.timeValueMillis(100)).build(), + Rounding.builder(TimeValue.timeValueMillis(50)).build(), + Rounding.builder(TimeValue.timeValueMillis(10)).build(), + Rounding.builder(TimeValue.timeValueMillis(1)).build(), }; + + private final Expression field; + private final Expression buckets; + private final Expression from; + private final Expression to; + + public AutoBucket(Source source, Expression field, Expression buckets, Expression from, Expression to) { + super(source, List.of(field, buckets, from, to)); + this.field = field; + this.buckets = buckets; + this.from = from; + this.to = to; + } + + @Override + public boolean foldable() { + return field.foldable() && buckets.foldable() && from.foldable() && to.foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + int b = ((Number) buckets.fold()).intValue(); + + if (field.dataType() == DataTypes.DATETIME) { + long f = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis(((BytesRef) from.fold()).utf8ToString()); + long t = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis(((BytesRef) to.fold()).utf8ToString()); + return DateTrunc.evaluator(toEvaluator.apply(field), new DateRoundingPicker(b, f, t).pickRounding().prepareForUnknown()); + } + if (field.dataType().isNumeric()) { + double f = ((Number) from.fold()).doubleValue(); + double t = ((Number) to.fold()).doubleValue(); + + // We could make this more efficient, either by generating the evaluators with byte code or hand rolling this one. + Literal rounding = new Literal(source(), pickRounding(b, f, t), DataTypes.DOUBLE); + Div div = new Div(source(), field, rounding); + Floor floor = new Floor(source(), div); + Mul mul = new Mul(source(), floor, rounding); + return toEvaluator.apply(mul); + } + throw new UnsupportedOperationException("unsupported type [" + field.dataType() + "]"); + } + + private record DateRoundingPicker(int buckets, long from, long to) { + Rounding pickRounding() { + Rounding prev = LARGEST_HUMAN_DATE_ROUNDING; + for (Rounding r : HUMAN_DATE_ROUNDINGS) { + if (roundingIsOk(r)) { + prev = r; + } else { + return prev; + } + } + return prev; + } + + /** + * True if the rounding produces less than or equal to the requested number of buckets. + */ + boolean roundingIsOk(Rounding rounding) { + Rounding.Prepared r = rounding.prepareForUnknown(); + long bucket = r.round(from); + int used = 0; + while (used < buckets) { + bucket = r.nextRoundingValue(bucket); + used++; + if (bucket > to) { + return true; + } + } + return false; + } + } + + private double pickRounding(int buckets, double from, double to) { + double precise = (to - from) / buckets; + double nextPowerOfTen = Math.pow(10, Math.ceil(Math.log10(precise))); + double halfPower = nextPowerOfTen / 2; + return precise < halfPower ? halfPower : nextPowerOfTen; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + if (field.dataType() == DataTypes.DATETIME) { + return resolveType((e, o) -> isString(e, sourceText(), o)); + } + if (field.dataType().isNumeric()) { + return resolveType((e, o) -> isNumeric(e, sourceText(), o)); + } + return isType(field, e -> false, sourceText(), FIRST, "datetime", "numeric"); + } + + private TypeResolution resolveType(BiFunction checkThirdAndForth) { + TypeResolution resolution = isInteger(buckets, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + resolution = isFoldable(buckets, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + + resolution = checkThirdAndForth.apply(from, THIRD); + if (resolution.unresolved()) { + return resolution; + } + resolution = isFoldable(from, sourceText(), THIRD); + if (resolution.unresolved()) { + return resolution; + } + + resolution = checkThirdAndForth.apply(to, FOURTH); + if (resolution.unresolved()) { + return resolution; + } + return isFoldable(to, sourceText(), FOURTH); + } + + @Override + public DataType dataType() { + if (field.dataType().isNumeric()) { + return DataTypes.DOUBLE; + } + return field.dataType(); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new AutoBucket(source(), newChildren.get(0), newChildren.get(1), newChildren.get(2), newChildren.get(3)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, AutoBucket::new, field, buckets, from, to); + } + + public Expression field() { + return field; + } + + public Expression buckets() { + return buckets; + } + + public Expression from() { + return from; + } + + public Expression to() { + return to; + } + + @Override + public String toString() { + return "AutoBucket{" + "field=" + field + ", buckets=" + buckets + ", from=" + from + ", to=" + to + '}'; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cast.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cast.java new file mode 100644 index 0000000000000..f9764ffd3bdc5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cast.java @@ -0,0 +1,101 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongToDouble; + +public class Cast { + /** + * Build the evaluator supplier to cast {@code in} from {@code current} to {@code required}. + */ + public static Supplier cast( + DataType current, + DataType required, + Supplier in + ) { + if (current == required) { + return in; + } + if (current == DataTypes.NULL || required == DataTypes.NULL) { + return () -> page -> Block.constantNullBlock(page.getPositionCount()); + } + if (required == DataTypes.DOUBLE) { + if (current == DataTypes.LONG) { + return () -> new CastLongToDoubleEvaluator(in.get()); + } + if (current == DataTypes.INTEGER) { + return () -> new CastIntToDoubleEvaluator(in.get()); + } + if (current == DataTypes.UNSIGNED_LONG) { + return () -> new CastUnsignedLongToDoubleEvaluator(in.get()); + } + throw cantCast(current, required); + } + if (required == DataTypes.UNSIGNED_LONG) { + if (current == DataTypes.LONG) { + return () -> new CastLongToUnsignedLongEvaluator(in.get()); + } + if (current == DataTypes.INTEGER) { + return () -> new CastIntToUnsignedLongEvaluator(in.get()); + } + } + if (required == DataTypes.LONG) { + if (current == DataTypes.INTEGER) { + return () -> new CastIntToLongEvaluator(in.get()); + } + throw cantCast(current, required); + } + throw cantCast(current, required); + } + + private static UnsupportedOperationException cantCast(DataType current, DataType required) { + return new UnsupportedOperationException("can't process [" + current.typeName() + " -> " + required.typeName() + "]"); + } + + @Evaluator(extraName = "IntToLong") + static long castIntToLong(int v) { + return v; + } + + @Evaluator(extraName = "IntToDouble") + static double castIntToDouble(int v) { + return v; + } + + @Evaluator(extraName = "LongToDouble") + static double castLongToDouble(long v) { + return v; + } + + @Evaluator(extraName = "UnsignedLongToDouble") + static double castUnsignedLongToDouble(long v) { + return unsignedLongToDouble(v); + } + + @Evaluator(extraName = "IntToUnsignedLong") + static long castIntToUnsignedLong(int v) { + return castLongToUnsignedLong(v); + } + + @Evaluator(extraName = "LongToUnsignedLong") + static long castLongToUnsignedLong(long v) { + if (v < 0) { + throw new QlIllegalArgumentException("[" + v + "] out of [unsigned_long] range"); + } + return v; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cos.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cos.java new file mode 100644 index 0000000000000..d9b3a592bcef1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cos.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Cosine trigonometric function. + */ +public class Cos extends AbstractTrigonometricFunction { + public Cos(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new CosEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Cos(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Cos::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.cos(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cosh.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cosh.java new file mode 100644 index 0000000000000..826334ed98ef6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Cosh.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Cosine hyperbolic function. + */ +public class Cosh extends AbstractTrigonometricFunction { + public Cosh(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new CoshEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Cosh(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Cosh::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.cosh(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/DoubleConstantFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/DoubleConstantFunction.java new file mode 100644 index 0000000000000..151c5c92a83ce --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/DoubleConstantFunction.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +/** + * Function that emits Euler's number. + */ +public abstract class DoubleConstantFunction extends ScalarFunction { + protected DoubleConstantFunction(Source source) { + super(source); + } + + @Override + public final boolean foldable() { + return true; + } + + @Override + public final DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + public final ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + protected final NodeInfo info() { + return NodeInfo.create(this); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/E.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/E.java new file mode 100644 index 0000000000000..d2900062f7875 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/E.java @@ -0,0 +1,32 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Function that emits Euler's number. + */ +public class E extends DoubleConstantFunction { + public E(Source source) { + super(source); + } + + @Override + public Object fold() { + return Math.E; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new E(source()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Floor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Floor.java new file mode 100644 index 0000000000000..67427f2a127f0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Floor.java @@ -0,0 +1,76 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +/** + * Round a number down to the nearest integer. + *

+ * Note that doubles are rounded down to the nearest valid double that is + * an integer ala {@link Math#floor}. + *

+ */ +public class Floor extends UnaryScalarFunction implements Mappable { + public Floor(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + if (dataType().isInteger()) { + return toEvaluator.apply(field()); + } + Supplier fieldEval = toEvaluator.apply(field()); + return () -> new FloorDoubleEvaluator(fieldEval.get()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + return isNumeric(field, sourceText(), DEFAULT); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Floor(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Floor::new, field()); + } + + @Evaluator(extraName = "Double") + static double process(double val) { + return Math.floor(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFinite.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFinite.java new file mode 100644 index 0000000000000..0db8a3b98189c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFinite.java @@ -0,0 +1,47 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +public class IsFinite extends RationalUnaryPredicate { + public IsFinite(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + return () -> new IsFiniteEvaluator(field.get()); + } + + @Evaluator + static boolean process(double val) { + return Double.isFinite(val); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new IsFinite(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, IsFinite::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfinite.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfinite.java new file mode 100644 index 0000000000000..c5b6fce00b75d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfinite.java @@ -0,0 +1,47 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +public class IsInfinite extends RationalUnaryPredicate { + public IsInfinite(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + return () -> new IsInfiniteEvaluator(field.get()); + } + + @Evaluator + static boolean process(double val) { + return Double.isInfinite(val); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new IsInfinite(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, IsInfinite::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaN.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaN.java new file mode 100644 index 0000000000000..81bec68372639 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaN.java @@ -0,0 +1,47 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +public class IsNaN extends RationalUnaryPredicate { + public IsNaN(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + return () -> new IsNaNEvaluator(field.get()); + } + + @Evaluator + static boolean process(double val) { + return Double.isNaN(val); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new IsNaN(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, IsNaN::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10.java new file mode 100644 index 0000000000000..693c754b98cab --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10.java @@ -0,0 +1,96 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Log10 extends UnaryScalarFunction implements Mappable { + public Log10(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + var fieldType = field().dataType(); + var eval = field.get(); + + if (fieldType == DataTypes.DOUBLE) { + return () -> new Log10DoubleEvaluator(eval); + } + if (fieldType == DataTypes.INTEGER) { + return () -> new Log10IntEvaluator(eval); + } + if (fieldType == DataTypes.LONG) { + return () -> new Log10LongEvaluator(eval); + } + + throw new UnsupportedOperationException("Unsupported type " + fieldType); + } + + @Evaluator(extraName = "Double") + static double process(double val) { + return Math.log10(val); + } + + @Evaluator(extraName = "Long") + static double process(long val) { + return Math.log10(val); + } + + @Evaluator(extraName = "Int") + static double process(int val) { + return Math.log10(val); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new Log10(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Log10::new, field()); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new Expression.TypeResolution("Unresolved children"); + } + + return isNumeric(field, sourceText(), DEFAULT); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pi.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pi.java new file mode 100644 index 0000000000000..bd36be56b356c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pi.java @@ -0,0 +1,32 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Function that emits pi. + */ +public class Pi extends DoubleConstantFunction { + public Pi(Source source) { + super(source); + } + + @Override + public Object fold() { + return Math.PI; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Pi(source()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pow.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pow.java new file mode 100644 index 0000000000000..5243432492ab8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Pow.java @@ -0,0 +1,206 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.expression.function.scalar.math.Cast.cast; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Pow extends ScalarFunction implements OptionalArgument, Mappable { + + private final Expression base, exponent; + private final DataType dataType; + + public Pow(Source source, Expression base, Expression exponent) { + super(source, Arrays.asList(base, exponent)); + this.base = base; + this.exponent = exponent; + this.dataType = determineDataType(base, exponent); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isNumeric(base, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + return isNumeric(exponent, sourceText(), SECOND); + } + + @Override + public boolean foldable() { + return base.foldable() && exponent.foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "Double", warnExceptions = { ArithmeticException.class }) + static double process(double base, double exponent) { + return validateAsDouble(base, exponent); + } + + @Evaluator(extraName = "Long", warnExceptions = { ArithmeticException.class }) + static long processLong(double base, double exponent) { + return exponent == 1 ? validateAsLong(base) : validateAsLong(base, exponent); + } + + @Evaluator(extraName = "Int", warnExceptions = { ArithmeticException.class }) + static int processInt(double base, double exponent) { + return exponent == 1 ? validateAsInt(base) : validateAsInt(base, exponent); + } + + private static double validateAsDouble(double base, double exponent) { + double result = Math.pow(base, exponent); + if (Double.isNaN(result)) { + throw new ArithmeticException("invalid result: pow(" + base + ", " + exponent + ")"); + } + return result; + } + + private static long validateAsLong(double base, double exponent) { + double result = Math.pow(base, exponent); + if (Double.isNaN(result)) { + throw new ArithmeticException("invalid result: pow(" + base + ", " + exponent + ")"); + } + return validateAsLong(result); + } + + private static long validateAsLong(double value) { + if (Double.compare(value, Long.MAX_VALUE) > 0) { + throw new ArithmeticException("long overflow"); + } + if (Double.compare(value, Long.MIN_VALUE) < 0) { + throw new ArithmeticException("long overflow"); + } + return (long) value; + } + + private static int validateAsInt(double base, double exponent) { + double result = Math.pow(base, exponent); + if (Double.isNaN(result)) { + throw new ArithmeticException("invalid result: pow(" + base + ", " + exponent + ")"); + } + return validateAsInt(result); + } + + private static int validateAsInt(double value) { + if (Double.compare(value, Integer.MAX_VALUE) > 0) { + throw new ArithmeticException("integer overflow"); + } + if (Double.compare(value, Integer.MIN_VALUE) < 0) { + throw new ArithmeticException("integer overflow"); + } + return (int) value; + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new Pow(source(), newChildren.get(0), newChildren.get(1)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Pow::new, base(), exponent()); + } + + public Expression base() { + return base; + } + + public Expression exponent() { + return exponent; + } + + @Override + public DataType dataType() { + return dataType; + } + + private static DataType determineDataType(Expression base, Expression exponent) { + if (base.dataType().isRational() || exponent.dataType().isRational()) { + return DataTypes.DOUBLE; + } + if (base.dataType().size() == Long.BYTES || exponent.dataType().size() == Long.BYTES) { + return DataTypes.LONG; + } + return DataTypes.INTEGER; + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException("functions do not support scripting"); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + var baseEvaluator = toEvaluator.apply(base); + var exponentEvaluator = toEvaluator.apply(exponent); + if (dataType == DataTypes.DOUBLE) { + return () -> new PowDoubleEvaluator( + source(), + cast(base.dataType(), DataTypes.DOUBLE, baseEvaluator).get(), + cast(exponent.dataType(), DataTypes.DOUBLE, exponentEvaluator).get() + ); + } else if (dataType == DataTypes.LONG) { + return () -> new PowLongEvaluator( + source(), + cast(base.dataType(), DataTypes.DOUBLE, baseEvaluator).get(), + cast(exponent.dataType(), DataTypes.DOUBLE, exponentEvaluator).get() + ); + } else { + return () -> new PowIntEvaluator( + source(), + cast(base.dataType(), DataTypes.DOUBLE, baseEvaluator).get(), + cast(exponent.dataType(), DataTypes.DOUBLE, exponentEvaluator).get() + ); + } + } + + @Override + public int hashCode() { + return Objects.hash(base, exponent); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != getClass()) { + return false; + } + Pow other = (Pow) obj; + return Objects.equals(other.base, base) && Objects.equals(other.exponent, exponent); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RationalUnaryPredicate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RationalUnaryPredicate.java new file mode 100644 index 0000000000000..1439f5ee8ab42 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RationalUnaryPredicate.java @@ -0,0 +1,42 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +public abstract class RationalUnaryPredicate extends UnaryScalarFunction implements Mappable { + public RationalUnaryPredicate(Source source, Expression field) { + super(source, field); + } + + @Override + protected final TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + return isType(field(), DataType::isRational, sourceText(), null, DataTypes.DOUBLE.typeName()); + } + + @Override + public final DataType dataType() { + return DataTypes.BOOLEAN; + } + + @Override + public final Object fold() { + return Mappable.super.fold(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Round.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Round.java new file mode 100644 index 0000000000000..5814bbcfbabd1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Round.java @@ -0,0 +1,192 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.expression.predicate.operator.math.Maths; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isInteger; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToLong; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asUnsignedLong; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; + +public class Round extends ScalarFunction implements OptionalArgument, Mappable { + + private final Expression field, decimals; + + public Round(Source source, Expression field, Expression decimals) { + super(source, decimals != null ? Arrays.asList(field, decimals) : Arrays.asList(field)); + this.field = field; + this.decimals = decimals; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isNumeric(field, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + return decimals == null ? TypeResolution.TYPE_RESOLVED : isInteger(decimals, sourceText(), SECOND); + } + + @Override + public boolean foldable() { + return field.foldable() && (decimals == null || decimals.foldable()); + } + + @Override + public Object fold() { + if (field.dataType() == DataTypes.UNSIGNED_LONG) { + return decimals == null + ? field.fold() + : processUnsignedLong(safeToLong((Number) field.fold()), safeToLong((Number) decimals.fold())); + } + if (decimals == null) { + return Maths.round((Number) field.fold(), 0L); + } + return Maths.round((Number) field.fold(), ((Number) decimals.fold()).longValue()); + } + + @Evaluator(extraName = "DoubleNoDecimals") + static double process(double val) { + return Maths.round(val, 0).doubleValue(); + } + + @Evaluator(extraName = "Int") + static int process(int val, long decimals) { + return Maths.round(val, decimals).intValue(); + } + + @Evaluator(extraName = "Long") + static long process(long val, long decimals) { + return Maths.round(val, decimals).longValue(); + } + + @Evaluator(extraName = "UnsignedLong") + static long processUnsignedLong(long val, long decimals) { + Number ul = unsignedLongAsNumber(val); + if (ul instanceof BigInteger bi) { + BigInteger rounded = Maths.round(bi, decimals); + BigInteger unsignedLong = asUnsignedLong(rounded); + return asLongUnsigned(unsignedLong); + } else { + return asLongUnsigned(Maths.round(ul.longValue(), decimals)); + } + } + + @Evaluator(extraName = "Double") + static double process(double val, long decimals) { + return Maths.round(val, decimals).doubleValue(); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new Round(source(), newChildren.get(0), decimals() == null ? null : newChildren.get(1)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Round::new, field(), decimals()); + } + + public Expression field() { + return field; + } + + public Expression decimals() { + return decimals; + } + + @Override + public DataType dataType() { + return field.dataType(); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException("functions do not support scripting"); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + DataType fieldType = dataType(); + if (fieldType == DataTypes.DOUBLE) { + return toEvaluator(toEvaluator, RoundDoubleNoDecimalsEvaluator::new, RoundDoubleEvaluator::new); + } + if (fieldType == DataTypes.INTEGER) { + return toEvaluator(toEvaluator, Function.identity(), RoundIntEvaluator::new); + } + if (fieldType == DataTypes.LONG) { + return toEvaluator(toEvaluator, Function.identity(), RoundLongEvaluator::new); + } + if (fieldType == DataTypes.UNSIGNED_LONG) { + return toEvaluator(toEvaluator, Function.identity(), RoundUnsignedLongEvaluator::new); + } + throw new UnsupportedOperationException(); + } + + private Supplier toEvaluator( + Function> toEvaluator, + Function noDecimals, + BiFunction withDecimals + ) { + Supplier fieldEvaluator = toEvaluator.apply(field()); + if (decimals == null) { + return () -> noDecimals.apply(fieldEvaluator.get()); + } + Supplier decimalsEvaluator = Cast.cast( + decimals().dataType(), + DataTypes.LONG, + toEvaluator.apply(decimals()) + ); + return () -> withDecimals.apply(fieldEvaluator.get(), decimalsEvaluator.get()); + } + + @Override + public int hashCode() { + return Objects.hash(field, decimals); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || obj.getClass() != getClass()) { + return false; + } + Round other = (Round) obj; + return Objects.equals(other.field, field) && Objects.equals(other.decimals, decimals); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sin.java new file mode 100644 index 0000000000000..1339e0a2130c8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sin.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Sine trigonometric function. + */ +public class Sin extends AbstractTrigonometricFunction { + public Sin(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new SinEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Sin(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Sin::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.sin(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sinh.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sinh.java new file mode 100644 index 0000000000000..d40d6bee299a5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sinh.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Sine hyperbolic function. + */ +public class Sinh extends AbstractTrigonometricFunction { + public Sinh(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new SinhEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Sinh(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Sinh::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.sinh(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sqrt.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sqrt.java new file mode 100644 index 0000000000000..5cd6db9e4f364 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Sqrt.java @@ -0,0 +1,96 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isNumeric; + +public class Sqrt extends UnaryScalarFunction implements Mappable { + public Sqrt(Source source, Expression field) { + super(source, field); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + var fieldType = field().dataType(); + var eval = field.get(); + + if (fieldType == DataTypes.DOUBLE) { + return () -> new SqrtDoubleEvaluator(eval); + } + if (fieldType == DataTypes.INTEGER) { + return () -> new SqrtIntEvaluator(eval); + } + if (fieldType == DataTypes.LONG) { + return () -> new SqrtLongEvaluator(eval); + } + + throw new UnsupportedOperationException("Unsupported type " + fieldType); + } + + @Evaluator(extraName = "Double") + static double process(double val) { + return Math.sqrt(val); + } + + @Evaluator(extraName = "Long") + static double process(long val) { + return Math.sqrt(val); + } + + @Evaluator(extraName = "Int") + static double process(int val) { + return Math.sqrt(val); + } + + @Override + public final Expression replaceChildren(List newChildren) { + return new Sqrt(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Sqrt::new, field()); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + return isNumeric(field, sourceText(), DEFAULT); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tan.java new file mode 100644 index 0000000000000..40d0ee5d9de64 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tan.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Tangent trigonometric function. + */ +public class Tan extends AbstractTrigonometricFunction { + public Tan(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new TanEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Tan(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Tan::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.tan(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tanh.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tanh.java new file mode 100644 index 0000000000000..31876aff69b33 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tanh.java @@ -0,0 +1,45 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Tangent hyperbolic function. + */ +public class Tanh extends AbstractTrigonometricFunction { + public Tanh(Source source, Expression field) { + super(source, field); + } + + @Override + protected EvalOperator.ExpressionEvaluator doubleEvaluator(EvalOperator.ExpressionEvaluator field) { + return new TanhEvaluator(field); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Tanh(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Tanh::new, field()); + } + + @Evaluator + static double process(double val) { + return Math.tanh(val); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tau.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tau.java new file mode 100644 index 0000000000000..e40d979886d0c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Tau.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.expression.function.scalar.math; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * Function that emits tau, also known as 2 * pi. + */ +public class Tau extends DoubleConstantFunction { + public static final double TAU = Math.PI * 2; + + public Tau(Source source) { + super(source); + } + + @Override + public Object fold() { + return TAU; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Tau(source()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java new file mode 100644 index 0000000000000..03455a4769dfb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunction.java @@ -0,0 +1,133 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.function.Supplier; + +/** + * Base class for functions that reduce multivalued fields into single valued fields. + */ +public abstract class AbstractMultivalueFunction extends UnaryScalarFunction implements Mappable { + protected AbstractMultivalueFunction(Source source, Expression field) { + super(source, field); + } + + /** + * Build the evaluator given the evaluator a multivalued field. + */ + protected abstract Supplier evaluator(Supplier fieldEval); + + @Override + protected final TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + return resolveFieldType(); + } + + protected abstract TypeResolution resolveFieldType(); + + @Override + public final Object fold() { + return Mappable.super.fold(); + } + + @Override + public final Supplier toEvaluator( + java.util.function.Function> toEvaluator + ) { + return evaluator(toEvaluator.apply(field())); + } + + /** + * Base evaluator that can handle both nulls- and no-nulls-containing blocks. + */ + public abstract static class AbstractEvaluator extends AbstractNullableEvaluator { + protected AbstractEvaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + /** + * Called when evaluating a {@link Block} that does not contain null values. + * It's useful to specialize this from {@link #evalNullable} because it knows + * that it's producing an "array vector" because it only ever emits single + * valued fields and no null values. Building an array vector directly is + * generally faster than building it via a {@link Block.Builder}. + */ + protected abstract Vector evalNotNullable(Block fieldVal); + + /** + * Called to evaluate single valued fields when the target block does not + * have null values. + */ + protected Vector evalSingleValuedNotNullable(Block fieldVal) { + return fieldVal.asVector(); + } + + @Override + public final Block eval(Page page) { + Block fieldVal = field.eval(page); + if (fieldVal.mayHaveMultivaluedFields() == false) { + if (fieldVal.mayHaveNulls()) { + return evalSingleValuedNullable(fieldVal); + } + return evalSingleValuedNotNullable(fieldVal).asBlock(); + } + if (fieldVal.mayHaveNulls()) { + return evalNullable(fieldVal); + } + return evalNotNullable(fieldVal).asBlock(); + } + } + + /** + * Base evaluator that can handle evaluator-checked exceptions; i.e. for expressions that can be evaluated to null. + */ + public abstract static class AbstractNullableEvaluator implements EvalOperator.ExpressionEvaluator { + protected final EvalOperator.ExpressionEvaluator field; + + protected AbstractNullableEvaluator(EvalOperator.ExpressionEvaluator field) { + this.field = field; + } + + protected abstract String name(); + + /** + * Called when evaluating a {@link Block} that contains null values. + */ + protected abstract Block evalNullable(Block fieldVal); + + /** + * Called to evaluate single valued fields when the target block has null + * values. + */ + protected Block evalSingleValuedNullable(Block fieldVal) { + return fieldVal; + } + + @Override + public Block eval(Page page) { + Block fieldVal = field.eval(page); + return fieldVal.mayHaveMultivaluedFields() ? evalNullable(fieldVal) : evalSingleValuedNullable(fieldVal); + } + + @Override + public final String toString() { + return name() + "[field=" + field + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvg.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvg.java new file mode 100644 index 0000000000000..713f5bcefda0f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvg.java @@ -0,0 +1,105 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.elasticsearch.compute.ann.MvEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.isRepresentable; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongToDouble; + +/** + * Reduce a multivalued field to a single valued field containing the average value. + */ +public class MvAvg extends AbstractMultivalueFunction { + public MvAvg(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), t -> t.isNumeric() && isRepresentable(t), sourceText(), null, "numeric"); + } + + @Override + public DataType dataType() { + return DataTypes.DOUBLE; + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return switch (LocalExecutionPlanner.toElementType(field().dataType())) { + case DOUBLE -> () -> new MvAvgDoubleEvaluator(fieldEval.get()); + case INT -> () -> new MvAvgIntEvaluator(fieldEval.get()); + case LONG -> field().dataType() == DataTypes.UNSIGNED_LONG + ? () -> new MvAvgUnsignedLongEvaluator(fieldEval.get()) + : () -> new MvAvgLongEvaluator(fieldEval.get()); + case NULL -> () -> EvalOperator.CONSTANT_NULL; + default -> throw new UnsupportedOperationException("unsupported type [" + field().dataType() + "]"); + }; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvAvg(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvAvg::new, field()); + } + + @MvEvaluator(extraName = "Double", finish = "finish") + public static void process(CompensatedSum sum, double v) { + sum.add(v); + } + + public static double finish(CompensatedSum sum, int valueCount) { + double value = sum.value(); + sum.reset(0, 0); + return value / valueCount; + } + + @MvEvaluator(extraName = "Int", finish = "finish", single = "single") + static void process(CompensatedSum sum, int v) { + sum.add(v); + } + + static double single(int value) { + return value; + } + + @MvEvaluator(extraName = "Long", finish = "finish", single = "single") + static void process(CompensatedSum sum, long v) { + sum.add(v); + } + + static double single(long value) { + return value; + } + + @MvEvaluator(extraName = "UnsignedLong", finish = "finish", single = "singleUnsignedLong") + static void processUnsignedLong(CompensatedSum sum, long v) { + sum.add(unsignedLongToDouble(v)); + } + + static double singleUnsignedLong(long value) { + return unsignedLongToDouble(value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcat.java new file mode 100644 index 0000000000000..7fcdcc722ce54 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcat.java @@ -0,0 +1,148 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.function.scalar.BinaryScalarFunction; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +/** + * Reduce a multivalued string field to a single valued field by concatenating all values. + */ +public class MvConcat extends BinaryScalarFunction implements Mappable { + public MvConcat(Source source, Expression field, Expression delim) { + super(source, field, delim); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isString(left(), sourceText(), TypeResolutions.ParamOrdinal.FIRST); + if (resolution.unresolved()) { + return resolution; + } + + return isString(right(), sourceText(), TypeResolutions.ParamOrdinal.SECOND); + } + + @Override + public DataType dataType() { + return DataTypes.KEYWORD; + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier fieldEval = toEvaluator.apply(left()); + Supplier delimEval = toEvaluator.apply(right()); + return () -> new MvConcatEvaluator(fieldEval.get(), delimEval.get()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + protected BinaryScalarFunction replaceChildren(Expression newLeft, Expression newRight) { + return new MvConcat(source(), newLeft, newRight); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvConcat::new, left(), right()); + } + + /** + * Evaluator for {@link MvConcat}. Not generated and doesn't extend from + * {@link AbstractMultivalueFunction.AbstractEvaluator} because it's just + * too different from all the other mv operators: + *
    + *
  • It takes an extra parameter - the delimiter
  • + *
  • That extra parameter makes it much more likely to be {@code null}
  • + *
  • The actual joining process needs init step per row - {@link BytesRefBuilder#clear()}
  • + *
+ */ + private class MvConcatEvaluator implements EvalOperator.ExpressionEvaluator { + private final EvalOperator.ExpressionEvaluator field; + private final EvalOperator.ExpressionEvaluator delim; + + MvConcatEvaluator(EvalOperator.ExpressionEvaluator field, EvalOperator.ExpressionEvaluator delim) { + this.field = field; + this.delim = delim; + } + + @Override + public final Block eval(Page page) { + Block fieldUncast = field.eval(page); + Block delimUncast = delim.eval(page); + if (fieldUncast.areAllValuesNull() || delimUncast.areAllValuesNull()) { + return Block.constantNullBlock(page.getPositionCount()); + } + BytesRefBlock fieldVal = (BytesRefBlock) fieldUncast; + BytesRefBlock delimVal = (BytesRefBlock) delimUncast; + + int positionCount = page.getPositionCount(); + BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(positionCount); + BytesRefBuilder work = new BytesRefBuilder(); + BytesRef fieldScratch = new BytesRef(); + BytesRef delimScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + int fieldValueCount = fieldVal.getValueCount(p); + if (fieldValueCount == 0) { + builder.appendNull(); + continue; + } + if (delimVal.getValueCount(p) != 1) { + builder.appendNull(); + continue; + } + int first = fieldVal.getFirstValueIndex(p); + if (fieldValueCount == 1) { + builder.appendBytesRef(fieldVal.getBytesRef(first, fieldScratch)); + continue; + } + int end = first + fieldValueCount; + BytesRef delim = delimVal.getBytesRef(delimVal.getFirstValueIndex(p), delimScratch); + work.clear(); + work.append(fieldVal.getBytesRef(first, fieldScratch)); + for (int i = first + 1; i < end; i++) { + work.append(delim); + work.append(fieldVal.getBytesRef(i, fieldScratch)); + } + builder.appendBytesRef(work.get()); + } + return builder.build(); + } + + @Override + public final String toString() { + return "MvConcat[field=" + field + ", delim=" + delim + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java new file mode 100644 index 0000000000000..7aec87cf95f89 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCount.java @@ -0,0 +1,104 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.multivalue; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Reduce a multivalued field to a single valued field containing the minimum value. + */ +public class MvCount extends AbstractMultivalueFunction { + public MvCount(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), EsqlDataTypes::isRepresentable, sourceText(), null, "representable"); + } + + @Override + public DataType dataType() { + return DataTypes.INTEGER; + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return () -> new Evaluator(fieldEval.get()); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvCount(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvCount::new, field()); + } + + private static class Evaluator extends AbstractEvaluator { + protected Evaluator(EvalOperator.ExpressionEvaluator field) { + super(field); + } + + @Override + protected String name() { + return "MvCount"; + } + + @Override + protected Block evalNullable(Block fieldVal) { + IntBlock.Builder builder = IntBlock.newBlockBuilder(fieldVal.getPositionCount()); + for (int p = 0; p < fieldVal.getPositionCount(); p++) { + int valueCount = fieldVal.getValueCount(p); + if (valueCount == 0) { + builder.appendNull(); + continue; + } + builder.appendInt(valueCount); + } + return builder.build(); + } + + @Override + protected Vector evalNotNullable(Block fieldVal) { + int[] values = new int[fieldVal.getPositionCount()]; + for (int p = 0; p < fieldVal.getPositionCount(); p++) { + values[p] = fieldVal.getValueCount(p); + } + return new IntArrayVector(values, values.length); + } + + @Override + protected Block evalSingleValuedNullable(Block fieldVal) { + return evalNullable(fieldVal); + } + + @Override + protected Vector evalSingleValuedNotNullable(Block fieldVal) { + return new ConstantIntVector(1, fieldVal.getPositionCount()); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupe.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupe.java new file mode 100644 index 0000000000000..536e231a6956a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupe.java @@ -0,0 +1,50 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.compute.operator.MultivalueDedupe; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Removes duplicate values from a multivalued field. + */ +public class MvDedupe extends AbstractMultivalueFunction { + public MvDedupe(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), EsqlDataTypes::isRepresentable, sourceText(), null, "representable"); + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return MultivalueDedupe.evaluator(LocalExecutionPlanner.toElementType(dataType()), fieldEval); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvDedupe(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvDedupe::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMax.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMax.java new file mode 100644 index 0000000000000..bec716a2f681d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMax.java @@ -0,0 +1,95 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.MvEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Reduce a multivalued field to a single valued field containing the maximum value. + */ +public class MvMax extends AbstractMultivalueFunction { + public MvMax(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), EsqlDataTypes::isRepresentable, sourceText(), null, "representable"); + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return switch (LocalExecutionPlanner.toElementType(field().dataType())) { + case BOOLEAN -> () -> new MvMaxBooleanEvaluator(fieldEval.get()); + case BYTES_REF -> () -> new MvMaxBytesRefEvaluator(fieldEval.get()); + case DOUBLE -> () -> new MvMaxDoubleEvaluator(fieldEval.get()); + case INT -> () -> new MvMaxIntEvaluator(fieldEval.get()); + case LONG -> () -> new MvMaxLongEvaluator(fieldEval.get()); + case NULL -> () -> EvalOperator.CONSTANT_NULL; + default -> throw new UnsupportedOperationException("unsupported type [" + field().dataType() + "]"); + }; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvMax(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvMax::new, field()); + } + + @MvEvaluator(extraName = "Boolean", ascending = "ascendingIndex") + static boolean process(boolean current, boolean v) { + return current || v; + } + + @MvEvaluator(extraName = "BytesRef", ascending = "ascendingIndex") + static void process(BytesRef current, BytesRef v) { + if (v.compareTo(current) > 0) { + current.bytes = v.bytes; + current.offset = v.offset; + current.length = v.length; + } + } + + @MvEvaluator(extraName = "Double", ascending = "ascendingIndex") + static double process(double current, double v) { + return Math.max(current, v); + } + + @MvEvaluator(extraName = "Int", ascending = "ascendingIndex") + static int process(int current, int v) { + return Math.max(current, v); + } + + @MvEvaluator(extraName = "Long", ascending = "ascendingIndex") + static long process(long current, long v) { + return Math.max(current, v); + } + + /** + * If the values as ascending pick the final value. + */ + static int ascendingIndex(int count) { + return count - 1; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedian.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedian.java new file mode 100644 index 0000000000000..263c00af25620 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedian.java @@ -0,0 +1,213 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.apache.lucene.util.ArrayUtil; +import org.elasticsearch.compute.ann.MvEvaluator; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.isRepresentable; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; + +/** + * Reduce a multivalued field to a single valued field containing the average value. + */ +public class MvMedian extends AbstractMultivalueFunction { + public MvMedian(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), t -> t.isNumeric() && isRepresentable(t), sourceText(), null, "numeric"); + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return switch (LocalExecutionPlanner.toElementType(field().dataType())) { + case DOUBLE -> () -> new MvMedianDoubleEvaluator(fieldEval.get()); + case INT -> () -> new MvMedianIntEvaluator(fieldEval.get()); + case LONG -> field().dataType() == DataTypes.UNSIGNED_LONG + ? () -> new MvMedianUnsignedLongEvaluator(fieldEval.get()) + : () -> new MvMedianLongEvaluator(fieldEval.get()); + default -> throw new UnsupportedOperationException("unsupported type [" + field().dataType() + "]"); + }; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvMedian(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvMedian::new, field()); + } + + static class Doubles { + public double[] values = new double[2]; + public int count; + } + + @MvEvaluator(extraName = "Double", finish = "finish") + static void process(Doubles doubles, double v) { + if (doubles.values.length < doubles.count + 1) { + doubles.values = ArrayUtil.grow(doubles.values, doubles.count + 1); + } + doubles.values[doubles.count++] = v; + } + + static double finish(Doubles doubles) { + // TODO quickselect + Arrays.sort(doubles.values, 0, doubles.count); + int middle = doubles.count / 2; + double median = doubles.count % 2 == 1 ? doubles.values[middle] : (doubles.values[middle - 1] + doubles.values[middle]) / 2; + doubles.count = 0; + return median; + } + + static double ascending(DoubleBlock values, int firstValue, int count) { + int middle = firstValue + count / 2; + if (count % 2 == 1) { + return values.getDouble(middle); + } + return (values.getDouble(middle - 1) + values.getDouble(middle)) / 2; + } + + static class Longs { + public long[] values = new long[2]; + public int count; + } + + @MvEvaluator(extraName = "Long", finish = "finish", ascending = "ascending") + static void process(Longs longs, long v) { + if (longs.values.length < longs.count + 1) { + longs.values = ArrayUtil.grow(longs.values, longs.count + 1); + } + longs.values[longs.count++] = v; + } + + static long finish(Longs longs) { + // TODO quickselect + Arrays.sort(longs.values, 0, longs.count); + int middle = longs.count / 2; + if (longs.count % 2 == 1) { + longs.count = 0; + return longs.values[middle]; + } + longs.count = 0; + return avgWithoutOverflow(longs.values[middle - 1], longs.values[middle]); + } + + /** + * If the values are ascending pick the middle value or average the two middle values together. + */ + static long ascending(LongBlock values, int firstValue, int count) { + int middle = firstValue + count / 2; + if (count % 2 == 1) { + return values.getLong(middle); + } + return avgWithoutOverflow(values.getLong(middle - 1), values.getLong(middle)); + } + + /** + * Average two {@code long}s without any overflow. + */ + static long avgWithoutOverflow(long a, long b) { + return (a & b) + ((a ^ b) >> 1); + } + + @MvEvaluator(extraName = "UnsignedLong", finish = "finishUnsignedLong", ascending = "ascendingUnsignedLong") + static void processUnsignedLong(Longs longs, long v) { + process(longs, v); + } + + static long finishUnsignedLong(Longs longs) { + if (longs.count % 2 == 1) { + return finish(longs); + } + // TODO quickselect + Arrays.sort(longs.values, 0, longs.count); + int middle = longs.count / 2; + longs.count = 0; + BigInteger a = unsignedLongAsBigInteger(longs.values[middle - 1]); + BigInteger b = unsignedLongAsBigInteger(longs.values[middle]); + return asLongUnsigned(a.add(b).shiftRight(1).longValue()); + } + + /** + * If the values are ascending pick the middle value or average the two middle values together. + */ + static long ascendingUnsignedLong(LongBlock values, int firstValue, int count) { + int middle = firstValue + count / 2; + if (count % 2 == 1) { + return values.getLong(middle); + } + BigInteger a = unsignedLongAsBigInteger(values.getLong(middle - 1)); + BigInteger b = unsignedLongAsBigInteger(values.getLong(middle)); + return asLongUnsigned(a.add(b).shiftRight(1).longValue()); + } + + static class Ints { + public int[] values = new int[2]; + public int count; + } + + @MvEvaluator(extraName = "Int", finish = "finish", ascending = "ascending") + static void process(Ints ints, int v) { + if (ints.values.length < ints.count + 1) { + ints.values = ArrayUtil.grow(ints.values, ints.count + 1); + } + ints.values[ints.count++] = v; + } + + static int finish(Ints ints) { + // TODO quickselect + Arrays.sort(ints.values, 0, ints.count); + int middle = ints.count / 2; + if (ints.count % 2 == 1) { + ints.count = 0; + return ints.values[middle]; + } + ints.count = 0; + return avgWithoutOverflow(ints.values[middle - 1], ints.values[middle]); + } + + /** + * If the values are ascending pick the middle value or average the two middle values together. + */ + static int ascending(IntBlock values, int firstValue, int count) { + int middle = firstValue + count / 2; + if (count % 2 == 1) { + return values.getInt(middle); + } + return avgWithoutOverflow(values.getInt(middle - 1), values.getInt(middle)); + } + + /** + * Average two {@code int}s together without overflow. + */ + static int avgWithoutOverflow(int a, int b) { + return (a & b) + ((a ^ b) >> 1); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMin.java new file mode 100644 index 0000000000000..c6e708d06d89b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMin.java @@ -0,0 +1,95 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.MvEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; + +/** + * Reduce a multivalued field to a single valued field containing the minimum value. + */ +public class MvMin extends AbstractMultivalueFunction { + public MvMin(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), EsqlDataTypes::isRepresentable, sourceText(), null, "representable"); + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return switch (LocalExecutionPlanner.toElementType(field().dataType())) { + case BOOLEAN -> () -> new MvMinBooleanEvaluator(fieldEval.get()); + case BYTES_REF -> () -> new MvMinBytesRefEvaluator(fieldEval.get()); + case DOUBLE -> () -> new MvMinDoubleEvaluator(fieldEval.get()); + case INT -> () -> new MvMinIntEvaluator(fieldEval.get()); + case LONG -> () -> new MvMinLongEvaluator(fieldEval.get()); + case NULL -> () -> EvalOperator.CONSTANT_NULL; + default -> throw new UnsupportedOperationException("unsupported type [" + field().dataType() + "]"); + }; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvMin(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvMin::new, field()); + } + + @MvEvaluator(extraName = "Boolean", ascending = "ascendingIndex") + static boolean process(boolean current, boolean v) { + return current && v; + } + + @MvEvaluator(extraName = "BytesRef", ascending = "ascendingIndex") + static void process(BytesRef current, BytesRef v) { + if (v.compareTo(current) < 0) { + current.bytes = v.bytes; + current.offset = v.offset; + current.length = v.length; + } + } + + @MvEvaluator(extraName = "Double", ascending = "ascendingIndex") + static double process(double current, double v) { + return Math.min(current, v); + } + + @MvEvaluator(extraName = "Int", ascending = "ascendingIndex") + static int process(int current, int v) { + return Math.min(current, v); + } + + @MvEvaluator(extraName = "Long", ascending = "ascendingIndex") + static long process(long current, long v) { + return Math.min(current, v); + } + + /** + * If the values are ascending pick the first value. + */ + static int ascendingIndex(int count) { + return 0; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSum.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSum.java new file mode 100644 index 0000000000000..645639fc5c605 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSum.java @@ -0,0 +1,87 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.multivalue; + +import org.elasticsearch.compute.ann.MvEvaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.isRepresentable; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isType; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAddExact; + +/** + * Reduce a multivalued field to a single valued field containing the sum of all values. + */ +public class MvSum extends AbstractMultivalueFunction { + public MvSum(Source source, Expression field) { + super(source, field); + } + + @Override + protected TypeResolution resolveFieldType() { + return isType(field(), t -> t.isNumeric() && isRepresentable(t), sourceText(), null, "numeric"); + } + + @Override + protected Supplier evaluator(Supplier fieldEval) { + return switch (LocalExecutionPlanner.toElementType(field().dataType())) { + case DOUBLE -> () -> new MvSumDoubleEvaluator(fieldEval.get()); + case INT -> () -> new MvSumIntEvaluator(source(), fieldEval.get()); + case LONG -> field().dataType() == DataTypes.UNSIGNED_LONG + ? () -> new MvSumUnsignedLongEvaluator(source(), fieldEval.get()) + : () -> new MvSumLongEvaluator(source(), fieldEval.get()); + case NULL -> () -> EvalOperator.CONSTANT_NULL; + default -> throw new UnsupportedOperationException("unsupported type [" + field().dataType() + "]"); + }; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MvSum(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvSum::new, field()); + } + + @MvEvaluator(extraName = "Double", finish = "finish") + public static void process(CompensatedSum sum, double v) { + sum.add(v); + } + + public static double finish(CompensatedSum sum) { + double value = sum.value(); + sum.reset(0, 0); + return value; + } + + @MvEvaluator(extraName = "Int", warnExceptions = { ArithmeticException.class }) + static int process(int current, int v) { + return Math.addExact(current, v); + } + + @MvEvaluator(extraName = "Long", warnExceptions = { ArithmeticException.class }) + static long process(long current, long v) { + return Math.addExact(current, v); + } + + @MvEvaluator(extraName = "UnsignedLong", warnExceptions = { ArithmeticException.class }) + static long processUnsignedLong(long current, long v) { + return unsignedLongAddExact(current, v); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java new file mode 100644 index 0000000000000..fa892265a3746 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java @@ -0,0 +1,108 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Stream; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +/** + * Join strings. + */ +public class Concat extends ScalarFunction implements Mappable { + public Concat(Source source, Expression first, List rest) { + super(source, Stream.concat(Stream.of(first), rest.stream()).toList()); + } + + @Override + public DataType dataType() { + return DataTypes.KEYWORD; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = TypeResolution.TYPE_RESOLVED; + for (Expression value : children()) { + resolution = isString(value, sourceText(), DEFAULT); + + if (resolution.unresolved()) { + return resolution; + } + } + + return resolution; + } + + @Override + public boolean foldable() { + return Expressions.foldable(children()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + List> values = children().stream().map(toEvaluator).toList(); + return () -> new ConcatEvaluator( + new BytesRefBuilder(), + values.stream().map(Supplier::get).toArray(EvalOperator.ExpressionEvaluator[]::new) + ); + } + + @Evaluator + static BytesRef process(@Fixed(includeInToString = false) BytesRefBuilder scratch, BytesRef[] values) { + scratch.clear(); + for (int i = 0; i < values.length; i++) { + scratch.append(values[i]); + } + return scratch.get(); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Concat(source(), newChildren.get(0), newChildren.subList(1, newChildren.size())); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Concat::new, children().get(0), children().subList(1, children().size())); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Length.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Length.java new file mode 100644 index 0000000000000..9794d243e7acf --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Length.java @@ -0,0 +1,81 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.UnicodeUtil; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +public class Length extends UnaryScalarFunction implements Mappable { + + public Length(Source source, Expression field) { + super(source, field); + } + + @Override + public DataType dataType() { + return DataTypes.INTEGER; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + return isString(field(), sourceText(), DEFAULT); + } + + @Override + public boolean foldable() { + return field().foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator + static int process(BytesRef val) { + return UnicodeUtil.codePointCount(val); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Length(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Length::new, field()); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + return () -> new LengthEvaluator(field.get()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java new file mode 100644 index 0000000000000..0387725b8e529 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java @@ -0,0 +1,132 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.BinaryScalarFunction; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isStringAndExact; + +/** + * Splits a string on some delimiter into a multivalued string field. + */ +public class Split extends BinaryScalarFunction implements Mappable { + public Split(Source source, Expression str, Expression delim) { + super(source, str, delim); + } + + @Override + public DataType dataType() { + return DataTypes.KEYWORD; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isStringAndExact(left(), sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + return isString(right(), sourceText(), SECOND); + } + + @Override + public boolean foldable() { + return left().foldable() && right().foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "SingleByte") + static void process( + BytesRefBlock.Builder builder, + BytesRef str, + @Fixed byte delim, + @Fixed(includeInToString = false) BytesRef scratch + ) { + scratch.bytes = str.bytes; + scratch.offset = str.offset; + int end = str.offset + str.length; + for (int i = str.offset; i < end; i++) { + if (str.bytes[i] == delim) { + scratch.length = i - scratch.offset; + if (scratch.offset == str.offset) { + builder.beginPositionEntry(); + } + builder.appendBytesRef(scratch); + scratch.offset = i + 1; + } + } + if (scratch.offset == str.offset) { + // Delimiter not found, single valued + builder.appendBytesRef(str); + return; + } + scratch.length = str.length - (scratch.offset - str.offset); + builder.appendBytesRef(scratch); + builder.endPositionEntry(); + } + + @Evaluator(extraName = "Variable") + static void process(BytesRefBlock.Builder builder, BytesRef str, BytesRef delim, @Fixed(includeInToString = false) BytesRef scratch) { + if (delim.length != 1) { + throw new QlIllegalArgumentException("delimiter must be single byte for now"); + } + process(builder, str, delim.bytes[delim.offset], scratch); + } + + @Override + protected BinaryScalarFunction replaceChildren(Expression newLeft, Expression newRight) { + return new Split(source(), newLeft, newRight); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Split::new, left(), right()); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier str = toEvaluator.apply(left()); + if (right().foldable() == false) { + Supplier delim = toEvaluator.apply(right()); + return () -> new SplitVariableEvaluator(str.get(), delim.get(), new BytesRef()); + } + BytesRef delim = (BytesRef) right().fold(); + if (delim.length != 1) { + throw new QlIllegalArgumentException("for now delimiter must be a single byte"); + } + return () -> new SplitSingleByteEvaluator(str.get(), delim.bytes[delim.offset], new BytesRef()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWith.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWith.java new file mode 100644 index 0000000000000..c650c7c3e0199 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWith.java @@ -0,0 +1,101 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +public class StartsWith extends ScalarFunction implements Mappable { + + private final Expression str; + private final Expression prefix; + + public StartsWith(Source source, Expression str, Expression prefix) { + super(source, Arrays.asList(str, prefix)); + this.str = str; + this.prefix = prefix; + } + + @Override + public DataType dataType() { + return DataTypes.BOOLEAN; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isString(str, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + return isString(prefix, sourceText(), SECOND); + } + + @Override + public boolean foldable() { + return str.foldable() && prefix.foldable(); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator + static boolean process(BytesRef str, BytesRef prefix) { + if (str.length < prefix.length) { + return false; + } + return Arrays.equals(str.bytes, str.offset, str.offset + prefix.length, prefix.bytes, prefix.offset, prefix.offset + prefix.length); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new StartsWith(source(), newChildren.get(0), newChildren.get(1)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, StartsWith::new, str, prefix); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier strEval = toEvaluator.apply(str); + Supplier prefixEval = toEvaluator.apply(prefix); + return () -> new StartsWithEvaluator(strEval.get(), prefixEval.get()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Substring.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Substring.java new file mode 100644 index 0000000000000..d061e395e5ad5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Substring.java @@ -0,0 +1,146 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.UnicodeUtil; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.THIRD; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isInteger; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +public class Substring extends ScalarFunction implements OptionalArgument, Mappable { + + private final Expression str, start, length; + + public Substring(Source source, Expression str, Expression start, Expression length) { + super(source, length == null ? Arrays.asList(str, start) : Arrays.asList(str, start, length)); + this.str = str; + this.start = start; + this.length = length; + } + + @Override + public DataType dataType() { + return DataTypes.KEYWORD; + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new TypeResolution("Unresolved children"); + } + + TypeResolution resolution = isString(str, sourceText(), FIRST); + if (resolution.unresolved()) { + return resolution; + } + + resolution = isInteger(start, sourceText(), SECOND); + if (resolution.unresolved()) { + return resolution; + } + + return length == null ? TypeResolution.TYPE_RESOLVED : isInteger(length, sourceText(), THIRD); + } + + @Override + public boolean foldable() { + return str.foldable() && start.foldable() && (length == null || length.foldable()); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Evaluator(extraName = "NoLength") + static BytesRef process(BytesRef str, int start) { + if (str.length == 0) { + return null; + } + int codePointCount = UnicodeUtil.codePointCount(str); + int indexStart = indexStart(codePointCount, start); + return new BytesRef(str.utf8ToString().substring(indexStart)); + } + + @Evaluator + static BytesRef process(BytesRef str, int start, int length) { + if (str.length == 0) { + return null; + } + if (length < 0) { + throw new IllegalArgumentException("Length parameter cannot be negative, found [" + length + "]"); + } + int codePointCount = UnicodeUtil.codePointCount(str); + int indexStart = indexStart(codePointCount, start); + int indexEnd = Math.min(codePointCount, indexStart + length); + String s = str.utf8ToString(); + return new BytesRef(s.substring(s.offsetByCodePoints(0, indexStart), s.offsetByCodePoints(0, indexEnd))); + } + + private static int indexStart(int codePointCount, int start) { + // esql is 1-based when it comes to string manipulation. We treat start = 0 and 1 the same + // a negative value is relative to the end of the string + int indexStart; + if (start > 0) { + indexStart = start - 1; + } else if (start < 0) { + indexStart = codePointCount + start; // start is negative, so this is a subtraction + } else { + indexStart = start; // start == 0 + } + return Math.min(Math.max(0, indexStart), codePointCount); // sanitise string start index + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Substring(source(), newChildren.get(0), newChildren.get(1), length == null ? null : newChildren.get(2)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Substring::new, str, start, length); + } + + @Override + public ScriptTemplate asScript() { + throw new UnsupportedOperationException(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier strSupplier = toEvaluator.apply(str); + Supplier startSupplier = toEvaluator.apply(start); + if (length == null) { + return () -> new SubstringNoLengthEvaluator(strSupplier.get(), startSupplier.get()); + } + Supplier lengthSupplier = toEvaluator.apply(length); + return () -> new SubstringEvaluator(strSupplier.get(), startSupplier.get(), lengthSupplier.get()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Trim.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Trim.java new file mode 100644 index 0000000000000..c9a454eb16105 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Trim.java @@ -0,0 +1,79 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.planner.Mappable; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString; + +/** + * Removes leading and trailing whitespaces from a string. + */ +public final class Trim extends UnaryScalarFunction implements Mappable { + + public Trim(Source source, Expression str) { + super(source, str); + } + + @Override + protected TypeResolution resolveType() { + if (childrenResolved() == false) { + return new Expression.TypeResolution("Unresolved children"); + } + + return isString(field, sourceText(), TypeResolutions.ParamOrdinal.DEFAULT); + } + + @Override + public Object fold() { + return Mappable.super.fold(); + } + + @Override + public Supplier toEvaluator( + Function> toEvaluator + ) { + Supplier field = toEvaluator.apply(field()); + return () -> new TrimEvaluator(field.get()); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new Trim(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Trim::new, field()); + } + + @Evaluator + static BytesRef process(BytesRef val) { + int offset = val.offset; + int length = val.length; + while ((offset < length) && ((val.bytes[offset] & 0xff) <= 0x20)) { + offset++; + } + while ((offset < length) && ((val.bytes[length - 1] & 0xff) <= 0x20)) { + length--; + } + return new BytesRef(val.bytes, offset, length - (offset - val.offset)); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/logical/Not.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/logical/Not.java new file mode 100644 index 0000000000000..8c1774d33bef5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/logical/Not.java @@ -0,0 +1,17 @@ +/* + * 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.expression.predicate.logical; + +import org.elasticsearch.compute.ann.Evaluator; + +public class Not { + @Evaluator + static boolean process(boolean v) { + return false == v; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Add.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Add.java new file mode 100644 index 0000000000000..417a2ed437516 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Add.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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAddExact; + +public class Add { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int lhs, int rhs) { + return Math.addExact(lhs, rhs); + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long lhs, long rhs) { + return Math.addExact(lhs, rhs); + } + + @Evaluator(extraName = "UnsignedLongs", warnExceptions = { ArithmeticException.class }) + public static long processUnsignedLongs(long lhs, long rhs) { + return unsignedLongAddExact(lhs, rhs); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double lhs, double rhs) { + return lhs + rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Div.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Div.java new file mode 100644 index 0000000000000..43a246fbc5c88 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Div.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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; + +public class Div { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int lhs, int rhs) { + return lhs / rhs; + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long lhs, long rhs) { + return lhs / rhs; + } + + @Evaluator(extraName = "UnsignedLongs", warnExceptions = { ArithmeticException.class }) + static long processUnsignedLongs(long lhs, long rhs) { + return asLongUnsigned(Long.divideUnsigned(asLongUnsigned(lhs), asLongUnsigned(rhs))); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double lhs, double rhs) { + return lhs / rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mod.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mod.java new file mode 100644 index 0000000000000..c37d9c0b90dcd --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mod.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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; + +public class Mod { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int lhs, int rhs) { + return lhs % rhs; + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long lhs, long rhs) { + return lhs % rhs; + } + + @Evaluator(extraName = "UnsignedLongs", warnExceptions = { ArithmeticException.class }) + static long processUnsignedLongs(long lhs, long rhs) { + return asLongUnsigned(Long.remainderUnsigned(asLongUnsigned(lhs), asLongUnsigned(rhs))); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double lhs, double rhs) { + return lhs % rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mul.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mul.java new file mode 100644 index 0000000000000..3359d8bea1b0f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Mul.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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongMultiplyExact; + +public class Mul { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int lhs, int rhs) { + return Math.multiplyExact(lhs, rhs); + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long lhs, long rhs) { + return Math.multiplyExact(lhs, rhs); + } + + @Evaluator(extraName = "UnsignedLongs", warnExceptions = { ArithmeticException.class }) + static long processUnsignedLongs(long lhs, long rhs) { + return unsignedLongMultiplyExact(lhs, rhs); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double lhs, double rhs) { + return lhs * rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Neg.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Neg.java new file mode 100644 index 0000000000000..ac4a47d7e8049 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Neg.java @@ -0,0 +1,28 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +public class Neg { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int v) { + return Math.negateExact(v); + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long v) { + return Math.negateExact(v); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double v) { + // This can never fail (including when `v` is +/- infinity or NaN) since negating a double is just a bit flip. + return -v; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Sub.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Sub.java new file mode 100644 index 0000000000000..b7823e14c7310 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/Sub.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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.compute.ann.Evaluator; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongSubtractExact; + +public class Sub { + @Evaluator(extraName = "Ints", warnExceptions = { ArithmeticException.class }) + static int processInts(int lhs, int rhs) { + return Math.subtractExact(lhs, rhs); + } + + @Evaluator(extraName = "Longs", warnExceptions = { ArithmeticException.class }) + static long processLongs(long lhs, long rhs) { + return Math.subtractExact(lhs, rhs); + } + + @Evaluator(extraName = "UnsignedLongs", warnExceptions = { ArithmeticException.class }) + static long processUnsignedLongs(long lhs, long rhs) { + return unsignedLongSubtractExact(lhs, rhs); + } + + @Evaluator(extraName = "Doubles") + static double processDoubles(double lhs, double rhs) { + return lhs - rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/Equals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/Equals.java new file mode 100644 index 0000000000000..7793dc0f8e167 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/Equals.java @@ -0,0 +1,37 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class Equals { + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs == rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs == rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs == rhs; + } + + @Evaluator(extraName = "Keywords") + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return lhs.equals(rhs); + } + + @Evaluator(extraName = "Bools") + static boolean processBools(boolean lhs, boolean rhs) { + return lhs == rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThan.java new file mode 100644 index 0000000000000..84be8eb00c99e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThan.java @@ -0,0 +1,32 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class GreaterThan { + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs > rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs > rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs > rhs; + } + + @Evaluator(extraName = "Keywords") + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return lhs.compareTo(rhs) > 0; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqual.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqual.java new file mode 100644 index 0000000000000..86ce56c7e3bc5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqual.java @@ -0,0 +1,33 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class GreaterThanOrEqual { + + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs >= rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs >= rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs >= rhs; + } + + @Evaluator(extraName = "Keywords") + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return lhs.compareTo(rhs) >= 0; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java new file mode 100644 index 0000000000000..b20160ac936d6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java @@ -0,0 +1,95 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.InProcessor; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; +import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.ql.util.StringUtils.ordinal; + +public class In extends org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.In { + public In(Source source, Expression value, List list) { + super(source, value, list); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, In::new, value(), list()); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new In(source(), newChildren.get(newChildren.size() - 1), newChildren.subList(0, newChildren.size() - 1)); + } + + @Override + public boolean foldable() { + // QL's In fold()s to null, if value() is null, but isn't foldable() unless all children are + // TODO: update this null check in QL too? + return Expressions.isNull(value()) || super.foldable(); + } + + @Override + public Boolean fold() { + if (Expressions.isNull(value()) || list().stream().allMatch(Expressions::isNull)) { + return null; + } + // QL's `In` fold() doesn't handle BytesRef and can't know if this is Keyword/Text, Version or IP anyway. + // `In` allows comparisons of same type only (safe for numerics), so it's safe to apply InProcessor directly with no implicit + // (non-numerical) conversions. + return InProcessor.apply(value().fold(), list().stream().map(Expression::fold).toList()); + } + + @Override + protected boolean areCompatible(DataType left, DataType right) { + if (left == DataTypes.UNSIGNED_LONG || right == DataTypes.UNSIGNED_LONG) { + // automatic numerical conversions not applicable for UNSIGNED_LONG, see Verifier#validateUnsignedLongOperator(). + return left == right; + } + return EsqlDataTypes.areCompatible(left, right); + } + + @Override + protected TypeResolution resolveType() { // TODO: move the foldability check from QL's In to SQL's and remove this method + TypeResolution resolution = TypeResolutions.isExact(value(), functionName(), DEFAULT); + if (resolution.unresolved()) { + return resolution; + } + + DataType dt = value().dataType(); + for (int i = 0; i < list().size(); i++) { + Expression listValue = list().get(i); + if (areCompatible(dt, listValue.dataType()) == false) { + return new TypeResolution( + format( + null, + "{} argument of [{}] must be [{}], found value [{}] type [{}]", + ordinal(i + 1), + sourceText(), + dt.typeName(), + Expressions.name(listValue), + listValue.dataType().typeName() + ) + ); + } + } + + return TypeResolution.TYPE_RESOLVED; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThan.java new file mode 100644 index 0000000000000..da9540b1f0442 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThan.java @@ -0,0 +1,32 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class LessThan { + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs < rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs < rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs < rhs; + } + + @Evaluator(extraName = "Keywords") // TODO rename to "Bytes" + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return lhs.compareTo(rhs) < 0; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqual.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqual.java new file mode 100644 index 0000000000000..af06e80f922c2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqual.java @@ -0,0 +1,32 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class LessThanOrEqual { + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs <= rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs <= rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs <= rhs; + } + + @Evaluator(extraName = "Keywords") + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return lhs.compareTo(rhs) <= 0; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEquals.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEquals.java new file mode 100644 index 0000000000000..9d31e661c93d2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEquals.java @@ -0,0 +1,37 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; + +public class NotEquals { + @Evaluator(extraName = "Ints") + static boolean processInts(int lhs, int rhs) { + return lhs != rhs; + } + + @Evaluator(extraName = "Longs") + static boolean processLongs(long lhs, long rhs) { + return lhs != rhs; + } + + @Evaluator(extraName = "Doubles") + static boolean processDoubles(double lhs, double rhs) { + return lhs != rhs; + } + + @Evaluator(extraName = "Keywords") + static boolean processKeywords(BytesRef lhs, BytesRef rhs) { + return false == lhs.equals(rhs); + } + + @Evaluator(extraName = "Bools") + static boolean processBools(boolean lhs, boolean rhs) { + return lhs != rhs; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatch.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatch.java new file mode 100644 index 0000000000000..a1f4bcd2aa34c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/regex/RegexMatch.java @@ -0,0 +1,23 @@ +/* + * 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.expression.predicate.operator.regex; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.ann.Fixed; + +public class RegexMatch { + @Evaluator + static boolean process(BytesRef input, @Fixed CharacterRunAutomaton pattern) { + if (input == null) { + return false; + } + return pattern.run(input.utf8ToString()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java new file mode 100644 index 0000000000000..95a91da7ab5a0 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java @@ -0,0 +1,338 @@ +/* + * 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.formatter; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.xcontent.MediaType; +import org.elasticsearch.xpack.esql.action.ColumnInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; +import org.elasticsearch.xpack.ql.util.StringUtils; + +import java.net.URLDecoder; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; + +/** + * Templating class for displaying ESQL responses in text formats. + */ +public enum TextFormat implements MediaType { + + /** + * Default text writer. + */ + PLAIN_TEXT() { + @Override + public String format(RestRequest request, EsqlQueryResponse esqlResponse) { + return new TextFormatter(esqlResponse).format(hasHeader(request)); + } + + @Override + public String queryParameter() { + return FORMAT_TEXT; + } + + @Override + String contentType() { + return CONTENT_TYPE_TXT; + } + + @Override + protected Character delimiter() { + throw new UnsupportedOperationException(); + } + + @Override + protected String eol() { + throw new UnsupportedOperationException(); + } + + @Override + public Set headerValues() { + return Set.of( + new HeaderValue(CONTENT_TYPE_TXT, Map.of("header", "present|absent")), + new HeaderValue( + VENDOR_CONTENT_TYPE_TXT, + Map.of("header", "present|absent", COMPATIBLE_WITH_PARAMETER_NAME, VERSION_PATTERN) + ) + ); + } + + }, + + /** + * Comma Separated Values implementation. + * + * Based on: + * https://tools.ietf.org/html/rfc4180 + * https://www.iana.org/assignments/media-types/text/csv + * https://www.w3.org/TR/sparql11-results-csv-tsv/ + * + */ + CSV() { + @Override + protected Character delimiter() { + return ','; + } + + @Override + protected String eol() { + // CRLF + return "\r\n"; + } + + @Override + public String queryParameter() { + return FORMAT_CSV; + } + + @Override + String contentType() { + return CONTENT_TYPE_CSV; + } + + @Override + public String contentType(RestRequest request) { + return contentType() + + "; charset=utf-8; " + + URL_PARAM_HEADER + + "=" + + (hasHeader(request) ? PARAM_HEADER_PRESENT : PARAM_HEADER_ABSENT); + } + + @Override + protected Character delimiter(RestRequest request) { + String delimiterParam = request.param(URL_PARAM_DELIMITER); + if (delimiterParam == null) { + return delimiter(); + } + delimiterParam = URLDecoder.decode(delimiterParam, StandardCharsets.UTF_8); + if (delimiterParam.length() != 1) { + throw new IllegalArgumentException( + "invalid " + (delimiterParam.length() > 0 ? "multi-character" : "empty") + " delimiter [" + delimiterParam + "]" + ); + } + Character delimiter = delimiterParam.charAt(0); + switch (delimiter) { + case '"', '\n', '\r' -> throw new IllegalArgumentException( + "illegal reserved character specified as delimiter [" + delimiter + "]" + ); + case '\t' -> throw new IllegalArgumentException( + "illegal delimiter [TAB] specified as delimiter for the [csv] format; " + "choose the [tsv] format instead" + ); + } + return delimiter; + } + + @Override + String maybeEscape(String value, Character delimiter) { + boolean needsEscaping = false; + + for (int i = 0; i < value.length(); i++) { + char c = value.charAt(i); + if (c == '"' || c == '\n' || c == '\r' || c == delimiter) { + needsEscaping = true; + break; + } + } + + if (needsEscaping) { + StringBuilder sb = new StringBuilder(); + + sb.append('"'); + for (int i = 0; i < value.length(); i++) { + char c = value.charAt(i); + if (value.charAt(i) == '"') { + sb.append('"'); + } + sb.append(c); + } + sb.append('"'); + value = sb.toString(); + } + + return value; + } + + @Override + boolean hasHeader(RestRequest request) { + String header = request.param(URL_PARAM_HEADER); + if (header == null) { + List values = request.getAllHeaderValues("Accept"); + if (values != null) { + // header values are separated by `;` so try breaking it down + for (String value : values) { + String[] params = Strings.tokenizeToStringArray(value, ";"); + for (String param : params) { + if (param.toLowerCase(Locale.ROOT).equals(URL_PARAM_HEADER + "=" + PARAM_HEADER_ABSENT)) { + return false; + } + } + } + } + return true; + } else { + return header.toLowerCase(Locale.ROOT).equals(PARAM_HEADER_ABSENT) == false; + } + } + + @Override + public Set headerValues() { + return Set.of( + new HeaderValue(CONTENT_TYPE_CSV, Map.of("header", "present|absent", "delimiter", ".+")),// more detailed parsing is in + // TextFormat.CSV#delimiter + new HeaderValue( + VENDOR_CONTENT_TYPE_CSV, + Map.of("header", "present|absent", "delimiter", ".+", COMPATIBLE_WITH_PARAMETER_NAME, VERSION_PATTERN) + ) + ); + } + }, + + TSV() { + @Override + protected Character delimiter() { + return '\t'; + } + + @Override + protected String eol() { + // only LF + return "\n"; + } + + @Override + public String queryParameter() { + return FORMAT_TSV; + } + + @Override + String contentType() { + return CONTENT_TYPE_TSV; + } + + @Override + public String contentType(RestRequest request) { + return contentType() + "; charset=utf-8"; + } + + @Override + String maybeEscape(String value, Character __) { + StringBuilder sb = new StringBuilder(); + + for (int i = 0; i < value.length(); i++) { + char c = value.charAt(i); + switch (c) { + case '\n' -> sb.append("\\n"); + case '\t' -> sb.append("\\t"); + default -> sb.append(c); + } + } + + return sb.toString(); + } + + @Override + public Set headerValues() { + return Set.of( + new HeaderValue(CONTENT_TYPE_TSV, Map.of("header", "present|absent")), + new HeaderValue( + VENDOR_CONTENT_TYPE_TSV, + Map.of("header", "present|absent", COMPATIBLE_WITH_PARAMETER_NAME, VERSION_PATTERN) + ) + ); + } + }; + + private static final String FORMAT_TEXT = "txt"; + private static final String FORMAT_CSV = "csv"; + private static final String FORMAT_TSV = "tsv"; + private static final String CONTENT_TYPE_TXT = "text/plain"; + private static final String VENDOR_CONTENT_TYPE_TXT = "text/vnd.elasticsearch+plain"; + private static final String CONTENT_TYPE_CSV = "text/csv"; + private static final String VENDOR_CONTENT_TYPE_CSV = "text/vnd.elasticsearch+csv"; + private static final String CONTENT_TYPE_TSV = "text/tab-separated-values"; + private static final String VENDOR_CONTENT_TYPE_TSV = "text/vnd.elasticsearch+tab-separated-values"; + private static final String URL_PARAM_HEADER = "header"; + private static final String PARAM_HEADER_ABSENT = "absent"; + private static final String PARAM_HEADER_PRESENT = "present"; + /* + * URL parameters + */ + public static final String URL_PARAM_FORMAT = "format"; + public static final String URL_PARAM_DELIMITER = "delimiter"; + + public String format(RestRequest request, EsqlQueryResponse esqlResponse) { + StringBuilder sb = new StringBuilder(); + + // if the header is requested return the info + if (hasHeader(request) && esqlResponse.columns() != null) { + row(sb, esqlResponse.columns(), ColumnInfo::name, delimiter(request)); + } + + for (List row : esqlResponse.values()) { + row(sb, row, f -> Objects.toString(f, StringUtils.EMPTY), delimiter(request)); + } + + return sb.toString(); + } + + boolean hasHeader(RestRequest request) { + return true; + } + + /** + * Formal IANA mime type. + */ + abstract String contentType(); + + /** + * Content type depending on the request. + * Might be used by some formatters (like CSV) to specify certain metadata like + * whether the header is returned or not. + */ + public String contentType(RestRequest request) { + return contentType(); + } + + // utility method for consuming a row. + void row(StringBuilder sb, List row, Function toString, Character delimiter) { + for (int i = 0; i < row.size(); i++) { + sb.append(maybeEscape(toString.apply(row.get(i)), delimiter)); + if (i < row.size() - 1) { + sb.append(delimiter); + } + } + sb.append(eol()); + } + + /** + * Delimiter between fields + */ + protected abstract Character delimiter(); + + protected Character delimiter(RestRequest request) { + return delimiter(); + } + + /** + * String indicating end-of-line or row. + */ + protected abstract String eol(); + + /** + * Method used for escaping (if needed) a given value. + */ + String maybeEscape(String value, Character delimiter) { + return value; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java new file mode 100644 index 0000000000000..4592c3be47c91 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java @@ -0,0 +1,125 @@ +/* + * 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.formatter; + +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; + +import java.util.Objects; +import java.util.function.Function; + +/** + * Formats {@link EsqlQueryResponse} for the textual representation. + */ +public class TextFormatter { + /** + * The minimum width for any column in the formatted results. + */ + private static final int MIN_COLUMN_WIDTH = 15; + + private final EsqlQueryResponse response; + private final int[] width; + private final Function FORMATTER = Objects::toString; + + /** + * Create a new {@linkplain TextFormatter} for formatting responses. + */ + public TextFormatter(EsqlQueryResponse response) { + this.response = response; + var columns = response.columns(); + // Figure out the column widths: + // 1. Start with the widths of the column names + width = new int[columns.size()]; + for (int i = 0; i < width.length; i++) { + // TODO read the width from the data type? + width[i] = Math.max(MIN_COLUMN_WIDTH, columns.get(i).name().length()); + } + + // 2. Expand columns to fit the largest value + for (var row : response.values()) { + for (int i = 0; i < width.length; i++) { + width[i] = Math.max(width[i], FORMATTER.apply(row.get(i)).length()); + } + } + } + + /** + * Format the provided {@linkplain EsqlQueryResponse} optionally including the header lines. + */ + public String format(boolean includeHeader) { + StringBuilder sb = new StringBuilder(estimateSize(response.values().size() + 2)); + + // The header lines + if (includeHeader && response.columns().size() > 0) { + formatHeader(sb); + } + // Now format the results. + formatResults(sb); + + return sb.toString(); + } + + private void formatHeader(StringBuilder sb) { + for (int i = 0; i < width.length; i++) { + if (i > 0) { + sb.append('|'); + } + + String name = response.columns().get(i).name(); + // left padding + int leftPadding = (width[i] - name.length()) / 2; + sb.append(" ".repeat(Math.max(0, leftPadding))); + sb.append(name); + // right padding + sb.append(" ".repeat(Math.max(0, width[i] - name.length() - leftPadding))); + } + sb.append('\n'); + + for (int i = 0; i < width.length; i++) { + if (i > 0) { + sb.append('+'); + } + sb.append("-".repeat(Math.max(0, width[i]))); // emdash creates issues + } + sb.append('\n'); + } + + private void formatResults(StringBuilder sb) { + for (var row : response.values()) { + for (int i = 0; i < width.length; i++) { + if (i > 0) { + sb.append('|'); + } + String string = FORMATTER.apply(row.get(i)); + if (string.length() <= width[i]) { + // Pad + sb.append(string); + sb.append(" ".repeat(Math.max(0, width[i] - string.length()))); + } else { + // Trim + sb.append(string, 0, width[i] - 1); + sb.append('~'); + } + } + sb.append('\n'); + } + } + + /** + * Pick a good estimate of the buffer size needed to contain the rows. + */ + int estimateSize(int rows) { + /* Each column has either a '|' or a '\n' after it + * so initialize size to number of columns then add + * up the actual widths of each column. */ + int rowWidthEstimate = width.length; + for (int w : width) { + rowWidthEstimate += w; + } + return rowWidthEstimate * rows; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNameRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNameRegistry.java new file mode 100644 index 0000000000000..0c10a424d9603 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNameRegistry.java @@ -0,0 +1,261 @@ +/* + * 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.io.stream; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A registry of ESQL names to readers and writers, that can be used to serialize a physical plan + * fragment. Allows to serialize the non-(Named)Writable types in both the QL and ESQL modules. + * Serialization is from the outside in, rather than from within. + *

+ * This class is somewhat analogous to NamedWriteableRegistry, but does not require the types to + * be NamedWriteable. + */ +public class PlanNameRegistry { + + public static final PlanNameRegistry INSTANCE = new PlanNameRegistry(); + + /** Adaptable writer interface to bridge between ESQL and regular stream outputs. */ + @FunctionalInterface + public interface PlanWriter extends Writeable.Writer { + + void write(PlanStreamOutput out, V value) throws IOException; + + @Override + default void write(StreamOutput out, V value) throws IOException { + write((PlanStreamOutput) out, value); + } + + static Writeable.Writer writerFromPlanWriter(PlanWriter planWriter) { + return planWriter; + } + } + + /** Adaptable reader interface to bridge between ESQL and regular stream inputs. */ + @FunctionalInterface + public interface PlanReader extends Writeable.Reader { + + V read(PlanStreamInput in) throws IOException; + + @Override + default V read(StreamInput in) throws IOException { + return read((PlanStreamInput) in); + } + + static Writeable.Reader readerFromPlanReader(PlanReader planReader) { + return planReader; + } + } + + /** Adaptable reader interface that allows access to the reader name. */ + @FunctionalInterface + interface PlanNamedReader extends PlanReader { + + V read(PlanStreamInput in, String name) throws IOException; + + default V read(PlanStreamInput in) throws IOException { + throw new UnsupportedOperationException("should not reach here"); + } + } + + record Entry( + /** The superclass of a writeable category will be read by a reader. */ + Class categoryClass, + /** A name for the writeable which is unique to the categoryClass. */ + String name, + /** A writer for non-NamedWriteable class */ + PlanWriter writer, + /** A reader capability of reading the writeable. */ + PlanReader reader + ) { + + /** Creates a new entry which can be stored by the registry. */ + Entry { + Objects.requireNonNull(categoryClass); + Objects.requireNonNull(name); + Objects.requireNonNull(writer); + Objects.requireNonNull(reader); + } + + static Entry of( + Class categoryClass, + Class concreteClass, + PlanWriter writer, + PlanReader reader + ) { + return new Entry(categoryClass, PlanNamedTypes.name(concreteClass), writer, reader); + } + + static Entry of( + Class categoryClass, + Class concreteClass, + PlanWriter writer, + PlanNamedReader reader + ) { + return new Entry(categoryClass, PlanNamedTypes.name(concreteClass), writer, reader); + } + } + + /** + * The underlying data of the registry maps from the category to an inner + * map of name unique to that category, to the actual reader. + */ + private final Map, Map>> readerRegistry; + + /** + * The underlying data of the registry maps from the category to an inner + * map of name unique to that category, to the actual writer. + */ + private final Map, Map>> writerRegistry; + + public PlanNameRegistry() { + this(PlanNamedTypes.namedTypeEntries()); + } + + /** Constructs a new registry from the given entries. */ + PlanNameRegistry(List entries) { + entries = new ArrayList<>(entries); + entries.sort(Comparator.comparing(e -> e.categoryClass().getName())); + + Map, Map>> rr = new HashMap<>(); + Map, Map>> wr = new HashMap<>(); + for (Entry entry : entries) { + Class categoryClass = entry.categoryClass; + Map> readers = rr.computeIfAbsent(categoryClass, v -> new HashMap<>()); + Map> writers = wr.computeIfAbsent(categoryClass, v -> new HashMap<>()); + + PlanReader oldReader = readers.put(entry.name, entry.reader); + if (oldReader != null) { + throwAlreadyRegisteredReader(categoryClass, entry.name, oldReader.getClass(), entry.reader.getClass()); + } + PlanWriter oldWriter = writers.put(entry.name, entry.writer); + if (oldWriter != null) { + throwAlreadyRegisteredReader(categoryClass, entry.name, oldWriter.getClass(), entry.writer.getClass()); + } + } + + // add subclass categories, e.g. NamedExpressions are also Expressions + Map, List>> subCategories = subCategories(entries); + for (var entry : subCategories.entrySet()) { + var readers = rr.get(entry.getKey()); + var writers = wr.get(entry.getKey()); + for (Class subCategory : entry.getValue()) { + readers.putAll(rr.get(subCategory)); + writers.putAll(wr.get(subCategory)); + } + } + + this.readerRegistry = Map.copyOf(rr); + this.writerRegistry = Map.copyOf(wr); + } + + /** Determines the subclass relation of category classes.*/ + static Map, List>> subCategories(List entries) { + Map, Set>> map = new HashMap<>(); + for (Entry entry : entries) { + Class category = entry.categoryClass; + for (Entry entry1 : entries) { + Class category1 = entry1.categoryClass; + if (category == category1) { + continue; + } + if (category.isAssignableFrom(category1)) { // category is a superclass/interface of category1 + Set> set = map.computeIfAbsent(category, v -> new HashSet<>()); + set.add(category1); + } + } + } + return map.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, s -> new ArrayList<>(s.getValue()))); + } + + PlanReader getReader(Class categoryClass, String name) { + Map> readers = getReaders(categoryClass); + return getReader(categoryClass, name, readers); + } + + static PlanReader getReader(Class categoryClass, String name, Map> readers) { + @SuppressWarnings("unchecked") + PlanReader reader = (PlanReader) readers.get(name); + if (reader == null) { + throwOnUnknownReadable(categoryClass, name); + } + return reader; + } + + Map> getReaders(Class categoryClass) { + Map> readers = readerRegistry.get(categoryClass); + if (readers == null) { + throwOnUnknownCategory(categoryClass); + } + return readers; + } + + PlanWriter getWriter(Class categoryClass, String name, Map> writers) { + @SuppressWarnings("unchecked") + PlanWriter writer = (PlanWriter) writers.get(name); + if (writer == null) { + throwOnUnknownWritable(categoryClass, name); + } + return writer; + } + + public Map> getWriters(Class categoryClass) { + Map> writers = writerRegistry.get(categoryClass); + if (writers == null) { + throwOnUnknownCategory(categoryClass); + } + return writers; + } + + public PlanWriter getWriter(Class categoryClass, String name) { + Map> writers = getWriters(categoryClass); + return getWriter(categoryClass, name, writers); + } + + private static void throwAlreadyRegisteredReader(Class categoryClass, String entryName, Class oldReader, Class entryReader) { + throw new IllegalArgumentException( + "PlanReader [" + + categoryClass.getName() + + "][" + + entryName + + "]" + + " is already registered for [" + + oldReader.getName() + + "]," + + " cannot register [" + + entryReader.getName() + + "]" + ); + } + + private static void throwOnUnknownWritable(Class categoryClass, String name) { + throw new IllegalArgumentException("Unknown writeable [" + categoryClass.getName() + "][" + name + "]"); + } + + private static void throwOnUnknownCategory(Class categoryClass) { + throw new IllegalArgumentException("Unknown writeable category [" + categoryClass.getName() + "]"); + } + + private static void throwOnUnknownReadable(Class categoryClass, String name) { + throw new IllegalArgumentException("Unknown readable [" + categoryClass.getName() + "][" + name + "]"); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java new file mode 100644 index 0000000000000..1ea0fa690880b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java @@ -0,0 +1,1430 @@ +/* + * 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.io.stream; + +import org.elasticsearch.common.TriFunction; +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.dissect.DissectParser; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Avg; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.function.aggregate.CountDistinct; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Max; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Median; +import org.elasticsearch.xpack.esql.expression.function.aggregate.MedianAbsoluteDeviation; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Min; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Percentile; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.esql.expression.function.scalar.UnaryScalarFunction; +import org.elasticsearch.xpack.esql.expression.function.scalar.conditional.Case; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToBoolean; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDatetime; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDegrees; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDouble; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToIP; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToInteger; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToLong; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToRadians; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToString; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToUnsignedLong; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToVersion; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateExtract; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateFormat; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateParse; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.Now; +import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Abs; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Acos; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Asin; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Atan; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Atan2; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.AutoBucket; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Cos; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Cosh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.E; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Floor; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsFinite; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsInfinite; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.IsNaN; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Log10; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pi; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sin; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sinh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Sqrt; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tan; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tanh; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Tau; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.AbstractMultivalueFunction; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvAvg; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvConcat; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvCount; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvDedupe; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMax; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMedian; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvMin; +import org.elasticsearch.xpack.esql.expression.function.scalar.multivalue.MvSum; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Concat; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Length; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Split; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.StartsWith; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Trim; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Dissect.Parser; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.TopN; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.DissectExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.GrokExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.MvExpandExec; +import org.elasticsearch.xpack.esql.plan.physical.OrderExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.RowExec; +import org.elasticsearch.xpack.esql.plan.physical.ShowExec; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.predicate.logical.And; +import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.ArithmeticOperation; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.DefaultBinaryArithmeticOperation; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mod; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparisonProcessor; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NullEquals; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLikePattern; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardPattern; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DateEsField; +import org.elasticsearch.xpack.ql.type.EsField; +import org.elasticsearch.xpack.ql.type.KeywordEsField; +import org.elasticsearch.xpack.ql.type.TextEsField; +import org.elasticsearch.xpack.ql.type.UnsupportedEsField; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.Function; + +import static java.util.Map.entry; +import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.Entry.of; +import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanReader.readerFromPlanReader; +import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanWriter.writerFromPlanWriter; + +/** + * A utility class that consists solely of static methods that describe how to serialize and + * deserialize QL and ESQL plan types. + *

+ * All types that require to be serialized should have a pair of co-located `readFoo` and `writeFoo` + * methods that deserialize and serialize respectively. + *

+ * A type can be named or non-named. A named type has a name written to the stream before its + * contents (similar to NamedWriteable), whereas a non-named type does not (similar to Writable). + * Named types allow to determine specific deserialization implementations for more general types, + * e.g. Literal, which is an Expression. Named types must have an entries in the namedTypeEntries + * list. + */ +public final class PlanNamedTypes { + + private PlanNamedTypes() {} + + /** + * Determines the writeable name of the give class. The simple class name is commonly used for + * {@link NamedWriteable}s and is sufficient here too, but it could be almost anything else. + */ + public static String name(Class cls) { + return cls.getSimpleName(); + } + + static final Class QL_UNARY_SCLR_CLS = + org.elasticsearch.xpack.ql.expression.function.scalar.UnaryScalarFunction.class; + + static final Class ESQL_UNARY_SCLR_CLS = UnaryScalarFunction.class; + + /** + * List of named type entries that link concrete names to stream reader and writer implementations. + * Entries have the form; category, name, serializer method, deserializer method. + */ + public static List namedTypeEntries() { + return List.of( + // Physical Plan Nodes + of(PhysicalPlan.class, AggregateExec.class, PlanNamedTypes::writeAggregateExec, PlanNamedTypes::readAggregateExec), + of(PhysicalPlan.class, DissectExec.class, PlanNamedTypes::writeDissectExec, PlanNamedTypes::readDissectExec), + of(PhysicalPlan.class, EsQueryExec.class, PlanNamedTypes::writeEsQueryExec, PlanNamedTypes::readEsQueryExec), + of(PhysicalPlan.class, EsSourceExec.class, PlanNamedTypes::writeEsSourceExec, PlanNamedTypes::readEsSourceExec), + of(PhysicalPlan.class, EvalExec.class, PlanNamedTypes::writeEvalExec, PlanNamedTypes::readEvalExec), + of(PhysicalPlan.class, EnrichExec.class, PlanNamedTypes::writeEnrichExec, PlanNamedTypes::readEnrichExec), + of(PhysicalPlan.class, ExchangeExec.class, PlanNamedTypes::writeExchangeExec, PlanNamedTypes::readExchangeExec), + of(PhysicalPlan.class, ExchangeSinkExec.class, PlanNamedTypes::writeExchangeSinkExec, PlanNamedTypes::readExchangeSinkExec), + of( + PhysicalPlan.class, + ExchangeSourceExec.class, + PlanNamedTypes::writeExchangeSourceExec, + PlanNamedTypes::readExchangeSourceExec + ), + of(PhysicalPlan.class, FieldExtractExec.class, PlanNamedTypes::writeFieldExtractExec, PlanNamedTypes::readFieldExtractExec), + of(PhysicalPlan.class, FilterExec.class, PlanNamedTypes::writeFilterExec, PlanNamedTypes::readFilterExec), + of(PhysicalPlan.class, FragmentExec.class, PlanNamedTypes::writeFragmentExec, PlanNamedTypes::readFragmentExec), + of(PhysicalPlan.class, GrokExec.class, PlanNamedTypes::writeGrokExec, PlanNamedTypes::readGrokExec), + of(PhysicalPlan.class, LimitExec.class, PlanNamedTypes::writeLimitExec, PlanNamedTypes::readLimitExec), + of(PhysicalPlan.class, MvExpandExec.class, PlanNamedTypes::writeMvExpandExec, PlanNamedTypes::readMvExpandExec), + of(PhysicalPlan.class, OrderExec.class, PlanNamedTypes::writeOrderExec, PlanNamedTypes::readOrderExec), + of(PhysicalPlan.class, ProjectExec.class, PlanNamedTypes::writeProjectExec, PlanNamedTypes::readProjectExec), + of(PhysicalPlan.class, RowExec.class, PlanNamedTypes::writeRowExec, PlanNamedTypes::readRowExec), + of(PhysicalPlan.class, ShowExec.class, PlanNamedTypes::writeShowExec, PlanNamedTypes::readShowExec), + of(PhysicalPlan.class, TopNExec.class, PlanNamedTypes::writeTopNExec, PlanNamedTypes::readTopNExec), + // Logical Plan Nodes - a subset of plans that end up being actually serialized + of(LogicalPlan.class, Aggregate.class, PlanNamedTypes::writeAggregate, PlanNamedTypes::readAggregate), + of(LogicalPlan.class, Dissect.class, PlanNamedTypes::writeDissect, PlanNamedTypes::readDissect), + of(LogicalPlan.class, EsRelation.class, PlanNamedTypes::writeEsRelation, PlanNamedTypes::readEsRelation), + of(LogicalPlan.class, Eval.class, PlanNamedTypes::writeEval, PlanNamedTypes::readEval), + of(LogicalPlan.class, Enrich.class, PlanNamedTypes::writeEnrich, PlanNamedTypes::readEnrich), + of(LogicalPlan.class, Filter.class, PlanNamedTypes::writeFilter, PlanNamedTypes::readFilter), + of(LogicalPlan.class, Grok.class, PlanNamedTypes::writeGrok, PlanNamedTypes::readGrok), + of(LogicalPlan.class, Limit.class, PlanNamedTypes::writeLimit, PlanNamedTypes::readLimit), + of(LogicalPlan.class, OrderBy.class, PlanNamedTypes::writeOrderBy, PlanNamedTypes::readOrderBy), + of(LogicalPlan.class, Project.class, PlanNamedTypes::writeProject, PlanNamedTypes::readProject), + of(LogicalPlan.class, TopN.class, PlanNamedTypes::writeTopN, PlanNamedTypes::readTopN), + // Attributes + of(Attribute.class, FieldAttribute.class, PlanNamedTypes::writeFieldAttribute, PlanNamedTypes::readFieldAttribute), + of(Attribute.class, ReferenceAttribute.class, PlanNamedTypes::writeReferenceAttr, PlanNamedTypes::readReferenceAttr), + of(Attribute.class, MetadataAttribute.class, PlanNamedTypes::writeMetadataAttr, PlanNamedTypes::readMetadataAttr), + of(Attribute.class, UnsupportedAttribute.class, PlanNamedTypes::writeUnsupportedAttr, PlanNamedTypes::readUnsupportedAttr), + // EsFields + of(EsField.class, EsField.class, PlanNamedTypes::writeEsField, PlanNamedTypes::readEsField), + of(EsField.class, DateEsField.class, PlanNamedTypes::writeDateEsField, PlanNamedTypes::readDateEsField), + of(EsField.class, KeywordEsField.class, PlanNamedTypes::writeKeywordEsField, PlanNamedTypes::readKeywordEsField), + of(EsField.class, TextEsField.class, PlanNamedTypes::writeTextEsField, PlanNamedTypes::readTextEsField), + of(EsField.class, UnsupportedEsField.class, PlanNamedTypes::writeUnsupportedEsField, PlanNamedTypes::readUnsupportedEsField), + // NamedExpressions + of(NamedExpression.class, Alias.class, PlanNamedTypes::writeAlias, PlanNamedTypes::readAlias), + // BinaryComparison + of(BinaryComparison.class, Equals.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, NullEquals.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, NotEquals.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, GreaterThan.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, GreaterThanOrEqual.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, LessThan.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + of(BinaryComparison.class, LessThanOrEqual.class, PlanNamedTypes::writeBinComparison, PlanNamedTypes::readBinComparison), + // InComparison + of(ScalarFunction.class, In.class, PlanNamedTypes::writeInComparison, PlanNamedTypes::readInComparison), + // RegexMatch + of(RegexMatch.class, WildcardLike.class, PlanNamedTypes::writeWildcardLike, PlanNamedTypes::readWildcardLike), + of(RegexMatch.class, RLike.class, PlanNamedTypes::writeRLike, PlanNamedTypes::readRLike), + // BinaryLogic + of(BinaryLogic.class, And.class, PlanNamedTypes::writeBinaryLogic, PlanNamedTypes::readBinaryLogic), + of(BinaryLogic.class, Or.class, PlanNamedTypes::writeBinaryLogic, PlanNamedTypes::readBinaryLogic), + // UnaryScalarFunction + of(QL_UNARY_SCLR_CLS, IsNotNull.class, PlanNamedTypes::writeQLUnaryScalar, PlanNamedTypes::readQLUnaryScalar), + of(QL_UNARY_SCLR_CLS, IsNull.class, PlanNamedTypes::writeQLUnaryScalar, PlanNamedTypes::readQLUnaryScalar), + of(QL_UNARY_SCLR_CLS, Not.class, PlanNamedTypes::writeQLUnaryScalar, PlanNamedTypes::readQLUnaryScalar), + of(QL_UNARY_SCLR_CLS, Neg.class, PlanNamedTypes::writeQLUnaryScalar, PlanNamedTypes::readQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Abs.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Acos.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Asin.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Atan.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Cos.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Cosh.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Floor.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, IsFinite.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, IsInfinite.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, IsNaN.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Length.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Log10.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Sin.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Sinh.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Sqrt.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Tan.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Tanh.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToBoolean.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToDatetime.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToDegrees.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToDouble.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToIP.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToInteger.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToLong.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToRadians.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToString.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToUnsignedLong.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, ToVersion.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + of(ESQL_UNARY_SCLR_CLS, Trim.class, PlanNamedTypes::writeESQLUnaryScalar, PlanNamedTypes::readESQLUnaryScalar), + // ScalarFunction + of(ScalarFunction.class, Atan2.class, PlanNamedTypes::writeAtan2, PlanNamedTypes::readAtan2), + of(ScalarFunction.class, AutoBucket.class, PlanNamedTypes::writeAutoBucket, PlanNamedTypes::readAutoBucket), + of(ScalarFunction.class, Case.class, PlanNamedTypes::writeCase, PlanNamedTypes::readCase), + of(ScalarFunction.class, CIDRMatch.class, PlanNamedTypes::writeCIDRMatch, PlanNamedTypes::readCIDRMatch), + of(ScalarFunction.class, Concat.class, PlanNamedTypes::writeConcat, PlanNamedTypes::readConcat), + of(ScalarFunction.class, DateExtract.class, PlanNamedTypes::writeDateExtract, PlanNamedTypes::readDateExtract), + of(ScalarFunction.class, DateFormat.class, PlanNamedTypes::writeDateFormat, PlanNamedTypes::readDateFormat), + of(ScalarFunction.class, DateParse.class, PlanNamedTypes::writeDateTimeParse, PlanNamedTypes::readDateTimeParse), + of(ScalarFunction.class, DateTrunc.class, PlanNamedTypes::writeDateTrunc, PlanNamedTypes::readDateTrunc), + of(ScalarFunction.class, E.class, PlanNamedTypes::writeNoArgScalar, PlanNamedTypes::readNoArgScalar), + of(ScalarFunction.class, Now.class, PlanNamedTypes::writeNow, PlanNamedTypes::readNow), + of(ScalarFunction.class, Pi.class, PlanNamedTypes::writeNoArgScalar, PlanNamedTypes::readNoArgScalar), + of(ScalarFunction.class, Round.class, PlanNamedTypes::writeRound, PlanNamedTypes::readRound), + of(ScalarFunction.class, Pow.class, PlanNamedTypes::writePow, PlanNamedTypes::readPow), + of(ScalarFunction.class, StartsWith.class, PlanNamedTypes::writeStartsWith, PlanNamedTypes::readStartsWith), + of(ScalarFunction.class, Substring.class, PlanNamedTypes::writeSubstring, PlanNamedTypes::readSubstring), + of(ScalarFunction.class, Split.class, PlanNamedTypes::writeSplit, PlanNamedTypes::readSplit), + of(ScalarFunction.class, Tau.class, PlanNamedTypes::writeNoArgScalar, PlanNamedTypes::readNoArgScalar), + // ArithmeticOperations + of(ArithmeticOperation.class, Add.class, PlanNamedTypes::writeArithmeticOperation, PlanNamedTypes::readArithmeticOperation), + of(ArithmeticOperation.class, Sub.class, PlanNamedTypes::writeArithmeticOperation, PlanNamedTypes::readArithmeticOperation), + of(ArithmeticOperation.class, Mul.class, PlanNamedTypes::writeArithmeticOperation, PlanNamedTypes::readArithmeticOperation), + of(ArithmeticOperation.class, Div.class, PlanNamedTypes::writeArithmeticOperation, PlanNamedTypes::readArithmeticOperation), + of(ArithmeticOperation.class, Mod.class, PlanNamedTypes::writeArithmeticOperation, PlanNamedTypes::readArithmeticOperation), + // AggregateFunctions + of(AggregateFunction.class, Avg.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, Count.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, CountDistinct.class, PlanNamedTypes::writeCountDistinct, PlanNamedTypes::readCountDistinct), + of(AggregateFunction.class, Min.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, Max.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, Median.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, MedianAbsoluteDeviation.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + of(AggregateFunction.class, Percentile.class, PlanNamedTypes::writePercentile, PlanNamedTypes::readPercentile), + of(AggregateFunction.class, Sum.class, PlanNamedTypes::writeAggFunction, PlanNamedTypes::readAggFunction), + // Multivalue functions + of(ScalarFunction.class, MvAvg.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvCount.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvConcat.class, PlanNamedTypes::writeMvConcat, PlanNamedTypes::readMvConcat), + of(ScalarFunction.class, MvDedupe.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvMax.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvMedian.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvMin.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + of(ScalarFunction.class, MvSum.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction), + // Expressions (other) + of(Expression.class, Literal.class, PlanNamedTypes::writeLiteral, PlanNamedTypes::readLiteral), + of(Expression.class, Order.class, PlanNamedTypes::writeOrder, PlanNamedTypes::readOrder) + ); + } + + // -- physical plan nodes + static AggregateExec readAggregateExec(PlanStreamInput in) throws IOException { + return new AggregateExec( + Source.EMPTY, + in.readPhysicalPlanNode(), + in.readList(readerFromPlanReader(PlanStreamInput::readExpression)), + readNamedExpressions(in), + in.readEnum(AggregateExec.Mode.class), + in.readOptionalVInt() + ); + } + + static void writeAggregateExec(PlanStreamOutput out, AggregateExec aggregateExec) throws IOException { + out.writePhysicalPlanNode(aggregateExec.child()); + out.writeCollection(aggregateExec.groupings(), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + writeNamedExpressions(out, aggregateExec.aggregates()); + out.writeEnum(aggregateExec.getMode()); + out.writeOptionalVInt(aggregateExec.estimatedRowSize()); + } + + static DissectExec readDissectExec(PlanStreamInput in) throws IOException { + return new DissectExec(Source.EMPTY, in.readPhysicalPlanNode(), in.readExpression(), readDissectParser(in), readAttributes(in)); + } + + static void writeDissectExec(PlanStreamOutput out, DissectExec dissectExec) throws IOException { + out.writePhysicalPlanNode(dissectExec.child()); + out.writeExpression(dissectExec.inputExpression()); + writeDissectParser(out, dissectExec.parser()); + writeAttributes(out, dissectExec.extractedFields()); + } + + static EsQueryExec readEsQueryExec(PlanStreamInput in) throws IOException { + return new EsQueryExec( + Source.EMPTY, + readEsIndex(in), + readAttributes(in), + in.readOptionalNamedWriteable(QueryBuilder.class), + in.readOptionalNamed(Expression.class), + in.readOptionalList(readerFromPlanReader(PlanNamedTypes::readFieldSort)), + in.readOptionalVInt() + ); + } + + static void writeEsQueryExec(PlanStreamOutput out, EsQueryExec esQueryExec) throws IOException { + assert esQueryExec.children().size() == 0; + writeEsIndex(out, esQueryExec.index()); + writeAttributes(out, esQueryExec.output()); + out.writeOptionalNamedWriteable(esQueryExec.query()); + out.writeOptionalExpression(esQueryExec.limit()); + out.writeOptionalCollection(esQueryExec.sorts(), writerFromPlanWriter(PlanNamedTypes::writeFieldSort)); + out.writeOptionalInt(esQueryExec.estimatedRowSize()); + } + + static EsSourceExec readEsSourceExec(PlanStreamInput in) throws IOException { + return new EsSourceExec(Source.EMPTY, readEsIndex(in), readAttributes(in), in.readOptionalNamedWriteable(QueryBuilder.class)); + } + + static void writeEsSourceExec(PlanStreamOutput out, EsSourceExec esSourceExec) throws IOException { + writeEsIndex(out, esSourceExec.index()); + writeAttributes(out, esSourceExec.output()); + out.writeOptionalNamedWriteable(esSourceExec.query()); + } + + static EvalExec readEvalExec(PlanStreamInput in) throws IOException { + return new EvalExec(Source.EMPTY, in.readPhysicalPlanNode(), readNamedExpressions(in)); + } + + static void writeEvalExec(PlanStreamOutput out, EvalExec evalExec) throws IOException { + out.writePhysicalPlanNode(evalExec.child()); + writeNamedExpressions(out, evalExec.fields()); + } + + static EnrichExec readEnrichExec(PlanStreamInput in) throws IOException { + return new EnrichExec( + Source.EMPTY, + in.readPhysicalPlanNode(), + in.readNamedExpression(), + in.readString(), + in.readString(), + readEsIndex(in), + readNamedExpressions(in) + ); + } + + static void writeEnrichExec(PlanStreamOutput out, EnrichExec enrich) throws IOException { + out.writePhysicalPlanNode(enrich.child()); + out.writeNamedExpression(enrich.matchField()); + out.writeString(enrich.policyName()); + out.writeString(enrich.policyMatchField()); + writeEsIndex(out, enrich.enrichIndex()); + writeNamedExpressions(out, enrich.enrichFields()); + } + + static ExchangeExec readExchangeExec(PlanStreamInput in) throws IOException { + return new ExchangeExec(Source.EMPTY, readAttributes(in), in.readBoolean(), in.readPhysicalPlanNode()); + } + + static void writeExchangeExec(PlanStreamOutput out, ExchangeExec exchangeExec) throws IOException { + writeAttributes(out, exchangeExec.output()); + out.writeBoolean(exchangeExec.isInBetweenAggs()); + out.writePhysicalPlanNode(exchangeExec.child()); + } + + static ExchangeSinkExec readExchangeSinkExec(PlanStreamInput in) throws IOException { + return new ExchangeSinkExec(Source.EMPTY, readAttributes(in), in.readPhysicalPlanNode()); + } + + static void writeExchangeSinkExec(PlanStreamOutput out, ExchangeSinkExec exchangeSinkExec) throws IOException { + writeAttributes(out, exchangeSinkExec.output()); + out.writePhysicalPlanNode(exchangeSinkExec.child()); + } + + static ExchangeSourceExec readExchangeSourceExec(PlanStreamInput in) throws IOException { + return new ExchangeSourceExec(Source.EMPTY, readAttributes(in), in.readBoolean()); + } + + static void writeExchangeSourceExec(PlanStreamOutput out, ExchangeSourceExec exchangeSourceExec) throws IOException { + writeAttributes(out, exchangeSourceExec.output()); + out.writeBoolean(exchangeSourceExec.isIntermediateAgg()); + } + + static FieldExtractExec readFieldExtractExec(PlanStreamInput in) throws IOException { + return new FieldExtractExec(Source.EMPTY, in.readPhysicalPlanNode(), readAttributes(in)); + } + + static void writeFieldExtractExec(PlanStreamOutput out, FieldExtractExec fieldExtractExec) throws IOException { + out.writePhysicalPlanNode(fieldExtractExec.child()); + writeAttributes(out, fieldExtractExec.attributesToExtract()); + } + + static FilterExec readFilterExec(PlanStreamInput in) throws IOException { + return new FilterExec(Source.EMPTY, in.readPhysicalPlanNode(), in.readExpression()); + } + + static void writeFilterExec(PlanStreamOutput out, FilterExec filterExec) throws IOException { + out.writePhysicalPlanNode(filterExec.child()); + out.writeExpression(filterExec.condition()); + } + + static FragmentExec readFragmentExec(PlanStreamInput in) throws IOException { + return new FragmentExec( + Source.EMPTY, + in.readLogicalPlanNode(), + in.readOptionalNamedWriteable(QueryBuilder.class), + in.readOptionalVInt() + ); + } + + static void writeFragmentExec(PlanStreamOutput out, FragmentExec fragmentExec) throws IOException { + out.writeLogicalPlanNode(fragmentExec.fragment()); + out.writeOptionalNamedWriteable(fragmentExec.esFilter()); + out.writeOptionalVInt(fragmentExec.estimatedRowSize()); + } + + static GrokExec readGrokExec(PlanStreamInput in) throws IOException { + return new GrokExec( + Source.EMPTY, + in.readPhysicalPlanNode(), + in.readExpression(), + Grok.pattern(Source.EMPTY, in.readString()), + readAttributes(in) + ); + } + + static void writeGrokExec(PlanStreamOutput out, GrokExec grokExec) throws IOException { + out.writePhysicalPlanNode(grokExec.child()); + out.writeExpression(grokExec.inputExpression()); + out.writeString(grokExec.pattern().pattern()); + writeAttributes(out, grokExec.extractedFields()); + } + + static LimitExec readLimitExec(PlanStreamInput in) throws IOException { + return new LimitExec(Source.EMPTY, in.readPhysicalPlanNode(), in.readNamed(Expression.class)); + } + + static void writeLimitExec(PlanStreamOutput out, LimitExec limitExec) throws IOException { + out.writePhysicalPlanNode(limitExec.child()); + out.writeExpression(limitExec.limit()); + } + + static MvExpandExec readMvExpandExec(PlanStreamInput in) throws IOException { + return new MvExpandExec(Source.EMPTY, in.readPhysicalPlanNode(), in.readNamedExpression()); + } + + static void writeMvExpandExec(PlanStreamOutput out, MvExpandExec mvExpandExec) throws IOException { + out.writePhysicalPlanNode(mvExpandExec.child()); + out.writeNamedExpression(mvExpandExec.target()); + } + + static OrderExec readOrderExec(PlanStreamInput in) throws IOException { + return new OrderExec(Source.EMPTY, in.readPhysicalPlanNode(), in.readList(readerFromPlanReader(PlanNamedTypes::readOrder))); + } + + static void writeOrderExec(PlanStreamOutput out, OrderExec orderExec) throws IOException { + out.writePhysicalPlanNode(orderExec.child()); + out.writeCollection(orderExec.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder)); + } + + static ProjectExec readProjectExec(PlanStreamInput in) throws IOException { + return new ProjectExec(Source.EMPTY, in.readPhysicalPlanNode(), readNamedExpressions(in)); + } + + static void writeProjectExec(PlanStreamOutput out, ProjectExec projectExec) throws IOException { + out.writePhysicalPlanNode(projectExec.child()); + writeNamedExpressions(out, projectExec.projections()); + } + + static RowExec readRowExec(PlanStreamInput in) throws IOException { + return new RowExec(Source.EMPTY, readNamedExpressions(in)); + } + + static void writeRowExec(PlanStreamOutput out, RowExec rowExec) throws IOException { + assert rowExec.children().size() == 0; + writeNamedExpressions(out, rowExec.fields()); + } + + @SuppressWarnings("unchecked") + static ShowExec readShowExec(PlanStreamInput in) throws IOException { + return new ShowExec(Source.EMPTY, readAttributes(in), (List>) in.readGenericValue()); + } + + static void writeShowExec(PlanStreamOutput out, ShowExec showExec) throws IOException { + writeAttributes(out, showExec.output()); + out.writeGenericValue(showExec.values()); + } + + static TopNExec readTopNExec(PlanStreamInput in) throws IOException { + return new TopNExec( + Source.EMPTY, + in.readPhysicalPlanNode(), + in.readList(readerFromPlanReader(PlanNamedTypes::readOrder)), + in.readNamed(Expression.class), + in.readOptionalVInt() + ); + } + + static void writeTopNExec(PlanStreamOutput out, TopNExec topNExec) throws IOException { + out.writePhysicalPlanNode(topNExec.child()); + out.writeCollection(topNExec.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder)); + out.writeExpression(topNExec.limit()); + out.writeOptionalVInt(topNExec.estimatedRowSize()); + } + + // -- Logical plan nodes + static Aggregate readAggregate(PlanStreamInput in) throws IOException { + return new Aggregate( + Source.EMPTY, + in.readLogicalPlanNode(), + in.readList(readerFromPlanReader(PlanStreamInput::readExpression)), + readNamedExpressions(in) + ); + } + + static void writeAggregate(PlanStreamOutput out, Aggregate aggregate) throws IOException { + out.writeLogicalPlanNode(aggregate.child()); + out.writeCollection(aggregate.groupings(), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + writeNamedExpressions(out, aggregate.aggregates()); + } + + static Dissect readDissect(PlanStreamInput in) throws IOException { + return new Dissect(Source.EMPTY, in.readLogicalPlanNode(), in.readExpression(), readDissectParser(in), readAttributes(in)); + } + + static void writeDissect(PlanStreamOutput out, Dissect dissect) throws IOException { + out.writeLogicalPlanNode(dissect.child()); + out.writeExpression(dissect.input()); + writeDissectParser(out, dissect.parser()); + writeAttributes(out, dissect.extractedFields()); + } + + static EsRelation readEsRelation(PlanStreamInput in) throws IOException { + return new EsRelation(Source.EMPTY, readEsIndex(in), readAttributes(in)); + } + + static void writeEsRelation(PlanStreamOutput out, EsRelation relation) throws IOException { + assert relation.children().size() == 0; + writeEsIndex(out, relation.index()); + writeAttributes(out, relation.output()); + } + + static Eval readEval(PlanStreamInput in) throws IOException { + return new Eval(Source.EMPTY, in.readLogicalPlanNode(), readNamedExpressions(in)); + } + + static void writeEval(PlanStreamOutput out, Eval eval) throws IOException { + out.writeLogicalPlanNode(eval.child()); + writeNamedExpressions(out, eval.fields()); + } + + static Enrich readEnrich(PlanStreamInput in) throws IOException { + return new Enrich( + Source.EMPTY, + in.readLogicalPlanNode(), + in.readExpression(), + in.readNamedExpression(), + new EnrichPolicyResolution(in.readString(), new EnrichPolicy(in), IndexResolution.valid(readEsIndex(in))), + readNamedExpressions(in) + ); + } + + static void writeEnrich(PlanStreamOutput out, Enrich enrich) throws IOException { + out.writeLogicalPlanNode(enrich.child()); + out.writeExpression(enrich.policyName()); + out.writeNamedExpression(enrich.matchField()); + out.writeString(enrich.policy().policyName()); + enrich.policy().policy().writeTo(out); + writeEsIndex(out, enrich.policy().index().get()); + writeNamedExpressions(out, enrich.enrichFields()); + } + + static Filter readFilter(PlanStreamInput in) throws IOException { + return new Filter(Source.EMPTY, in.readLogicalPlanNode(), in.readExpression()); + } + + static void writeFilter(PlanStreamOutput out, Filter filter) throws IOException { + out.writeLogicalPlanNode(filter.child()); + out.writeExpression(filter.condition()); + } + + static Grok readGrok(PlanStreamInput in) throws IOException { + return new Grok( + Source.EMPTY, + in.readLogicalPlanNode(), + in.readExpression(), + Grok.pattern(Source.EMPTY, in.readString()), + readAttributes(in) + ); + } + + static void writeGrok(PlanStreamOutput out, Grok grok) throws IOException { + out.writeLogicalPlanNode(grok.child()); + out.writeExpression(grok.input()); + out.writeString(grok.parser().pattern()); + writeAttributes(out, grok.extractedFields()); + } + + static Limit readLimit(PlanStreamInput in) throws IOException { + return new Limit(Source.EMPTY, in.readNamed(Expression.class), in.readLogicalPlanNode()); + } + + static void writeLimit(PlanStreamOutput out, Limit limit) throws IOException { + out.writeExpression(limit.limit()); + out.writeLogicalPlanNode(limit.child()); + } + + static OrderBy readOrderBy(PlanStreamInput in) throws IOException { + return new OrderBy(Source.EMPTY, in.readLogicalPlanNode(), in.readList(readerFromPlanReader(PlanNamedTypes::readOrder))); + } + + static void writeOrderBy(PlanStreamOutput out, OrderBy order) throws IOException { + out.writeLogicalPlanNode(order.child()); + out.writeCollection(order.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder)); + } + + static Project readProject(PlanStreamInput in) throws IOException { + return new Project(Source.EMPTY, in.readLogicalPlanNode(), readNamedExpressions(in)); + } + + static void writeProject(PlanStreamOutput out, Project project) throws IOException { + out.writeLogicalPlanNode(project.child()); + writeNamedExpressions(out, project.projections()); + } + + static TopN readTopN(PlanStreamInput in) throws IOException { + return new TopN( + Source.EMPTY, + in.readLogicalPlanNode(), + in.readList(readerFromPlanReader(PlanNamedTypes::readOrder)), + in.readNamed(Expression.class) + ); + } + + static void writeTopN(PlanStreamOutput out, TopN topN) throws IOException { + out.writeLogicalPlanNode(topN.child()); + out.writeCollection(topN.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder)); + out.writeExpression(topN.limit()); + } + + // + // -- Attributes + // + + private static List readAttributes(PlanStreamInput in) throws IOException { + return in.readList(readerFromPlanReader(PlanStreamInput::readAttribute)); + } + + static void writeAttributes(PlanStreamOutput out, List attributes) throws IOException { + out.writeCollection(attributes, writerFromPlanWriter(PlanStreamOutput::writeAttribute)); + } + + private static List readNamedExpressions(PlanStreamInput in) throws IOException { + return in.readList(readerFromPlanReader(PlanStreamInput::readNamedExpression)); + } + + static void writeNamedExpressions(PlanStreamOutput out, List namedExpressions) throws IOException { + out.writeCollection(namedExpressions, writerFromPlanWriter(PlanStreamOutput::writeNamedExpression)); + } + + static FieldAttribute readFieldAttribute(PlanStreamInput in) throws IOException { + return new FieldAttribute( + Source.EMPTY, + in.readOptionalWithReader(PlanNamedTypes::readFieldAttribute), + in.readString(), + in.dataTypeFromTypeName(in.readString()), + in.readEsFieldNamed(), + in.readOptionalString(), + in.readEnum(Nullability.class), + in.nameIdFromLongValue(in.readLong()), + in.readBoolean() + ); + } + + static void writeFieldAttribute(PlanStreamOutput out, FieldAttribute fileAttribute) throws IOException { + out.writeOptionalWriteable(fileAttribute.parent() == null ? null : o -> writeFieldAttribute(out, fileAttribute.parent())); + out.writeString(fileAttribute.name()); + out.writeString(fileAttribute.dataType().typeName()); + out.writeNamed(EsField.class, fileAttribute.field()); + out.writeOptionalString(fileAttribute.qualifier()); + out.writeEnum(fileAttribute.nullable()); + out.writeLong(Long.parseLong(fileAttribute.id().toString())); + out.writeBoolean(fileAttribute.synthetic()); + } + + static ReferenceAttribute readReferenceAttr(PlanStreamInput in) throws IOException { + return new ReferenceAttribute( + Source.EMPTY, + in.readString(), + in.dataTypeFromTypeName(in.readString()), + in.readOptionalString(), + in.readEnum(Nullability.class), + in.nameIdFromLongValue(in.readLong()), + in.readBoolean() + ); + } + + static void writeReferenceAttr(PlanStreamOutput out, ReferenceAttribute referenceAttribute) throws IOException { + out.writeString(referenceAttribute.name()); + out.writeString(referenceAttribute.dataType().typeName()); + out.writeOptionalString(referenceAttribute.qualifier()); + out.writeEnum(referenceAttribute.nullable()); + out.writeLong(Long.parseLong(referenceAttribute.id().toString())); + out.writeBoolean(referenceAttribute.synthetic()); + } + + static MetadataAttribute readMetadataAttr(PlanStreamInput in) throws IOException { + return new MetadataAttribute( + Source.EMPTY, + in.readString(), + in.dataTypeFromTypeName(in.readString()), + in.readOptionalString(), + in.readEnum(Nullability.class), + in.nameIdFromLongValue(in.readLong()), + in.readBoolean(), + in.readBoolean() + ); + } + + static void writeMetadataAttr(PlanStreamOutput out, MetadataAttribute metadataAttribute) throws IOException { + out.writeString(metadataAttribute.name()); + out.writeString(metadataAttribute.dataType().typeName()); + out.writeOptionalString(metadataAttribute.qualifier()); + out.writeEnum(metadataAttribute.nullable()); + out.writeLong(Long.parseLong(metadataAttribute.id().toString())); + out.writeBoolean(metadataAttribute.synthetic()); + out.writeBoolean(metadataAttribute.searchable()); + } + + static UnsupportedAttribute readUnsupportedAttr(PlanStreamInput in) throws IOException { + return new UnsupportedAttribute( + Source.EMPTY, + in.readString(), + readUnsupportedEsField(in), + in.readOptionalString(), + in.nameIdFromLongValue(in.readLong()) + ); + } + + static void writeUnsupportedAttr(PlanStreamOutput out, UnsupportedAttribute unsupportedAttribute) throws IOException { + out.writeString(unsupportedAttribute.name()); + writeUnsupportedEsField(out, unsupportedAttribute.field()); + out.writeOptionalString(unsupportedAttribute.hasCustomMessage() ? unsupportedAttribute.unresolvedMessage() : null); + out.writeLong(Long.parseLong(unsupportedAttribute.id().toString())); + } + + // -- EsFields + + static EsField readEsField(PlanStreamInput in) throws IOException { + return new EsField( + in.readString(), + in.dataTypeFromTypeName(in.readString()), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)), + in.readBoolean(), + in.readBoolean() + ); + } + + static void writeEsField(PlanStreamOutput out, EsField esField) throws IOException { + out.writeString(esField.getName()); + out.writeString(esField.getDataType().typeName()); + out.writeMap(esField.getProperties(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + out.writeBoolean(esField.isAggregatable()); + out.writeBoolean(esField.isAlias()); + } + + static DateEsField readDateEsField(PlanStreamInput in) throws IOException { + return DateEsField.dateEsField( + in.readString(), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)), + in.readBoolean() + ); + } + + static void writeDateEsField(PlanStreamOutput out, DateEsField dateEsField) throws IOException { + out.writeString(dateEsField.getName()); + out.writeMap(dateEsField.getProperties(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + out.writeBoolean(dateEsField.isAggregatable()); + } + + static KeywordEsField readKeywordEsField(PlanStreamInput in) throws IOException { + return new KeywordEsField( + in.readString(), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)), + in.readBoolean(), + in.readInt(), + in.readBoolean(), + in.readBoolean() + ); + } + + static void writeKeywordEsField(PlanStreamOutput out, KeywordEsField keywordEsField) throws IOException { + out.writeString(keywordEsField.getName()); + out.writeMap(keywordEsField.getProperties(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + out.writeBoolean(keywordEsField.isAggregatable()); + out.writeInt(keywordEsField.getPrecision()); + out.writeBoolean(keywordEsField.getNormalized()); + out.writeBoolean(keywordEsField.isAlias()); + } + + static TextEsField readTextEsField(PlanStreamInput in) throws IOException { + return new TextEsField( + in.readString(), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)), + in.readBoolean(), + in.readBoolean() + ); + } + + static void writeTextEsField(PlanStreamOutput out, TextEsField textEsField) throws IOException { + out.writeString(textEsField.getName()); + out.writeMap(textEsField.getProperties(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + out.writeBoolean(textEsField.isAggregatable()); + out.writeBoolean(textEsField.isAlias()); + } + + static UnsupportedEsField readUnsupportedEsField(PlanStreamInput in) throws IOException { + return new UnsupportedEsField( + in.readString(), + in.readString(), + in.readOptionalString(), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)) + ); + } + + static void writeUnsupportedEsField(PlanStreamOutput out, UnsupportedEsField unsupportedEsField) throws IOException { + out.writeString(unsupportedEsField.getName()); + out.writeString(unsupportedEsField.getOriginalType()); + out.writeOptionalString(unsupportedEsField.getInherited()); + out.writeMap(unsupportedEsField.getProperties(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + } + + // -- BinaryComparison + + static BinaryComparison readBinComparison(PlanStreamInput in, String name) throws IOException { + var operation = in.readEnum(BinaryComparisonProcessor.BinaryComparisonOperation.class); + var left = in.readExpression(); + var right = in.readExpression(); + var zoneId = in.readOptionalZoneId(); + return switch (operation) { + case EQ -> new Equals(Source.EMPTY, left, right, zoneId); + case NULLEQ -> new NullEquals(Source.EMPTY, left, right, zoneId); + case NEQ -> new NotEquals(Source.EMPTY, left, right, zoneId); + case GT -> new GreaterThan(Source.EMPTY, left, right, zoneId); + case GTE -> new GreaterThanOrEqual(Source.EMPTY, left, right, zoneId); + case LT -> new LessThan(Source.EMPTY, left, right, zoneId); + case LTE -> new LessThanOrEqual(Source.EMPTY, left, right, zoneId); + }; + } + + static void writeBinComparison(PlanStreamOutput out, BinaryComparison binaryComparison) throws IOException { + out.writeEnum(binaryComparison.function()); + out.writeExpression(binaryComparison.left()); + out.writeExpression(binaryComparison.right()); + out.writeOptionalZoneId(binaryComparison.zoneId()); + } + + // -- InComparison + + static In readInComparison(PlanStreamInput in) throws IOException { + return new In(Source.EMPTY, in.readExpression(), in.readList(readerFromPlanReader(PlanStreamInput::readExpression))); + } + + static void writeInComparison(PlanStreamOutput out, In in) throws IOException { + out.writeExpression(in.value()); + out.writeCollection(in.list(), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + } + + // -- RegexMatch + + static WildcardLike readWildcardLike(PlanStreamInput in, String name) throws IOException { + return new WildcardLike(Source.EMPTY, in.readExpression(), new WildcardPattern(in.readString())); + } + + static void writeWildcardLike(PlanStreamOutput out, WildcardLike like) throws IOException { + out.writeExpression(like.field()); + out.writeString(like.pattern().pattern()); + } + + static RLike readRLike(PlanStreamInput in, String name) throws IOException { + return new RLike(Source.EMPTY, in.readExpression(), new RLikePattern(in.readString())); + } + + static void writeRLike(PlanStreamOutput out, RLike like) throws IOException { + out.writeExpression(like.field()); + out.writeString(like.pattern().asJavaRegex()); + } + + // -- BinaryLogic + + static final Map> BINARY_LOGIC_CTRS = Map.ofEntries( + entry(name(And.class), And::new), + entry(name(Or.class), Or::new) + ); + + static BinaryLogic readBinaryLogic(PlanStreamInput in, String name) throws IOException { + var left = in.readExpression(); + var right = in.readExpression(); + return BINARY_LOGIC_CTRS.get(name).apply(Source.EMPTY, left, right); + } + + static void writeBinaryLogic(PlanStreamOutput out, BinaryLogic binaryLogic) throws IOException { + out.writeExpression(binaryLogic.left()); + out.writeExpression(binaryLogic.right()); + } + + // -- UnaryScalarFunction + + static final Map> ESQL_UNARY_SCALAR_CTRS = Map.ofEntries( + entry(name(Abs.class), Abs::new), + entry(name(Acos.class), Acos::new), + entry(name(Asin.class), Asin::new), + entry(name(Atan.class), Atan::new), + entry(name(Cos.class), Cos::new), + entry(name(Cosh.class), Cosh::new), + entry(name(Floor.class), Floor::new), + entry(name(IsFinite.class), IsFinite::new), + entry(name(IsInfinite.class), IsInfinite::new), + entry(name(IsNaN.class), IsNaN::new), + entry(name(Length.class), Length::new), + entry(name(Log10.class), Log10::new), + entry(name(Sin.class), Sin::new), + entry(name(Sinh.class), Sinh::new), + entry(name(Sqrt.class), Sqrt::new), + entry(name(Tan.class), Tan::new), + entry(name(Tanh.class), Tanh::new), + entry(name(ToBoolean.class), ToBoolean::new), + entry(name(ToDatetime.class), ToDatetime::new), + entry(name(ToDegrees.class), ToDegrees::new), + entry(name(ToDouble.class), ToDouble::new), + entry(name(ToIP.class), ToIP::new), + entry(name(ToInteger.class), ToInteger::new), + entry(name(ToLong.class), ToLong::new), + entry(name(ToRadians.class), ToRadians::new), + entry(name(ToString.class), ToString::new), + entry(name(ToUnsignedLong.class), ToUnsignedLong::new), + entry(name(ToVersion.class), ToVersion::new), + entry(name(Trim.class), Trim::new) + ); + + static UnaryScalarFunction readESQLUnaryScalar(PlanStreamInput in, String name) throws IOException { + var ctr = ESQL_UNARY_SCALAR_CTRS.get(name); + if (ctr == null) { + throw new IOException("Constructor for ESQLUnaryScalar not found for name:" + name); + } + return ctr.apply(Source.EMPTY, in.readExpression()); + } + + static void writeESQLUnaryScalar(PlanStreamOutput out, UnaryScalarFunction function) throws IOException { + out.writeExpression(function.field()); + } + + static final Map> NO_ARG_SCALAR_CTRS = Map.ofEntries( + entry(name(E.class), E::new), + entry(name(Pi.class), Pi::new), + entry(name(Tau.class), Tau::new) + ); + + static ScalarFunction readNoArgScalar(PlanStreamInput in, String name) throws IOException { + var ctr = NO_ARG_SCALAR_CTRS.get(name); + if (ctr == null) { + throw new IOException("Constructor not found:" + name); + } + return ctr.apply(Source.EMPTY); + } + + static void writeNoArgScalar(PlanStreamOutput out, ScalarFunction function) {} + + static final Map< + String, + BiFunction> QL_UNARY_SCALAR_CTRS = + Map.ofEntries( + entry(name(IsNotNull.class), IsNotNull::new), + entry(name(IsNull.class), IsNull::new), + entry(name(Not.class), Not::new), + entry(name(Neg.class), Neg::new) + ); + + static org.elasticsearch.xpack.ql.expression.function.scalar.UnaryScalarFunction readQLUnaryScalar(PlanStreamInput in, String name) + throws IOException { + var ctr = QL_UNARY_SCALAR_CTRS.get(name); + if (ctr == null) { + throw new IOException("Constructor for QLUnaryScalar not found for name:" + name); + } + return ctr.apply(Source.EMPTY, in.readExpression()); + } + + static void writeQLUnaryScalar(PlanStreamOutput out, org.elasticsearch.xpack.ql.expression.function.scalar.UnaryScalarFunction function) + throws IOException { + out.writeExpression(function.field()); + } + + // -- ScalarFunction + + static Atan2 readAtan2(PlanStreamInput in) throws IOException { + return new Atan2(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writeAtan2(PlanStreamOutput out, Atan2 atan2) throws IOException { + out.writeExpression(atan2.y()); + out.writeExpression(atan2.x()); + } + + static AutoBucket readAutoBucket(PlanStreamInput in) throws IOException { + return new AutoBucket(Source.EMPTY, in.readExpression(), in.readExpression(), in.readExpression(), in.readExpression()); + } + + static void writeAutoBucket(PlanStreamOutput out, AutoBucket bucket) throws IOException { + out.writeExpression(bucket.field()); + out.writeExpression(bucket.buckets()); + out.writeExpression(bucket.from()); + out.writeExpression(bucket.to()); + } + + static Case readCase(PlanStreamInput in) throws IOException { + return new Case(Source.EMPTY, in.readList(readerFromPlanReader(PlanStreamInput::readExpression))); + } + + static void writeCase(PlanStreamOutput out, Case caseValue) throws IOException { + out.writeCollection(caseValue.children(), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + } + + static Concat readConcat(PlanStreamInput in) throws IOException { + return new Concat(Source.EMPTY, in.readExpression(), in.readList(readerFromPlanReader(PlanStreamInput::readExpression))); + } + + static void writeConcat(PlanStreamOutput out, Concat concat) throws IOException { + List fields = concat.children(); + out.writeExpression(fields.get(0)); + out.writeCollection(fields.subList(1, fields.size()), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + } + + static CountDistinct readCountDistinct(PlanStreamInput in) throws IOException { + return new CountDistinct(Source.EMPTY, in.readExpression(), in.readOptionalNamed(Expression.class)); + } + + static void writeCountDistinct(PlanStreamOutput out, CountDistinct countDistinct) throws IOException { + List fields = countDistinct.children(); + assert fields.size() == 1 || fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeOptionalWriteable(fields.size() == 2 ? o -> out.writeExpression(fields.get(1)) : null); + } + + static DateExtract readDateExtract(PlanStreamInput in) throws IOException { + return new DateExtract(Source.EMPTY, in.readExpression(), in.readExpression(), in.configuration()); + } + + static void writeDateExtract(PlanStreamOutput out, DateExtract function) throws IOException { + List fields = function.children(); + assert fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeExpression(fields.get(1)); + } + + static DateFormat readDateFormat(PlanStreamInput in) throws IOException { + return new DateFormat(Source.EMPTY, in.readExpression(), in.readOptionalNamed(Expression.class), in.configuration()); + } + + static void writeDateFormat(PlanStreamOutput out, DateFormat dateFormat) throws IOException { + List fields = dateFormat.children(); + assert fields.size() == 1 || fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeOptionalWriteable(fields.size() == 2 ? o -> out.writeExpression(fields.get(1)) : null); + } + + static DateParse readDateTimeParse(PlanStreamInput in) throws IOException { + return new DateParse(Source.EMPTY, in.readExpression(), in.readOptionalNamed(Expression.class)); + } + + static void writeDateTimeParse(PlanStreamOutput out, DateParse function) throws IOException { + List fields = function.children(); + assert fields.size() == 1 || fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeOptionalWriteable(fields.size() == 2 ? o -> out.writeExpression(fields.get(1)) : null); + } + + static DateTrunc readDateTrunc(PlanStreamInput in) throws IOException { + return new DateTrunc(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writeDateTrunc(PlanStreamOutput out, DateTrunc dateTrunc) throws IOException { + List fields = dateTrunc.children(); + assert fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeExpression(fields.get(1)); + } + + static Now readNow(PlanStreamInput in) throws IOException { + return new Now(Source.EMPTY, in.configuration()); + } + + static void writeNow(PlanStreamOutput out, Now function) {} + + static Round readRound(PlanStreamInput in) throws IOException { + return new Round(Source.EMPTY, in.readExpression(), in.readOptionalNamed(Expression.class)); + } + + static void writeRound(PlanStreamOutput out, Round round) throws IOException { + out.writeExpression(round.field()); + out.writeOptionalExpression(round.decimals()); + } + + static Pow readPow(PlanStreamInput in) throws IOException { + return new Pow(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writePow(PlanStreamOutput out, Pow pow) throws IOException { + out.writeExpression(pow.base()); + out.writeExpression(pow.exponent()); + } + + static Percentile readPercentile(PlanStreamInput in) throws IOException { + return new Percentile(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writePercentile(PlanStreamOutput out, Percentile percentile) throws IOException { + List fields = percentile.children(); + assert fields.size() == 2 : "percentile() aggregation must have two arguments"; + out.writeExpression(fields.get(0)); + out.writeExpression(fields.get(1)); + } + + static StartsWith readStartsWith(PlanStreamInput in) throws IOException { + return new StartsWith(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writeStartsWith(PlanStreamOutput out, StartsWith startsWith) throws IOException { + List fields = startsWith.children(); + assert fields.size() == 2; + out.writeExpression(fields.get(0)); + out.writeExpression(fields.get(1)); + } + + static Substring readSubstring(PlanStreamInput in) throws IOException { + return new Substring(Source.EMPTY, in.readExpression(), in.readExpression(), in.readOptionalNamed(Expression.class)); + } + + static void writeSubstring(PlanStreamOutput out, Substring substring) throws IOException { + List fields = substring.children(); + assert fields.size() == 2 || fields.size() == 3; + out.writeExpression(fields.get(0)); + out.writeExpression(fields.get(1)); + out.writeOptionalWriteable(fields.size() == 3 ? o -> out.writeExpression(fields.get(2)) : null); + } + + static Split readSplit(PlanStreamInput in) throws IOException { + return new Split(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writeSplit(PlanStreamOutput out, Split split) throws IOException { + out.writeExpression(split.left()); + out.writeExpression(split.right()); + } + + static CIDRMatch readCIDRMatch(PlanStreamInput in) throws IOException { + return new CIDRMatch(Source.EMPTY, in.readExpression(), in.readList(readerFromPlanReader(PlanStreamInput::readExpression))); + } + + static void writeCIDRMatch(PlanStreamOutput out, CIDRMatch cidrMatch) throws IOException { + List children = cidrMatch.children(); + assert children.size() > 1; + out.writeExpression(children.get(0)); + out.writeCollection(children.subList(1, children.size()), writerFromPlanWriter(PlanStreamOutput::writeExpression)); + } + + // -- ArithmeticOperations + + static final Map> ARITHMETIC_CTRS = + Map.ofEntries( + entry(DefaultBinaryArithmeticOperation.ADD, Add::new), + entry(DefaultBinaryArithmeticOperation.SUB, Sub::new), + entry(DefaultBinaryArithmeticOperation.MUL, Mul::new), + entry(DefaultBinaryArithmeticOperation.DIV, Div::new), + entry(DefaultBinaryArithmeticOperation.MOD, Mod::new) + ); + + static ArithmeticOperation readArithmeticOperation(PlanStreamInput in, String name) throws IOException { + var left = in.readExpression(); + var right = in.readExpression(); + var operation = DefaultBinaryArithmeticOperation.valueOf(name.toUpperCase(Locale.ROOT)); + return ARITHMETIC_CTRS.get(operation).apply(Source.EMPTY, left, right); + } + + static void writeArithmeticOperation(PlanStreamOutput out, ArithmeticOperation arithmeticOperation) throws IOException { + out.writeExpression(arithmeticOperation.left()); + out.writeExpression(arithmeticOperation.right()); + } + + // -- Aggregations + static final Map> AGG_CTRS = Map.ofEntries( + entry(name(Avg.class), Avg::new), + entry(name(Count.class), Count::new), + entry(name(Sum.class), Sum::new), + entry(name(Min.class), Min::new), + entry(name(Max.class), Max::new), + entry(name(Median.class), Median::new), + entry(name(MedianAbsoluteDeviation.class), MedianAbsoluteDeviation::new) + ); + + static AggregateFunction readAggFunction(PlanStreamInput in, String name) throws IOException { + return AGG_CTRS.get(name).apply(Source.EMPTY, in.readExpression()); + } + + static void writeAggFunction(PlanStreamOutput out, AggregateFunction aggregateFunction) throws IOException { + out.writeExpression(aggregateFunction.field()); + } + + // -- Multivalue functions + static final Map> MV_CTRS = Map.ofEntries( + entry(name(MvAvg.class), MvAvg::new), + entry(name(MvCount.class), MvCount::new), + entry(name(MvDedupe.class), MvDedupe::new), + entry(name(MvMax.class), MvMax::new), + entry(name(MvMedian.class), MvMedian::new), + entry(name(MvMin.class), MvMin::new), + entry(name(MvSum.class), MvSum::new) + ); + + static AbstractMultivalueFunction readMvFunction(PlanStreamInput in, String name) throws IOException { + return MV_CTRS.get(name).apply(Source.EMPTY, in.readExpression()); + } + + static void writeMvFunction(PlanStreamOutput out, AbstractMultivalueFunction fn) throws IOException { + out.writeExpression(fn.field()); + } + + static MvConcat readMvConcat(PlanStreamInput in) throws IOException { + return new MvConcat(Source.EMPTY, in.readExpression(), in.readExpression()); + } + + static void writeMvConcat(PlanStreamOutput out, MvConcat fn) throws IOException { + out.writeExpression(fn.left()); + out.writeExpression(fn.right()); + } + + // -- NamedExpressions + + static Alias readAlias(PlanStreamInput in) throws IOException { + return new Alias( + Source.EMPTY, + in.readString(), + in.readOptionalString(), + in.readNamed(Expression.class), + in.nameIdFromLongValue(in.readLong()), + in.readBoolean() + ); + } + + static void writeAlias(PlanStreamOutput out, Alias alias) throws IOException { + out.writeString(alias.name()); + out.writeOptionalString(alias.qualifier()); + out.writeExpression(alias.child()); + out.writeLong(Long.parseLong(alias.id().toString())); + out.writeBoolean(alias.synthetic()); + } + + // -- Expressions (other) + + static Literal readLiteral(PlanStreamInput in) throws IOException { + return new Literal(Source.EMPTY, in.readGenericValue(), in.dataTypeFromTypeName(in.readString())); + } + + static void writeLiteral(PlanStreamOutput out, Literal literal) throws IOException { + out.writeGenericValue(literal.value()); + out.writeString(literal.dataType().typeName()); + } + + static Order readOrder(PlanStreamInput in) throws IOException { + return new Order( + Source.EMPTY, + in.readNamed(Expression.class), + in.readEnum(Order.OrderDirection.class), + in.readEnum(Order.NullsPosition.class) + ); + } + + static void writeOrder(PlanStreamOutput out, Order order) throws IOException { + out.writeExpression(order.child()); + out.writeEnum(order.direction()); + out.writeEnum(order.nullsPosition()); + } + + // -- ancillary supporting classes of plan nodes, etc + + static EsQueryExec.FieldSort readFieldSort(PlanStreamInput in) throws IOException { + return new EsQueryExec.FieldSort( + readFieldAttribute(in), + in.readEnum(Order.OrderDirection.class), + in.readEnum(Order.NullsPosition.class) + ); + } + + static void writeFieldSort(PlanStreamOutput out, EsQueryExec.FieldSort fieldSort) throws IOException { + writeFieldAttribute(out, fieldSort.field()); + out.writeEnum(fieldSort.direction()); + out.writeEnum(fieldSort.nulls()); + } + + @SuppressWarnings("unchecked") + static EsIndex readEsIndex(PlanStreamInput in) throws IOException { + return new EsIndex( + in.readString(), + in.readImmutableMap(StreamInput::readString, readerFromPlanReader(PlanStreamInput::readEsFieldNamed)), + (Set) in.readGenericValue() + ); + } + + static void writeEsIndex(PlanStreamOutput out, EsIndex esIndex) throws IOException { + out.writeString(esIndex.name()); + out.writeMap(esIndex.mapping(), StreamOutput::writeString, (o, v) -> out.writeNamed(EsField.class, v)); + out.writeGenericValue(esIndex.concreteIndices()); + } + + static Parser readDissectParser(PlanStreamInput in) throws IOException { + String pattern = in.readString(); + String appendSeparator = in.readString(); + return new Parser(pattern, appendSeparator, new DissectParser(pattern, appendSeparator)); + } + + static void writeDissectParser(PlanStreamOutput out, Parser dissectParser) throws IOException { + out.writeString(dissectParser.pattern()); + out.writeString(dissectParser.appendSeparator()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamInput.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamInput.java new file mode 100644 index 0000000000000..49173779406cc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamInput.java @@ -0,0 +1,174 @@ +/* + * 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.io.stream; + +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanNamedReader; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanReader; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.AttributeSet; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.function.LongFunction; + +/** + * A customized stream input used to deserialize ESQL physical plan fragments. Complements stream + * input with methods that read plan nodes, Attributes, Expressions, etc. + */ +public final class PlanStreamInput extends NamedWriteableAwareStreamInput { + + private static final LongFunction DEFAULT_NAME_ID_FUNC = NameId::new; + + private final PlanNameRegistry registry; + + // hook for nameId, where can cache and map, for now just return a NameId of the same long value. + private final LongFunction nameIdFunction; + + private EsqlConfiguration configuration; + + public PlanStreamInput( + StreamInput streamInput, + PlanNameRegistry registry, + NamedWriteableRegistry namedWriteableRegistry, + EsqlConfiguration configuration + ) { + this(streamInput, registry, namedWriteableRegistry, configuration, DEFAULT_NAME_ID_FUNC); + } + + public PlanStreamInput( + StreamInput streamInput, + PlanNameRegistry registry, + NamedWriteableRegistry namedWriteableRegistry, + EsqlConfiguration configuration, + LongFunction nameIdFunction + ) { + super(streamInput, namedWriteableRegistry); + this.registry = registry; + this.nameIdFunction = nameIdFunction; + this.configuration = configuration; + } + + NameId nameIdFromLongValue(long value) { + return nameIdFunction.apply(value); + } + + DataType dataTypeFromTypeName(String typeName) throws IOException { + DataType dataType; + if (typeName.equalsIgnoreCase(EsQueryExec.DOC_DATA_TYPE.name())) { + dataType = EsQueryExec.DOC_DATA_TYPE; + } else { + dataType = EsqlDataTypes.fromTypeName(typeName); + } + if (dataType == null) { + throw new IOException("Unknown DataType for type name: " + typeName); + } + return dataType; + } + + public LogicalPlan readLogicalPlanNode() throws IOException { + return readNamed(LogicalPlan.class); + } + + public PhysicalPlan readPhysicalPlanNode() throws IOException { + return readNamed(PhysicalPlan.class); + } + + public Expression readExpression() throws IOException { + return readNamed(Expression.class); + } + + public NamedExpression readNamedExpression() throws IOException { + return readNamed(NamedExpression.class); + } + + public Attribute readAttribute() throws IOException { + return readNamed(Attribute.class); + } + + public EsField readEsFieldNamed() throws IOException { + return readNamed(EsField.class); + } + + public T readNamed(Class type) throws IOException { + String name = readString(); + @SuppressWarnings("unchecked") + PlanReader reader = (PlanReader) registry.getReader(type, name); + if (reader instanceof PlanNamedReader namedReader) { + return namedReader.read(this, name); + } else { + return reader.read(this); + } + } + + public T readOptionalNamed(Class type) throws IOException { + if (readBoolean()) { + T t = readNamed(type); + if (t == null) { + throwOnNullOptionalRead(type); + } + return t; + } else { + return null; + } + } + + public T readOptionalWithReader(PlanReader reader) throws IOException { + if (readBoolean()) { + T t = reader.read(this); + if (t == null) { + throwOnNullOptionalRead(reader); + } + return t; + } else { + return null; + } + } + + public AttributeSet readAttributeSet(Writeable.Reader reader) throws IOException { + int count = readArraySize(); + if (count == 0) { + return new AttributeSet(); + } + Collection builder = new HashSet<>(); + for (int i = 0; i < count; i++) { + builder.add(reader.read(this)); + } + return new AttributeSet(builder); + } + + public EsqlConfiguration configuration() throws IOException { + return configuration; + } + + static void throwOnNullOptionalRead(Class type) throws IOException { + final IOException e = new IOException("read optional named returned null which is not allowed, type:" + type); + assert false : e; + throw e; + } + + static void throwOnNullOptionalRead(PlanReader reader) throws IOException { + final IOException e = new IOException("read optional named returned null which is not allowed, reader:" + reader); + assert false : e; + throw e; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamOutput.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamOutput.java new file mode 100644 index 0000000000000..41219f5481034 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanStreamOutput.java @@ -0,0 +1,80 @@ +/* + * 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.io.stream; + +import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanWriter; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; + +import java.io.IOException; +import java.util.function.Function; + +/** + * A customized stream output used to serialize ESQL physical plan fragments. Complements stream + * output with methods that write plan nodes, Attributes, Expressions, etc. + */ +public final class PlanStreamOutput extends OutputStreamStreamOutput { + + private final PlanNameRegistry registry; + + private final Function, String> nameSupplier; + + public PlanStreamOutput(StreamOutput streamOutput, PlanNameRegistry registry) { + this(streamOutput, registry, PlanNamedTypes::name); + } + + public PlanStreamOutput(StreamOutput streamOutput, PlanNameRegistry registry, Function, String> nameSupplier) { + super(streamOutput); + this.registry = registry; + this.nameSupplier = nameSupplier; + } + + public void writeLogicalPlanNode(LogicalPlan logicalPlan) throws IOException { + assert logicalPlan.children().size() <= 1; + writeNamed(LogicalPlan.class, logicalPlan); + } + + public void writePhysicalPlanNode(PhysicalPlan physicalPlan) throws IOException { + assert physicalPlan.children().size() <= 1; + writeNamed(PhysicalPlan.class, physicalPlan); + } + + public void writeExpression(Expression expression) throws IOException { + writeNamed(Expression.class, expression); + } + + public void writeNamedExpression(NamedExpression namedExpression) throws IOException { + writeNamed(NamedExpression.class, namedExpression); + } + + public void writeAttribute(Attribute attribute) throws IOException { + writeNamed(Attribute.class, attribute); + } + + public void writeOptionalExpression(Expression expression) throws IOException { + if (expression == null) { + writeBoolean(false); + } else { + writeBoolean(true); + writeExpression(expression); + } + } + + public void writeNamed(Class type, T value) throws IOException { + String name = nameSupplier.apply(value.getClass()); + @SuppressWarnings("unchecked") + PlanWriter writer = (PlanWriter) registry.getWriter(type, name); + writeString(name); + writer.write(this, value); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalOptimizerContext.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalOptimizerContext.java new file mode 100644 index 0000000000000..36d275909b47a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalOptimizerContext.java @@ -0,0 +1,13 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.SearchStats; + +public record LocalLogicalOptimizerContext(EsqlConfiguration configuration, SearchStats searchStats) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java new file mode 100644 index 0000000000000..851499567084f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java @@ -0,0 +1,130 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.TopN; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.rule.ParameterizedRule; +import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor; + +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.TransformDirection.UP; + +public class LocalLogicalPlanOptimizer extends ParameterizedRuleExecutor { + + public LocalLogicalPlanOptimizer(LocalLogicalOptimizerContext localLogicalOptimizerContext) { + super(localLogicalOptimizerContext); + } + + @Override + protected List> batches() { + var local = new Batch<>("Local rewrite", new ReplaceTopNWithLimitAndSort(), new ReplaceMissingFieldWithNull()); + + var rules = new ArrayList>(); + rules.add(local); + // TODO: if the local rules haven't touched the tree, the rest of the rules can be skipped + rules.addAll(LogicalPlanOptimizer.rules()); + return rules; + } + + public LogicalPlan localOptimize(LogicalPlan plan) { + return execute(plan); + } + + /** + * Break TopN back into Limit + OrderBy to allow the order rules to kick in. + */ + public static class ReplaceTopNWithLimitAndSort extends OptimizerRules.OptimizerRule { + public ReplaceTopNWithLimitAndSort() { + super(UP); + } + + @Override + protected LogicalPlan rule(TopN plan) { + return new Limit(plan.source(), plan.limit(), new OrderBy(plan.source(), plan.child(), plan.order())); + } + } + + /** + * Look for any fields used in the plan that are missing locally and replace them with null. + * This should minimize the plan execution, in the best scenario skipping its execution all together. + */ + private static class ReplaceMissingFieldWithNull extends ParameterizedRule { + + @Override + public LogicalPlan apply(LogicalPlan plan, LocalLogicalOptimizerContext localLogicalOptimizerContext) { + return plan.transformUp(p -> missingToNull(p, localLogicalOptimizerContext.searchStats())); + } + + private LogicalPlan missingToNull(LogicalPlan plan, SearchStats stats) { + if (plan instanceof EsRelation) { + return plan; + } + + if (plan instanceof Aggregate a) { + // don't do anything (for now) + return a; + } + // keep the aliased name + else if (plan instanceof Project project) { + var projections = project.projections(); + List newProjections = new ArrayList<>(projections.size()); + List literals = new ArrayList<>(); + + for (NamedExpression projection : projections) { + if (projection instanceof FieldAttribute f && stats.exists(f.qualifiedName()) == false) { + var alias = new Alias(f.source(), f.name(), null, Literal.of(f, null), f.id()); + literals.add(alias); + newProjections.add(alias.toAttribute()); + } else { + newProjections.add(projection); + } + } + if (literals.size() > 0) { + plan = new Eval(project.source(), project.child(), literals); + plan = new Project(project.source(), plan, newProjections); + } else { + plan = project; + } + } else { + plan = plan.transformExpressionsOnlyUp( + FieldAttribute.class, + f -> stats.exists(f.qualifiedName()) ? f : Literal.of(f, null) + ); + } + + return plan; + } + } + + public abstract static class ParameterizedOptimizerRule extends ParameterizedRule< + SubPlan, + LogicalPlan, + P> { + + public final LogicalPlan apply(LogicalPlan plan, P context) { + return plan.transformUp(typeToken(), t -> rule(t, context)); + } + + protected abstract LogicalPlan rule(SubPlan plan, P context); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalOptimizerContext.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalOptimizerContext.java new file mode 100644 index 0000000000000..cb7f1c96c7d3f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalOptimizerContext.java @@ -0,0 +1,12 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; + +public record LocalPhysicalOptimizerContext(EsqlConfiguration configuration) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java new file mode 100644 index 0000000000000..74e1a9e0cc180 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java @@ -0,0 +1,310 @@ +/* + * 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; + +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules.OptimizerRule; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.esql.plan.physical.UnaryExec; +import org.elasticsearch.xpack.esql.planner.PhysicalVerificationException; +import org.elasticsearch.xpack.esql.planner.PhysicalVerifier; +import org.elasticsearch.xpack.esql.querydsl.query.SingleValueQuery; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.TypedAttribute; +import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.ql.expression.predicate.Predicates; +import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike; +import org.elasticsearch.xpack.ql.planner.ExpressionTranslator; +import org.elasticsearch.xpack.ql.planner.QlTranslatorHandler; +import org.elasticsearch.xpack.ql.querydsl.query.Query; +import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor; +import org.elasticsearch.xpack.ql.rule.Rule; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.function.Supplier; + +import static java.util.Arrays.asList; +import static org.elasticsearch.index.query.QueryBuilders.boolQuery; +import static org.elasticsearch.xpack.ql.expression.predicate.Predicates.splitAnd; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.TransformDirection.UP; + +public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor { + private static final QlTranslatorHandler TRANSLATOR_HANDLER = new EsqlTranslatorHandler(); + + private final PhysicalVerifier verifier = new PhysicalVerifier(); + + public LocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context) { + super(context); + } + + public PhysicalPlan localOptimize(PhysicalPlan plan) { + return verify(execute(plan)); + } + + PhysicalPlan verify(PhysicalPlan plan) { + Collection failures = verifier.verify(plan); + if (failures.isEmpty() == false) { + throw new PhysicalVerificationException(failures); + } + return plan; + } + + protected List> rules(boolean optimizeForEsSource) { + List> esSourceRules = new ArrayList<>(4); + esSourceRules.add(new ReplaceAttributeSourceWithDocId()); + + if (optimizeForEsSource) { + esSourceRules.add(new PushTopNToSource()); + esSourceRules.add(new PushLimitToSource()); + esSourceRules.add(new PushFiltersToSource()); + } + + // execute the rules multiple times to improve the chances of things being pushed down + @SuppressWarnings("unchecked") + var pushdown = new Batch("Push to ES", esSourceRules.toArray(Rule[]::new)); + // add the field extraction in just one pass + // add it at the end after all the other rules have ran + var fieldExtraction = new Batch<>("Field extraction", Limiter.ONCE, new InsertFieldExtraction()); + return asList(pushdown, fieldExtraction); + } + + @Override + protected List> batches() { + return rules(true); + } + + private static class ReplaceAttributeSourceWithDocId extends OptimizerRule { + + ReplaceAttributeSourceWithDocId() { + super(UP); + } + + @Override + protected PhysicalPlan rule(EsSourceExec plan) { + return new EsQueryExec(plan.source(), plan.index(), plan.query()); + } + } + + // Materialize the concrete fields that need to be extracted from the storage until the last possible moment. + // Expects the local plan to already have a projection containing the fields needed upstream. + // + // 1. add the materialization right before usage inside the local plan + // 2. materialize any missing fields needed further up the chain + /** + * @see org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer.ProjectAwayColumns + */ + static class InsertFieldExtraction extends Rule { + + @Override + public PhysicalPlan apply(PhysicalPlan plan) { + // apply the plan locally, adding a field extractor right before data is loaded + // by going bottom-up + plan = plan.transformUp(UnaryExec.class, p -> { + var missing = missingAttributes(p); + + /* + * If there is a single grouping then we'll try to use ords. Either way + * it loads the field lazily. If we have more than one field we need to + * make sure the fields are loaded for the standard hash aggregator. + */ + if (p instanceof AggregateExec agg && agg.groupings().size() == 1) { + var leaves = new LinkedList<>(); + // TODO: this seems out of place + agg.aggregates() + .stream() + .filter(a -> agg.groupings().contains(a) == false) + .forEach(a -> leaves.addAll(a.collectLeaves())); + var remove = agg.groupings().stream().filter(g -> leaves.contains(g) == false).toList(); + missing.removeAll(Expressions.references(remove)); + } + + // add extractor + if (missing.isEmpty() == false) { + // collect source attributes and add the extractor + var extractor = new FieldExtractExec(p.source(), p.child(), List.copyOf(missing)); + p = p.replaceChild(extractor); + } + + return p; + }); + + return plan; + } + + private static Set missingAttributes(PhysicalPlan p) { + var missing = new LinkedHashSet(); + var input = p.inputSet(); + + // collect field attributes used inside expressions + p.forEachExpression(TypedAttribute.class, f -> { + if (f instanceof FieldAttribute || f instanceof MetadataAttribute) { + if (input.contains(f) == false) { + missing.add(f); + } + } + }); + return missing; + } + } + + private static class PushFiltersToSource extends OptimizerRule { + @Override + protected PhysicalPlan rule(FilterExec filterExec) { + PhysicalPlan plan = filterExec; + if (filterExec.child() instanceof EsQueryExec queryExec) { + List pushable = new ArrayList<>(); + List nonPushable = new ArrayList<>(); + for (Expression exp : splitAnd(filterExec.condition())) { + (canPushToSource(exp) ? pushable : nonPushable).add(exp); + } + if (pushable.size() > 0) { // update the executable with pushable conditions + QueryBuilder planQuery = TRANSLATOR_HANDLER.asQuery(Predicates.combineAnd(pushable)).asBuilder(); + QueryBuilder query = planQuery; + QueryBuilder filterQuery = queryExec.query(); + if (filterQuery != null) { + query = boolQuery().filter(filterQuery).filter(planQuery); + } + queryExec = new EsQueryExec( + queryExec.source(), + queryExec.index(), + queryExec.output(), + query, + queryExec.limit(), + queryExec.sorts(), + queryExec.estimatedRowSize() + ); + if (nonPushable.size() > 0) { // update filter with remaining non-pushable conditions + plan = new FilterExec(filterExec.source(), queryExec, Predicates.combineAnd(nonPushable)); + } else { // prune Filter entirely + plan = queryExec; + } + } // else: nothing changes + } + + return plan; + } + + private static boolean canPushToSource(Expression exp) { + if (exp instanceof BinaryComparison bc) { + return isAttributePushable(bc.left(), bc) && bc.right().foldable(); + } else if (exp instanceof BinaryLogic bl) { + return canPushToSource(bl.left()) && canPushToSource(bl.right()); + } else if (exp instanceof RegexMatch rm) { + return isAttributePushable(rm.field(), rm); + } else if (exp instanceof In in) { + return isAttributePushable(in.value(), null) && Expressions.foldable(in.list()); + } else if (exp instanceof Not not) { + return canPushToSource(not.field()); + } + return false; + } + + private static boolean isAttributePushable(Expression expression, ScalarFunction operation) { + if (expression instanceof FieldAttribute) { + return true; + } + if (expression instanceof MetadataAttribute ma && ma.searchable()) { + return operation == null + // no range or regex queries supported with metadata fields + || operation instanceof Equals + || operation instanceof NotEquals + || operation instanceof WildcardLike; + } + return false; + } + } + + private static class PushLimitToSource extends OptimizerRule { + @Override + protected PhysicalPlan rule(LimitExec limitExec) { + PhysicalPlan plan = limitExec; + PhysicalPlan child = limitExec.child(); + if (child instanceof EsQueryExec queryExec) { // add_task_parallelism_above_query: false + plan = queryExec.withLimit(limitExec.limit()); + } else if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) { + plan = exchangeExec.replaceChild(queryExec.withLimit(limitExec.limit())); + } + return plan; + } + } + + private static class PushTopNToSource extends OptimizerRule { + @Override + protected PhysicalPlan rule(TopNExec topNExec) { + PhysicalPlan plan = topNExec; + PhysicalPlan child = topNExec.child(); + + boolean canPushDownTopN = child instanceof EsQueryExec + || (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec); + if (canPushDownTopN && canPushDownOrders(topNExec.order())) { + var sorts = buildFieldSorts(topNExec.order()); + var limit = topNExec.limit(); + + if (child instanceof ExchangeExec exchangeExec && exchangeExec.child() instanceof EsQueryExec queryExec) { + plan = exchangeExec.replaceChild(queryExec.withSorts(sorts).withLimit(limit)); + } else { + plan = ((EsQueryExec) child).withSorts(sorts).withLimit(limit); + } + } + return plan; + } + + private boolean canPushDownOrders(List orders) { + // allow only FieldAttributes (no expressions) for sorting + return false == Expressions.match(orders, s -> ((Order) s).child() instanceof FieldAttribute == false); + } + + private List buildFieldSorts(List orders) { + List sorts = new ArrayList<>(orders.size()); + for (Order o : orders) { + sorts.add(new EsQueryExec.FieldSort(((FieldAttribute) o.child()), o.direction(), o.nullsPosition())); + } + return sorts; + } + } + + private static final class EsqlTranslatorHandler extends QlTranslatorHandler { + @Override + public Query wrapFunctionQuery(ScalarFunction sf, Expression field, Supplier querySupplier) { + if (field instanceof FieldAttribute fa) { + return ExpressionTranslator.wrapIfNested(new SingleValueQuery(querySupplier.get(), fa.name()), field); + } + if (field instanceof MetadataAttribute) { + return querySupplier.get(); // MetadataAttributes are always single valued + } + throw new EsqlIllegalArgumentException("Expected a FieldAttribute or MetadataAttribute but received [" + field + "]"); + } + } +} 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 new file mode 100644 index 0000000000000..eb20b167dbc7f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java @@ -0,0 +1,786 @@ +/* + * 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; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.expression.SurrogateExpression; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.RegexExtract; +import org.elasticsearch.xpack.esql.plan.logical.TopN; +import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.AttributeMap; +import org.elasticsearch.xpack.ql.expression.AttributeSet; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.ExpressionSet; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.predicate.Predicates; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.BinaryComparisonSimplification; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.BooleanFunctionEqualsElimination; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ConstantFolding; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.LiteralsOnTheRight; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PruneLiteralsInOrderBy; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.SetAsOptimized; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.SimplifyComparisonsArithmetics; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.rule.Rule; +import org.elasticsearch.xpack.ql.rule.RuleExecutor; +import org.elasticsearch.xpack.ql.util.CollectionUtils; +import org.elasticsearch.xpack.ql.util.Holder; + +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; + +import static java.util.Arrays.asList; +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputExpressions; +import static org.elasticsearch.xpack.ql.expression.Expressions.asAttributes; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.FoldNull; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PropagateEquals; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.PropagateNullable; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.ReplaceRegexMatch; +import static org.elasticsearch.xpack.ql.optimizer.OptimizerRules.TransformDirection; + +public class LogicalPlanOptimizer extends RuleExecutor { + + public LogicalPlan optimize(LogicalPlan verified) { + return verified.optimized() ? verified : execute(verified); + } + + @Override + protected List> batches() { + return rules(); + } + + protected static List> rules() { + var substitutions = new Batch<>("Substitutions", Limiter.ONCE, new SubstituteSurrogates(), new ReplaceRegexMatch()); + + var operators = new Batch<>( + "Operator Optimization", + new CombineProjections(), + new PruneEmptyPlans(), + new PropagateEmptyRelation(), + new ConvertStringToByteRef(), + new FoldNull(), + new SplitInWithFoldableValue(), + new ConstantFolding(), + // boolean + new BooleanSimplification(), + new LiteralsOnTheRight(), + new BinaryComparisonSimplification(), + // needs to occur before BinaryComparison combinations (see class) + new PropagateEquals(), + new PropagateNullable(), + new BooleanFunctionEqualsElimination(), + new CombineDisjunctionsToIn(), + new SimplifyComparisonsArithmetics(EsqlDataTypes::areCompatible), + // prune/elimination + new PruneFilters(), + new PruneColumns(), + new PruneLiteralsInOrderBy(), + new PushDownAndCombineLimits(), + new PushDownAndCombineFilters(), + new PushDownEval(), + new PushDownRegexExtract(), + new PushDownEnrich(), + new PushDownAndCombineOrderBy(), + new PruneOrderByBeforeStats(), + new PruneRedundantSortClauses() + ); + + var skip = new Batch<>("Skip Compute", new SkipQueryOnLimitZero()); + var cleanup = new Batch<>("Clean Up", new ReplaceLimitAndSortAsTopN()); + var label = new Batch<>("Set as Optimized", Limiter.ONCE, new SetAsOptimized()); + + return asList(substitutions, operators, skip, cleanup, label); + } + + // TODO: currently this rule only works for aggregate functions (AVG) + static class SubstituteSurrogates extends OptimizerRules.OptimizerRule { + + SubstituteSurrogates() { + super(TransformDirection.UP); + } + + @Override + protected LogicalPlan rule(Aggregate aggregate) { + var aggs = aggregate.aggregates(); + List newAggs = new ArrayList<>(aggs.size()); + // existing aggregate and their respective attributes + Map aggFuncToAttr = new HashMap<>(); + // surrogate functions eval + List transientEval = new ArrayList<>(); + boolean changed = false; + + // first pass to check existing aggregates (to avoid duplication and alias waste) + for (NamedExpression agg : aggs) { + if (agg instanceof Alias a && a.child() instanceof AggregateFunction af && af instanceof SurrogateExpression == false) { + aggFuncToAttr.put(af, a.toAttribute()); + } + } + + // 0. check list of surrogate expressions + for (NamedExpression agg : aggs) { + Expression e = agg instanceof Alias a ? a.child() : agg; + if (e instanceof SurrogateExpression sf) { + changed = true; + Expression s = sf.surrogate(); + + // if the expression is NOT a 1:1 replacement need to add an eval + if (s instanceof AggregateFunction == false) { + // 1. collect all aggregate functions from the expression + var surrogateWithRefs = s.transformUp(AggregateFunction.class, af -> { + // 2. check if they are already use otherwise add them to the Aggregate with some made-up aliases + // 3. replace them inside the expression using the given alias + var attr = aggFuncToAttr.get(af); + // the agg doesn't exist in the Aggregate, create an alias for it and save its attribute + if (attr == null) { + var temporaryName = temporaryName(agg, af); + // create a synthetic alias (so it doesn't clash with a user defined name) + var newAlias = new Alias(agg.source(), temporaryName, null, af, null, true); + attr = newAlias.toAttribute(); + aggFuncToAttr.put(af, attr); + newAggs.add(newAlias); + } + return attr; + }); + // 4. move the expression as an eval using the original alias + // copy the original alias id so that other nodes using it down stream (e.g. eval referring to the original agg) + // don't have to updated + var aliased = new Alias(agg.source(), agg.name(), null, surrogateWithRefs, agg.toAttribute().id()); + transientEval.add(aliased); + } + // the replacement is another aggregate function, so replace it in place + else { + newAggs.add((NamedExpression) agg.replaceChildren(Collections.singletonList(s))); + } + } else { + newAggs.add(agg); + } + } + + LogicalPlan plan = aggregate; + if (changed) { + var source = aggregate.source(); + plan = new Aggregate(aggregate.source(), aggregate.child(), aggregate.groupings(), newAggs); + // 5. force the initial projection in place + if (transientEval.size() > 0) { + plan = new Eval(source, plan, transientEval); + // project away transient fields and re-enforce the original order using references (not copies) to the original aggs + // this works since the replaced aliases have their nameId copied to avoid having to update all references (which has + // a cascading effect) + plan = new EsqlProject(source, plan, Expressions.asAttributes(aggs)); + } + } + + return plan; + } + + private static String temporaryName(NamedExpression agg, AggregateFunction af) { + return "__" + agg.name() + "_" + af.functionName() + "@" + Integer.toHexString(af.hashCode()); + } + } + + static class ConvertStringToByteRef extends OptimizerRules.OptimizerExpressionRule { + + ConvertStringToByteRef() { + super(TransformDirection.UP); + } + + @Override + protected Expression rule(Literal lit) { + if (lit.value() == null) { + return lit; + } + if (lit.value() instanceof String s) { + return Literal.of(lit, new BytesRef(s)); + } + if (lit.value() instanceof List l) { + if (l.isEmpty() || false == l.get(0) instanceof String) { + return lit; + } + return Literal.of(lit, l.stream().map(v -> new BytesRef((String) v)).toList()); + } + return lit; + } + } + + static class CombineProjections extends OptimizerRules.OptimizerRule { + + CombineProjections() { + super(TransformDirection.UP); + } + + @Override + protected LogicalPlan rule(UnaryPlan plan) { + LogicalPlan child = plan.child(); + + if (plan instanceof Project project) { + if (child instanceof Project p) { + // eliminate lower project but first replace the aliases in the upper one + return p.withProjections(combineProjections(project.projections(), p.projections())); + } else if (child instanceof Aggregate a) { + return new Aggregate(a.source(), a.child(), a.groupings(), combineProjections(project.projections(), a.aggregates())); + } + } + + // Agg with underlying Project (group by on sub-queries) + if (plan instanceof Aggregate a) { + if (child instanceof Project p) { + return new Aggregate(a.source(), p.child(), a.groupings(), combineProjections(a.aggregates(), p.projections())); + } + } + + return plan; + } + + // normally only the upper projections should survive but since the lower list might have aliases definitions + // that might be reused by the upper one, these need to be replaced. + // for example an alias defined in the lower list might be referred in the upper - without replacing it the alias becomes invalid + private List combineProjections(List upper, List lower) { + + // collect aliases in the lower list + AttributeMap.Builder aliasesBuilder = AttributeMap.builder(); + for (NamedExpression ne : lower) { + if ((ne instanceof Attribute) == false) { + aliasesBuilder.put(ne.toAttribute(), ne); + } + } + + AttributeMap aliases = aliasesBuilder.build(); + List replaced = new ArrayList<>(); + + // replace any matching attribute with a lower alias (if there's a match) + // but clean-up non-top aliases at the end + for (NamedExpression ne : upper) { + NamedExpression replacedExp = (NamedExpression) ne.transformUp(Attribute.class, a -> aliases.resolve(a, a)); + replaced.add((NamedExpression) trimNonTopLevelAliases(replacedExp)); + } + return replaced; + } + + public static Expression trimNonTopLevelAliases(Expression e) { + if (e instanceof Alias a) { + return new Alias(a.source(), a.name(), a.qualifier(), trimAliases(a.child()), a.id()); + } + return trimAliases(e); + } + + private static Expression trimAliases(Expression e) { + return e.transformDown(Alias.class, Alias::child); + } + } + + static class PushDownAndCombineLimits extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(Limit limit) { + if (limit.child() instanceof Limit childLimit) { + var limitSource = limit.limit(); + var l1 = (int) limitSource.fold(); + var l2 = (int) childLimit.limit().fold(); + return new Limit(limit.source(), Literal.of(limitSource, Math.min(l1, l2)), childLimit.child()); + } else if (limit.child() instanceof UnaryPlan unary) { + if (unary instanceof Eval || unary instanceof Project || unary instanceof RegexExtract || unary instanceof Enrich) { + return unary.replaceChild(limit.replaceChild(unary.child())); + } + // check if there's a 'visible' descendant limit lower than the current one + // and if so, align the current limit since it adds no value + // this applies for cases such as | limit 1 | sort field | limit 10 + else { + Limit descendantLimit = descendantLimit(unary); + if (descendantLimit != null) { + var l1 = (int) limit.limit().fold(); + var l2 = (int) descendantLimit.limit().fold(); + if (l2 <= l1) { + return new Limit(limit.source(), Literal.of(limit.limit(), l2), limit.child()); + } + } + } + } + return limit; + } + + /** + * Checks the existence of another 'visible' Limit, that exists behind an operation that doesn't produce output more data than + * its input (that is not a relation/source nor aggregation). + * P.S. Typically an aggregation produces less data than the input. + */ + private static Limit descendantLimit(UnaryPlan unary) { + UnaryPlan plan = unary; + while (plan instanceof Aggregate == false) { + if (plan instanceof Limit limit) { + return limit; + } + if (plan.child() instanceof UnaryPlan unaryPlan) { + plan = unaryPlan; + } else { + break; + } + } + return null; + } + } + + // 3 in (field, 4, 5) --> 3 in (field) or 3 in (4, 5) + public static class SplitInWithFoldableValue extends OptimizerRules.OptimizerExpressionRule { + + SplitInWithFoldableValue() { + super(TransformDirection.UP); + } + + @Override + protected Expression rule(In in) { + if (in.value().foldable()) { + List foldables = new ArrayList<>(in.list().size()); + List nonFoldables = new ArrayList<>(in.list().size()); + in.list().forEach(e -> { + if (e.foldable() && Expressions.isNull(e) == false) { // keep `null`s, needed for the 3VL + foldables.add(e); + } else { + nonFoldables.add(e); + } + }); + if (foldables.size() > 0 && nonFoldables.size() > 0) { + In withFoldables = new In(in.source(), in.value(), foldables); + In withoutFoldables = new In(in.source(), in.value(), nonFoldables); + return new Or(in.source(), withFoldables, withoutFoldables); + } + } + return in; + } + } + + private static class BooleanSimplification extends org.elasticsearch.xpack.ql.optimizer.OptimizerRules.BooleanSimplification { + + BooleanSimplification() { + super(); + } + + @Override + protected Expression maybeSimplifyNegatable(Expression e) { + return null; + } + + } + + static class PruneFilters extends OptimizerRules.PruneFilters { + + @Override + protected LogicalPlan skipPlan(Filter filter) { + return LogicalPlanOptimizer.skipPlan(filter); + } + } + + static class SkipQueryOnLimitZero extends OptimizerRules.SkipQueryOnLimitZero { + + @Override + protected LogicalPlan skipPlan(Limit limit) { + return LogicalPlanOptimizer.skipPlan(limit); + } + } + + static class PruneEmptyPlans extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(UnaryPlan plan) { + return plan.output().isEmpty() ? skipPlan(plan) : plan; + } + } + + static class PropagateEmptyRelation extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(UnaryPlan plan) { + LogicalPlan p = plan; + if (plan.child() instanceof LocalRelation local && local.supplier() == LocalSupplier.EMPTY) { + // only care about non-grouped aggs might return something (count) + if (plan instanceof Aggregate agg && agg.groupings().isEmpty()) { + p = skipPlan(plan, aggsFromEmpty(agg.aggregates())); + } else { + p = skipPlan(plan); + } + } + return p; + } + + private static LocalSupplier aggsFromEmpty(List aggs) { + var result = new ArrayList(aggs.size()); + for (var agg : aggs) { + // there needs to be an alias + if (agg instanceof Alias a && a.child() instanceof AggregateFunction aggFunc) { + result.add(aggFunc instanceof Count ? 0L : null); + } else { + throw new EsqlIllegalArgumentException("Did not expect a non-aliased aggregation {}", agg); + } + } + var blocks = BlockUtils.fromListRow(result); + return LocalSupplier.of(blocks); + } + } + + private static LogicalPlan skipPlan(UnaryPlan plan) { + return new LocalRelation(plan.source(), plan.output(), LocalSupplier.EMPTY); + } + + private static LogicalPlan skipPlan(UnaryPlan plan, LocalSupplier supplier) { + return new LocalRelation(plan.source(), plan.output(), supplier); + } + + protected static class PushDownAndCombineFilters extends OptimizerRules.OptimizerRule { + @Override + protected LogicalPlan rule(Filter filter) { + LogicalPlan plan = filter; + LogicalPlan child = filter.child(); + Expression condition = filter.condition(); + + if (child instanceof Filter f) { + // combine nodes into a single Filter with updated ANDed condition + plan = f.with(Predicates.combineAnd(List.of(f.condition(), condition))); + } else if (child instanceof Aggregate agg) { // TODO: re-evaluate along with multi-value support + // Only push [parts of] a filter past an agg if these/it operates on agg's grouping[s], not output. + plan = maybePushDownPastUnary( + filter, + agg, + e -> e instanceof Attribute && agg.output().contains(e) && agg.groupings().contains(e) == false + || e instanceof AggregateFunction + ); + } else if (child instanceof Eval eval) { + // Don't push if Filter (still) contains references of Eval's fields. + var attributes = new AttributeSet(Expressions.asAttributes(eval.fields())); + plan = maybePushDownPastUnary(filter, eval, attributes::contains); + } else if (child instanceof RegexExtract re) { + // Push down filters that do not rely on attributes created by RegexExtract + var attributes = new AttributeSet(Expressions.asAttributes(re.extractedFields())); + plan = maybePushDownPastUnary(filter, re, attributes::contains); + } else if (child instanceof Enrich enrich) { + // Push down filters that do not rely on attributes created by Enrich + var attributes = new AttributeSet(Expressions.asAttributes(enrich.enrichFields())); + plan = maybePushDownPastUnary(filter, enrich, attributes::contains); + } else if (child instanceof Project) { + return pushDownPastProject(filter); + } else if (child instanceof OrderBy orderBy) { + // swap the filter with its child + plan = orderBy.replaceChild(filter.with(orderBy.child(), condition)); + } + // cannot push past a Limit, this could change the tailing result set returned + return plan; + } + + private static LogicalPlan maybePushDownPastUnary(Filter filter, UnaryPlan unary, Predicate cannotPush) { + LogicalPlan plan; + List pushable = new ArrayList<>(); + List nonPushable = new ArrayList<>(); + for (Expression exp : Predicates.splitAnd(filter.condition())) { + (exp.anyMatch(cannotPush) ? nonPushable : pushable).add(exp); + } + // Push the filter down even if it might not be pushable all the way to ES eventually: eval'ing it closer to the source, + // potentially still in the Exec Engine, distributes the computation. + if (pushable.size() > 0) { + if (nonPushable.size() > 0) { + Filter pushed = new Filter(filter.source(), unary.child(), Predicates.combineAnd(pushable)); + plan = filter.with(unary.replaceChild(pushed), Predicates.combineAnd(nonPushable)); + } else { + plan = unary.replaceChild(filter.with(unary.child(), filter.condition())); + } + } else { + plan = filter; + } + return plan; + } + } + + /** + * Pushes Evals past OrderBys. Although it seems arbitrary whether the OrderBy or the Eval is executed first, + * this transformation ensures that OrderBys only separated by an eval can be combined by PushDownAndCombineOrderBy. + * + * E.g.: + * + * ... | sort a | eval x = b + 1 | sort x + * + * becomes + * + * ... | eval x = b + 1 | sort a | sort x + * + * Ordering the evals before the orderBys has the advantage that it's always possible to order the plans like this. + * E.g., in the example above it would not be possible to put the eval after the two orderBys. + */ + protected static class PushDownEval extends OptimizerRules.OptimizerRule { + @Override + protected LogicalPlan rule(Eval eval) { + LogicalPlan child = eval.child(); + + // TODO: combine with CombineEval from https://github.com/elastic/elasticsearch-internal/pull/511 when merged + if (child instanceof OrderBy orderBy) { + return orderBy.replaceChild(eval.replaceChild(orderBy.child())); + } else if (child instanceof Project) { + var projectWithEvalChild = pushDownPastProject(eval); + var fieldProjections = asAttributes(eval.fields()); + return projectWithEvalChild.withProjections(mergeOutputExpressions(fieldProjections, projectWithEvalChild.projections())); + } + + return eval; + } + } + + // same as for PushDownEval + protected static class PushDownRegexExtract extends OptimizerRules.OptimizerRule { + @Override + protected LogicalPlan rule(RegexExtract re) { + LogicalPlan child = re.child(); + + if (child instanceof OrderBy orderBy) { + return orderBy.replaceChild(re.replaceChild(orderBy.child())); + } else if (child instanceof Project) { + var projectWithChild = pushDownPastProject(re); + return projectWithChild.withProjections(mergeOutputExpressions(re.extractedFields(), projectWithChild.projections())); + } + + return re; + } + } + + // TODO double-check: this should be the same as EVAL and GROK/DISSECT, needed to avoid unbounded sort + protected static class PushDownEnrich extends OptimizerRules.OptimizerRule { + @Override + protected LogicalPlan rule(Enrich re) { + LogicalPlan child = re.child(); + + if (child instanceof OrderBy orderBy) { + return orderBy.replaceChild(re.replaceChild(orderBy.child())); + } else if (child instanceof Project) { + var projectWithChild = pushDownPastProject(re); + var attrs = asAttributes(re.enrichFields()); + return projectWithChild.withProjections(mergeOutputExpressions(attrs, projectWithChild.projections())); + } + + return re; + } + } + + protected static class PushDownAndCombineOrderBy extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(OrderBy orderBy) { + LogicalPlan child = orderBy.child(); + + if (child instanceof OrderBy childOrder) { + // combine orders + return new OrderBy(orderBy.source(), childOrder.child(), CollectionUtils.combine(orderBy.order(), childOrder.order())); + } else if (child instanceof Project) { + return pushDownPastProject(orderBy); + } + + return orderBy; + } + } + + /** + * Remove unused columns created in the plan, in fields inside eval or aggregations inside stats. + */ + static class PruneColumns extends Rule { + + @Override + public LogicalPlan apply(LogicalPlan plan) { + var used = new Holder<>(new AttributeSet()); + // don't remove Evals without any Project/Aggregate (which might not occur as the last node in the plan) + var seenProjection = new Holder<>(Boolean.FALSE); + + // start top-to-bottom + // and track used references + var pl = plan.transformDown(p -> { + // skip nodes that simply pass the input through + if (p instanceof Limit) { + return p; + } + + // remember used + var usedSet = used.get(); + boolean recheck; + // analyze the unused items against dedicated 'producer' nodes such as Eval and Aggregate + // perform a loop to retry checking if the current node is completely eliminated + do { + recheck = false; + if (p instanceof Aggregate aggregate) { + var remaining = seenProjection.get() ? removeUnused(aggregate.aggregates(), usedSet) : null; + // no aggregates, no need + if (remaining != null) { + if (remaining.isEmpty()) { + recheck = true; + p = aggregate.child(); + } else { + p = new Aggregate(aggregate.source(), aggregate.child(), aggregate.groupings(), remaining); + } + } + + seenProjection.set(Boolean.TRUE); + } else if (p instanceof Eval eval) { + var remaining = seenProjection.get() ? removeUnused(eval.fields(), usedSet) : null; + // no fields, no eval + if (remaining != null) { + if (remaining.isEmpty()) { + p = eval.child(); + recheck = true; + } else { + p = new Eval(eval.source(), eval.child(), remaining); + } + } + } else if (p instanceof Project) { + seenProjection.set(Boolean.TRUE); + } + } while (recheck); + + var inUse = usedSet.combine(references(p)); + used.set(inUse); + + // preserve the state before going to the next node + return p; + }); + + return pl; + } + + /** + * Prunes attributes from the list not found in the given set. + * Returns null if no changed occurred. + */ + private static List removeUnused(List named, AttributeSet used) { + var clone = new ArrayList<>(named); + var it = clone.listIterator(clone.size()); + + // due to Eval, go in reverse + while (it.hasPrevious()) { + N prev = it.previous(); + if (used.contains(prev.toAttribute()) == false) { + it.remove(); + } else { + used = used.combine(prev.references()); + } + } + return clone.size() != named.size() ? clone : null; + } + + private static List expressions(LogicalPlan plan) { + List exp = new ArrayList<>(); + plan.forEachExpression(exp::add); + return exp; + } + + private static AttributeSet references(LogicalPlan plan) { + return Expressions.references(expressions(plan)); + } + } + + static class PruneOrderByBeforeStats extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(Aggregate agg) { + OrderBy order = findPullableOrderBy(agg.child()); + + LogicalPlan p = agg; + if (order != null) { + p = agg.transformDown(OrderBy.class, o -> o == order ? order.child() : o); + } + return p; + } + + private static OrderBy findPullableOrderBy(LogicalPlan plan) { + OrderBy pullable = null; + if (plan instanceof OrderBy o) { + pullable = o; + } else if (plan instanceof Eval + || plan instanceof Filter + || plan instanceof Project + || plan instanceof RegexExtract + || plan instanceof Enrich) { + pullable = findPullableOrderBy(((UnaryPlan) plan).child()); + } + return pullable; + } + + } + + static class PruneRedundantSortClauses extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(OrderBy plan) { + var referencedAttributes = new ExpressionSet(); + var order = new ArrayList(); + for (Order o : plan.order()) { + Attribute a = (Attribute) o.child(); + if (referencedAttributes.add(a)) { + order.add(o); + } + } + + return plan.order().size() == order.size() ? plan : new OrderBy(plan.source(), plan.child(), order); + } + } + + private static Project pushDownPastProject(UnaryPlan parent) { + if (parent.child() instanceof Project project) { + AttributeMap.Builder aliasBuilder = AttributeMap.builder(); + project.forEachExpression(Alias.class, a -> aliasBuilder.put(a.toAttribute(), a.child())); + var aliases = aliasBuilder.build(); + + var expressionsWithResolvedAliases = (UnaryPlan) parent.transformExpressionsOnly( + ReferenceAttribute.class, + r -> aliases.resolve(r, r) + ); + + return project.replaceChild(expressionsWithResolvedAliases.replaceChild(project.child())); + } else { + throw new UnsupportedOperationException("Expected child to be instance of Project"); + } + } + + static class CombineDisjunctionsToIn extends org.elasticsearch.xpack.ql.optimizer.OptimizerRules.CombineDisjunctionsToIn { + @Override + protected In createIn(Expression key, List values, ZoneId zoneId) { + return new In(key.source(), key, values); + } + } + + static class ReplaceLimitAndSortAsTopN extends OptimizerRules.OptimizerRule { + + @Override + protected LogicalPlan rule(Limit plan) { + LogicalPlan p = plan; + if (plan.child() instanceof OrderBy o) { + p = new TopN(plan.source(), o.child(), o.order(), plan.limit()); + } + return p; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerContext.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerContext.java new file mode 100644 index 0000000000000..7e3ba256f4ffc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerContext.java @@ -0,0 +1,12 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; + +public record PhysicalOptimizerContext(EsqlConfiguration configuration) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java new file mode 100644 index 0000000000000..af72c8e8b1649 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java @@ -0,0 +1,95 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules.TransformDirection; +import org.elasticsearch.xpack.ql.rule.ParameterizedRule; +import org.elasticsearch.xpack.ql.rule.Rule; +import org.elasticsearch.xpack.ql.util.ReflectionUtils; + +public class PhysicalOptimizerRules { + + public abstract static class ParameterizedOptimizerRule extends ParameterizedRule< + SubPlan, + PhysicalPlan, + P> { + + private final TransformDirection direction; + + public ParameterizedOptimizerRule() { + this(TransformDirection.DOWN); + } + + protected ParameterizedOptimizerRule(TransformDirection direction) { + this.direction = direction; + } + + @Override + public final PhysicalPlan apply(PhysicalPlan plan, P context) { + return direction == TransformDirection.DOWN + ? plan.transformDown(typeToken(), t -> rule(t, context)) + : plan.transformUp(typeToken(), t -> rule(t, context)); + } + + protected abstract PhysicalPlan rule(SubPlan plan, P context); + } + + public abstract static class OptimizerRule extends Rule { + + private final TransformDirection direction; + + public OptimizerRule() { + this(TransformDirection.DOWN); + } + + protected OptimizerRule(TransformDirection direction) { + this.direction = direction; + } + + @Override + public final PhysicalPlan apply(PhysicalPlan plan) { + return direction == TransformDirection.DOWN + ? plan.transformDown(typeToken(), this::rule) + : plan.transformUp(typeToken(), this::rule); + } + + protected abstract PhysicalPlan rule(SubPlan plan); + } + + public abstract static class OptimizerExpressionRule extends Rule { + + private final TransformDirection direction; + // overriding type token which returns the correct class but does an uncheck cast to LogicalPlan due to its generic bound + // a proper solution is to wrap the Expression rule into a Plan rule but that would affect the rule declaration + // so instead this is hacked here + private final Class expressionTypeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + + public OptimizerExpressionRule(TransformDirection direction) { + this.direction = direction; + } + + @Override + public final PhysicalPlan apply(PhysicalPlan plan) { + return direction == TransformDirection.DOWN + ? plan.transformExpressionsDown(expressionTypeToken, this::rule) + : plan.transformExpressionsUp(expressionTypeToken, this::rule); + } + + protected PhysicalPlan rule(PhysicalPlan plan) { + return plan; + } + + protected abstract Expression rule(E e); + + public Class expressionToken() { + return expressionTypeToken; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java new file mode 100644 index 0000000000000..61049c52a01e8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizer.java @@ -0,0 +1,149 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.RegexExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.UnaryExec; +import org.elasticsearch.xpack.esql.planner.PhysicalVerificationException; +import org.elasticsearch.xpack.esql.planner.PhysicalVerifier; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor; +import org.elasticsearch.xpack.ql.rule.Rule; +import org.elasticsearch.xpack.ql.rule.RuleExecutor; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.util.Holder; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; + +import static java.lang.Boolean.FALSE; +import static java.lang.Boolean.TRUE; +import static java.util.Arrays.asList; + +/** + * Performs global (coordinator) optimization of the physical plan. + * Local (data-node) optimizations occur later by operating just on a plan fragment (subplan). + */ +public class PhysicalPlanOptimizer extends ParameterizedRuleExecutor { + private static final Iterable> rules = initializeRules(true); + + private final PhysicalVerifier verifier; + + public PhysicalPlanOptimizer(PhysicalOptimizerContext context) { + super(context); + this.verifier = new PhysicalVerifier(); + } + + public PhysicalPlan optimize(PhysicalPlan plan) { + return verify(execute(plan)); + } + + PhysicalPlan verify(PhysicalPlan plan) { + Collection failures = verifier.verify(plan); + if (failures.isEmpty() == false) { + throw new PhysicalVerificationException(failures); + } + return plan; + } + + static List> initializeRules(boolean isOptimizedForEsSource) { + var boundary = new Batch("Plan Boundary", Limiter.ONCE, new ProjectAwayColumns()); + return asList(boundary); + } + + @Override + protected Iterable> batches() { + return rules; + } + + /** + * Adds an explicit project to minimize the amount of attributes sent from the local plan to the coordinator. + * This is done here to localize the project close to the data source and simplify the upcoming field + * extraction. + */ + static class ProjectAwayColumns extends Rule { + + @Override + public PhysicalPlan apply(PhysicalPlan plan) { + var projectAll = new Holder<>(TRUE); + var keepCollecting = new Holder<>(TRUE); + var attributes = new LinkedHashSet(); + var aliases = new HashMap(); + + return plan.transformDown(UnaryExec.class, p -> { + // no need for project all + if (p instanceof ProjectExec || p instanceof AggregateExec) { + projectAll.set(FALSE); + } + if (keepCollecting.get()) { + p.forEachExpression(NamedExpression.class, ne -> { + var attr = ne.toAttribute(); + // filter out attributes declared as aliases before + if (ne instanceof Alias as) { + aliases.put(attr, as.child()); + attributes.remove(attr); + } else { + if (aliases.containsKey(attr) == false) { + attributes.add(attr); + } + } + }); + if (p instanceof RegexExtractExec ree) { + attributes.removeAll(ree.extractedFields()); + } + if (p instanceof EnrichExec ee) { + for (NamedExpression enrichField : ee.enrichFields()) { + attributes.remove(enrichField instanceof Alias a ? a.child() : enrichField); + } + } + } + if (p instanceof ExchangeExec exec) { + keepCollecting.set(FALSE); + var child = exec.child(); + // otherwise expect a Fragment + if (child instanceof FragmentExec fragmentExec) { + var logicalFragment = fragmentExec.fragment(); + // no need for projection when dealing with aggs + if (logicalFragment instanceof Aggregate) { + attributes.clear(); + } + var selectAll = projectAll.get(); + if (attributes.isEmpty() == false || selectAll) { + var output = selectAll ? exec.child().output() : new ArrayList<>(attributes); + // add a logical projection (let the local replanning remove it if needed) + p = exec.replaceChild( + new FragmentExec( + Source.EMPTY, + new Project(logicalFragment.source(), logicalFragment, output), + fragmentExec.esFilter(), + fragmentExec.estimatedRowSize() + ) + ); + } + } + } + return p; + }); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AbstractBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AbstractBuilder.java new file mode 100644 index 0000000000000..aff66b6485db6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AbstractBuilder.java @@ -0,0 +1,63 @@ +/* + * 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.parser; + +import org.antlr.v4.runtime.tree.ParseTree; +import org.antlr.v4.runtime.tree.TerminalNode; +import org.elasticsearch.xpack.ql.parser.ParserUtils; +import org.elasticsearch.xpack.ql.tree.Source; + +abstract class AbstractBuilder extends EsqlBaseParserBaseVisitor { + + @Override + public Object visit(ParseTree tree) { + return ParserUtils.visit(super::visit, tree); + } + + @Override + public Object visitTerminal(TerminalNode node) { + return ParserUtils.source(node); + } + + static String unquoteString(Source source) { + String text = source.text(); + if (text == null) { + return null; + } + + // unescaped strings can be interpreted directly + if (text.startsWith("\"\"\"")) { + return text.substring(3, text.length() - 3); + } + + text = text.substring(1, text.length() - 1); + StringBuilder sb = new StringBuilder(); + + for (int i = 0; i < text.length();) { + if (text.charAt(i) == '\\') { + // ANTLR4 Grammar guarantees there is always a character after the `\` + switch (text.charAt(++i)) { + case 't' -> sb.append('\t'); + case 'n' -> sb.append('\n'); + case 'r' -> sb.append('\r'); + case '"' -> sb.append('\"'); + case '\\' -> sb.append('\\'); + + // will be interpreted as regex, so we have to escape it + default -> + // unknown escape sequence, pass through as-is, e.g: `...\w...` + sb.append('\\').append(text.charAt(i)); + } + i++; + } else { + sb.append(text.charAt(i++)); + } + } + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AstBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AstBuilder.java new file mode 100644 index 0000000000000..406b9e21e1d59 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/AstBuilder.java @@ -0,0 +1,18 @@ +/* + * 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.parser; + +import org.antlr.v4.runtime.Token; + +import java.util.Map; + +public class AstBuilder extends LogicalPlanBuilder { + public AstBuilder(Map params) { + super(params); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ContentLocation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ContentLocation.java new file mode 100644 index 0000000000000..6b1b50df32f5e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ContentLocation.java @@ -0,0 +1,30 @@ +/* + * 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.parser; + +/** + * Light clone of XContentLocation + */ +public class ContentLocation { + + public static final ContentLocation UNKNOWN = new ContentLocation(-1, -1); + + public final int lineNumber; + public final int columnNumber; + + public ContentLocation(int lineNumber, int columnNumber) { + super(); + this.lineNumber = lineNumber; + this.columnNumber = columnNumber; + } + + @Override + public String toString() { + return lineNumber + ":" + columnNumber; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp new file mode 100644 index 0000000000000..12542878c3ed3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp @@ -0,0 +1,275 @@ +token literal names: +null +'dissect' +'drop' +'enrich' +'eval' +'explain' +'from' +'grok' +'inlinestats' +'keep' +'limit' +'mv_expand' +'project' +'rename' +'row' +'show' +'sort' +'stats' +'where' +null +null +null +null +null +null +null +null +null +null +null +'by' +'and' +'asc' +null +null +'desc' +'.' +'false' +'first' +'last' +'(' +'in' +'is' +'like' +'not' +'null' +'nulls' +'or' +'?' +'rlike' +')' +'true' +'info' +'functions' +'==' +'!=' +'<' +'<=' +'>' +'>=' +'+' +'-' +'*' +'/' +'%' +null +']' +null +null +null +null +null +'as' +'metadata' +'on' +'with' +null +null +null +null +null +null + +token symbolic names: +null +DISSECT +DROP +ENRICH +EVAL +EXPLAIN +FROM +GROK +INLINESTATS +KEEP +LIMIT +MV_EXPAND +PROJECT +RENAME +ROW +SHOW +SORT +STATS +WHERE +UNKNOWN_CMD +LINE_COMMENT +MULTILINE_COMMENT +WS +EXPLAIN_WS +EXPLAIN_LINE_COMMENT +EXPLAIN_MULTILINE_COMMENT +PIPE +STRING +INTEGER_LITERAL +DECIMAL_LITERAL +BY +AND +ASC +ASSIGN +COMMA +DESC +DOT +FALSE +FIRST +LAST +LP +IN +IS +LIKE +NOT +NULL +NULLS +OR +PARAM +RLIKE +RP +TRUE +INFO +FUNCTIONS +EQ +NEQ +LT +LTE +GT +GTE +PLUS +MINUS +ASTERISK +SLASH +PERCENT +OPENING_BRACKET +CLOSING_BRACKET +UNQUOTED_IDENTIFIER +QUOTED_IDENTIFIER +EXPR_LINE_COMMENT +EXPR_MULTILINE_COMMENT +EXPR_WS +AS +METADATA +ON +WITH +SRC_UNQUOTED_IDENTIFIER +SRC_QUOTED_IDENTIFIER +SRC_LINE_COMMENT +SRC_MULTILINE_COMMENT +SRC_WS +EXPLAIN_PIPE + +rule names: +DISSECT +DROP +ENRICH +EVAL +EXPLAIN +FROM +GROK +INLINESTATS +KEEP +LIMIT +MV_EXPAND +PROJECT +RENAME +ROW +SHOW +SORT +STATS +WHERE +UNKNOWN_CMD +LINE_COMMENT +MULTILINE_COMMENT +WS +EXPLAIN_OPENING_BRACKET +EXPLAIN_PIPE +EXPLAIN_WS +EXPLAIN_LINE_COMMENT +EXPLAIN_MULTILINE_COMMENT +PIPE +DIGIT +LETTER +ESCAPE_SEQUENCE +UNESCAPED_CHARS +EXPONENT +STRING +INTEGER_LITERAL +DECIMAL_LITERAL +BY +AND +ASC +ASSIGN +COMMA +DESC +DOT +FALSE +FIRST +LAST +LP +IN +IS +LIKE +NOT +NULL +NULLS +OR +PARAM +RLIKE +RP +TRUE +INFO +FUNCTIONS +EQ +NEQ +LT +LTE +GT +GTE +PLUS +MINUS +ASTERISK +SLASH +PERCENT +OPENING_BRACKET +CLOSING_BRACKET +UNQUOTED_IDENTIFIER +QUOTED_IDENTIFIER +EXPR_LINE_COMMENT +EXPR_MULTILINE_COMMENT +EXPR_WS +SRC_PIPE +SRC_OPENING_BRACKET +SRC_CLOSING_BRACKET +SRC_COMMA +SRC_ASSIGN +AS +METADATA +ON +WITH +SRC_UNQUOTED_IDENTIFIER +SRC_UNQUOTED_IDENTIFIER_PART +SRC_QUOTED_IDENTIFIER +SRC_LINE_COMMENT +SRC_MULTILINE_COMMENT +SRC_WS + +channel names: +DEFAULT_TOKEN_CHANNEL +HIDDEN + +mode names: +DEFAULT_MODE +EXPLAIN_MODE +EXPRESSION +SOURCE_IDENTIFIERS + +atn: +[4, 0, 81, 764, 6, -1, 6, -1, 6, -1, 6, -1, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 2, 47, 7, 47, 2, 48, 7, 48, 2, 49, 7, 49, 2, 50, 7, 50, 2, 51, 7, 51, 2, 52, 7, 52, 2, 53, 7, 53, 2, 54, 7, 54, 2, 55, 7, 55, 2, 56, 7, 56, 2, 57, 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 2, 77, 7, 77, 2, 78, 7, 78, 2, 79, 7, 79, 2, 80, 7, 80, 2, 81, 7, 81, 2, 82, 7, 82, 2, 83, 7, 83, 2, 84, 7, 84, 2, 85, 7, 85, 2, 86, 7, 86, 2, 87, 7, 87, 2, 88, 7, 88, 2, 89, 7, 89, 2, 90, 7, 90, 2, 91, 7, 91, 2, 92, 7, 92, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 15, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 16, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 17, 1, 18, 4, 18, 345, 8, 18, 11, 18, 12, 18, 346, 1, 18, 1, 18, 1, 19, 1, 19, 1, 19, 1, 19, 5, 19, 355, 8, 19, 10, 19, 12, 19, 358, 9, 19, 1, 19, 3, 19, 361, 8, 19, 1, 19, 3, 19, 364, 8, 19, 1, 19, 1, 19, 1, 20, 1, 20, 1, 20, 1, 20, 1, 20, 5, 20, 373, 8, 20, 10, 20, 12, 20, 376, 9, 20, 1, 20, 1, 20, 1, 20, 1, 20, 1, 20, 1, 21, 4, 21, 384, 8, 21, 11, 21, 12, 21, 385, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 23, 1, 23, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 24, 1, 24, 1, 25, 1, 25, 1, 25, 1, 25, 1, 26, 1, 26, 1, 26, 1, 26, 1, 27, 1, 27, 1, 27, 1, 27, 1, 28, 1, 28, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 1, 31, 1, 31, 1, 32, 1, 32, 3, 32, 427, 8, 32, 1, 32, 4, 32, 430, 8, 32, 11, 32, 12, 32, 431, 1, 33, 1, 33, 1, 33, 5, 33, 437, 8, 33, 10, 33, 12, 33, 440, 9, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 5, 33, 448, 8, 33, 10, 33, 12, 33, 451, 9, 33, 1, 33, 1, 33, 1, 33, 1, 33, 1, 33, 3, 33, 458, 8, 33, 1, 33, 3, 33, 461, 8, 33, 3, 33, 463, 8, 33, 1, 34, 4, 34, 466, 8, 34, 11, 34, 12, 34, 467, 1, 35, 4, 35, 471, 8, 35, 11, 35, 12, 35, 472, 1, 35, 1, 35, 5, 35, 477, 8, 35, 10, 35, 12, 35, 480, 9, 35, 1, 35, 1, 35, 4, 35, 484, 8, 35, 11, 35, 12, 35, 485, 1, 35, 4, 35, 489, 8, 35, 11, 35, 12, 35, 490, 1, 35, 1, 35, 5, 35, 495, 8, 35, 10, 35, 12, 35, 498, 9, 35, 3, 35, 500, 8, 35, 1, 35, 1, 35, 1, 35, 1, 35, 4, 35, 506, 8, 35, 11, 35, 12, 35, 507, 1, 35, 1, 35, 3, 35, 512, 8, 35, 1, 36, 1, 36, 1, 36, 1, 37, 1, 37, 1, 37, 1, 37, 1, 38, 1, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1, 40, 1, 40, 1, 41, 1, 41, 1, 41, 1, 41, 1, 41, 1, 42, 1, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 43, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 44, 1, 45, 1, 45, 1, 45, 1, 45, 1, 45, 1, 46, 1, 46, 1, 47, 1, 47, 1, 47, 1, 48, 1, 48, 1, 48, 1, 49, 1, 49, 1, 49, 1, 49, 1, 49, 1, 50, 1, 50, 1, 50, 1, 50, 1, 51, 1, 51, 1, 51, 1, 51, 1, 51, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 1, 52, 1, 53, 1, 53, 1, 53, 1, 54, 1, 54, 1, 55, 1, 55, 1, 55, 1, 55, 1, 55, 1, 55, 1, 56, 1, 56, 1, 57, 1, 57, 1, 57, 1, 57, 1, 57, 1, 58, 1, 58, 1, 58, 1, 58, 1, 58, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 59, 1, 60, 1, 60, 1, 60, 1, 61, 1, 61, 1, 61, 1, 62, 1, 62, 1, 63, 1, 63, 1, 63, 1, 64, 1, 64, 1, 65, 1, 65, 1, 65, 1, 66, 1, 66, 1, 67, 1, 67, 1, 68, 1, 68, 1, 69, 1, 69, 1, 70, 1, 70, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 1, 73, 1, 73, 1, 73, 1, 73, 5, 73, 654, 8, 73, 10, 73, 12, 73, 657, 9, 73, 1, 73, 1, 73, 1, 73, 1, 73, 4, 73, 663, 8, 73, 11, 73, 12, 73, 664, 3, 73, 667, 8, 73, 1, 74, 1, 74, 1, 74, 1, 74, 5, 74, 673, 8, 74, 10, 74, 12, 74, 676, 9, 74, 1, 74, 1, 74, 1, 75, 1, 75, 1, 75, 1, 75, 1, 76, 1, 76, 1, 76, 1, 76, 1, 77, 1, 77, 1, 77, 1, 77, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 79, 1, 79, 1, 79, 1, 79, 1, 79, 1, 79, 1, 80, 1, 80, 1, 80, 1, 80, 1, 80, 1, 80, 1, 81, 1, 81, 1, 81, 1, 81, 1, 82, 1, 82, 1, 82, 1, 82, 1, 83, 1, 83, 1, 83, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 85, 1, 85, 1, 85, 1, 86, 1, 86, 1, 86, 1, 86, 1, 86, 1, 87, 4, 87, 738, 8, 87, 11, 87, 12, 87, 739, 1, 88, 4, 88, 743, 8, 88, 11, 88, 12, 88, 744, 1, 88, 1, 88, 3, 88, 749, 8, 88, 1, 89, 1, 89, 1, 90, 1, 90, 1, 90, 1, 90, 1, 91, 1, 91, 1, 91, 1, 91, 1, 92, 1, 92, 1, 92, 1, 92, 2, 374, 449, 0, 93, 4, 1, 6, 2, 8, 3, 10, 4, 12, 5, 14, 6, 16, 7, 18, 8, 20, 9, 22, 10, 24, 11, 26, 12, 28, 13, 30, 14, 32, 15, 34, 16, 36, 17, 38, 18, 40, 19, 42, 20, 44, 21, 46, 22, 48, 0, 50, 81, 52, 23, 54, 24, 56, 25, 58, 26, 60, 0, 62, 0, 64, 0, 66, 0, 68, 0, 70, 27, 72, 28, 74, 29, 76, 30, 78, 31, 80, 32, 82, 33, 84, 34, 86, 35, 88, 36, 90, 37, 92, 38, 94, 39, 96, 40, 98, 41, 100, 42, 102, 43, 104, 44, 106, 45, 108, 46, 110, 47, 112, 48, 114, 49, 116, 50, 118, 51, 120, 52, 122, 53, 124, 54, 126, 55, 128, 56, 130, 57, 132, 58, 134, 59, 136, 60, 138, 61, 140, 62, 142, 63, 144, 64, 146, 65, 148, 66, 150, 67, 152, 68, 154, 69, 156, 70, 158, 71, 160, 0, 162, 0, 164, 0, 166, 0, 168, 0, 170, 72, 172, 73, 174, 74, 176, 75, 178, 76, 180, 0, 182, 77, 184, 78, 186, 79, 188, 80, 4, 0, 1, 2, 3, 13, 6, 0, 9, 10, 13, 13, 32, 32, 47, 47, 91, 91, 93, 93, 2, 0, 10, 10, 13, 13, 3, 0, 9, 10, 13, 13, 32, 32, 1, 0, 48, 57, 2, 0, 65, 90, 97, 122, 5, 0, 34, 34, 92, 92, 110, 110, 114, 114, 116, 116, 4, 0, 10, 10, 13, 13, 34, 34, 92, 92, 2, 0, 69, 69, 101, 101, 2, 0, 43, 43, 45, 45, 2, 0, 64, 64, 95, 95, 1, 0, 96, 96, 10, 0, 9, 10, 13, 13, 32, 32, 44, 44, 47, 47, 61, 61, 91, 91, 93, 93, 96, 96, 124, 124, 2, 0, 42, 42, 47, 47, 792, 0, 4, 1, 0, 0, 0, 0, 6, 1, 0, 0, 0, 0, 8, 1, 0, 0, 0, 0, 10, 1, 0, 0, 0, 0, 12, 1, 0, 0, 0, 0, 14, 1, 0, 0, 0, 0, 16, 1, 0, 0, 0, 0, 18, 1, 0, 0, 0, 0, 20, 1, 0, 0, 0, 0, 22, 1, 0, 0, 0, 0, 24, 1, 0, 0, 0, 0, 26, 1, 0, 0, 0, 0, 28, 1, 0, 0, 0, 0, 30, 1, 0, 0, 0, 0, 32, 1, 0, 0, 0, 0, 34, 1, 0, 0, 0, 0, 36, 1, 0, 0, 0, 0, 38, 1, 0, 0, 0, 0, 40, 1, 0, 0, 0, 0, 42, 1, 0, 0, 0, 0, 44, 1, 0, 0, 0, 0, 46, 1, 0, 0, 0, 1, 48, 1, 0, 0, 0, 1, 50, 1, 0, 0, 0, 1, 52, 1, 0, 0, 0, 1, 54, 1, 0, 0, 0, 1, 56, 1, 0, 0, 0, 2, 58, 1, 0, 0, 0, 2, 70, 1, 0, 0, 0, 2, 72, 1, 0, 0, 0, 2, 74, 1, 0, 0, 0, 2, 76, 1, 0, 0, 0, 2, 78, 1, 0, 0, 0, 2, 80, 1, 0, 0, 0, 2, 82, 1, 0, 0, 0, 2, 84, 1, 0, 0, 0, 2, 86, 1, 0, 0, 0, 2, 88, 1, 0, 0, 0, 2, 90, 1, 0, 0, 0, 2, 92, 1, 0, 0, 0, 2, 94, 1, 0, 0, 0, 2, 96, 1, 0, 0, 0, 2, 98, 1, 0, 0, 0, 2, 100, 1, 0, 0, 0, 2, 102, 1, 0, 0, 0, 2, 104, 1, 0, 0, 0, 2, 106, 1, 0, 0, 0, 2, 108, 1, 0, 0, 0, 2, 110, 1, 0, 0, 0, 2, 112, 1, 0, 0, 0, 2, 114, 1, 0, 0, 0, 2, 116, 1, 0, 0, 0, 2, 118, 1, 0, 0, 0, 2, 120, 1, 0, 0, 0, 2, 122, 1, 0, 0, 0, 2, 124, 1, 0, 0, 0, 2, 126, 1, 0, 0, 0, 2, 128, 1, 0, 0, 0, 2, 130, 1, 0, 0, 0, 2, 132, 1, 0, 0, 0, 2, 134, 1, 0, 0, 0, 2, 136, 1, 0, 0, 0, 2, 138, 1, 0, 0, 0, 2, 140, 1, 0, 0, 0, 2, 142, 1, 0, 0, 0, 2, 144, 1, 0, 0, 0, 2, 146, 1, 0, 0, 0, 2, 148, 1, 0, 0, 0, 2, 150, 1, 0, 0, 0, 2, 152, 1, 0, 0, 0, 2, 154, 1, 0, 0, 0, 2, 156, 1, 0, 0, 0, 2, 158, 1, 0, 0, 0, 3, 160, 1, 0, 0, 0, 3, 162, 1, 0, 0, 0, 3, 164, 1, 0, 0, 0, 3, 166, 1, 0, 0, 0, 3, 168, 1, 0, 0, 0, 3, 170, 1, 0, 0, 0, 3, 172, 1, 0, 0, 0, 3, 174, 1, 0, 0, 0, 3, 176, 1, 0, 0, 0, 3, 178, 1, 0, 0, 0, 3, 182, 1, 0, 0, 0, 3, 184, 1, 0, 0, 0, 3, 186, 1, 0, 0, 0, 3, 188, 1, 0, 0, 0, 4, 190, 1, 0, 0, 0, 6, 200, 1, 0, 0, 0, 8, 207, 1, 0, 0, 0, 10, 216, 1, 0, 0, 0, 12, 223, 1, 0, 0, 0, 14, 233, 1, 0, 0, 0, 16, 240, 1, 0, 0, 0, 18, 247, 1, 0, 0, 0, 20, 261, 1, 0, 0, 0, 22, 268, 1, 0, 0, 0, 24, 276, 1, 0, 0, 0, 26, 288, 1, 0, 0, 0, 28, 298, 1, 0, 0, 0, 30, 307, 1, 0, 0, 0, 32, 313, 1, 0, 0, 0, 34, 320, 1, 0, 0, 0, 36, 327, 1, 0, 0, 0, 38, 335, 1, 0, 0, 0, 40, 344, 1, 0, 0, 0, 42, 350, 1, 0, 0, 0, 44, 367, 1, 0, 0, 0, 46, 383, 1, 0, 0, 0, 48, 389, 1, 0, 0, 0, 50, 394, 1, 0, 0, 0, 52, 399, 1, 0, 0, 0, 54, 403, 1, 0, 0, 0, 56, 407, 1, 0, 0, 0, 58, 411, 1, 0, 0, 0, 60, 415, 1, 0, 0, 0, 62, 417, 1, 0, 0, 0, 64, 419, 1, 0, 0, 0, 66, 422, 1, 0, 0, 0, 68, 424, 1, 0, 0, 0, 70, 462, 1, 0, 0, 0, 72, 465, 1, 0, 0, 0, 74, 511, 1, 0, 0, 0, 76, 513, 1, 0, 0, 0, 78, 516, 1, 0, 0, 0, 80, 520, 1, 0, 0, 0, 82, 524, 1, 0, 0, 0, 84, 526, 1, 0, 0, 0, 86, 528, 1, 0, 0, 0, 88, 533, 1, 0, 0, 0, 90, 535, 1, 0, 0, 0, 92, 541, 1, 0, 0, 0, 94, 547, 1, 0, 0, 0, 96, 552, 1, 0, 0, 0, 98, 554, 1, 0, 0, 0, 100, 557, 1, 0, 0, 0, 102, 560, 1, 0, 0, 0, 104, 565, 1, 0, 0, 0, 106, 569, 1, 0, 0, 0, 108, 574, 1, 0, 0, 0, 110, 580, 1, 0, 0, 0, 112, 583, 1, 0, 0, 0, 114, 585, 1, 0, 0, 0, 116, 591, 1, 0, 0, 0, 118, 593, 1, 0, 0, 0, 120, 598, 1, 0, 0, 0, 122, 603, 1, 0, 0, 0, 124, 613, 1, 0, 0, 0, 126, 616, 1, 0, 0, 0, 128, 619, 1, 0, 0, 0, 130, 621, 1, 0, 0, 0, 132, 624, 1, 0, 0, 0, 134, 626, 1, 0, 0, 0, 136, 629, 1, 0, 0, 0, 138, 631, 1, 0, 0, 0, 140, 633, 1, 0, 0, 0, 142, 635, 1, 0, 0, 0, 144, 637, 1, 0, 0, 0, 146, 639, 1, 0, 0, 0, 148, 644, 1, 0, 0, 0, 150, 666, 1, 0, 0, 0, 152, 668, 1, 0, 0, 0, 154, 679, 1, 0, 0, 0, 156, 683, 1, 0, 0, 0, 158, 687, 1, 0, 0, 0, 160, 691, 1, 0, 0, 0, 162, 696, 1, 0, 0, 0, 164, 702, 1, 0, 0, 0, 166, 708, 1, 0, 0, 0, 168, 712, 1, 0, 0, 0, 170, 716, 1, 0, 0, 0, 172, 719, 1, 0, 0, 0, 174, 728, 1, 0, 0, 0, 176, 731, 1, 0, 0, 0, 178, 737, 1, 0, 0, 0, 180, 748, 1, 0, 0, 0, 182, 750, 1, 0, 0, 0, 184, 752, 1, 0, 0, 0, 186, 756, 1, 0, 0, 0, 188, 760, 1, 0, 0, 0, 190, 191, 5, 100, 0, 0, 191, 192, 5, 105, 0, 0, 192, 193, 5, 115, 0, 0, 193, 194, 5, 115, 0, 0, 194, 195, 5, 101, 0, 0, 195, 196, 5, 99, 0, 0, 196, 197, 5, 116, 0, 0, 197, 198, 1, 0, 0, 0, 198, 199, 6, 0, 0, 0, 199, 5, 1, 0, 0, 0, 200, 201, 5, 100, 0, 0, 201, 202, 5, 114, 0, 0, 202, 203, 5, 111, 0, 0, 203, 204, 5, 112, 0, 0, 204, 205, 1, 0, 0, 0, 205, 206, 6, 1, 1, 0, 206, 7, 1, 0, 0, 0, 207, 208, 5, 101, 0, 0, 208, 209, 5, 110, 0, 0, 209, 210, 5, 114, 0, 0, 210, 211, 5, 105, 0, 0, 211, 212, 5, 99, 0, 0, 212, 213, 5, 104, 0, 0, 213, 214, 1, 0, 0, 0, 214, 215, 6, 2, 1, 0, 215, 9, 1, 0, 0, 0, 216, 217, 5, 101, 0, 0, 217, 218, 5, 118, 0, 0, 218, 219, 5, 97, 0, 0, 219, 220, 5, 108, 0, 0, 220, 221, 1, 0, 0, 0, 221, 222, 6, 3, 0, 0, 222, 11, 1, 0, 0, 0, 223, 224, 5, 101, 0, 0, 224, 225, 5, 120, 0, 0, 225, 226, 5, 112, 0, 0, 226, 227, 5, 108, 0, 0, 227, 228, 5, 97, 0, 0, 228, 229, 5, 105, 0, 0, 229, 230, 5, 110, 0, 0, 230, 231, 1, 0, 0, 0, 231, 232, 6, 4, 2, 0, 232, 13, 1, 0, 0, 0, 233, 234, 5, 102, 0, 0, 234, 235, 5, 114, 0, 0, 235, 236, 5, 111, 0, 0, 236, 237, 5, 109, 0, 0, 237, 238, 1, 0, 0, 0, 238, 239, 6, 5, 1, 0, 239, 15, 1, 0, 0, 0, 240, 241, 5, 103, 0, 0, 241, 242, 5, 114, 0, 0, 242, 243, 5, 111, 0, 0, 243, 244, 5, 107, 0, 0, 244, 245, 1, 0, 0, 0, 245, 246, 6, 6, 0, 0, 246, 17, 1, 0, 0, 0, 247, 248, 5, 105, 0, 0, 248, 249, 5, 110, 0, 0, 249, 250, 5, 108, 0, 0, 250, 251, 5, 105, 0, 0, 251, 252, 5, 110, 0, 0, 252, 253, 5, 101, 0, 0, 253, 254, 5, 115, 0, 0, 254, 255, 5, 116, 0, 0, 255, 256, 5, 97, 0, 0, 256, 257, 5, 116, 0, 0, 257, 258, 5, 115, 0, 0, 258, 259, 1, 0, 0, 0, 259, 260, 6, 7, 0, 0, 260, 19, 1, 0, 0, 0, 261, 262, 5, 107, 0, 0, 262, 263, 5, 101, 0, 0, 263, 264, 5, 101, 0, 0, 264, 265, 5, 112, 0, 0, 265, 266, 1, 0, 0, 0, 266, 267, 6, 8, 1, 0, 267, 21, 1, 0, 0, 0, 268, 269, 5, 108, 0, 0, 269, 270, 5, 105, 0, 0, 270, 271, 5, 109, 0, 0, 271, 272, 5, 105, 0, 0, 272, 273, 5, 116, 0, 0, 273, 274, 1, 0, 0, 0, 274, 275, 6, 9, 0, 0, 275, 23, 1, 0, 0, 0, 276, 277, 5, 109, 0, 0, 277, 278, 5, 118, 0, 0, 278, 279, 5, 95, 0, 0, 279, 280, 5, 101, 0, 0, 280, 281, 5, 120, 0, 0, 281, 282, 5, 112, 0, 0, 282, 283, 5, 97, 0, 0, 283, 284, 5, 110, 0, 0, 284, 285, 5, 100, 0, 0, 285, 286, 1, 0, 0, 0, 286, 287, 6, 10, 1, 0, 287, 25, 1, 0, 0, 0, 288, 289, 5, 112, 0, 0, 289, 290, 5, 114, 0, 0, 290, 291, 5, 111, 0, 0, 291, 292, 5, 106, 0, 0, 292, 293, 5, 101, 0, 0, 293, 294, 5, 99, 0, 0, 294, 295, 5, 116, 0, 0, 295, 296, 1, 0, 0, 0, 296, 297, 6, 11, 1, 0, 297, 27, 1, 0, 0, 0, 298, 299, 5, 114, 0, 0, 299, 300, 5, 101, 0, 0, 300, 301, 5, 110, 0, 0, 301, 302, 5, 97, 0, 0, 302, 303, 5, 109, 0, 0, 303, 304, 5, 101, 0, 0, 304, 305, 1, 0, 0, 0, 305, 306, 6, 12, 1, 0, 306, 29, 1, 0, 0, 0, 307, 308, 5, 114, 0, 0, 308, 309, 5, 111, 0, 0, 309, 310, 5, 119, 0, 0, 310, 311, 1, 0, 0, 0, 311, 312, 6, 13, 0, 0, 312, 31, 1, 0, 0, 0, 313, 314, 5, 115, 0, 0, 314, 315, 5, 104, 0, 0, 315, 316, 5, 111, 0, 0, 316, 317, 5, 119, 0, 0, 317, 318, 1, 0, 0, 0, 318, 319, 6, 14, 0, 0, 319, 33, 1, 0, 0, 0, 320, 321, 5, 115, 0, 0, 321, 322, 5, 111, 0, 0, 322, 323, 5, 114, 0, 0, 323, 324, 5, 116, 0, 0, 324, 325, 1, 0, 0, 0, 325, 326, 6, 15, 0, 0, 326, 35, 1, 0, 0, 0, 327, 328, 5, 115, 0, 0, 328, 329, 5, 116, 0, 0, 329, 330, 5, 97, 0, 0, 330, 331, 5, 116, 0, 0, 331, 332, 5, 115, 0, 0, 332, 333, 1, 0, 0, 0, 333, 334, 6, 16, 0, 0, 334, 37, 1, 0, 0, 0, 335, 336, 5, 119, 0, 0, 336, 337, 5, 104, 0, 0, 337, 338, 5, 101, 0, 0, 338, 339, 5, 114, 0, 0, 339, 340, 5, 101, 0, 0, 340, 341, 1, 0, 0, 0, 341, 342, 6, 17, 0, 0, 342, 39, 1, 0, 0, 0, 343, 345, 8, 0, 0, 0, 344, 343, 1, 0, 0, 0, 345, 346, 1, 0, 0, 0, 346, 344, 1, 0, 0, 0, 346, 347, 1, 0, 0, 0, 347, 348, 1, 0, 0, 0, 348, 349, 6, 18, 0, 0, 349, 41, 1, 0, 0, 0, 350, 351, 5, 47, 0, 0, 351, 352, 5, 47, 0, 0, 352, 356, 1, 0, 0, 0, 353, 355, 8, 1, 0, 0, 354, 353, 1, 0, 0, 0, 355, 358, 1, 0, 0, 0, 356, 354, 1, 0, 0, 0, 356, 357, 1, 0, 0, 0, 357, 360, 1, 0, 0, 0, 358, 356, 1, 0, 0, 0, 359, 361, 5, 13, 0, 0, 360, 359, 1, 0, 0, 0, 360, 361, 1, 0, 0, 0, 361, 363, 1, 0, 0, 0, 362, 364, 5, 10, 0, 0, 363, 362, 1, 0, 0, 0, 363, 364, 1, 0, 0, 0, 364, 365, 1, 0, 0, 0, 365, 366, 6, 19, 3, 0, 366, 43, 1, 0, 0, 0, 367, 368, 5, 47, 0, 0, 368, 369, 5, 42, 0, 0, 369, 374, 1, 0, 0, 0, 370, 373, 3, 44, 20, 0, 371, 373, 9, 0, 0, 0, 372, 370, 1, 0, 0, 0, 372, 371, 1, 0, 0, 0, 373, 376, 1, 0, 0, 0, 374, 375, 1, 0, 0, 0, 374, 372, 1, 0, 0, 0, 375, 377, 1, 0, 0, 0, 376, 374, 1, 0, 0, 0, 377, 378, 5, 42, 0, 0, 378, 379, 5, 47, 0, 0, 379, 380, 1, 0, 0, 0, 380, 381, 6, 20, 3, 0, 381, 45, 1, 0, 0, 0, 382, 384, 7, 2, 0, 0, 383, 382, 1, 0, 0, 0, 384, 385, 1, 0, 0, 0, 385, 383, 1, 0, 0, 0, 385, 386, 1, 0, 0, 0, 386, 387, 1, 0, 0, 0, 387, 388, 6, 21, 3, 0, 388, 47, 1, 0, 0, 0, 389, 390, 5, 91, 0, 0, 390, 391, 1, 0, 0, 0, 391, 392, 6, 22, 4, 0, 392, 393, 6, 22, 5, 0, 393, 49, 1, 0, 0, 0, 394, 395, 5, 124, 0, 0, 395, 396, 1, 0, 0, 0, 396, 397, 6, 23, 6, 0, 397, 398, 6, 23, 7, 0, 398, 51, 1, 0, 0, 0, 399, 400, 3, 46, 21, 0, 400, 401, 1, 0, 0, 0, 401, 402, 6, 24, 3, 0, 402, 53, 1, 0, 0, 0, 403, 404, 3, 42, 19, 0, 404, 405, 1, 0, 0, 0, 405, 406, 6, 25, 3, 0, 406, 55, 1, 0, 0, 0, 407, 408, 3, 44, 20, 0, 408, 409, 1, 0, 0, 0, 409, 410, 6, 26, 3, 0, 410, 57, 1, 0, 0, 0, 411, 412, 5, 124, 0, 0, 412, 413, 1, 0, 0, 0, 413, 414, 6, 27, 7, 0, 414, 59, 1, 0, 0, 0, 415, 416, 7, 3, 0, 0, 416, 61, 1, 0, 0, 0, 417, 418, 7, 4, 0, 0, 418, 63, 1, 0, 0, 0, 419, 420, 5, 92, 0, 0, 420, 421, 7, 5, 0, 0, 421, 65, 1, 0, 0, 0, 422, 423, 8, 6, 0, 0, 423, 67, 1, 0, 0, 0, 424, 426, 7, 7, 0, 0, 425, 427, 7, 8, 0, 0, 426, 425, 1, 0, 0, 0, 426, 427, 1, 0, 0, 0, 427, 429, 1, 0, 0, 0, 428, 430, 3, 60, 28, 0, 429, 428, 1, 0, 0, 0, 430, 431, 1, 0, 0, 0, 431, 429, 1, 0, 0, 0, 431, 432, 1, 0, 0, 0, 432, 69, 1, 0, 0, 0, 433, 438, 5, 34, 0, 0, 434, 437, 3, 64, 30, 0, 435, 437, 3, 66, 31, 0, 436, 434, 1, 0, 0, 0, 436, 435, 1, 0, 0, 0, 437, 440, 1, 0, 0, 0, 438, 436, 1, 0, 0, 0, 438, 439, 1, 0, 0, 0, 439, 441, 1, 0, 0, 0, 440, 438, 1, 0, 0, 0, 441, 463, 5, 34, 0, 0, 442, 443, 5, 34, 0, 0, 443, 444, 5, 34, 0, 0, 444, 445, 5, 34, 0, 0, 445, 449, 1, 0, 0, 0, 446, 448, 8, 1, 0, 0, 447, 446, 1, 0, 0, 0, 448, 451, 1, 0, 0, 0, 449, 450, 1, 0, 0, 0, 449, 447, 1, 0, 0, 0, 450, 452, 1, 0, 0, 0, 451, 449, 1, 0, 0, 0, 452, 453, 5, 34, 0, 0, 453, 454, 5, 34, 0, 0, 454, 455, 5, 34, 0, 0, 455, 457, 1, 0, 0, 0, 456, 458, 5, 34, 0, 0, 457, 456, 1, 0, 0, 0, 457, 458, 1, 0, 0, 0, 458, 460, 1, 0, 0, 0, 459, 461, 5, 34, 0, 0, 460, 459, 1, 0, 0, 0, 460, 461, 1, 0, 0, 0, 461, 463, 1, 0, 0, 0, 462, 433, 1, 0, 0, 0, 462, 442, 1, 0, 0, 0, 463, 71, 1, 0, 0, 0, 464, 466, 3, 60, 28, 0, 465, 464, 1, 0, 0, 0, 466, 467, 1, 0, 0, 0, 467, 465, 1, 0, 0, 0, 467, 468, 1, 0, 0, 0, 468, 73, 1, 0, 0, 0, 469, 471, 3, 60, 28, 0, 470, 469, 1, 0, 0, 0, 471, 472, 1, 0, 0, 0, 472, 470, 1, 0, 0, 0, 472, 473, 1, 0, 0, 0, 473, 474, 1, 0, 0, 0, 474, 478, 3, 88, 42, 0, 475, 477, 3, 60, 28, 0, 476, 475, 1, 0, 0, 0, 477, 480, 1, 0, 0, 0, 478, 476, 1, 0, 0, 0, 478, 479, 1, 0, 0, 0, 479, 512, 1, 0, 0, 0, 480, 478, 1, 0, 0, 0, 481, 483, 3, 88, 42, 0, 482, 484, 3, 60, 28, 0, 483, 482, 1, 0, 0, 0, 484, 485, 1, 0, 0, 0, 485, 483, 1, 0, 0, 0, 485, 486, 1, 0, 0, 0, 486, 512, 1, 0, 0, 0, 487, 489, 3, 60, 28, 0, 488, 487, 1, 0, 0, 0, 489, 490, 1, 0, 0, 0, 490, 488, 1, 0, 0, 0, 490, 491, 1, 0, 0, 0, 491, 499, 1, 0, 0, 0, 492, 496, 3, 88, 42, 0, 493, 495, 3, 60, 28, 0, 494, 493, 1, 0, 0, 0, 495, 498, 1, 0, 0, 0, 496, 494, 1, 0, 0, 0, 496, 497, 1, 0, 0, 0, 497, 500, 1, 0, 0, 0, 498, 496, 1, 0, 0, 0, 499, 492, 1, 0, 0, 0, 499, 500, 1, 0, 0, 0, 500, 501, 1, 0, 0, 0, 501, 502, 3, 68, 32, 0, 502, 512, 1, 0, 0, 0, 503, 505, 3, 88, 42, 0, 504, 506, 3, 60, 28, 0, 505, 504, 1, 0, 0, 0, 506, 507, 1, 0, 0, 0, 507, 505, 1, 0, 0, 0, 507, 508, 1, 0, 0, 0, 508, 509, 1, 0, 0, 0, 509, 510, 3, 68, 32, 0, 510, 512, 1, 0, 0, 0, 511, 470, 1, 0, 0, 0, 511, 481, 1, 0, 0, 0, 511, 488, 1, 0, 0, 0, 511, 503, 1, 0, 0, 0, 512, 75, 1, 0, 0, 0, 513, 514, 5, 98, 0, 0, 514, 515, 5, 121, 0, 0, 515, 77, 1, 0, 0, 0, 516, 517, 5, 97, 0, 0, 517, 518, 5, 110, 0, 0, 518, 519, 5, 100, 0, 0, 519, 79, 1, 0, 0, 0, 520, 521, 5, 97, 0, 0, 521, 522, 5, 115, 0, 0, 522, 523, 5, 99, 0, 0, 523, 81, 1, 0, 0, 0, 524, 525, 5, 61, 0, 0, 525, 83, 1, 0, 0, 0, 526, 527, 5, 44, 0, 0, 527, 85, 1, 0, 0, 0, 528, 529, 5, 100, 0, 0, 529, 530, 5, 101, 0, 0, 530, 531, 5, 115, 0, 0, 531, 532, 5, 99, 0, 0, 532, 87, 1, 0, 0, 0, 533, 534, 5, 46, 0, 0, 534, 89, 1, 0, 0, 0, 535, 536, 5, 102, 0, 0, 536, 537, 5, 97, 0, 0, 537, 538, 5, 108, 0, 0, 538, 539, 5, 115, 0, 0, 539, 540, 5, 101, 0, 0, 540, 91, 1, 0, 0, 0, 541, 542, 5, 102, 0, 0, 542, 543, 5, 105, 0, 0, 543, 544, 5, 114, 0, 0, 544, 545, 5, 115, 0, 0, 545, 546, 5, 116, 0, 0, 546, 93, 1, 0, 0, 0, 547, 548, 5, 108, 0, 0, 548, 549, 5, 97, 0, 0, 549, 550, 5, 115, 0, 0, 550, 551, 5, 116, 0, 0, 551, 95, 1, 0, 0, 0, 552, 553, 5, 40, 0, 0, 553, 97, 1, 0, 0, 0, 554, 555, 5, 105, 0, 0, 555, 556, 5, 110, 0, 0, 556, 99, 1, 0, 0, 0, 557, 558, 5, 105, 0, 0, 558, 559, 5, 115, 0, 0, 559, 101, 1, 0, 0, 0, 560, 561, 5, 108, 0, 0, 561, 562, 5, 105, 0, 0, 562, 563, 5, 107, 0, 0, 563, 564, 5, 101, 0, 0, 564, 103, 1, 0, 0, 0, 565, 566, 5, 110, 0, 0, 566, 567, 5, 111, 0, 0, 567, 568, 5, 116, 0, 0, 568, 105, 1, 0, 0, 0, 569, 570, 5, 110, 0, 0, 570, 571, 5, 117, 0, 0, 571, 572, 5, 108, 0, 0, 572, 573, 5, 108, 0, 0, 573, 107, 1, 0, 0, 0, 574, 575, 5, 110, 0, 0, 575, 576, 5, 117, 0, 0, 576, 577, 5, 108, 0, 0, 577, 578, 5, 108, 0, 0, 578, 579, 5, 115, 0, 0, 579, 109, 1, 0, 0, 0, 580, 581, 5, 111, 0, 0, 581, 582, 5, 114, 0, 0, 582, 111, 1, 0, 0, 0, 583, 584, 5, 63, 0, 0, 584, 113, 1, 0, 0, 0, 585, 586, 5, 114, 0, 0, 586, 587, 5, 108, 0, 0, 587, 588, 5, 105, 0, 0, 588, 589, 5, 107, 0, 0, 589, 590, 5, 101, 0, 0, 590, 115, 1, 0, 0, 0, 591, 592, 5, 41, 0, 0, 592, 117, 1, 0, 0, 0, 593, 594, 5, 116, 0, 0, 594, 595, 5, 114, 0, 0, 595, 596, 5, 117, 0, 0, 596, 597, 5, 101, 0, 0, 597, 119, 1, 0, 0, 0, 598, 599, 5, 105, 0, 0, 599, 600, 5, 110, 0, 0, 600, 601, 5, 102, 0, 0, 601, 602, 5, 111, 0, 0, 602, 121, 1, 0, 0, 0, 603, 604, 5, 102, 0, 0, 604, 605, 5, 117, 0, 0, 605, 606, 5, 110, 0, 0, 606, 607, 5, 99, 0, 0, 607, 608, 5, 116, 0, 0, 608, 609, 5, 105, 0, 0, 609, 610, 5, 111, 0, 0, 610, 611, 5, 110, 0, 0, 611, 612, 5, 115, 0, 0, 612, 123, 1, 0, 0, 0, 613, 614, 5, 61, 0, 0, 614, 615, 5, 61, 0, 0, 615, 125, 1, 0, 0, 0, 616, 617, 5, 33, 0, 0, 617, 618, 5, 61, 0, 0, 618, 127, 1, 0, 0, 0, 619, 620, 5, 60, 0, 0, 620, 129, 1, 0, 0, 0, 621, 622, 5, 60, 0, 0, 622, 623, 5, 61, 0, 0, 623, 131, 1, 0, 0, 0, 624, 625, 5, 62, 0, 0, 625, 133, 1, 0, 0, 0, 626, 627, 5, 62, 0, 0, 627, 628, 5, 61, 0, 0, 628, 135, 1, 0, 0, 0, 629, 630, 5, 43, 0, 0, 630, 137, 1, 0, 0, 0, 631, 632, 5, 45, 0, 0, 632, 139, 1, 0, 0, 0, 633, 634, 5, 42, 0, 0, 634, 141, 1, 0, 0, 0, 635, 636, 5, 47, 0, 0, 636, 143, 1, 0, 0, 0, 637, 638, 5, 37, 0, 0, 638, 145, 1, 0, 0, 0, 639, 640, 5, 91, 0, 0, 640, 641, 1, 0, 0, 0, 641, 642, 6, 71, 0, 0, 642, 643, 6, 71, 0, 0, 643, 147, 1, 0, 0, 0, 644, 645, 5, 93, 0, 0, 645, 646, 1, 0, 0, 0, 646, 647, 6, 72, 7, 0, 647, 648, 6, 72, 7, 0, 648, 149, 1, 0, 0, 0, 649, 655, 3, 62, 29, 0, 650, 654, 3, 62, 29, 0, 651, 654, 3, 60, 28, 0, 652, 654, 5, 95, 0, 0, 653, 650, 1, 0, 0, 0, 653, 651, 1, 0, 0, 0, 653, 652, 1, 0, 0, 0, 654, 657, 1, 0, 0, 0, 655, 653, 1, 0, 0, 0, 655, 656, 1, 0, 0, 0, 656, 667, 1, 0, 0, 0, 657, 655, 1, 0, 0, 0, 658, 662, 7, 9, 0, 0, 659, 663, 3, 62, 29, 0, 660, 663, 3, 60, 28, 0, 661, 663, 5, 95, 0, 0, 662, 659, 1, 0, 0, 0, 662, 660, 1, 0, 0, 0, 662, 661, 1, 0, 0, 0, 663, 664, 1, 0, 0, 0, 664, 662, 1, 0, 0, 0, 664, 665, 1, 0, 0, 0, 665, 667, 1, 0, 0, 0, 666, 649, 1, 0, 0, 0, 666, 658, 1, 0, 0, 0, 667, 151, 1, 0, 0, 0, 668, 674, 5, 96, 0, 0, 669, 673, 8, 10, 0, 0, 670, 671, 5, 96, 0, 0, 671, 673, 5, 96, 0, 0, 672, 669, 1, 0, 0, 0, 672, 670, 1, 0, 0, 0, 673, 676, 1, 0, 0, 0, 674, 672, 1, 0, 0, 0, 674, 675, 1, 0, 0, 0, 675, 677, 1, 0, 0, 0, 676, 674, 1, 0, 0, 0, 677, 678, 5, 96, 0, 0, 678, 153, 1, 0, 0, 0, 679, 680, 3, 42, 19, 0, 680, 681, 1, 0, 0, 0, 681, 682, 6, 75, 3, 0, 682, 155, 1, 0, 0, 0, 683, 684, 3, 44, 20, 0, 684, 685, 1, 0, 0, 0, 685, 686, 6, 76, 3, 0, 686, 157, 1, 0, 0, 0, 687, 688, 3, 46, 21, 0, 688, 689, 1, 0, 0, 0, 689, 690, 6, 77, 3, 0, 690, 159, 1, 0, 0, 0, 691, 692, 5, 124, 0, 0, 692, 693, 1, 0, 0, 0, 693, 694, 6, 78, 6, 0, 694, 695, 6, 78, 7, 0, 695, 161, 1, 0, 0, 0, 696, 697, 5, 91, 0, 0, 697, 698, 1, 0, 0, 0, 698, 699, 6, 79, 4, 0, 699, 700, 6, 79, 1, 0, 700, 701, 6, 79, 1, 0, 701, 163, 1, 0, 0, 0, 702, 703, 5, 93, 0, 0, 703, 704, 1, 0, 0, 0, 704, 705, 6, 80, 7, 0, 705, 706, 6, 80, 7, 0, 706, 707, 6, 80, 8, 0, 707, 165, 1, 0, 0, 0, 708, 709, 5, 44, 0, 0, 709, 710, 1, 0, 0, 0, 710, 711, 6, 81, 9, 0, 711, 167, 1, 0, 0, 0, 712, 713, 5, 61, 0, 0, 713, 714, 1, 0, 0, 0, 714, 715, 6, 82, 10, 0, 715, 169, 1, 0, 0, 0, 716, 717, 5, 97, 0, 0, 717, 718, 5, 115, 0, 0, 718, 171, 1, 0, 0, 0, 719, 720, 5, 109, 0, 0, 720, 721, 5, 101, 0, 0, 721, 722, 5, 116, 0, 0, 722, 723, 5, 97, 0, 0, 723, 724, 5, 100, 0, 0, 724, 725, 5, 97, 0, 0, 725, 726, 5, 116, 0, 0, 726, 727, 5, 97, 0, 0, 727, 173, 1, 0, 0, 0, 728, 729, 5, 111, 0, 0, 729, 730, 5, 110, 0, 0, 730, 175, 1, 0, 0, 0, 731, 732, 5, 119, 0, 0, 732, 733, 5, 105, 0, 0, 733, 734, 5, 116, 0, 0, 734, 735, 5, 104, 0, 0, 735, 177, 1, 0, 0, 0, 736, 738, 3, 180, 88, 0, 737, 736, 1, 0, 0, 0, 738, 739, 1, 0, 0, 0, 739, 737, 1, 0, 0, 0, 739, 740, 1, 0, 0, 0, 740, 179, 1, 0, 0, 0, 741, 743, 8, 11, 0, 0, 742, 741, 1, 0, 0, 0, 743, 744, 1, 0, 0, 0, 744, 742, 1, 0, 0, 0, 744, 745, 1, 0, 0, 0, 745, 749, 1, 0, 0, 0, 746, 747, 5, 47, 0, 0, 747, 749, 8, 12, 0, 0, 748, 742, 1, 0, 0, 0, 748, 746, 1, 0, 0, 0, 749, 181, 1, 0, 0, 0, 750, 751, 3, 152, 74, 0, 751, 183, 1, 0, 0, 0, 752, 753, 3, 42, 19, 0, 753, 754, 1, 0, 0, 0, 754, 755, 6, 90, 3, 0, 755, 185, 1, 0, 0, 0, 756, 757, 3, 44, 20, 0, 757, 758, 1, 0, 0, 0, 758, 759, 6, 91, 3, 0, 759, 187, 1, 0, 0, 0, 760, 761, 3, 46, 21, 0, 761, 762, 1, 0, 0, 0, 762, 763, 6, 92, 3, 0, 763, 189, 1, 0, 0, 0, 38, 0, 1, 2, 3, 346, 356, 360, 363, 372, 374, 385, 426, 431, 436, 438, 449, 457, 460, 462, 467, 472, 478, 485, 490, 496, 499, 507, 511, 653, 655, 662, 664, 666, 672, 674, 739, 744, 748, 11, 5, 2, 0, 5, 3, 0, 5, 1, 0, 0, 1, 0, 7, 65, 0, 5, 0, 0, 7, 26, 0, 4, 0, 0, 7, 66, 0, 7, 34, 0, 7, 33, 0] \ No newline at end of file diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java new file mode 100644 index 0000000000000..be46b6c6e1797 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java @@ -0,0 +1,638 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; +import org.antlr.v4.runtime.Lexer; +import org.antlr.v4.runtime.CharStream; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenStream; +import org.antlr.v4.runtime.*; +import org.antlr.v4.runtime.atn.*; +import org.antlr.v4.runtime.dfa.DFA; +import org.antlr.v4.runtime.misc.*; + +@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast", "CheckReturnValue"}) +public class EsqlBaseLexer extends Lexer { + static { RuntimeMetaData.checkVersion("4.11.1", RuntimeMetaData.VERSION); } + + protected static final DFA[] _decisionToDFA; + protected static final PredictionContextCache _sharedContextCache = + new PredictionContextCache(); + public static final int + DISSECT=1, DROP=2, ENRICH=3, EVAL=4, EXPLAIN=5, FROM=6, GROK=7, INLINESTATS=8, + KEEP=9, LIMIT=10, MV_EXPAND=11, PROJECT=12, RENAME=13, ROW=14, SHOW=15, + SORT=16, STATS=17, WHERE=18, UNKNOWN_CMD=19, LINE_COMMENT=20, MULTILINE_COMMENT=21, + WS=22, EXPLAIN_WS=23, EXPLAIN_LINE_COMMENT=24, EXPLAIN_MULTILINE_COMMENT=25, + PIPE=26, STRING=27, INTEGER_LITERAL=28, DECIMAL_LITERAL=29, BY=30, AND=31, + ASC=32, ASSIGN=33, COMMA=34, DESC=35, DOT=36, FALSE=37, FIRST=38, LAST=39, + LP=40, IN=41, IS=42, LIKE=43, NOT=44, NULL=45, NULLS=46, OR=47, PARAM=48, + RLIKE=49, RP=50, TRUE=51, INFO=52, FUNCTIONS=53, EQ=54, NEQ=55, LT=56, + LTE=57, GT=58, GTE=59, PLUS=60, MINUS=61, ASTERISK=62, SLASH=63, PERCENT=64, + OPENING_BRACKET=65, CLOSING_BRACKET=66, UNQUOTED_IDENTIFIER=67, QUOTED_IDENTIFIER=68, + EXPR_LINE_COMMENT=69, EXPR_MULTILINE_COMMENT=70, EXPR_WS=71, AS=72, METADATA=73, + ON=74, WITH=75, SRC_UNQUOTED_IDENTIFIER=76, SRC_QUOTED_IDENTIFIER=77, + SRC_LINE_COMMENT=78, SRC_MULTILINE_COMMENT=79, SRC_WS=80, EXPLAIN_PIPE=81; + public static final int + EXPLAIN_MODE=1, EXPRESSION=2, SOURCE_IDENTIFIERS=3; + public static String[] channelNames = { + "DEFAULT_TOKEN_CHANNEL", "HIDDEN" + }; + + public static String[] modeNames = { + "DEFAULT_MODE", "EXPLAIN_MODE", "EXPRESSION", "SOURCE_IDENTIFIERS" + }; + + private static String[] makeRuleNames() { + return new String[] { + "DISSECT", "DROP", "ENRICH", "EVAL", "EXPLAIN", "FROM", "GROK", "INLINESTATS", + "KEEP", "LIMIT", "MV_EXPAND", "PROJECT", "RENAME", "ROW", "SHOW", "SORT", + "STATS", "WHERE", "UNKNOWN_CMD", "LINE_COMMENT", "MULTILINE_COMMENT", + "WS", "EXPLAIN_OPENING_BRACKET", "EXPLAIN_PIPE", "EXPLAIN_WS", "EXPLAIN_LINE_COMMENT", + "EXPLAIN_MULTILINE_COMMENT", "PIPE", "DIGIT", "LETTER", "ESCAPE_SEQUENCE", + "UNESCAPED_CHARS", "EXPONENT", "STRING", "INTEGER_LITERAL", "DECIMAL_LITERAL", + "BY", "AND", "ASC", "ASSIGN", "COMMA", "DESC", "DOT", "FALSE", "FIRST", + "LAST", "LP", "IN", "IS", "LIKE", "NOT", "NULL", "NULLS", "OR", "PARAM", + "RLIKE", "RP", "TRUE", "INFO", "FUNCTIONS", "EQ", "NEQ", "LT", "LTE", + "GT", "GTE", "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "OPENING_BRACKET", + "CLOSING_BRACKET", "UNQUOTED_IDENTIFIER", "QUOTED_IDENTIFIER", "EXPR_LINE_COMMENT", + "EXPR_MULTILINE_COMMENT", "EXPR_WS", "SRC_PIPE", "SRC_OPENING_BRACKET", + "SRC_CLOSING_BRACKET", "SRC_COMMA", "SRC_ASSIGN", "AS", "METADATA", "ON", + "WITH", "SRC_UNQUOTED_IDENTIFIER", "SRC_UNQUOTED_IDENTIFIER_PART", "SRC_QUOTED_IDENTIFIER", + "SRC_LINE_COMMENT", "SRC_MULTILINE_COMMENT", "SRC_WS" + }; + } + public static final String[] ruleNames = makeRuleNames(); + + private static String[] makeLiteralNames() { + return new String[] { + null, "'dissect'", "'drop'", "'enrich'", "'eval'", "'explain'", "'from'", + "'grok'", "'inlinestats'", "'keep'", "'limit'", "'mv_expand'", "'project'", + "'rename'", "'row'", "'show'", "'sort'", "'stats'", "'where'", null, + null, null, null, null, null, null, null, null, null, null, "'by'", "'and'", + "'asc'", null, null, "'desc'", "'.'", "'false'", "'first'", "'last'", + "'('", "'in'", "'is'", "'like'", "'not'", "'null'", "'nulls'", "'or'", + "'?'", "'rlike'", "')'", "'true'", "'info'", "'functions'", "'=='", "'!='", + "'<'", "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", null, + "']'", null, null, null, null, null, "'as'", "'metadata'", "'on'", "'with'" + }; + } + private static final String[] _LITERAL_NAMES = makeLiteralNames(); + private static String[] makeSymbolicNames() { + return new String[] { + null, "DISSECT", "DROP", "ENRICH", "EVAL", "EXPLAIN", "FROM", "GROK", + "INLINESTATS", "KEEP", "LIMIT", "MV_EXPAND", "PROJECT", "RENAME", "ROW", + "SHOW", "SORT", "STATS", "WHERE", "UNKNOWN_CMD", "LINE_COMMENT", "MULTILINE_COMMENT", + "WS", "EXPLAIN_WS", "EXPLAIN_LINE_COMMENT", "EXPLAIN_MULTILINE_COMMENT", + "PIPE", "STRING", "INTEGER_LITERAL", "DECIMAL_LITERAL", "BY", "AND", + "ASC", "ASSIGN", "COMMA", "DESC", "DOT", "FALSE", "FIRST", "LAST", "LP", + "IN", "IS", "LIKE", "NOT", "NULL", "NULLS", "OR", "PARAM", "RLIKE", "RP", + "TRUE", "INFO", "FUNCTIONS", "EQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", + "MINUS", "ASTERISK", "SLASH", "PERCENT", "OPENING_BRACKET", "CLOSING_BRACKET", + "UNQUOTED_IDENTIFIER", "QUOTED_IDENTIFIER", "EXPR_LINE_COMMENT", "EXPR_MULTILINE_COMMENT", + "EXPR_WS", "AS", "METADATA", "ON", "WITH", "SRC_UNQUOTED_IDENTIFIER", + "SRC_QUOTED_IDENTIFIER", "SRC_LINE_COMMENT", "SRC_MULTILINE_COMMENT", + "SRC_WS", "EXPLAIN_PIPE" + }; + } + private static final String[] _SYMBOLIC_NAMES = makeSymbolicNames(); + public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); + + /** + * @deprecated Use {@link #VOCABULARY} instead. + */ + @Deprecated + public static final String[] tokenNames; + static { + tokenNames = new String[_SYMBOLIC_NAMES.length]; + for (int i = 0; i < tokenNames.length; i++) { + tokenNames[i] = VOCABULARY.getLiteralName(i); + if (tokenNames[i] == null) { + tokenNames[i] = VOCABULARY.getSymbolicName(i); + } + + if (tokenNames[i] == null) { + tokenNames[i] = ""; + } + } + } + + @Override + @Deprecated + public String[] getTokenNames() { + return tokenNames; + } + + @Override + + public Vocabulary getVocabulary() { + return VOCABULARY; + } + + + public EsqlBaseLexer(CharStream input) { + super(input); + _interp = new LexerATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache); + } + + @Override + public String getGrammarFileName() { return "EsqlBaseLexer.g4"; } + + @Override + public String[] getRuleNames() { return ruleNames; } + + @Override + public String getSerializedATN() { return _serializedATN; } + + @Override + public String[] getChannelNames() { return channelNames; } + + @Override + public String[] getModeNames() { return modeNames; } + + @Override + public ATN getATN() { return _ATN; } + + public static final String _serializedATN = + "\u0004\u0000Q\u02fc\u0006\uffff\uffff\u0006\uffff\uffff\u0006\uffff\uffff"+ + "\u0006\uffff\uffff\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001\u0002"+ + "\u0002\u0007\u0002\u0002\u0003\u0007\u0003\u0002\u0004\u0007\u0004\u0002"+ + "\u0005\u0007\u0005\u0002\u0006\u0007\u0006\u0002\u0007\u0007\u0007\u0002"+ + "\b\u0007\b\u0002\t\u0007\t\u0002\n\u0007\n\u0002\u000b\u0007\u000b\u0002"+ + "\f\u0007\f\u0002\r\u0007\r\u0002\u000e\u0007\u000e\u0002\u000f\u0007\u000f"+ + "\u0002\u0010\u0007\u0010\u0002\u0011\u0007\u0011\u0002\u0012\u0007\u0012"+ + "\u0002\u0013\u0007\u0013\u0002\u0014\u0007\u0014\u0002\u0015\u0007\u0015"+ + "\u0002\u0016\u0007\u0016\u0002\u0017\u0007\u0017\u0002\u0018\u0007\u0018"+ + "\u0002\u0019\u0007\u0019\u0002\u001a\u0007\u001a\u0002\u001b\u0007\u001b"+ + "\u0002\u001c\u0007\u001c\u0002\u001d\u0007\u001d\u0002\u001e\u0007\u001e"+ + "\u0002\u001f\u0007\u001f\u0002 \u0007 \u0002!\u0007!\u0002\"\u0007\"\u0002"+ + "#\u0007#\u0002$\u0007$\u0002%\u0007%\u0002&\u0007&\u0002\'\u0007\'\u0002"+ + "(\u0007(\u0002)\u0007)\u0002*\u0007*\u0002+\u0007+\u0002,\u0007,\u0002"+ + "-\u0007-\u0002.\u0007.\u0002/\u0007/\u00020\u00070\u00021\u00071\u0002"+ + "2\u00072\u00023\u00073\u00024\u00074\u00025\u00075\u00026\u00076\u0002"+ + "7\u00077\u00028\u00078\u00029\u00079\u0002:\u0007:\u0002;\u0007;\u0002"+ + "<\u0007<\u0002=\u0007=\u0002>\u0007>\u0002?\u0007?\u0002@\u0007@\u0002"+ + "A\u0007A\u0002B\u0007B\u0002C\u0007C\u0002D\u0007D\u0002E\u0007E\u0002"+ + "F\u0007F\u0002G\u0007G\u0002H\u0007H\u0002I\u0007I\u0002J\u0007J\u0002"+ + "K\u0007K\u0002L\u0007L\u0002M\u0007M\u0002N\u0007N\u0002O\u0007O\u0002"+ + "P\u0007P\u0002Q\u0007Q\u0002R\u0007R\u0002S\u0007S\u0002T\u0007T\u0002"+ + "U\u0007U\u0002V\u0007V\u0002W\u0007W\u0002X\u0007X\u0002Y\u0007Y\u0002"+ + "Z\u0007Z\u0002[\u0007[\u0002\\\u0007\\\u0001\u0000\u0001\u0000\u0001\u0000"+ + "\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0000"+ + "\u0001\u0000\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0001"+ + "\u0001\u0001\u0001\u0001\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0002"+ + "\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0003"+ + "\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003"+ + "\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0004"+ + "\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0004\u0001\u0005\u0001\u0005"+ + "\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0006"+ + "\u0001\u0006\u0001\u0006\u0001\u0006\u0001\u0006\u0001\u0006\u0001\u0006"+ + "\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007"+ + "\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007\u0001\u0007"+ + "\u0001\u0007\u0001\u0007\u0001\b\u0001\b\u0001\b\u0001\b\u0001\b\u0001"+ + "\b\u0001\b\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001"+ + "\t\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0001\n\u0001"+ + "\n\u0001\n\u0001\n\u0001\n\u0001\u000b\u0001\u000b\u0001\u000b\u0001\u000b"+ + "\u0001\u000b\u0001\u000b\u0001\u000b\u0001\u000b\u0001\u000b\u0001\u000b"+ + "\u0001\f\u0001\f\u0001\f\u0001\f\u0001\f\u0001\f\u0001\f\u0001\f\u0001"+ + "\f\u0001\r\u0001\r\u0001\r\u0001\r\u0001\r\u0001\r\u0001\u000e\u0001\u000e"+ + "\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000f"+ + "\u0001\u000f\u0001\u000f\u0001\u000f\u0001\u000f\u0001\u000f\u0001\u000f"+ + "\u0001\u0010\u0001\u0010\u0001\u0010\u0001\u0010\u0001\u0010\u0001\u0010"+ + "\u0001\u0010\u0001\u0010\u0001\u0011\u0001\u0011\u0001\u0011\u0001\u0011"+ + "\u0001\u0011\u0001\u0011\u0001\u0011\u0001\u0011\u0001\u0012\u0004\u0012"+ + "\u0159\b\u0012\u000b\u0012\f\u0012\u015a\u0001\u0012\u0001\u0012\u0001"+ + "\u0013\u0001\u0013\u0001\u0013\u0001\u0013\u0005\u0013\u0163\b\u0013\n"+ + "\u0013\f\u0013\u0166\t\u0013\u0001\u0013\u0003\u0013\u0169\b\u0013\u0001"+ + "\u0013\u0003\u0013\u016c\b\u0013\u0001\u0013\u0001\u0013\u0001\u0014\u0001"+ + "\u0014\u0001\u0014\u0001\u0014\u0001\u0014\u0005\u0014\u0175\b\u0014\n"+ + "\u0014\f\u0014\u0178\t\u0014\u0001\u0014\u0001\u0014\u0001\u0014\u0001"+ + "\u0014\u0001\u0014\u0001\u0015\u0004\u0015\u0180\b\u0015\u000b\u0015\f"+ + "\u0015\u0181\u0001\u0015\u0001\u0015\u0001\u0016\u0001\u0016\u0001\u0016"+ + "\u0001\u0016\u0001\u0016\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0017"+ + "\u0001\u0017\u0001\u0018\u0001\u0018\u0001\u0018\u0001\u0018\u0001\u0019"+ + "\u0001\u0019\u0001\u0019\u0001\u0019\u0001\u001a\u0001\u001a\u0001\u001a"+ + "\u0001\u001a\u0001\u001b\u0001\u001b\u0001\u001b\u0001\u001b\u0001\u001c"+ + "\u0001\u001c\u0001\u001d\u0001\u001d\u0001\u001e\u0001\u001e\u0001\u001e"+ + "\u0001\u001f\u0001\u001f\u0001 \u0001 \u0003 \u01ab\b \u0001 \u0004 \u01ae"+ + "\b \u000b \f \u01af\u0001!\u0001!\u0001!\u0005!\u01b5\b!\n!\f!\u01b8\t"+ + "!\u0001!\u0001!\u0001!\u0001!\u0001!\u0001!\u0005!\u01c0\b!\n!\f!\u01c3"+ + "\t!\u0001!\u0001!\u0001!\u0001!\u0001!\u0003!\u01ca\b!\u0001!\u0003!\u01cd"+ + "\b!\u0003!\u01cf\b!\u0001\"\u0004\"\u01d2\b\"\u000b\"\f\"\u01d3\u0001"+ + "#\u0004#\u01d7\b#\u000b#\f#\u01d8\u0001#\u0001#\u0005#\u01dd\b#\n#\f#"+ + "\u01e0\t#\u0001#\u0001#\u0004#\u01e4\b#\u000b#\f#\u01e5\u0001#\u0004#"+ + "\u01e9\b#\u000b#\f#\u01ea\u0001#\u0001#\u0005#\u01ef\b#\n#\f#\u01f2\t"+ + "#\u0003#\u01f4\b#\u0001#\u0001#\u0001#\u0001#\u0004#\u01fa\b#\u000b#\f"+ + "#\u01fb\u0001#\u0001#\u0003#\u0200\b#\u0001$\u0001$\u0001$\u0001%\u0001"+ + "%\u0001%\u0001%\u0001&\u0001&\u0001&\u0001&\u0001\'\u0001\'\u0001(\u0001"+ + "(\u0001)\u0001)\u0001)\u0001)\u0001)\u0001*\u0001*\u0001+\u0001+\u0001"+ + "+\u0001+\u0001+\u0001+\u0001,\u0001,\u0001,\u0001,\u0001,\u0001,\u0001"+ + "-\u0001-\u0001-\u0001-\u0001-\u0001.\u0001.\u0001/\u0001/\u0001/\u0001"+ + "0\u00010\u00010\u00011\u00011\u00011\u00011\u00011\u00012\u00012\u0001"+ + "2\u00012\u00013\u00013\u00013\u00013\u00013\u00014\u00014\u00014\u0001"+ + "4\u00014\u00014\u00015\u00015\u00015\u00016\u00016\u00017\u00017\u0001"+ + "7\u00017\u00017\u00017\u00018\u00018\u00019\u00019\u00019\u00019\u0001"+ + "9\u0001:\u0001:\u0001:\u0001:\u0001:\u0001;\u0001;\u0001;\u0001;\u0001"+ + ";\u0001;\u0001;\u0001;\u0001;\u0001;\u0001<\u0001<\u0001<\u0001=\u0001"+ + "=\u0001=\u0001>\u0001>\u0001?\u0001?\u0001?\u0001@\u0001@\u0001A\u0001"+ + "A\u0001A\u0001B\u0001B\u0001C\u0001C\u0001D\u0001D\u0001E\u0001E\u0001"+ + "F\u0001F\u0001G\u0001G\u0001G\u0001G\u0001G\u0001H\u0001H\u0001H\u0001"+ + "H\u0001H\u0001I\u0001I\u0001I\u0001I\u0005I\u028e\bI\nI\fI\u0291\tI\u0001"+ + "I\u0001I\u0001I\u0001I\u0004I\u0297\bI\u000bI\fI\u0298\u0003I\u029b\b"+ + "I\u0001J\u0001J\u0001J\u0001J\u0005J\u02a1\bJ\nJ\fJ\u02a4\tJ\u0001J\u0001"+ + "J\u0001K\u0001K\u0001K\u0001K\u0001L\u0001L\u0001L\u0001L\u0001M\u0001"+ + "M\u0001M\u0001M\u0001N\u0001N\u0001N\u0001N\u0001N\u0001O\u0001O\u0001"+ + "O\u0001O\u0001O\u0001O\u0001P\u0001P\u0001P\u0001P\u0001P\u0001P\u0001"+ + "Q\u0001Q\u0001Q\u0001Q\u0001R\u0001R\u0001R\u0001R\u0001S\u0001S\u0001"+ + "S\u0001T\u0001T\u0001T\u0001T\u0001T\u0001T\u0001T\u0001T\u0001T\u0001"+ + "U\u0001U\u0001U\u0001V\u0001V\u0001V\u0001V\u0001V\u0001W\u0004W\u02e2"+ + "\bW\u000bW\fW\u02e3\u0001X\u0004X\u02e7\bX\u000bX\fX\u02e8\u0001X\u0001"+ + "X\u0003X\u02ed\bX\u0001Y\u0001Y\u0001Z\u0001Z\u0001Z\u0001Z\u0001[\u0001"+ + "[\u0001[\u0001[\u0001\\\u0001\\\u0001\\\u0001\\\u0002\u0176\u01c1\u0000"+ + "]\u0004\u0001\u0006\u0002\b\u0003\n\u0004\f\u0005\u000e\u0006\u0010\u0007"+ + "\u0012\b\u0014\t\u0016\n\u0018\u000b\u001a\f\u001c\r\u001e\u000e \u000f"+ + "\"\u0010$\u0011&\u0012(\u0013*\u0014,\u0015.\u00160\u00002Q4\u00176\u0018"+ + "8\u0019:\u001a<\u0000>\u0000@\u0000B\u0000D\u0000F\u001bH\u001cJ\u001d"+ + "L\u001eN\u001fP R!T\"V#X$Z%\\&^\'`(b)d*f+h,j-l.n/p0r1t2v3x4z5|6~7\u0080"+ + "8\u00829\u0084:\u0086;\u0088<\u008a=\u008c>\u008e?\u0090@\u0092A\u0094"+ + "B\u0096C\u0098D\u009aE\u009cF\u009eG\u00a0\u0000\u00a2\u0000\u00a4\u0000"+ + "\u00a6\u0000\u00a8\u0000\u00aaH\u00acI\u00aeJ\u00b0K\u00b2L\u00b4\u0000"+ + "\u00b6M\u00b8N\u00baO\u00bcP\u0004\u0000\u0001\u0002\u0003\r\u0006\u0000"+ + "\t\n\r\r //[[]]\u0002\u0000\n\n\r\r\u0003\u0000\t\n\r\r \u0001\u0000"+ + "09\u0002\u0000AZaz\u0005\u0000\"\"\\\\nnrrtt\u0004\u0000\n\n\r\r\"\"\\"+ + "\\\u0002\u0000EEee\u0002\u0000++--\u0002\u0000@@__\u0001\u0000``\n\u0000"+ + "\t\n\r\r ,,//==[[]]``||\u0002\u0000**//\u0318\u0000\u0004\u0001\u0000"+ + "\u0000\u0000\u0000\u0006\u0001\u0000\u0000\u0000\u0000\b\u0001\u0000\u0000"+ + "\u0000\u0000\n\u0001\u0000\u0000\u0000\u0000\f\u0001\u0000\u0000\u0000"+ + "\u0000\u000e\u0001\u0000\u0000\u0000\u0000\u0010\u0001\u0000\u0000\u0000"+ + "\u0000\u0012\u0001\u0000\u0000\u0000\u0000\u0014\u0001\u0000\u0000\u0000"+ + "\u0000\u0016\u0001\u0000\u0000\u0000\u0000\u0018\u0001\u0000\u0000\u0000"+ + "\u0000\u001a\u0001\u0000\u0000\u0000\u0000\u001c\u0001\u0000\u0000\u0000"+ + "\u0000\u001e\u0001\u0000\u0000\u0000\u0000 \u0001\u0000\u0000\u0000\u0000"+ + "\"\u0001\u0000\u0000\u0000\u0000$\u0001\u0000\u0000\u0000\u0000&\u0001"+ + "\u0000\u0000\u0000\u0000(\u0001\u0000\u0000\u0000\u0000*\u0001\u0000\u0000"+ + "\u0000\u0000,\u0001\u0000\u0000\u0000\u0000.\u0001\u0000\u0000\u0000\u0001"+ + "0\u0001\u0000\u0000\u0000\u00012\u0001\u0000\u0000\u0000\u00014\u0001"+ + "\u0000\u0000\u0000\u00016\u0001\u0000\u0000\u0000\u00018\u0001\u0000\u0000"+ + "\u0000\u0002:\u0001\u0000\u0000\u0000\u0002F\u0001\u0000\u0000\u0000\u0002"+ + "H\u0001\u0000\u0000\u0000\u0002J\u0001\u0000\u0000\u0000\u0002L\u0001"+ + "\u0000\u0000\u0000\u0002N\u0001\u0000\u0000\u0000\u0002P\u0001\u0000\u0000"+ + "\u0000\u0002R\u0001\u0000\u0000\u0000\u0002T\u0001\u0000\u0000\u0000\u0002"+ + "V\u0001\u0000\u0000\u0000\u0002X\u0001\u0000\u0000\u0000\u0002Z\u0001"+ + "\u0000\u0000\u0000\u0002\\\u0001\u0000\u0000\u0000\u0002^\u0001\u0000"+ + "\u0000\u0000\u0002`\u0001\u0000\u0000\u0000\u0002b\u0001\u0000\u0000\u0000"+ + "\u0002d\u0001\u0000\u0000\u0000\u0002f\u0001\u0000\u0000\u0000\u0002h"+ + "\u0001\u0000\u0000\u0000\u0002j\u0001\u0000\u0000\u0000\u0002l\u0001\u0000"+ + "\u0000\u0000\u0002n\u0001\u0000\u0000\u0000\u0002p\u0001\u0000\u0000\u0000"+ + "\u0002r\u0001\u0000\u0000\u0000\u0002t\u0001\u0000\u0000\u0000\u0002v"+ + "\u0001\u0000\u0000\u0000\u0002x\u0001\u0000\u0000\u0000\u0002z\u0001\u0000"+ + "\u0000\u0000\u0002|\u0001\u0000\u0000\u0000\u0002~\u0001\u0000\u0000\u0000"+ + "\u0002\u0080\u0001\u0000\u0000\u0000\u0002\u0082\u0001\u0000\u0000\u0000"+ + "\u0002\u0084\u0001\u0000\u0000\u0000\u0002\u0086\u0001\u0000\u0000\u0000"+ + "\u0002\u0088\u0001\u0000\u0000\u0000\u0002\u008a\u0001\u0000\u0000\u0000"+ + "\u0002\u008c\u0001\u0000\u0000\u0000\u0002\u008e\u0001\u0000\u0000\u0000"+ + "\u0002\u0090\u0001\u0000\u0000\u0000\u0002\u0092\u0001\u0000\u0000\u0000"+ + "\u0002\u0094\u0001\u0000\u0000\u0000\u0002\u0096\u0001\u0000\u0000\u0000"+ + "\u0002\u0098\u0001\u0000\u0000\u0000\u0002\u009a\u0001\u0000\u0000\u0000"+ + "\u0002\u009c\u0001\u0000\u0000\u0000\u0002\u009e\u0001\u0000\u0000\u0000"+ + "\u0003\u00a0\u0001\u0000\u0000\u0000\u0003\u00a2\u0001\u0000\u0000\u0000"+ + "\u0003\u00a4\u0001\u0000\u0000\u0000\u0003\u00a6\u0001\u0000\u0000\u0000"+ + "\u0003\u00a8\u0001\u0000\u0000\u0000\u0003\u00aa\u0001\u0000\u0000\u0000"+ + "\u0003\u00ac\u0001\u0000\u0000\u0000\u0003\u00ae\u0001\u0000\u0000\u0000"+ + "\u0003\u00b0\u0001\u0000\u0000\u0000\u0003\u00b2\u0001\u0000\u0000\u0000"+ + "\u0003\u00b6\u0001\u0000\u0000\u0000\u0003\u00b8\u0001\u0000\u0000\u0000"+ + "\u0003\u00ba\u0001\u0000\u0000\u0000\u0003\u00bc\u0001\u0000\u0000\u0000"+ + "\u0004\u00be\u0001\u0000\u0000\u0000\u0006\u00c8\u0001\u0000\u0000\u0000"+ + "\b\u00cf\u0001\u0000\u0000\u0000\n\u00d8\u0001\u0000\u0000\u0000\f\u00df"+ + "\u0001\u0000\u0000\u0000\u000e\u00e9\u0001\u0000\u0000\u0000\u0010\u00f0"+ + "\u0001\u0000\u0000\u0000\u0012\u00f7\u0001\u0000\u0000\u0000\u0014\u0105"+ + "\u0001\u0000\u0000\u0000\u0016\u010c\u0001\u0000\u0000\u0000\u0018\u0114"+ + "\u0001\u0000\u0000\u0000\u001a\u0120\u0001\u0000\u0000\u0000\u001c\u012a"+ + "\u0001\u0000\u0000\u0000\u001e\u0133\u0001\u0000\u0000\u0000 \u0139\u0001"+ + "\u0000\u0000\u0000\"\u0140\u0001\u0000\u0000\u0000$\u0147\u0001\u0000"+ + "\u0000\u0000&\u014f\u0001\u0000\u0000\u0000(\u0158\u0001\u0000\u0000\u0000"+ + "*\u015e\u0001\u0000\u0000\u0000,\u016f\u0001\u0000\u0000\u0000.\u017f"+ + "\u0001\u0000\u0000\u00000\u0185\u0001\u0000\u0000\u00002\u018a\u0001\u0000"+ + "\u0000\u00004\u018f\u0001\u0000\u0000\u00006\u0193\u0001\u0000\u0000\u0000"+ + "8\u0197\u0001\u0000\u0000\u0000:\u019b\u0001\u0000\u0000\u0000<\u019f"+ + "\u0001\u0000\u0000\u0000>\u01a1\u0001\u0000\u0000\u0000@\u01a3\u0001\u0000"+ + "\u0000\u0000B\u01a6\u0001\u0000\u0000\u0000D\u01a8\u0001\u0000\u0000\u0000"+ + "F\u01ce\u0001\u0000\u0000\u0000H\u01d1\u0001\u0000\u0000\u0000J\u01ff"+ + "\u0001\u0000\u0000\u0000L\u0201\u0001\u0000\u0000\u0000N\u0204\u0001\u0000"+ + "\u0000\u0000P\u0208\u0001\u0000\u0000\u0000R\u020c\u0001\u0000\u0000\u0000"+ + "T\u020e\u0001\u0000\u0000\u0000V\u0210\u0001\u0000\u0000\u0000X\u0215"+ + "\u0001\u0000\u0000\u0000Z\u0217\u0001\u0000\u0000\u0000\\\u021d\u0001"+ + "\u0000\u0000\u0000^\u0223\u0001\u0000\u0000\u0000`\u0228\u0001\u0000\u0000"+ + "\u0000b\u022a\u0001\u0000\u0000\u0000d\u022d\u0001\u0000\u0000\u0000f"+ + "\u0230\u0001\u0000\u0000\u0000h\u0235\u0001\u0000\u0000\u0000j\u0239\u0001"+ + "\u0000\u0000\u0000l\u023e\u0001\u0000\u0000\u0000n\u0244\u0001\u0000\u0000"+ + "\u0000p\u0247\u0001\u0000\u0000\u0000r\u0249\u0001\u0000\u0000\u0000t"+ + "\u024f\u0001\u0000\u0000\u0000v\u0251\u0001\u0000\u0000\u0000x\u0256\u0001"+ + "\u0000\u0000\u0000z\u025b\u0001\u0000\u0000\u0000|\u0265\u0001\u0000\u0000"+ + "\u0000~\u0268\u0001\u0000\u0000\u0000\u0080\u026b\u0001\u0000\u0000\u0000"+ + "\u0082\u026d\u0001\u0000\u0000\u0000\u0084\u0270\u0001\u0000\u0000\u0000"+ + "\u0086\u0272\u0001\u0000\u0000\u0000\u0088\u0275\u0001\u0000\u0000\u0000"+ + "\u008a\u0277\u0001\u0000\u0000\u0000\u008c\u0279\u0001\u0000\u0000\u0000"+ + "\u008e\u027b\u0001\u0000\u0000\u0000\u0090\u027d\u0001\u0000\u0000\u0000"+ + "\u0092\u027f\u0001\u0000\u0000\u0000\u0094\u0284\u0001\u0000\u0000\u0000"+ + "\u0096\u029a\u0001\u0000\u0000\u0000\u0098\u029c\u0001\u0000\u0000\u0000"+ + "\u009a\u02a7\u0001\u0000\u0000\u0000\u009c\u02ab\u0001\u0000\u0000\u0000"+ + "\u009e\u02af\u0001\u0000\u0000\u0000\u00a0\u02b3\u0001\u0000\u0000\u0000"+ + "\u00a2\u02b8\u0001\u0000\u0000\u0000\u00a4\u02be\u0001\u0000\u0000\u0000"+ + "\u00a6\u02c4\u0001\u0000\u0000\u0000\u00a8\u02c8\u0001\u0000\u0000\u0000"+ + "\u00aa\u02cc\u0001\u0000\u0000\u0000\u00ac\u02cf\u0001\u0000\u0000\u0000"+ + "\u00ae\u02d8\u0001\u0000\u0000\u0000\u00b0\u02db\u0001\u0000\u0000\u0000"+ + "\u00b2\u02e1\u0001\u0000\u0000\u0000\u00b4\u02ec\u0001\u0000\u0000\u0000"+ + "\u00b6\u02ee\u0001\u0000\u0000\u0000\u00b8\u02f0\u0001\u0000\u0000\u0000"+ + "\u00ba\u02f4\u0001\u0000\u0000\u0000\u00bc\u02f8\u0001\u0000\u0000\u0000"+ + "\u00be\u00bf\u0005d\u0000\u0000\u00bf\u00c0\u0005i\u0000\u0000\u00c0\u00c1"+ + "\u0005s\u0000\u0000\u00c1\u00c2\u0005s\u0000\u0000\u00c2\u00c3\u0005e"+ + "\u0000\u0000\u00c3\u00c4\u0005c\u0000\u0000\u00c4\u00c5\u0005t\u0000\u0000"+ + "\u00c5\u00c6\u0001\u0000\u0000\u0000\u00c6\u00c7\u0006\u0000\u0000\u0000"+ + "\u00c7\u0005\u0001\u0000\u0000\u0000\u00c8\u00c9\u0005d\u0000\u0000\u00c9"+ + "\u00ca\u0005r\u0000\u0000\u00ca\u00cb\u0005o\u0000\u0000\u00cb\u00cc\u0005"+ + "p\u0000\u0000\u00cc\u00cd\u0001\u0000\u0000\u0000\u00cd\u00ce\u0006\u0001"+ + "\u0001\u0000\u00ce\u0007\u0001\u0000\u0000\u0000\u00cf\u00d0\u0005e\u0000"+ + "\u0000\u00d0\u00d1\u0005n\u0000\u0000\u00d1\u00d2\u0005r\u0000\u0000\u00d2"+ + "\u00d3\u0005i\u0000\u0000\u00d3\u00d4\u0005c\u0000\u0000\u00d4\u00d5\u0005"+ + "h\u0000\u0000\u00d5\u00d6\u0001\u0000\u0000\u0000\u00d6\u00d7\u0006\u0002"+ + "\u0001\u0000\u00d7\t\u0001\u0000\u0000\u0000\u00d8\u00d9\u0005e\u0000"+ + "\u0000\u00d9\u00da\u0005v\u0000\u0000\u00da\u00db\u0005a\u0000\u0000\u00db"+ + "\u00dc\u0005l\u0000\u0000\u00dc\u00dd\u0001\u0000\u0000\u0000\u00dd\u00de"+ + "\u0006\u0003\u0000\u0000\u00de\u000b\u0001\u0000\u0000\u0000\u00df\u00e0"+ + "\u0005e\u0000\u0000\u00e0\u00e1\u0005x\u0000\u0000\u00e1\u00e2\u0005p"+ + "\u0000\u0000\u00e2\u00e3\u0005l\u0000\u0000\u00e3\u00e4\u0005a\u0000\u0000"+ + "\u00e4\u00e5\u0005i\u0000\u0000\u00e5\u00e6\u0005n\u0000\u0000\u00e6\u00e7"+ + "\u0001\u0000\u0000\u0000\u00e7\u00e8\u0006\u0004\u0002\u0000\u00e8\r\u0001"+ + "\u0000\u0000\u0000\u00e9\u00ea\u0005f\u0000\u0000\u00ea\u00eb\u0005r\u0000"+ + "\u0000\u00eb\u00ec\u0005o\u0000\u0000\u00ec\u00ed\u0005m\u0000\u0000\u00ed"+ + "\u00ee\u0001\u0000\u0000\u0000\u00ee\u00ef\u0006\u0005\u0001\u0000\u00ef"+ + "\u000f\u0001\u0000\u0000\u0000\u00f0\u00f1\u0005g\u0000\u0000\u00f1\u00f2"+ + "\u0005r\u0000\u0000\u00f2\u00f3\u0005o\u0000\u0000\u00f3\u00f4\u0005k"+ + "\u0000\u0000\u00f4\u00f5\u0001\u0000\u0000\u0000\u00f5\u00f6\u0006\u0006"+ + "\u0000\u0000\u00f6\u0011\u0001\u0000\u0000\u0000\u00f7\u00f8\u0005i\u0000"+ + "\u0000\u00f8\u00f9\u0005n\u0000\u0000\u00f9\u00fa\u0005l\u0000\u0000\u00fa"+ + "\u00fb\u0005i\u0000\u0000\u00fb\u00fc\u0005n\u0000\u0000\u00fc\u00fd\u0005"+ + "e\u0000\u0000\u00fd\u00fe\u0005s\u0000\u0000\u00fe\u00ff\u0005t\u0000"+ + "\u0000\u00ff\u0100\u0005a\u0000\u0000\u0100\u0101\u0005t\u0000\u0000\u0101"+ + "\u0102\u0005s\u0000\u0000\u0102\u0103\u0001\u0000\u0000\u0000\u0103\u0104"+ + "\u0006\u0007\u0000\u0000\u0104\u0013\u0001\u0000\u0000\u0000\u0105\u0106"+ + "\u0005k\u0000\u0000\u0106\u0107\u0005e\u0000\u0000\u0107\u0108\u0005e"+ + "\u0000\u0000\u0108\u0109\u0005p\u0000\u0000\u0109\u010a\u0001\u0000\u0000"+ + "\u0000\u010a\u010b\u0006\b\u0001\u0000\u010b\u0015\u0001\u0000\u0000\u0000"+ + "\u010c\u010d\u0005l\u0000\u0000\u010d\u010e\u0005i\u0000\u0000\u010e\u010f"+ + "\u0005m\u0000\u0000\u010f\u0110\u0005i\u0000\u0000\u0110\u0111\u0005t"+ + "\u0000\u0000\u0111\u0112\u0001\u0000\u0000\u0000\u0112\u0113\u0006\t\u0000"+ + "\u0000\u0113\u0017\u0001\u0000\u0000\u0000\u0114\u0115\u0005m\u0000\u0000"+ + "\u0115\u0116\u0005v\u0000\u0000\u0116\u0117\u0005_\u0000\u0000\u0117\u0118"+ + "\u0005e\u0000\u0000\u0118\u0119\u0005x\u0000\u0000\u0119\u011a\u0005p"+ + "\u0000\u0000\u011a\u011b\u0005a\u0000\u0000\u011b\u011c\u0005n\u0000\u0000"+ + "\u011c\u011d\u0005d\u0000\u0000\u011d\u011e\u0001\u0000\u0000\u0000\u011e"+ + "\u011f\u0006\n\u0001\u0000\u011f\u0019\u0001\u0000\u0000\u0000\u0120\u0121"+ + "\u0005p\u0000\u0000\u0121\u0122\u0005r\u0000\u0000\u0122\u0123\u0005o"+ + "\u0000\u0000\u0123\u0124\u0005j\u0000\u0000\u0124\u0125\u0005e\u0000\u0000"+ + "\u0125\u0126\u0005c\u0000\u0000\u0126\u0127\u0005t\u0000\u0000\u0127\u0128"+ + "\u0001\u0000\u0000\u0000\u0128\u0129\u0006\u000b\u0001\u0000\u0129\u001b"+ + "\u0001\u0000\u0000\u0000\u012a\u012b\u0005r\u0000\u0000\u012b\u012c\u0005"+ + "e\u0000\u0000\u012c\u012d\u0005n\u0000\u0000\u012d\u012e\u0005a\u0000"+ + "\u0000\u012e\u012f\u0005m\u0000\u0000\u012f\u0130\u0005e\u0000\u0000\u0130"+ + "\u0131\u0001\u0000\u0000\u0000\u0131\u0132\u0006\f\u0001\u0000\u0132\u001d"+ + "\u0001\u0000\u0000\u0000\u0133\u0134\u0005r\u0000\u0000\u0134\u0135\u0005"+ + "o\u0000\u0000\u0135\u0136\u0005w\u0000\u0000\u0136\u0137\u0001\u0000\u0000"+ + "\u0000\u0137\u0138\u0006\r\u0000\u0000\u0138\u001f\u0001\u0000\u0000\u0000"+ + "\u0139\u013a\u0005s\u0000\u0000\u013a\u013b\u0005h\u0000\u0000\u013b\u013c"+ + "\u0005o\u0000\u0000\u013c\u013d\u0005w\u0000\u0000\u013d\u013e\u0001\u0000"+ + "\u0000\u0000\u013e\u013f\u0006\u000e\u0000\u0000\u013f!\u0001\u0000\u0000"+ + "\u0000\u0140\u0141\u0005s\u0000\u0000\u0141\u0142\u0005o\u0000\u0000\u0142"+ + "\u0143\u0005r\u0000\u0000\u0143\u0144\u0005t\u0000\u0000\u0144\u0145\u0001"+ + "\u0000\u0000\u0000\u0145\u0146\u0006\u000f\u0000\u0000\u0146#\u0001\u0000"+ + "\u0000\u0000\u0147\u0148\u0005s\u0000\u0000\u0148\u0149\u0005t\u0000\u0000"+ + "\u0149\u014a\u0005a\u0000\u0000\u014a\u014b\u0005t\u0000\u0000\u014b\u014c"+ + "\u0005s\u0000\u0000\u014c\u014d\u0001\u0000\u0000\u0000\u014d\u014e\u0006"+ + "\u0010\u0000\u0000\u014e%\u0001\u0000\u0000\u0000\u014f\u0150\u0005w\u0000"+ + "\u0000\u0150\u0151\u0005h\u0000\u0000\u0151\u0152\u0005e\u0000\u0000\u0152"+ + "\u0153\u0005r\u0000\u0000\u0153\u0154\u0005e\u0000\u0000\u0154\u0155\u0001"+ + "\u0000\u0000\u0000\u0155\u0156\u0006\u0011\u0000\u0000\u0156\'\u0001\u0000"+ + "\u0000\u0000\u0157\u0159\b\u0000\u0000\u0000\u0158\u0157\u0001\u0000\u0000"+ + "\u0000\u0159\u015a\u0001\u0000\u0000\u0000\u015a\u0158\u0001\u0000\u0000"+ + "\u0000\u015a\u015b\u0001\u0000\u0000\u0000\u015b\u015c\u0001\u0000\u0000"+ + "\u0000\u015c\u015d\u0006\u0012\u0000\u0000\u015d)\u0001\u0000\u0000\u0000"+ + "\u015e\u015f\u0005/\u0000\u0000\u015f\u0160\u0005/\u0000\u0000\u0160\u0164"+ + "\u0001\u0000\u0000\u0000\u0161\u0163\b\u0001\u0000\u0000\u0162\u0161\u0001"+ + "\u0000\u0000\u0000\u0163\u0166\u0001\u0000\u0000\u0000\u0164\u0162\u0001"+ + "\u0000\u0000\u0000\u0164\u0165\u0001\u0000\u0000\u0000\u0165\u0168\u0001"+ + "\u0000\u0000\u0000\u0166\u0164\u0001\u0000\u0000\u0000\u0167\u0169\u0005"+ + "\r\u0000\u0000\u0168\u0167\u0001\u0000\u0000\u0000\u0168\u0169\u0001\u0000"+ + "\u0000\u0000\u0169\u016b\u0001\u0000\u0000\u0000\u016a\u016c\u0005\n\u0000"+ + "\u0000\u016b\u016a\u0001\u0000\u0000\u0000\u016b\u016c\u0001\u0000\u0000"+ + "\u0000\u016c\u016d\u0001\u0000\u0000\u0000\u016d\u016e\u0006\u0013\u0003"+ + "\u0000\u016e+\u0001\u0000\u0000\u0000\u016f\u0170\u0005/\u0000\u0000\u0170"+ + "\u0171\u0005*\u0000\u0000\u0171\u0176\u0001\u0000\u0000\u0000\u0172\u0175"+ + "\u0003,\u0014\u0000\u0173\u0175\t\u0000\u0000\u0000\u0174\u0172\u0001"+ + "\u0000\u0000\u0000\u0174\u0173\u0001\u0000\u0000\u0000\u0175\u0178\u0001"+ + "\u0000\u0000\u0000\u0176\u0177\u0001\u0000\u0000\u0000\u0176\u0174\u0001"+ + "\u0000\u0000\u0000\u0177\u0179\u0001\u0000\u0000\u0000\u0178\u0176\u0001"+ + "\u0000\u0000\u0000\u0179\u017a\u0005*\u0000\u0000\u017a\u017b\u0005/\u0000"+ + "\u0000\u017b\u017c\u0001\u0000\u0000\u0000\u017c\u017d\u0006\u0014\u0003"+ + "\u0000\u017d-\u0001\u0000\u0000\u0000\u017e\u0180\u0007\u0002\u0000\u0000"+ + "\u017f\u017e\u0001\u0000\u0000\u0000\u0180\u0181\u0001\u0000\u0000\u0000"+ + "\u0181\u017f\u0001\u0000\u0000\u0000\u0181\u0182\u0001\u0000\u0000\u0000"+ + "\u0182\u0183\u0001\u0000\u0000\u0000\u0183\u0184\u0006\u0015\u0003\u0000"+ + "\u0184/\u0001\u0000\u0000\u0000\u0185\u0186\u0005[\u0000\u0000\u0186\u0187"+ + "\u0001\u0000\u0000\u0000\u0187\u0188\u0006\u0016\u0004\u0000\u0188\u0189"+ + "\u0006\u0016\u0005\u0000\u01891\u0001\u0000\u0000\u0000\u018a\u018b\u0005"+ + "|\u0000\u0000\u018b\u018c\u0001\u0000\u0000\u0000\u018c\u018d\u0006\u0017"+ + "\u0006\u0000\u018d\u018e\u0006\u0017\u0007\u0000\u018e3\u0001\u0000\u0000"+ + "\u0000\u018f\u0190\u0003.\u0015\u0000\u0190\u0191\u0001\u0000\u0000\u0000"+ + "\u0191\u0192\u0006\u0018\u0003\u0000\u01925\u0001\u0000\u0000\u0000\u0193"+ + "\u0194\u0003*\u0013\u0000\u0194\u0195\u0001\u0000\u0000\u0000\u0195\u0196"+ + "\u0006\u0019\u0003\u0000\u01967\u0001\u0000\u0000\u0000\u0197\u0198\u0003"+ + ",\u0014\u0000\u0198\u0199\u0001\u0000\u0000\u0000\u0199\u019a\u0006\u001a"+ + "\u0003\u0000\u019a9\u0001\u0000\u0000\u0000\u019b\u019c\u0005|\u0000\u0000"+ + "\u019c\u019d\u0001\u0000\u0000\u0000\u019d\u019e\u0006\u001b\u0007\u0000"+ + "\u019e;\u0001\u0000\u0000\u0000\u019f\u01a0\u0007\u0003\u0000\u0000\u01a0"+ + "=\u0001\u0000\u0000\u0000\u01a1\u01a2\u0007\u0004\u0000\u0000\u01a2?\u0001"+ + "\u0000\u0000\u0000\u01a3\u01a4\u0005\\\u0000\u0000\u01a4\u01a5\u0007\u0005"+ + "\u0000\u0000\u01a5A\u0001\u0000\u0000\u0000\u01a6\u01a7\b\u0006\u0000"+ + "\u0000\u01a7C\u0001\u0000\u0000\u0000\u01a8\u01aa\u0007\u0007\u0000\u0000"+ + "\u01a9\u01ab\u0007\b\u0000\u0000\u01aa\u01a9\u0001\u0000\u0000\u0000\u01aa"+ + "\u01ab\u0001\u0000\u0000\u0000\u01ab\u01ad\u0001\u0000\u0000\u0000\u01ac"+ + "\u01ae\u0003<\u001c\u0000\u01ad\u01ac\u0001\u0000\u0000\u0000\u01ae\u01af"+ + "\u0001\u0000\u0000\u0000\u01af\u01ad\u0001\u0000\u0000\u0000\u01af\u01b0"+ + "\u0001\u0000\u0000\u0000\u01b0E\u0001\u0000\u0000\u0000\u01b1\u01b6\u0005"+ + "\"\u0000\u0000\u01b2\u01b5\u0003@\u001e\u0000\u01b3\u01b5\u0003B\u001f"+ + "\u0000\u01b4\u01b2\u0001\u0000\u0000\u0000\u01b4\u01b3\u0001\u0000\u0000"+ + "\u0000\u01b5\u01b8\u0001\u0000\u0000\u0000\u01b6\u01b4\u0001\u0000\u0000"+ + "\u0000\u01b6\u01b7\u0001\u0000\u0000\u0000\u01b7\u01b9\u0001\u0000\u0000"+ + "\u0000\u01b8\u01b6\u0001\u0000\u0000\u0000\u01b9\u01cf\u0005\"\u0000\u0000"+ + "\u01ba\u01bb\u0005\"\u0000\u0000\u01bb\u01bc\u0005\"\u0000\u0000\u01bc"+ + "\u01bd\u0005\"\u0000\u0000\u01bd\u01c1\u0001\u0000\u0000\u0000\u01be\u01c0"+ + "\b\u0001\u0000\u0000\u01bf\u01be\u0001\u0000\u0000\u0000\u01c0\u01c3\u0001"+ + "\u0000\u0000\u0000\u01c1\u01c2\u0001\u0000\u0000\u0000\u01c1\u01bf\u0001"+ + "\u0000\u0000\u0000\u01c2\u01c4\u0001\u0000\u0000\u0000\u01c3\u01c1\u0001"+ + "\u0000\u0000\u0000\u01c4\u01c5\u0005\"\u0000\u0000\u01c5\u01c6\u0005\""+ + "\u0000\u0000\u01c6\u01c7\u0005\"\u0000\u0000\u01c7\u01c9\u0001\u0000\u0000"+ + "\u0000\u01c8\u01ca\u0005\"\u0000\u0000\u01c9\u01c8\u0001\u0000\u0000\u0000"+ + "\u01c9\u01ca\u0001\u0000\u0000\u0000\u01ca\u01cc\u0001\u0000\u0000\u0000"+ + "\u01cb\u01cd\u0005\"\u0000\u0000\u01cc\u01cb\u0001\u0000\u0000\u0000\u01cc"+ + "\u01cd\u0001\u0000\u0000\u0000\u01cd\u01cf\u0001\u0000\u0000\u0000\u01ce"+ + "\u01b1\u0001\u0000\u0000\u0000\u01ce\u01ba\u0001\u0000\u0000\u0000\u01cf"+ + "G\u0001\u0000\u0000\u0000\u01d0\u01d2\u0003<\u001c\u0000\u01d1\u01d0\u0001"+ + "\u0000\u0000\u0000\u01d2\u01d3\u0001\u0000\u0000\u0000\u01d3\u01d1\u0001"+ + "\u0000\u0000\u0000\u01d3\u01d4\u0001\u0000\u0000\u0000\u01d4I\u0001\u0000"+ + "\u0000\u0000\u01d5\u01d7\u0003<\u001c\u0000\u01d6\u01d5\u0001\u0000\u0000"+ + "\u0000\u01d7\u01d8\u0001\u0000\u0000\u0000\u01d8\u01d6\u0001\u0000\u0000"+ + "\u0000\u01d8\u01d9\u0001\u0000\u0000\u0000\u01d9\u01da\u0001\u0000\u0000"+ + "\u0000\u01da\u01de\u0003X*\u0000\u01db\u01dd\u0003<\u001c\u0000\u01dc"+ + "\u01db\u0001\u0000\u0000\u0000\u01dd\u01e0\u0001\u0000\u0000\u0000\u01de"+ + "\u01dc\u0001\u0000\u0000\u0000\u01de\u01df\u0001\u0000\u0000\u0000\u01df"+ + "\u0200\u0001\u0000\u0000\u0000\u01e0\u01de\u0001\u0000\u0000\u0000\u01e1"+ + "\u01e3\u0003X*\u0000\u01e2\u01e4\u0003<\u001c\u0000\u01e3\u01e2\u0001"+ + "\u0000\u0000\u0000\u01e4\u01e5\u0001\u0000\u0000\u0000\u01e5\u01e3\u0001"+ + "\u0000\u0000\u0000\u01e5\u01e6\u0001\u0000\u0000\u0000\u01e6\u0200\u0001"+ + "\u0000\u0000\u0000\u01e7\u01e9\u0003<\u001c\u0000\u01e8\u01e7\u0001\u0000"+ + "\u0000\u0000\u01e9\u01ea\u0001\u0000\u0000\u0000\u01ea\u01e8\u0001\u0000"+ + "\u0000\u0000\u01ea\u01eb\u0001\u0000\u0000\u0000\u01eb\u01f3\u0001\u0000"+ + "\u0000\u0000\u01ec\u01f0\u0003X*\u0000\u01ed\u01ef\u0003<\u001c\u0000"+ + "\u01ee\u01ed\u0001\u0000\u0000\u0000\u01ef\u01f2\u0001\u0000\u0000\u0000"+ + "\u01f0\u01ee\u0001\u0000\u0000\u0000\u01f0\u01f1\u0001\u0000\u0000\u0000"+ + "\u01f1\u01f4\u0001\u0000\u0000\u0000\u01f2\u01f0\u0001\u0000\u0000\u0000"+ + "\u01f3\u01ec\u0001\u0000\u0000\u0000\u01f3\u01f4\u0001\u0000\u0000\u0000"+ + "\u01f4\u01f5\u0001\u0000\u0000\u0000\u01f5\u01f6\u0003D \u0000\u01f6\u0200"+ + "\u0001\u0000\u0000\u0000\u01f7\u01f9\u0003X*\u0000\u01f8\u01fa\u0003<"+ + "\u001c\u0000\u01f9\u01f8\u0001\u0000\u0000\u0000\u01fa\u01fb\u0001\u0000"+ + "\u0000\u0000\u01fb\u01f9\u0001\u0000\u0000\u0000\u01fb\u01fc\u0001\u0000"+ + "\u0000\u0000\u01fc\u01fd\u0001\u0000\u0000\u0000\u01fd\u01fe\u0003D \u0000"+ + "\u01fe\u0200\u0001\u0000\u0000\u0000\u01ff\u01d6\u0001\u0000\u0000\u0000"+ + "\u01ff\u01e1\u0001\u0000\u0000\u0000\u01ff\u01e8\u0001\u0000\u0000\u0000"+ + "\u01ff\u01f7\u0001\u0000\u0000\u0000\u0200K\u0001\u0000\u0000\u0000\u0201"+ + "\u0202\u0005b\u0000\u0000\u0202\u0203\u0005y\u0000\u0000\u0203M\u0001"+ + "\u0000\u0000\u0000\u0204\u0205\u0005a\u0000\u0000\u0205\u0206\u0005n\u0000"+ + "\u0000\u0206\u0207\u0005d\u0000\u0000\u0207O\u0001\u0000\u0000\u0000\u0208"+ + "\u0209\u0005a\u0000\u0000\u0209\u020a\u0005s\u0000\u0000\u020a\u020b\u0005"+ + "c\u0000\u0000\u020bQ\u0001\u0000\u0000\u0000\u020c\u020d\u0005=\u0000"+ + "\u0000\u020dS\u0001\u0000\u0000\u0000\u020e\u020f\u0005,\u0000\u0000\u020f"+ + "U\u0001\u0000\u0000\u0000\u0210\u0211\u0005d\u0000\u0000\u0211\u0212\u0005"+ + "e\u0000\u0000\u0212\u0213\u0005s\u0000\u0000\u0213\u0214\u0005c\u0000"+ + "\u0000\u0214W\u0001\u0000\u0000\u0000\u0215\u0216\u0005.\u0000\u0000\u0216"+ + "Y\u0001\u0000\u0000\u0000\u0217\u0218\u0005f\u0000\u0000\u0218\u0219\u0005"+ + "a\u0000\u0000\u0219\u021a\u0005l\u0000\u0000\u021a\u021b\u0005s\u0000"+ + "\u0000\u021b\u021c\u0005e\u0000\u0000\u021c[\u0001\u0000\u0000\u0000\u021d"+ + "\u021e\u0005f\u0000\u0000\u021e\u021f\u0005i\u0000\u0000\u021f\u0220\u0005"+ + "r\u0000\u0000\u0220\u0221\u0005s\u0000\u0000\u0221\u0222\u0005t\u0000"+ + "\u0000\u0222]\u0001\u0000\u0000\u0000\u0223\u0224\u0005l\u0000\u0000\u0224"+ + "\u0225\u0005a\u0000\u0000\u0225\u0226\u0005s\u0000\u0000\u0226\u0227\u0005"+ + "t\u0000\u0000\u0227_\u0001\u0000\u0000\u0000\u0228\u0229\u0005(\u0000"+ + "\u0000\u0229a\u0001\u0000\u0000\u0000\u022a\u022b\u0005i\u0000\u0000\u022b"+ + "\u022c\u0005n\u0000\u0000\u022cc\u0001\u0000\u0000\u0000\u022d\u022e\u0005"+ + "i\u0000\u0000\u022e\u022f\u0005s\u0000\u0000\u022fe\u0001\u0000\u0000"+ + "\u0000\u0230\u0231\u0005l\u0000\u0000\u0231\u0232\u0005i\u0000\u0000\u0232"+ + "\u0233\u0005k\u0000\u0000\u0233\u0234\u0005e\u0000\u0000\u0234g\u0001"+ + "\u0000\u0000\u0000\u0235\u0236\u0005n\u0000\u0000\u0236\u0237\u0005o\u0000"+ + "\u0000\u0237\u0238\u0005t\u0000\u0000\u0238i\u0001\u0000\u0000\u0000\u0239"+ + "\u023a\u0005n\u0000\u0000\u023a\u023b\u0005u\u0000\u0000\u023b\u023c\u0005"+ + "l\u0000\u0000\u023c\u023d\u0005l\u0000\u0000\u023dk\u0001\u0000\u0000"+ + "\u0000\u023e\u023f\u0005n\u0000\u0000\u023f\u0240\u0005u\u0000\u0000\u0240"+ + "\u0241\u0005l\u0000\u0000\u0241\u0242\u0005l\u0000\u0000\u0242\u0243\u0005"+ + "s\u0000\u0000\u0243m\u0001\u0000\u0000\u0000\u0244\u0245\u0005o\u0000"+ + "\u0000\u0245\u0246\u0005r\u0000\u0000\u0246o\u0001\u0000\u0000\u0000\u0247"+ + "\u0248\u0005?\u0000\u0000\u0248q\u0001\u0000\u0000\u0000\u0249\u024a\u0005"+ + "r\u0000\u0000\u024a\u024b\u0005l\u0000\u0000\u024b\u024c\u0005i\u0000"+ + "\u0000\u024c\u024d\u0005k\u0000\u0000\u024d\u024e\u0005e\u0000\u0000\u024e"+ + "s\u0001\u0000\u0000\u0000\u024f\u0250\u0005)\u0000\u0000\u0250u\u0001"+ + "\u0000\u0000\u0000\u0251\u0252\u0005t\u0000\u0000\u0252\u0253\u0005r\u0000"+ + "\u0000\u0253\u0254\u0005u\u0000\u0000\u0254\u0255\u0005e\u0000\u0000\u0255"+ + "w\u0001\u0000\u0000\u0000\u0256\u0257\u0005i\u0000\u0000\u0257\u0258\u0005"+ + "n\u0000\u0000\u0258\u0259\u0005f\u0000\u0000\u0259\u025a\u0005o\u0000"+ + "\u0000\u025ay\u0001\u0000\u0000\u0000\u025b\u025c\u0005f\u0000\u0000\u025c"+ + "\u025d\u0005u\u0000\u0000\u025d\u025e\u0005n\u0000\u0000\u025e\u025f\u0005"+ + "c\u0000\u0000\u025f\u0260\u0005t\u0000\u0000\u0260\u0261\u0005i\u0000"+ + "\u0000\u0261\u0262\u0005o\u0000\u0000\u0262\u0263\u0005n\u0000\u0000\u0263"+ + "\u0264\u0005s\u0000\u0000\u0264{\u0001\u0000\u0000\u0000\u0265\u0266\u0005"+ + "=\u0000\u0000\u0266\u0267\u0005=\u0000\u0000\u0267}\u0001\u0000\u0000"+ + "\u0000\u0268\u0269\u0005!\u0000\u0000\u0269\u026a\u0005=\u0000\u0000\u026a"+ + "\u007f\u0001\u0000\u0000\u0000\u026b\u026c\u0005<\u0000\u0000\u026c\u0081"+ + "\u0001\u0000\u0000\u0000\u026d\u026e\u0005<\u0000\u0000\u026e\u026f\u0005"+ + "=\u0000\u0000\u026f\u0083\u0001\u0000\u0000\u0000\u0270\u0271\u0005>\u0000"+ + "\u0000\u0271\u0085\u0001\u0000\u0000\u0000\u0272\u0273\u0005>\u0000\u0000"+ + "\u0273\u0274\u0005=\u0000\u0000\u0274\u0087\u0001\u0000\u0000\u0000\u0275"+ + "\u0276\u0005+\u0000\u0000\u0276\u0089\u0001\u0000\u0000\u0000\u0277\u0278"+ + "\u0005-\u0000\u0000\u0278\u008b\u0001\u0000\u0000\u0000\u0279\u027a\u0005"+ + "*\u0000\u0000\u027a\u008d\u0001\u0000\u0000\u0000\u027b\u027c\u0005/\u0000"+ + "\u0000\u027c\u008f\u0001\u0000\u0000\u0000\u027d\u027e\u0005%\u0000\u0000"+ + "\u027e\u0091\u0001\u0000\u0000\u0000\u027f\u0280\u0005[\u0000\u0000\u0280"+ + "\u0281\u0001\u0000\u0000\u0000\u0281\u0282\u0006G\u0000\u0000\u0282\u0283"+ + "\u0006G\u0000\u0000\u0283\u0093\u0001\u0000\u0000\u0000\u0284\u0285\u0005"+ + "]\u0000\u0000\u0285\u0286\u0001\u0000\u0000\u0000\u0286\u0287\u0006H\u0007"+ + "\u0000\u0287\u0288\u0006H\u0007\u0000\u0288\u0095\u0001\u0000\u0000\u0000"+ + "\u0289\u028f\u0003>\u001d\u0000\u028a\u028e\u0003>\u001d\u0000\u028b\u028e"+ + "\u0003<\u001c\u0000\u028c\u028e\u0005_\u0000\u0000\u028d\u028a\u0001\u0000"+ + "\u0000\u0000\u028d\u028b\u0001\u0000\u0000\u0000\u028d\u028c\u0001\u0000"+ + "\u0000\u0000\u028e\u0291\u0001\u0000\u0000\u0000\u028f\u028d\u0001\u0000"+ + "\u0000\u0000\u028f\u0290\u0001\u0000\u0000\u0000\u0290\u029b\u0001\u0000"+ + "\u0000\u0000\u0291\u028f\u0001\u0000\u0000\u0000\u0292\u0296\u0007\t\u0000"+ + "\u0000\u0293\u0297\u0003>\u001d\u0000\u0294\u0297\u0003<\u001c\u0000\u0295"+ + "\u0297\u0005_\u0000\u0000\u0296\u0293\u0001\u0000\u0000\u0000\u0296\u0294"+ + "\u0001\u0000\u0000\u0000\u0296\u0295\u0001\u0000\u0000\u0000\u0297\u0298"+ + "\u0001\u0000\u0000\u0000\u0298\u0296\u0001\u0000\u0000\u0000\u0298\u0299"+ + "\u0001\u0000\u0000\u0000\u0299\u029b\u0001\u0000\u0000\u0000\u029a\u0289"+ + "\u0001\u0000\u0000\u0000\u029a\u0292\u0001\u0000\u0000\u0000\u029b\u0097"+ + "\u0001\u0000\u0000\u0000\u029c\u02a2\u0005`\u0000\u0000\u029d\u02a1\b"+ + "\n\u0000\u0000\u029e\u029f\u0005`\u0000\u0000\u029f\u02a1\u0005`\u0000"+ + "\u0000\u02a0\u029d\u0001\u0000\u0000\u0000\u02a0\u029e\u0001\u0000\u0000"+ + "\u0000\u02a1\u02a4\u0001\u0000\u0000\u0000\u02a2\u02a0\u0001\u0000\u0000"+ + "\u0000\u02a2\u02a3\u0001\u0000\u0000\u0000\u02a3\u02a5\u0001\u0000\u0000"+ + "\u0000\u02a4\u02a2\u0001\u0000\u0000\u0000\u02a5\u02a6\u0005`\u0000\u0000"+ + "\u02a6\u0099\u0001\u0000\u0000\u0000\u02a7\u02a8\u0003*\u0013\u0000\u02a8"+ + "\u02a9\u0001\u0000\u0000\u0000\u02a9\u02aa\u0006K\u0003\u0000\u02aa\u009b"+ + "\u0001\u0000\u0000\u0000\u02ab\u02ac\u0003,\u0014\u0000\u02ac\u02ad\u0001"+ + "\u0000\u0000\u0000\u02ad\u02ae\u0006L\u0003\u0000\u02ae\u009d\u0001\u0000"+ + "\u0000\u0000\u02af\u02b0\u0003.\u0015\u0000\u02b0\u02b1\u0001\u0000\u0000"+ + "\u0000\u02b1\u02b2\u0006M\u0003\u0000\u02b2\u009f\u0001\u0000\u0000\u0000"+ + "\u02b3\u02b4\u0005|\u0000\u0000\u02b4\u02b5\u0001\u0000\u0000\u0000\u02b5"+ + "\u02b6\u0006N\u0006\u0000\u02b6\u02b7\u0006N\u0007\u0000\u02b7\u00a1\u0001"+ + "\u0000\u0000\u0000\u02b8\u02b9\u0005[\u0000\u0000\u02b9\u02ba\u0001\u0000"+ + "\u0000\u0000\u02ba\u02bb\u0006O\u0004\u0000\u02bb\u02bc\u0006O\u0001\u0000"+ + "\u02bc\u02bd\u0006O\u0001\u0000\u02bd\u00a3\u0001\u0000\u0000\u0000\u02be"+ + "\u02bf\u0005]\u0000\u0000\u02bf\u02c0\u0001\u0000\u0000\u0000\u02c0\u02c1"+ + "\u0006P\u0007\u0000\u02c1\u02c2\u0006P\u0007\u0000\u02c2\u02c3\u0006P"+ + "\b\u0000\u02c3\u00a5\u0001\u0000\u0000\u0000\u02c4\u02c5\u0005,\u0000"+ + "\u0000\u02c5\u02c6\u0001\u0000\u0000\u0000\u02c6\u02c7\u0006Q\t\u0000"+ + "\u02c7\u00a7\u0001\u0000\u0000\u0000\u02c8\u02c9\u0005=\u0000\u0000\u02c9"+ + "\u02ca\u0001\u0000\u0000\u0000\u02ca\u02cb\u0006R\n\u0000\u02cb\u00a9"+ + "\u0001\u0000\u0000\u0000\u02cc\u02cd\u0005a\u0000\u0000\u02cd\u02ce\u0005"+ + "s\u0000\u0000\u02ce\u00ab\u0001\u0000\u0000\u0000\u02cf\u02d0\u0005m\u0000"+ + "\u0000\u02d0\u02d1\u0005e\u0000\u0000\u02d1\u02d2\u0005t\u0000\u0000\u02d2"+ + "\u02d3\u0005a\u0000\u0000\u02d3\u02d4\u0005d\u0000\u0000\u02d4\u02d5\u0005"+ + "a\u0000\u0000\u02d5\u02d6\u0005t\u0000\u0000\u02d6\u02d7\u0005a\u0000"+ + "\u0000\u02d7\u00ad\u0001\u0000\u0000\u0000\u02d8\u02d9\u0005o\u0000\u0000"+ + "\u02d9\u02da\u0005n\u0000\u0000\u02da\u00af\u0001\u0000\u0000\u0000\u02db"+ + "\u02dc\u0005w\u0000\u0000\u02dc\u02dd\u0005i\u0000\u0000\u02dd\u02de\u0005"+ + "t\u0000\u0000\u02de\u02df\u0005h\u0000\u0000\u02df\u00b1\u0001\u0000\u0000"+ + "\u0000\u02e0\u02e2\u0003\u00b4X\u0000\u02e1\u02e0\u0001\u0000\u0000\u0000"+ + "\u02e2\u02e3\u0001\u0000\u0000\u0000\u02e3\u02e1\u0001\u0000\u0000\u0000"+ + "\u02e3\u02e4\u0001\u0000\u0000\u0000\u02e4\u00b3\u0001\u0000\u0000\u0000"+ + "\u02e5\u02e7\b\u000b\u0000\u0000\u02e6\u02e5\u0001\u0000\u0000\u0000\u02e7"+ + "\u02e8\u0001\u0000\u0000\u0000\u02e8\u02e6\u0001\u0000\u0000\u0000\u02e8"+ + "\u02e9\u0001\u0000\u0000\u0000\u02e9\u02ed\u0001\u0000\u0000\u0000\u02ea"+ + "\u02eb\u0005/\u0000\u0000\u02eb\u02ed\b\f\u0000\u0000\u02ec\u02e6\u0001"+ + "\u0000\u0000\u0000\u02ec\u02ea\u0001\u0000\u0000\u0000\u02ed\u00b5\u0001"+ + "\u0000\u0000\u0000\u02ee\u02ef\u0003\u0098J\u0000\u02ef\u00b7\u0001\u0000"+ + "\u0000\u0000\u02f0\u02f1\u0003*\u0013\u0000\u02f1\u02f2\u0001\u0000\u0000"+ + "\u0000\u02f2\u02f3\u0006Z\u0003\u0000\u02f3\u00b9\u0001\u0000\u0000\u0000"+ + "\u02f4\u02f5\u0003,\u0014\u0000\u02f5\u02f6\u0001\u0000\u0000\u0000\u02f6"+ + "\u02f7\u0006[\u0003\u0000\u02f7\u00bb\u0001\u0000\u0000\u0000\u02f8\u02f9"+ + "\u0003.\u0015\u0000\u02f9\u02fa\u0001\u0000\u0000\u0000\u02fa\u02fb\u0006"+ + "\\\u0003\u0000\u02fb\u00bd\u0001\u0000\u0000\u0000&\u0000\u0001\u0002"+ + "\u0003\u015a\u0164\u0168\u016b\u0174\u0176\u0181\u01aa\u01af\u01b4\u01b6"+ + "\u01c1\u01c9\u01cc\u01ce\u01d3\u01d8\u01de\u01e5\u01ea\u01f0\u01f3\u01fb"+ + "\u01ff\u028d\u028f\u0296\u0298\u029a\u02a0\u02a2\u02e3\u02e8\u02ec\u000b"+ + "\u0005\u0002\u0000\u0005\u0003\u0000\u0005\u0001\u0000\u0000\u0001\u0000"+ + "\u0007A\u0000\u0005\u0000\u0000\u0007\u001a\u0000\u0004\u0000\u0000\u0007"+ + "B\u0000\u0007\"\u0000\u0007!\u0000"; + public static final ATN _ATN = + new ATNDeserializer().deserialize(_serializedATN.toCharArray()); + static { + _decisionToDFA = new DFA[_ATN.getNumberOfDecisions()]; + for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) { + _decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp new file mode 100644 index 0000000000000..349f31f7c476d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp @@ -0,0 +1,219 @@ +token literal names: +null +'dissect' +'drop' +'enrich' +'eval' +'explain' +'from' +'grok' +'inlinestats' +'keep' +'limit' +'mv_expand' +'project' +'rename' +'row' +'show' +'sort' +'stats' +'where' +null +null +null +null +null +null +null +null +null +null +null +'by' +'and' +'asc' +null +null +'desc' +'.' +'false' +'first' +'last' +'(' +'in' +'is' +'like' +'not' +'null' +'nulls' +'or' +'?' +'rlike' +')' +'true' +'info' +'functions' +'==' +'!=' +'<' +'<=' +'>' +'>=' +'+' +'-' +'*' +'/' +'%' +null +']' +null +null +null +null +null +'as' +'metadata' +'on' +'with' +null +null +null +null +null +null + +token symbolic names: +null +DISSECT +DROP +ENRICH +EVAL +EXPLAIN +FROM +GROK +INLINESTATS +KEEP +LIMIT +MV_EXPAND +PROJECT +RENAME +ROW +SHOW +SORT +STATS +WHERE +UNKNOWN_CMD +LINE_COMMENT +MULTILINE_COMMENT +WS +EXPLAIN_WS +EXPLAIN_LINE_COMMENT +EXPLAIN_MULTILINE_COMMENT +PIPE +STRING +INTEGER_LITERAL +DECIMAL_LITERAL +BY +AND +ASC +ASSIGN +COMMA +DESC +DOT +FALSE +FIRST +LAST +LP +IN +IS +LIKE +NOT +NULL +NULLS +OR +PARAM +RLIKE +RP +TRUE +INFO +FUNCTIONS +EQ +NEQ +LT +LTE +GT +GTE +PLUS +MINUS +ASTERISK +SLASH +PERCENT +OPENING_BRACKET +CLOSING_BRACKET +UNQUOTED_IDENTIFIER +QUOTED_IDENTIFIER +EXPR_LINE_COMMENT +EXPR_MULTILINE_COMMENT +EXPR_WS +AS +METADATA +ON +WITH +SRC_UNQUOTED_IDENTIFIER +SRC_QUOTED_IDENTIFIER +SRC_LINE_COMMENT +SRC_MULTILINE_COMMENT +SRC_WS +EXPLAIN_PIPE + +rule names: +singleStatement +query +sourceCommand +processingCommand +whereCommand +booleanExpression +regexBooleanExpression +valueExpression +operatorExpression +primaryExpression +rowCommand +fields +field +fromCommand +metadata +evalCommand +statsCommand +inlinestatsCommand +grouping +sourceIdentifier +qualifiedName +identifier +constant +limitCommand +sortCommand +orderExpression +keepCommand +dropCommand +renameCommand +renameClause +dissectCommand +grokCommand +mvExpandCommand +commandOptions +commandOption +booleanValue +numericValue +decimalValue +integerValue +string +comparisonOperator +explainCommand +subqueryExpression +showCommand +enrichCommand +enrichWithClause + + +atn: +[4, 1, 81, 501, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, 1, 102, 8, 1, 10, 1, 12, 1, 105, 9, 1, 1, 2, 1, 2, 1, 2, 1, 2, 3, 2, 111, 8, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 3, 3, 126, 8, 3, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 138, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 145, 8, 5, 10, 5, 12, 5, 148, 9, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 3, 5, 155, 8, 5, 1, 5, 1, 5, 3, 5, 159, 8, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 1, 5, 5, 5, 167, 8, 5, 10, 5, 12, 5, 170, 9, 5, 1, 6, 1, 6, 3, 6, 174, 8, 6, 1, 6, 1, 6, 1, 6, 1, 6, 1, 6, 3, 6, 181, 8, 6, 1, 6, 1, 6, 1, 6, 3, 6, 186, 8, 6, 1, 7, 1, 7, 1, 7, 1, 7, 1, 7, 3, 7, 193, 8, 7, 1, 8, 1, 8, 1, 8, 1, 8, 3, 8, 199, 8, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 1, 8, 5, 8, 207, 8, 8, 10, 8, 12, 8, 210, 9, 8, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 1, 9, 5, 9, 223, 8, 9, 10, 9, 12, 9, 226, 9, 9, 3, 9, 228, 8, 9, 1, 9, 1, 9, 3, 9, 232, 8, 9, 1, 10, 1, 10, 1, 10, 1, 11, 1, 11, 1, 11, 5, 11, 240, 8, 11, 10, 11, 12, 11, 243, 9, 11, 1, 12, 1, 12, 1, 12, 1, 12, 1, 12, 3, 12, 250, 8, 12, 1, 13, 1, 13, 1, 13, 1, 13, 5, 13, 256, 8, 13, 10, 13, 12, 13, 259, 9, 13, 1, 13, 3, 13, 262, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 5, 14, 269, 8, 14, 10, 14, 12, 14, 272, 9, 14, 1, 14, 1, 14, 1, 15, 1, 15, 1, 15, 1, 16, 1, 16, 3, 16, 281, 8, 16, 1, 16, 1, 16, 3, 16, 285, 8, 16, 1, 17, 1, 17, 1, 17, 1, 17, 3, 17, 291, 8, 17, 1, 18, 1, 18, 1, 18, 5, 18, 296, 8, 18, 10, 18, 12, 18, 299, 9, 18, 1, 19, 1, 19, 1, 20, 1, 20, 1, 20, 5, 20, 306, 8, 20, 10, 20, 12, 20, 309, 9, 20, 1, 21, 1, 21, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 326, 8, 22, 10, 22, 12, 22, 329, 9, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 337, 8, 22, 10, 22, 12, 22, 340, 9, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 1, 22, 5, 22, 348, 8, 22, 10, 22, 12, 22, 351, 9, 22, 1, 22, 1, 22, 3, 22, 355, 8, 22, 1, 23, 1, 23, 1, 23, 1, 24, 1, 24, 1, 24, 1, 24, 5, 24, 364, 8, 24, 10, 24, 12, 24, 367, 9, 24, 1, 25, 1, 25, 3, 25, 371, 8, 25, 1, 25, 1, 25, 3, 25, 375, 8, 25, 1, 26, 1, 26, 1, 26, 1, 26, 5, 26, 381, 8, 26, 10, 26, 12, 26, 384, 9, 26, 1, 26, 1, 26, 1, 26, 1, 26, 5, 26, 390, 8, 26, 10, 26, 12, 26, 393, 9, 26, 3, 26, 395, 8, 26, 1, 27, 1, 27, 1, 27, 1, 27, 5, 27, 401, 8, 27, 10, 27, 12, 27, 404, 9, 27, 1, 28, 1, 28, 1, 28, 1, 28, 5, 28, 410, 8, 28, 10, 28, 12, 28, 413, 9, 28, 1, 29, 1, 29, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 1, 30, 3, 30, 423, 8, 30, 1, 31, 1, 31, 1, 31, 1, 31, 1, 32, 1, 32, 1, 32, 1, 33, 1, 33, 1, 33, 5, 33, 435, 8, 33, 10, 33, 12, 33, 438, 9, 33, 1, 34, 1, 34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 36, 1, 36, 3, 36, 448, 8, 36, 1, 37, 3, 37, 451, 8, 37, 1, 37, 1, 37, 1, 38, 3, 38, 456, 8, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1, 40, 1, 40, 1, 41, 1, 41, 1, 41, 1, 42, 1, 42, 1, 42, 1, 42, 1, 43, 1, 43, 1, 43, 1, 43, 3, 43, 475, 8, 43, 1, 44, 1, 44, 1, 44, 1, 44, 3, 44, 481, 8, 44, 1, 44, 1, 44, 1, 44, 1, 44, 5, 44, 487, 8, 44, 10, 44, 12, 44, 490, 9, 44, 3, 44, 492, 8, 44, 1, 45, 1, 45, 1, 45, 3, 45, 497, 8, 45, 1, 45, 1, 45, 1, 45, 0, 3, 2, 10, 16, 46, 0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38, 40, 42, 44, 46, 48, 50, 52, 54, 56, 58, 60, 62, 64, 66, 68, 70, 72, 74, 76, 78, 80, 82, 84, 86, 88, 90, 0, 8, 1, 0, 60, 61, 1, 0, 62, 64, 1, 0, 76, 77, 1, 0, 67, 68, 2, 0, 32, 32, 35, 35, 1, 0, 38, 39, 2, 0, 37, 37, 51, 51, 1, 0, 54, 59, 531, 0, 92, 1, 0, 0, 0, 2, 95, 1, 0, 0, 0, 4, 110, 1, 0, 0, 0, 6, 125, 1, 0, 0, 0, 8, 127, 1, 0, 0, 0, 10, 158, 1, 0, 0, 0, 12, 185, 1, 0, 0, 0, 14, 192, 1, 0, 0, 0, 16, 198, 1, 0, 0, 0, 18, 231, 1, 0, 0, 0, 20, 233, 1, 0, 0, 0, 22, 236, 1, 0, 0, 0, 24, 249, 1, 0, 0, 0, 26, 251, 1, 0, 0, 0, 28, 263, 1, 0, 0, 0, 30, 275, 1, 0, 0, 0, 32, 278, 1, 0, 0, 0, 34, 286, 1, 0, 0, 0, 36, 292, 1, 0, 0, 0, 38, 300, 1, 0, 0, 0, 40, 302, 1, 0, 0, 0, 42, 310, 1, 0, 0, 0, 44, 354, 1, 0, 0, 0, 46, 356, 1, 0, 0, 0, 48, 359, 1, 0, 0, 0, 50, 368, 1, 0, 0, 0, 52, 394, 1, 0, 0, 0, 54, 396, 1, 0, 0, 0, 56, 405, 1, 0, 0, 0, 58, 414, 1, 0, 0, 0, 60, 418, 1, 0, 0, 0, 62, 424, 1, 0, 0, 0, 64, 428, 1, 0, 0, 0, 66, 431, 1, 0, 0, 0, 68, 439, 1, 0, 0, 0, 70, 443, 1, 0, 0, 0, 72, 447, 1, 0, 0, 0, 74, 450, 1, 0, 0, 0, 76, 455, 1, 0, 0, 0, 78, 459, 1, 0, 0, 0, 80, 461, 1, 0, 0, 0, 82, 463, 1, 0, 0, 0, 84, 466, 1, 0, 0, 0, 86, 474, 1, 0, 0, 0, 88, 476, 1, 0, 0, 0, 90, 496, 1, 0, 0, 0, 92, 93, 3, 2, 1, 0, 93, 94, 5, 0, 0, 1, 94, 1, 1, 0, 0, 0, 95, 96, 6, 1, -1, 0, 96, 97, 3, 4, 2, 0, 97, 103, 1, 0, 0, 0, 98, 99, 10, 1, 0, 0, 99, 100, 5, 26, 0, 0, 100, 102, 3, 6, 3, 0, 101, 98, 1, 0, 0, 0, 102, 105, 1, 0, 0, 0, 103, 101, 1, 0, 0, 0, 103, 104, 1, 0, 0, 0, 104, 3, 1, 0, 0, 0, 105, 103, 1, 0, 0, 0, 106, 111, 3, 82, 41, 0, 107, 111, 3, 26, 13, 0, 108, 111, 3, 20, 10, 0, 109, 111, 3, 86, 43, 0, 110, 106, 1, 0, 0, 0, 110, 107, 1, 0, 0, 0, 110, 108, 1, 0, 0, 0, 110, 109, 1, 0, 0, 0, 111, 5, 1, 0, 0, 0, 112, 126, 3, 30, 15, 0, 113, 126, 3, 34, 17, 0, 114, 126, 3, 46, 23, 0, 115, 126, 3, 52, 26, 0, 116, 126, 3, 48, 24, 0, 117, 126, 3, 32, 16, 0, 118, 126, 3, 8, 4, 0, 119, 126, 3, 54, 27, 0, 120, 126, 3, 56, 28, 0, 121, 126, 3, 60, 30, 0, 122, 126, 3, 62, 31, 0, 123, 126, 3, 88, 44, 0, 124, 126, 3, 64, 32, 0, 125, 112, 1, 0, 0, 0, 125, 113, 1, 0, 0, 0, 125, 114, 1, 0, 0, 0, 125, 115, 1, 0, 0, 0, 125, 116, 1, 0, 0, 0, 125, 117, 1, 0, 0, 0, 125, 118, 1, 0, 0, 0, 125, 119, 1, 0, 0, 0, 125, 120, 1, 0, 0, 0, 125, 121, 1, 0, 0, 0, 125, 122, 1, 0, 0, 0, 125, 123, 1, 0, 0, 0, 125, 124, 1, 0, 0, 0, 126, 7, 1, 0, 0, 0, 127, 128, 5, 18, 0, 0, 128, 129, 3, 10, 5, 0, 129, 9, 1, 0, 0, 0, 130, 131, 6, 5, -1, 0, 131, 132, 5, 44, 0, 0, 132, 159, 3, 10, 5, 7, 133, 159, 3, 14, 7, 0, 134, 159, 3, 12, 6, 0, 135, 137, 3, 14, 7, 0, 136, 138, 5, 44, 0, 0, 137, 136, 1, 0, 0, 0, 137, 138, 1, 0, 0, 0, 138, 139, 1, 0, 0, 0, 139, 140, 5, 41, 0, 0, 140, 141, 5, 40, 0, 0, 141, 146, 3, 14, 7, 0, 142, 143, 5, 34, 0, 0, 143, 145, 3, 14, 7, 0, 144, 142, 1, 0, 0, 0, 145, 148, 1, 0, 0, 0, 146, 144, 1, 0, 0, 0, 146, 147, 1, 0, 0, 0, 147, 149, 1, 0, 0, 0, 148, 146, 1, 0, 0, 0, 149, 150, 5, 50, 0, 0, 150, 159, 1, 0, 0, 0, 151, 152, 3, 14, 7, 0, 152, 154, 5, 42, 0, 0, 153, 155, 5, 44, 0, 0, 154, 153, 1, 0, 0, 0, 154, 155, 1, 0, 0, 0, 155, 156, 1, 0, 0, 0, 156, 157, 5, 45, 0, 0, 157, 159, 1, 0, 0, 0, 158, 130, 1, 0, 0, 0, 158, 133, 1, 0, 0, 0, 158, 134, 1, 0, 0, 0, 158, 135, 1, 0, 0, 0, 158, 151, 1, 0, 0, 0, 159, 168, 1, 0, 0, 0, 160, 161, 10, 4, 0, 0, 161, 162, 5, 31, 0, 0, 162, 167, 3, 10, 5, 5, 163, 164, 10, 3, 0, 0, 164, 165, 5, 47, 0, 0, 165, 167, 3, 10, 5, 4, 166, 160, 1, 0, 0, 0, 166, 163, 1, 0, 0, 0, 167, 170, 1, 0, 0, 0, 168, 166, 1, 0, 0, 0, 168, 169, 1, 0, 0, 0, 169, 11, 1, 0, 0, 0, 170, 168, 1, 0, 0, 0, 171, 173, 3, 14, 7, 0, 172, 174, 5, 44, 0, 0, 173, 172, 1, 0, 0, 0, 173, 174, 1, 0, 0, 0, 174, 175, 1, 0, 0, 0, 175, 176, 5, 43, 0, 0, 176, 177, 3, 78, 39, 0, 177, 186, 1, 0, 0, 0, 178, 180, 3, 14, 7, 0, 179, 181, 5, 44, 0, 0, 180, 179, 1, 0, 0, 0, 180, 181, 1, 0, 0, 0, 181, 182, 1, 0, 0, 0, 182, 183, 5, 49, 0, 0, 183, 184, 3, 78, 39, 0, 184, 186, 1, 0, 0, 0, 185, 171, 1, 0, 0, 0, 185, 178, 1, 0, 0, 0, 186, 13, 1, 0, 0, 0, 187, 193, 3, 16, 8, 0, 188, 189, 3, 16, 8, 0, 189, 190, 3, 80, 40, 0, 190, 191, 3, 16, 8, 0, 191, 193, 1, 0, 0, 0, 192, 187, 1, 0, 0, 0, 192, 188, 1, 0, 0, 0, 193, 15, 1, 0, 0, 0, 194, 195, 6, 8, -1, 0, 195, 199, 3, 18, 9, 0, 196, 197, 7, 0, 0, 0, 197, 199, 3, 16, 8, 3, 198, 194, 1, 0, 0, 0, 198, 196, 1, 0, 0, 0, 199, 208, 1, 0, 0, 0, 200, 201, 10, 2, 0, 0, 201, 202, 7, 1, 0, 0, 202, 207, 3, 16, 8, 3, 203, 204, 10, 1, 0, 0, 204, 205, 7, 0, 0, 0, 205, 207, 3, 16, 8, 2, 206, 200, 1, 0, 0, 0, 206, 203, 1, 0, 0, 0, 207, 210, 1, 0, 0, 0, 208, 206, 1, 0, 0, 0, 208, 209, 1, 0, 0, 0, 209, 17, 1, 0, 0, 0, 210, 208, 1, 0, 0, 0, 211, 232, 3, 44, 22, 0, 212, 232, 3, 40, 20, 0, 213, 214, 5, 40, 0, 0, 214, 215, 3, 10, 5, 0, 215, 216, 5, 50, 0, 0, 216, 232, 1, 0, 0, 0, 217, 218, 3, 42, 21, 0, 218, 227, 5, 40, 0, 0, 219, 224, 3, 10, 5, 0, 220, 221, 5, 34, 0, 0, 221, 223, 3, 10, 5, 0, 222, 220, 1, 0, 0, 0, 223, 226, 1, 0, 0, 0, 224, 222, 1, 0, 0, 0, 224, 225, 1, 0, 0, 0, 225, 228, 1, 0, 0, 0, 226, 224, 1, 0, 0, 0, 227, 219, 1, 0, 0, 0, 227, 228, 1, 0, 0, 0, 228, 229, 1, 0, 0, 0, 229, 230, 5, 50, 0, 0, 230, 232, 1, 0, 0, 0, 231, 211, 1, 0, 0, 0, 231, 212, 1, 0, 0, 0, 231, 213, 1, 0, 0, 0, 231, 217, 1, 0, 0, 0, 232, 19, 1, 0, 0, 0, 233, 234, 5, 14, 0, 0, 234, 235, 3, 22, 11, 0, 235, 21, 1, 0, 0, 0, 236, 241, 3, 24, 12, 0, 237, 238, 5, 34, 0, 0, 238, 240, 3, 24, 12, 0, 239, 237, 1, 0, 0, 0, 240, 243, 1, 0, 0, 0, 241, 239, 1, 0, 0, 0, 241, 242, 1, 0, 0, 0, 242, 23, 1, 0, 0, 0, 243, 241, 1, 0, 0, 0, 244, 250, 3, 10, 5, 0, 245, 246, 3, 40, 20, 0, 246, 247, 5, 33, 0, 0, 247, 248, 3, 10, 5, 0, 248, 250, 1, 0, 0, 0, 249, 244, 1, 0, 0, 0, 249, 245, 1, 0, 0, 0, 250, 25, 1, 0, 0, 0, 251, 252, 5, 6, 0, 0, 252, 257, 3, 38, 19, 0, 253, 254, 5, 34, 0, 0, 254, 256, 3, 38, 19, 0, 255, 253, 1, 0, 0, 0, 256, 259, 1, 0, 0, 0, 257, 255, 1, 0, 0, 0, 257, 258, 1, 0, 0, 0, 258, 261, 1, 0, 0, 0, 259, 257, 1, 0, 0, 0, 260, 262, 3, 28, 14, 0, 261, 260, 1, 0, 0, 0, 261, 262, 1, 0, 0, 0, 262, 27, 1, 0, 0, 0, 263, 264, 5, 65, 0, 0, 264, 265, 5, 73, 0, 0, 265, 270, 3, 38, 19, 0, 266, 267, 5, 34, 0, 0, 267, 269, 3, 38, 19, 0, 268, 266, 1, 0, 0, 0, 269, 272, 1, 0, 0, 0, 270, 268, 1, 0, 0, 0, 270, 271, 1, 0, 0, 0, 271, 273, 1, 0, 0, 0, 272, 270, 1, 0, 0, 0, 273, 274, 5, 66, 0, 0, 274, 29, 1, 0, 0, 0, 275, 276, 5, 4, 0, 0, 276, 277, 3, 22, 11, 0, 277, 31, 1, 0, 0, 0, 278, 280, 5, 17, 0, 0, 279, 281, 3, 22, 11, 0, 280, 279, 1, 0, 0, 0, 280, 281, 1, 0, 0, 0, 281, 284, 1, 0, 0, 0, 282, 283, 5, 30, 0, 0, 283, 285, 3, 36, 18, 0, 284, 282, 1, 0, 0, 0, 284, 285, 1, 0, 0, 0, 285, 33, 1, 0, 0, 0, 286, 287, 5, 8, 0, 0, 287, 290, 3, 22, 11, 0, 288, 289, 5, 30, 0, 0, 289, 291, 3, 36, 18, 0, 290, 288, 1, 0, 0, 0, 290, 291, 1, 0, 0, 0, 291, 35, 1, 0, 0, 0, 292, 297, 3, 40, 20, 0, 293, 294, 5, 34, 0, 0, 294, 296, 3, 40, 20, 0, 295, 293, 1, 0, 0, 0, 296, 299, 1, 0, 0, 0, 297, 295, 1, 0, 0, 0, 297, 298, 1, 0, 0, 0, 298, 37, 1, 0, 0, 0, 299, 297, 1, 0, 0, 0, 300, 301, 7, 2, 0, 0, 301, 39, 1, 0, 0, 0, 302, 307, 3, 42, 21, 0, 303, 304, 5, 36, 0, 0, 304, 306, 3, 42, 21, 0, 305, 303, 1, 0, 0, 0, 306, 309, 1, 0, 0, 0, 307, 305, 1, 0, 0, 0, 307, 308, 1, 0, 0, 0, 308, 41, 1, 0, 0, 0, 309, 307, 1, 0, 0, 0, 310, 311, 7, 3, 0, 0, 311, 43, 1, 0, 0, 0, 312, 355, 5, 45, 0, 0, 313, 314, 3, 76, 38, 0, 314, 315, 5, 67, 0, 0, 315, 355, 1, 0, 0, 0, 316, 355, 3, 74, 37, 0, 317, 355, 3, 76, 38, 0, 318, 355, 3, 70, 35, 0, 319, 355, 5, 48, 0, 0, 320, 355, 3, 78, 39, 0, 321, 322, 5, 65, 0, 0, 322, 327, 3, 72, 36, 0, 323, 324, 5, 34, 0, 0, 324, 326, 3, 72, 36, 0, 325, 323, 1, 0, 0, 0, 326, 329, 1, 0, 0, 0, 327, 325, 1, 0, 0, 0, 327, 328, 1, 0, 0, 0, 328, 330, 1, 0, 0, 0, 329, 327, 1, 0, 0, 0, 330, 331, 5, 66, 0, 0, 331, 355, 1, 0, 0, 0, 332, 333, 5, 65, 0, 0, 333, 338, 3, 70, 35, 0, 334, 335, 5, 34, 0, 0, 335, 337, 3, 70, 35, 0, 336, 334, 1, 0, 0, 0, 337, 340, 1, 0, 0, 0, 338, 336, 1, 0, 0, 0, 338, 339, 1, 0, 0, 0, 339, 341, 1, 0, 0, 0, 340, 338, 1, 0, 0, 0, 341, 342, 5, 66, 0, 0, 342, 355, 1, 0, 0, 0, 343, 344, 5, 65, 0, 0, 344, 349, 3, 78, 39, 0, 345, 346, 5, 34, 0, 0, 346, 348, 3, 78, 39, 0, 347, 345, 1, 0, 0, 0, 348, 351, 1, 0, 0, 0, 349, 347, 1, 0, 0, 0, 349, 350, 1, 0, 0, 0, 350, 352, 1, 0, 0, 0, 351, 349, 1, 0, 0, 0, 352, 353, 5, 66, 0, 0, 353, 355, 1, 0, 0, 0, 354, 312, 1, 0, 0, 0, 354, 313, 1, 0, 0, 0, 354, 316, 1, 0, 0, 0, 354, 317, 1, 0, 0, 0, 354, 318, 1, 0, 0, 0, 354, 319, 1, 0, 0, 0, 354, 320, 1, 0, 0, 0, 354, 321, 1, 0, 0, 0, 354, 332, 1, 0, 0, 0, 354, 343, 1, 0, 0, 0, 355, 45, 1, 0, 0, 0, 356, 357, 5, 10, 0, 0, 357, 358, 5, 28, 0, 0, 358, 47, 1, 0, 0, 0, 359, 360, 5, 16, 0, 0, 360, 365, 3, 50, 25, 0, 361, 362, 5, 34, 0, 0, 362, 364, 3, 50, 25, 0, 363, 361, 1, 0, 0, 0, 364, 367, 1, 0, 0, 0, 365, 363, 1, 0, 0, 0, 365, 366, 1, 0, 0, 0, 366, 49, 1, 0, 0, 0, 367, 365, 1, 0, 0, 0, 368, 370, 3, 10, 5, 0, 369, 371, 7, 4, 0, 0, 370, 369, 1, 0, 0, 0, 370, 371, 1, 0, 0, 0, 371, 374, 1, 0, 0, 0, 372, 373, 5, 46, 0, 0, 373, 375, 7, 5, 0, 0, 374, 372, 1, 0, 0, 0, 374, 375, 1, 0, 0, 0, 375, 51, 1, 0, 0, 0, 376, 377, 5, 9, 0, 0, 377, 382, 3, 38, 19, 0, 378, 379, 5, 34, 0, 0, 379, 381, 3, 38, 19, 0, 380, 378, 1, 0, 0, 0, 381, 384, 1, 0, 0, 0, 382, 380, 1, 0, 0, 0, 382, 383, 1, 0, 0, 0, 383, 395, 1, 0, 0, 0, 384, 382, 1, 0, 0, 0, 385, 386, 5, 12, 0, 0, 386, 391, 3, 38, 19, 0, 387, 388, 5, 34, 0, 0, 388, 390, 3, 38, 19, 0, 389, 387, 1, 0, 0, 0, 390, 393, 1, 0, 0, 0, 391, 389, 1, 0, 0, 0, 391, 392, 1, 0, 0, 0, 392, 395, 1, 0, 0, 0, 393, 391, 1, 0, 0, 0, 394, 376, 1, 0, 0, 0, 394, 385, 1, 0, 0, 0, 395, 53, 1, 0, 0, 0, 396, 397, 5, 2, 0, 0, 397, 402, 3, 38, 19, 0, 398, 399, 5, 34, 0, 0, 399, 401, 3, 38, 19, 0, 400, 398, 1, 0, 0, 0, 401, 404, 1, 0, 0, 0, 402, 400, 1, 0, 0, 0, 402, 403, 1, 0, 0, 0, 403, 55, 1, 0, 0, 0, 404, 402, 1, 0, 0, 0, 405, 406, 5, 13, 0, 0, 406, 411, 3, 58, 29, 0, 407, 408, 5, 34, 0, 0, 408, 410, 3, 58, 29, 0, 409, 407, 1, 0, 0, 0, 410, 413, 1, 0, 0, 0, 411, 409, 1, 0, 0, 0, 411, 412, 1, 0, 0, 0, 412, 57, 1, 0, 0, 0, 413, 411, 1, 0, 0, 0, 414, 415, 3, 38, 19, 0, 415, 416, 5, 72, 0, 0, 416, 417, 3, 38, 19, 0, 417, 59, 1, 0, 0, 0, 418, 419, 5, 1, 0, 0, 419, 420, 3, 18, 9, 0, 420, 422, 3, 78, 39, 0, 421, 423, 3, 66, 33, 0, 422, 421, 1, 0, 0, 0, 422, 423, 1, 0, 0, 0, 423, 61, 1, 0, 0, 0, 424, 425, 5, 7, 0, 0, 425, 426, 3, 18, 9, 0, 426, 427, 3, 78, 39, 0, 427, 63, 1, 0, 0, 0, 428, 429, 5, 11, 0, 0, 429, 430, 3, 38, 19, 0, 430, 65, 1, 0, 0, 0, 431, 436, 3, 68, 34, 0, 432, 433, 5, 34, 0, 0, 433, 435, 3, 68, 34, 0, 434, 432, 1, 0, 0, 0, 435, 438, 1, 0, 0, 0, 436, 434, 1, 0, 0, 0, 436, 437, 1, 0, 0, 0, 437, 67, 1, 0, 0, 0, 438, 436, 1, 0, 0, 0, 439, 440, 3, 42, 21, 0, 440, 441, 5, 33, 0, 0, 441, 442, 3, 44, 22, 0, 442, 69, 1, 0, 0, 0, 443, 444, 7, 6, 0, 0, 444, 71, 1, 0, 0, 0, 445, 448, 3, 74, 37, 0, 446, 448, 3, 76, 38, 0, 447, 445, 1, 0, 0, 0, 447, 446, 1, 0, 0, 0, 448, 73, 1, 0, 0, 0, 449, 451, 7, 0, 0, 0, 450, 449, 1, 0, 0, 0, 450, 451, 1, 0, 0, 0, 451, 452, 1, 0, 0, 0, 452, 453, 5, 29, 0, 0, 453, 75, 1, 0, 0, 0, 454, 456, 7, 0, 0, 0, 455, 454, 1, 0, 0, 0, 455, 456, 1, 0, 0, 0, 456, 457, 1, 0, 0, 0, 457, 458, 5, 28, 0, 0, 458, 77, 1, 0, 0, 0, 459, 460, 5, 27, 0, 0, 460, 79, 1, 0, 0, 0, 461, 462, 7, 7, 0, 0, 462, 81, 1, 0, 0, 0, 463, 464, 5, 5, 0, 0, 464, 465, 3, 84, 42, 0, 465, 83, 1, 0, 0, 0, 466, 467, 5, 65, 0, 0, 467, 468, 3, 2, 1, 0, 468, 469, 5, 66, 0, 0, 469, 85, 1, 0, 0, 0, 470, 471, 5, 15, 0, 0, 471, 475, 5, 52, 0, 0, 472, 473, 5, 15, 0, 0, 473, 475, 5, 53, 0, 0, 474, 470, 1, 0, 0, 0, 474, 472, 1, 0, 0, 0, 475, 87, 1, 0, 0, 0, 476, 477, 5, 3, 0, 0, 477, 480, 3, 38, 19, 0, 478, 479, 5, 74, 0, 0, 479, 481, 3, 38, 19, 0, 480, 478, 1, 0, 0, 0, 480, 481, 1, 0, 0, 0, 481, 491, 1, 0, 0, 0, 482, 483, 5, 75, 0, 0, 483, 488, 3, 90, 45, 0, 484, 485, 5, 34, 0, 0, 485, 487, 3, 90, 45, 0, 486, 484, 1, 0, 0, 0, 487, 490, 1, 0, 0, 0, 488, 486, 1, 0, 0, 0, 488, 489, 1, 0, 0, 0, 489, 492, 1, 0, 0, 0, 490, 488, 1, 0, 0, 0, 491, 482, 1, 0, 0, 0, 491, 492, 1, 0, 0, 0, 492, 89, 1, 0, 0, 0, 493, 494, 3, 38, 19, 0, 494, 495, 5, 33, 0, 0, 495, 497, 1, 0, 0, 0, 496, 493, 1, 0, 0, 0, 496, 497, 1, 0, 0, 0, 497, 498, 1, 0, 0, 0, 498, 499, 3, 38, 19, 0, 499, 91, 1, 0, 0, 0, 51, 103, 110, 125, 137, 146, 154, 158, 166, 168, 173, 180, 185, 192, 198, 206, 208, 224, 227, 231, 241, 249, 257, 261, 270, 280, 284, 290, 297, 307, 327, 338, 349, 354, 365, 370, 374, 382, 391, 394, 402, 411, 422, 436, 447, 450, 455, 474, 480, 488, 491, 496] \ No newline at end of file diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java new file mode 100644 index 0000000000000..b5eac5f58f9f6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java @@ -0,0 +1,4621 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; +import org.antlr.v4.runtime.atn.*; +import org.antlr.v4.runtime.dfa.DFA; +import org.antlr.v4.runtime.*; +import org.antlr.v4.runtime.misc.*; +import org.antlr.v4.runtime.tree.*; +import java.util.List; +import java.util.Iterator; +import java.util.ArrayList; + +@SuppressWarnings({"all", "warnings", "unchecked", "unused", "cast", "CheckReturnValue"}) +public class EsqlBaseParser extends Parser { + static { RuntimeMetaData.checkVersion("4.11.1", RuntimeMetaData.VERSION); } + + protected static final DFA[] _decisionToDFA; + protected static final PredictionContextCache _sharedContextCache = + new PredictionContextCache(); + public static final int + DISSECT=1, DROP=2, ENRICH=3, EVAL=4, EXPLAIN=5, FROM=6, GROK=7, INLINESTATS=8, + KEEP=9, LIMIT=10, MV_EXPAND=11, PROJECT=12, RENAME=13, ROW=14, SHOW=15, + SORT=16, STATS=17, WHERE=18, UNKNOWN_CMD=19, LINE_COMMENT=20, MULTILINE_COMMENT=21, + WS=22, EXPLAIN_WS=23, EXPLAIN_LINE_COMMENT=24, EXPLAIN_MULTILINE_COMMENT=25, + PIPE=26, STRING=27, INTEGER_LITERAL=28, DECIMAL_LITERAL=29, BY=30, AND=31, + ASC=32, ASSIGN=33, COMMA=34, DESC=35, DOT=36, FALSE=37, FIRST=38, LAST=39, + LP=40, IN=41, IS=42, LIKE=43, NOT=44, NULL=45, NULLS=46, OR=47, PARAM=48, + RLIKE=49, RP=50, TRUE=51, INFO=52, FUNCTIONS=53, EQ=54, NEQ=55, LT=56, + LTE=57, GT=58, GTE=59, PLUS=60, MINUS=61, ASTERISK=62, SLASH=63, PERCENT=64, + OPENING_BRACKET=65, CLOSING_BRACKET=66, UNQUOTED_IDENTIFIER=67, QUOTED_IDENTIFIER=68, + EXPR_LINE_COMMENT=69, EXPR_MULTILINE_COMMENT=70, EXPR_WS=71, AS=72, METADATA=73, + ON=74, WITH=75, SRC_UNQUOTED_IDENTIFIER=76, SRC_QUOTED_IDENTIFIER=77, + SRC_LINE_COMMENT=78, SRC_MULTILINE_COMMENT=79, SRC_WS=80, EXPLAIN_PIPE=81; + public static final int + RULE_singleStatement = 0, RULE_query = 1, RULE_sourceCommand = 2, RULE_processingCommand = 3, + RULE_whereCommand = 4, RULE_booleanExpression = 5, RULE_regexBooleanExpression = 6, + RULE_valueExpression = 7, RULE_operatorExpression = 8, RULE_primaryExpression = 9, + RULE_rowCommand = 10, RULE_fields = 11, RULE_field = 12, RULE_fromCommand = 13, + RULE_metadata = 14, RULE_evalCommand = 15, RULE_statsCommand = 16, RULE_inlinestatsCommand = 17, + RULE_grouping = 18, RULE_sourceIdentifier = 19, RULE_qualifiedName = 20, + RULE_identifier = 21, RULE_constant = 22, RULE_limitCommand = 23, RULE_sortCommand = 24, + RULE_orderExpression = 25, RULE_keepCommand = 26, RULE_dropCommand = 27, + RULE_renameCommand = 28, RULE_renameClause = 29, RULE_dissectCommand = 30, + RULE_grokCommand = 31, RULE_mvExpandCommand = 32, RULE_commandOptions = 33, + RULE_commandOption = 34, RULE_booleanValue = 35, RULE_numericValue = 36, + RULE_decimalValue = 37, RULE_integerValue = 38, RULE_string = 39, RULE_comparisonOperator = 40, + RULE_explainCommand = 41, RULE_subqueryExpression = 42, RULE_showCommand = 43, + RULE_enrichCommand = 44, RULE_enrichWithClause = 45; + private static String[] makeRuleNames() { + return new String[] { + "singleStatement", "query", "sourceCommand", "processingCommand", "whereCommand", + "booleanExpression", "regexBooleanExpression", "valueExpression", "operatorExpression", + "primaryExpression", "rowCommand", "fields", "field", "fromCommand", + "metadata", "evalCommand", "statsCommand", "inlinestatsCommand", "grouping", + "sourceIdentifier", "qualifiedName", "identifier", "constant", "limitCommand", + "sortCommand", "orderExpression", "keepCommand", "dropCommand", "renameCommand", + "renameClause", "dissectCommand", "grokCommand", "mvExpandCommand", "commandOptions", + "commandOption", "booleanValue", "numericValue", "decimalValue", "integerValue", + "string", "comparisonOperator", "explainCommand", "subqueryExpression", + "showCommand", "enrichCommand", "enrichWithClause" + }; + } + public static final String[] ruleNames = makeRuleNames(); + + private static String[] makeLiteralNames() { + return new String[] { + null, "'dissect'", "'drop'", "'enrich'", "'eval'", "'explain'", "'from'", + "'grok'", "'inlinestats'", "'keep'", "'limit'", "'mv_expand'", "'project'", + "'rename'", "'row'", "'show'", "'sort'", "'stats'", "'where'", null, + null, null, null, null, null, null, null, null, null, null, "'by'", "'and'", + "'asc'", null, null, "'desc'", "'.'", "'false'", "'first'", "'last'", + "'('", "'in'", "'is'", "'like'", "'not'", "'null'", "'nulls'", "'or'", + "'?'", "'rlike'", "')'", "'true'", "'info'", "'functions'", "'=='", "'!='", + "'<'", "'<='", "'>'", "'>='", "'+'", "'-'", "'*'", "'/'", "'%'", null, + "']'", null, null, null, null, null, "'as'", "'metadata'", "'on'", "'with'" + }; + } + private static final String[] _LITERAL_NAMES = makeLiteralNames(); + private static String[] makeSymbolicNames() { + return new String[] { + null, "DISSECT", "DROP", "ENRICH", "EVAL", "EXPLAIN", "FROM", "GROK", + "INLINESTATS", "KEEP", "LIMIT", "MV_EXPAND", "PROJECT", "RENAME", "ROW", + "SHOW", "SORT", "STATS", "WHERE", "UNKNOWN_CMD", "LINE_COMMENT", "MULTILINE_COMMENT", + "WS", "EXPLAIN_WS", "EXPLAIN_LINE_COMMENT", "EXPLAIN_MULTILINE_COMMENT", + "PIPE", "STRING", "INTEGER_LITERAL", "DECIMAL_LITERAL", "BY", "AND", + "ASC", "ASSIGN", "COMMA", "DESC", "DOT", "FALSE", "FIRST", "LAST", "LP", + "IN", "IS", "LIKE", "NOT", "NULL", "NULLS", "OR", "PARAM", "RLIKE", "RP", + "TRUE", "INFO", "FUNCTIONS", "EQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", + "MINUS", "ASTERISK", "SLASH", "PERCENT", "OPENING_BRACKET", "CLOSING_BRACKET", + "UNQUOTED_IDENTIFIER", "QUOTED_IDENTIFIER", "EXPR_LINE_COMMENT", "EXPR_MULTILINE_COMMENT", + "EXPR_WS", "AS", "METADATA", "ON", "WITH", "SRC_UNQUOTED_IDENTIFIER", + "SRC_QUOTED_IDENTIFIER", "SRC_LINE_COMMENT", "SRC_MULTILINE_COMMENT", + "SRC_WS", "EXPLAIN_PIPE" + }; + } + private static final String[] _SYMBOLIC_NAMES = makeSymbolicNames(); + public static final Vocabulary VOCABULARY = new VocabularyImpl(_LITERAL_NAMES, _SYMBOLIC_NAMES); + + /** + * @deprecated Use {@link #VOCABULARY} instead. + */ + @Deprecated + public static final String[] tokenNames; + static { + tokenNames = new String[_SYMBOLIC_NAMES.length]; + for (int i = 0; i < tokenNames.length; i++) { + tokenNames[i] = VOCABULARY.getLiteralName(i); + if (tokenNames[i] == null) { + tokenNames[i] = VOCABULARY.getSymbolicName(i); + } + + if (tokenNames[i] == null) { + tokenNames[i] = ""; + } + } + } + + @Override + @Deprecated + public String[] getTokenNames() { + return tokenNames; + } + + @Override + + public Vocabulary getVocabulary() { + return VOCABULARY; + } + + @Override + public String getGrammarFileName() { return "java-escape"; } + + @Override + public String[] getRuleNames() { return ruleNames; } + + @Override + public String getSerializedATN() { return _serializedATN; } + + @Override + public ATN getATN() { return _ATN; } + + public EsqlBaseParser(TokenStream input) { + super(input); + _interp = new ParserATNSimulator(this,_ATN,_decisionToDFA,_sharedContextCache); + } + + @SuppressWarnings("CheckReturnValue") + public static class SingleStatementContext extends ParserRuleContext { + public QueryContext query() { + return getRuleContext(QueryContext.class,0); + } + public TerminalNode EOF() { return getToken(EsqlBaseParser.EOF, 0); } + public SingleStatementContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_singleStatement; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSingleStatement(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSingleStatement(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSingleStatement(this); + else return visitor.visitChildren(this); + } + } + + public final SingleStatementContext singleStatement() throws RecognitionException { + SingleStatementContext _localctx = new SingleStatementContext(_ctx, getState()); + enterRule(_localctx, 0, RULE_singleStatement); + try { + enterOuterAlt(_localctx, 1); + { + setState(92); + query(0); + setState(93); + match(EOF); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class QueryContext extends ParserRuleContext { + public QueryContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_query; } + + public QueryContext() { } + public void copyFrom(QueryContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class CompositeQueryContext extends QueryContext { + public QueryContext query() { + return getRuleContext(QueryContext.class,0); + } + public TerminalNode PIPE() { return getToken(EsqlBaseParser.PIPE, 0); } + public ProcessingCommandContext processingCommand() { + return getRuleContext(ProcessingCommandContext.class,0); + } + public CompositeQueryContext(QueryContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterCompositeQuery(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitCompositeQuery(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitCompositeQuery(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class SingleCommandQueryContext extends QueryContext { + public SourceCommandContext sourceCommand() { + return getRuleContext(SourceCommandContext.class,0); + } + public SingleCommandQueryContext(QueryContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSingleCommandQuery(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSingleCommandQuery(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSingleCommandQuery(this); + else return visitor.visitChildren(this); + } + } + + public final QueryContext query() throws RecognitionException { + return query(0); + } + + private QueryContext query(int _p) throws RecognitionException { + ParserRuleContext _parentctx = _ctx; + int _parentState = getState(); + QueryContext _localctx = new QueryContext(_ctx, _parentState); + QueryContext _prevctx = _localctx; + int _startState = 2; + enterRecursionRule(_localctx, 2, RULE_query, _p); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + { + _localctx = new SingleCommandQueryContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + + setState(96); + sourceCommand(); + } + _ctx.stop = _input.LT(-1); + setState(103); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,0,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + if ( _parseListeners!=null ) triggerExitRuleEvent(); + _prevctx = _localctx; + { + { + _localctx = new CompositeQueryContext(new QueryContext(_parentctx, _parentState)); + pushNewRecursionContext(_localctx, _startState, RULE_query); + setState(98); + if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); + setState(99); + match(PIPE); + setState(100); + processingCommand(); + } + } + } + setState(105); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,0,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + unrollRecursionContexts(_parentctx); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class SourceCommandContext extends ParserRuleContext { + public ExplainCommandContext explainCommand() { + return getRuleContext(ExplainCommandContext.class,0); + } + public FromCommandContext fromCommand() { + return getRuleContext(FromCommandContext.class,0); + } + public RowCommandContext rowCommand() { + return getRuleContext(RowCommandContext.class,0); + } + public ShowCommandContext showCommand() { + return getRuleContext(ShowCommandContext.class,0); + } + public SourceCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_sourceCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSourceCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSourceCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSourceCommand(this); + else return visitor.visitChildren(this); + } + } + + public final SourceCommandContext sourceCommand() throws RecognitionException { + SourceCommandContext _localctx = new SourceCommandContext(_ctx, getState()); + enterRule(_localctx, 4, RULE_sourceCommand); + try { + setState(110); + _errHandler.sync(this); + switch (_input.LA(1)) { + case EXPLAIN: + enterOuterAlt(_localctx, 1); + { + setState(106); + explainCommand(); + } + break; + case FROM: + enterOuterAlt(_localctx, 2); + { + setState(107); + fromCommand(); + } + break; + case ROW: + enterOuterAlt(_localctx, 3); + { + setState(108); + rowCommand(); + } + break; + case SHOW: + enterOuterAlt(_localctx, 4); + { + setState(109); + showCommand(); + } + break; + default: + throw new NoViableAltException(this); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ProcessingCommandContext extends ParserRuleContext { + public EvalCommandContext evalCommand() { + return getRuleContext(EvalCommandContext.class,0); + } + public InlinestatsCommandContext inlinestatsCommand() { + return getRuleContext(InlinestatsCommandContext.class,0); + } + public LimitCommandContext limitCommand() { + return getRuleContext(LimitCommandContext.class,0); + } + public KeepCommandContext keepCommand() { + return getRuleContext(KeepCommandContext.class,0); + } + public SortCommandContext sortCommand() { + return getRuleContext(SortCommandContext.class,0); + } + public StatsCommandContext statsCommand() { + return getRuleContext(StatsCommandContext.class,0); + } + public WhereCommandContext whereCommand() { + return getRuleContext(WhereCommandContext.class,0); + } + public DropCommandContext dropCommand() { + return getRuleContext(DropCommandContext.class,0); + } + public RenameCommandContext renameCommand() { + return getRuleContext(RenameCommandContext.class,0); + } + public DissectCommandContext dissectCommand() { + return getRuleContext(DissectCommandContext.class,0); + } + public GrokCommandContext grokCommand() { + return getRuleContext(GrokCommandContext.class,0); + } + public EnrichCommandContext enrichCommand() { + return getRuleContext(EnrichCommandContext.class,0); + } + public MvExpandCommandContext mvExpandCommand() { + return getRuleContext(MvExpandCommandContext.class,0); + } + public ProcessingCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_processingCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterProcessingCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitProcessingCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitProcessingCommand(this); + else return visitor.visitChildren(this); + } + } + + public final ProcessingCommandContext processingCommand() throws RecognitionException { + ProcessingCommandContext _localctx = new ProcessingCommandContext(_ctx, getState()); + enterRule(_localctx, 6, RULE_processingCommand); + try { + setState(125); + _errHandler.sync(this); + switch (_input.LA(1)) { + case EVAL: + enterOuterAlt(_localctx, 1); + { + setState(112); + evalCommand(); + } + break; + case INLINESTATS: + enterOuterAlt(_localctx, 2); + { + setState(113); + inlinestatsCommand(); + } + break; + case LIMIT: + enterOuterAlt(_localctx, 3); + { + setState(114); + limitCommand(); + } + break; + case KEEP: + case PROJECT: + enterOuterAlt(_localctx, 4); + { + setState(115); + keepCommand(); + } + break; + case SORT: + enterOuterAlt(_localctx, 5); + { + setState(116); + sortCommand(); + } + break; + case STATS: + enterOuterAlt(_localctx, 6); + { + setState(117); + statsCommand(); + } + break; + case WHERE: + enterOuterAlt(_localctx, 7); + { + setState(118); + whereCommand(); + } + break; + case DROP: + enterOuterAlt(_localctx, 8); + { + setState(119); + dropCommand(); + } + break; + case RENAME: + enterOuterAlt(_localctx, 9); + { + setState(120); + renameCommand(); + } + break; + case DISSECT: + enterOuterAlt(_localctx, 10); + { + setState(121); + dissectCommand(); + } + break; + case GROK: + enterOuterAlt(_localctx, 11); + { + setState(122); + grokCommand(); + } + break; + case ENRICH: + enterOuterAlt(_localctx, 12); + { + setState(123); + enrichCommand(); + } + break; + case MV_EXPAND: + enterOuterAlt(_localctx, 13); + { + setState(124); + mvExpandCommand(); + } + break; + default: + throw new NoViableAltException(this); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class WhereCommandContext extends ParserRuleContext { + public TerminalNode WHERE() { return getToken(EsqlBaseParser.WHERE, 0); } + public BooleanExpressionContext booleanExpression() { + return getRuleContext(BooleanExpressionContext.class,0); + } + public WhereCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_whereCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterWhereCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitWhereCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitWhereCommand(this); + else return visitor.visitChildren(this); + } + } + + public final WhereCommandContext whereCommand() throws RecognitionException { + WhereCommandContext _localctx = new WhereCommandContext(_ctx, getState()); + enterRule(_localctx, 8, RULE_whereCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(127); + match(WHERE); + setState(128); + booleanExpression(0); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class BooleanExpressionContext extends ParserRuleContext { + public BooleanExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_booleanExpression; } + + public BooleanExpressionContext() { } + public void copyFrom(BooleanExpressionContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class LogicalNotContext extends BooleanExpressionContext { + public TerminalNode NOT() { return getToken(EsqlBaseParser.NOT, 0); } + public BooleanExpressionContext booleanExpression() { + return getRuleContext(BooleanExpressionContext.class,0); + } + public LogicalNotContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterLogicalNot(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitLogicalNot(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitLogicalNot(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class BooleanDefaultContext extends BooleanExpressionContext { + public ValueExpressionContext valueExpression() { + return getRuleContext(ValueExpressionContext.class,0); + } + public BooleanDefaultContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterBooleanDefault(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitBooleanDefault(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitBooleanDefault(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class IsNullContext extends BooleanExpressionContext { + public ValueExpressionContext valueExpression() { + return getRuleContext(ValueExpressionContext.class,0); + } + public TerminalNode IS() { return getToken(EsqlBaseParser.IS, 0); } + public TerminalNode NULL() { return getToken(EsqlBaseParser.NULL, 0); } + public TerminalNode NOT() { return getToken(EsqlBaseParser.NOT, 0); } + public IsNullContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterIsNull(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitIsNull(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitIsNull(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class RegexExpressionContext extends BooleanExpressionContext { + public RegexBooleanExpressionContext regexBooleanExpression() { + return getRuleContext(RegexBooleanExpressionContext.class,0); + } + public RegexExpressionContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterRegexExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitRegexExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitRegexExpression(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class LogicalInContext extends BooleanExpressionContext { + public List valueExpression() { + return getRuleContexts(ValueExpressionContext.class); + } + public ValueExpressionContext valueExpression(int i) { + return getRuleContext(ValueExpressionContext.class,i); + } + public TerminalNode IN() { return getToken(EsqlBaseParser.IN, 0); } + public TerminalNode LP() { return getToken(EsqlBaseParser.LP, 0); } + public TerminalNode RP() { return getToken(EsqlBaseParser.RP, 0); } + public TerminalNode NOT() { return getToken(EsqlBaseParser.NOT, 0); } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public LogicalInContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterLogicalIn(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitLogicalIn(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitLogicalIn(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class LogicalBinaryContext extends BooleanExpressionContext { + public BooleanExpressionContext left; + public Token operator; + public BooleanExpressionContext right; + public List booleanExpression() { + return getRuleContexts(BooleanExpressionContext.class); + } + public BooleanExpressionContext booleanExpression(int i) { + return getRuleContext(BooleanExpressionContext.class,i); + } + public TerminalNode AND() { return getToken(EsqlBaseParser.AND, 0); } + public TerminalNode OR() { return getToken(EsqlBaseParser.OR, 0); } + public LogicalBinaryContext(BooleanExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterLogicalBinary(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitLogicalBinary(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitLogicalBinary(this); + else return visitor.visitChildren(this); + } + } + + public final BooleanExpressionContext booleanExpression() throws RecognitionException { + return booleanExpression(0); + } + + private BooleanExpressionContext booleanExpression(int _p) throws RecognitionException { + ParserRuleContext _parentctx = _ctx; + int _parentState = getState(); + BooleanExpressionContext _localctx = new BooleanExpressionContext(_ctx, _parentState); + BooleanExpressionContext _prevctx = _localctx; + int _startState = 10; + enterRecursionRule(_localctx, 10, RULE_booleanExpression, _p); + int _la; + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(158); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,6,_ctx) ) { + case 1: + { + _localctx = new LogicalNotContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + + setState(131); + match(NOT); + setState(132); + booleanExpression(7); + } + break; + case 2: + { + _localctx = new BooleanDefaultContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(133); + valueExpression(); + } + break; + case 3: + { + _localctx = new RegexExpressionContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(134); + regexBooleanExpression(); + } + break; + case 4: + { + _localctx = new LogicalInContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(135); + valueExpression(); + setState(137); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==NOT) { + { + setState(136); + match(NOT); + } + } + + setState(139); + match(IN); + setState(140); + match(LP); + setState(141); + valueExpression(); + setState(146); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(142); + match(COMMA); + setState(143); + valueExpression(); + } + } + setState(148); + _errHandler.sync(this); + _la = _input.LA(1); + } + setState(149); + match(RP); + } + break; + case 5: + { + _localctx = new IsNullContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(151); + valueExpression(); + setState(152); + match(IS); + setState(154); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==NOT) { + { + setState(153); + match(NOT); + } + } + + setState(156); + match(NULL); + } + break; + } + _ctx.stop = _input.LT(-1); + setState(168); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,8,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + if ( _parseListeners!=null ) triggerExitRuleEvent(); + _prevctx = _localctx; + { + setState(166); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,7,_ctx) ) { + case 1: + { + _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); + ((LogicalBinaryContext)_localctx).left = _prevctx; + pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); + setState(160); + if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)"); + setState(161); + ((LogicalBinaryContext)_localctx).operator = match(AND); + setState(162); + ((LogicalBinaryContext)_localctx).right = booleanExpression(5); + } + break; + case 2: + { + _localctx = new LogicalBinaryContext(new BooleanExpressionContext(_parentctx, _parentState)); + ((LogicalBinaryContext)_localctx).left = _prevctx; + pushNewRecursionContext(_localctx, _startState, RULE_booleanExpression); + setState(163); + if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); + setState(164); + ((LogicalBinaryContext)_localctx).operator = match(OR); + setState(165); + ((LogicalBinaryContext)_localctx).right = booleanExpression(4); + } + break; + } + } + } + setState(170); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,8,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + unrollRecursionContexts(_parentctx); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class RegexBooleanExpressionContext extends ParserRuleContext { + public Token kind; + public StringContext pattern; + public ValueExpressionContext valueExpression() { + return getRuleContext(ValueExpressionContext.class,0); + } + public TerminalNode LIKE() { return getToken(EsqlBaseParser.LIKE, 0); } + public StringContext string() { + return getRuleContext(StringContext.class,0); + } + public TerminalNode NOT() { return getToken(EsqlBaseParser.NOT, 0); } + public TerminalNode RLIKE() { return getToken(EsqlBaseParser.RLIKE, 0); } + public RegexBooleanExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_regexBooleanExpression; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterRegexBooleanExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitRegexBooleanExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitRegexBooleanExpression(this); + else return visitor.visitChildren(this); + } + } + + public final RegexBooleanExpressionContext regexBooleanExpression() throws RecognitionException { + RegexBooleanExpressionContext _localctx = new RegexBooleanExpressionContext(_ctx, getState()); + enterRule(_localctx, 12, RULE_regexBooleanExpression); + int _la; + try { + setState(185); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) { + case 1: + enterOuterAlt(_localctx, 1); + { + setState(171); + valueExpression(); + setState(173); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==NOT) { + { + setState(172); + match(NOT); + } + } + + setState(175); + ((RegexBooleanExpressionContext)_localctx).kind = match(LIKE); + setState(176); + ((RegexBooleanExpressionContext)_localctx).pattern = string(); + } + break; + case 2: + enterOuterAlt(_localctx, 2); + { + setState(178); + valueExpression(); + setState(180); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==NOT) { + { + setState(179); + match(NOT); + } + } + + setState(182); + ((RegexBooleanExpressionContext)_localctx).kind = match(RLIKE); + setState(183); + ((RegexBooleanExpressionContext)_localctx).pattern = string(); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ValueExpressionContext extends ParserRuleContext { + public ValueExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_valueExpression; } + + public ValueExpressionContext() { } + public void copyFrom(ValueExpressionContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ValueExpressionDefaultContext extends ValueExpressionContext { + public OperatorExpressionContext operatorExpression() { + return getRuleContext(OperatorExpressionContext.class,0); + } + public ValueExpressionDefaultContext(ValueExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterValueExpressionDefault(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitValueExpressionDefault(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitValueExpressionDefault(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ComparisonContext extends ValueExpressionContext { + public OperatorExpressionContext left; + public OperatorExpressionContext right; + public ComparisonOperatorContext comparisonOperator() { + return getRuleContext(ComparisonOperatorContext.class,0); + } + public List operatorExpression() { + return getRuleContexts(OperatorExpressionContext.class); + } + public OperatorExpressionContext operatorExpression(int i) { + return getRuleContext(OperatorExpressionContext.class,i); + } + public ComparisonContext(ValueExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterComparison(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitComparison(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitComparison(this); + else return visitor.visitChildren(this); + } + } + + public final ValueExpressionContext valueExpression() throws RecognitionException { + ValueExpressionContext _localctx = new ValueExpressionContext(_ctx, getState()); + enterRule(_localctx, 14, RULE_valueExpression); + try { + setState(192); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,12,_ctx) ) { + case 1: + _localctx = new ValueExpressionDefaultContext(_localctx); + enterOuterAlt(_localctx, 1); + { + setState(187); + operatorExpression(0); + } + break; + case 2: + _localctx = new ComparisonContext(_localctx); + enterOuterAlt(_localctx, 2); + { + setState(188); + ((ComparisonContext)_localctx).left = operatorExpression(0); + setState(189); + comparisonOperator(); + setState(190); + ((ComparisonContext)_localctx).right = operatorExpression(0); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class OperatorExpressionContext extends ParserRuleContext { + public OperatorExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_operatorExpression; } + + public OperatorExpressionContext() { } + public void copyFrom(OperatorExpressionContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class OperatorExpressionDefaultContext extends OperatorExpressionContext { + public PrimaryExpressionContext primaryExpression() { + return getRuleContext(PrimaryExpressionContext.class,0); + } + public OperatorExpressionDefaultContext(OperatorExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterOperatorExpressionDefault(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitOperatorExpressionDefault(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitOperatorExpressionDefault(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ArithmeticBinaryContext extends OperatorExpressionContext { + public OperatorExpressionContext left; + public Token operator; + public OperatorExpressionContext right; + public List operatorExpression() { + return getRuleContexts(OperatorExpressionContext.class); + } + public OperatorExpressionContext operatorExpression(int i) { + return getRuleContext(OperatorExpressionContext.class,i); + } + public TerminalNode ASTERISK() { return getToken(EsqlBaseParser.ASTERISK, 0); } + public TerminalNode SLASH() { return getToken(EsqlBaseParser.SLASH, 0); } + public TerminalNode PERCENT() { return getToken(EsqlBaseParser.PERCENT, 0); } + public TerminalNode PLUS() { return getToken(EsqlBaseParser.PLUS, 0); } + public TerminalNode MINUS() { return getToken(EsqlBaseParser.MINUS, 0); } + public ArithmeticBinaryContext(OperatorExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterArithmeticBinary(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitArithmeticBinary(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitArithmeticBinary(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ArithmeticUnaryContext extends OperatorExpressionContext { + public Token operator; + public OperatorExpressionContext operatorExpression() { + return getRuleContext(OperatorExpressionContext.class,0); + } + public TerminalNode MINUS() { return getToken(EsqlBaseParser.MINUS, 0); } + public TerminalNode PLUS() { return getToken(EsqlBaseParser.PLUS, 0); } + public ArithmeticUnaryContext(OperatorExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterArithmeticUnary(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitArithmeticUnary(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitArithmeticUnary(this); + else return visitor.visitChildren(this); + } + } + + public final OperatorExpressionContext operatorExpression() throws RecognitionException { + return operatorExpression(0); + } + + private OperatorExpressionContext operatorExpression(int _p) throws RecognitionException { + ParserRuleContext _parentctx = _ctx; + int _parentState = getState(); + OperatorExpressionContext _localctx = new OperatorExpressionContext(_ctx, _parentState); + OperatorExpressionContext _prevctx = _localctx; + int _startState = 16; + enterRecursionRule(_localctx, 16, RULE_operatorExpression, _p); + int _la; + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(198); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,13,_ctx) ) { + case 1: + { + _localctx = new OperatorExpressionDefaultContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + + setState(195); + primaryExpression(); + } + break; + case 2: + { + _localctx = new ArithmeticUnaryContext(_localctx); + _ctx = _localctx; + _prevctx = _localctx; + setState(196); + ((ArithmeticUnaryContext)_localctx).operator = _input.LT(1); + _la = _input.LA(1); + if ( !(_la==PLUS || _la==MINUS) ) { + ((ArithmeticUnaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + setState(197); + operatorExpression(3); + } + break; + } + _ctx.stop = _input.LT(-1); + setState(208); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,15,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + if ( _parseListeners!=null ) triggerExitRuleEvent(); + _prevctx = _localctx; + { + setState(206); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,14,_ctx) ) { + case 1: + { + _localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState)); + ((ArithmeticBinaryContext)_localctx).left = _prevctx; + pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression); + setState(200); + if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); + setState(201); + ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); + _la = _input.LA(1); + if ( !((((_la - 62)) & ~0x3f) == 0 && ((1L << (_la - 62)) & 7L) != 0) ) { + ((ArithmeticBinaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + setState(202); + ((ArithmeticBinaryContext)_localctx).right = operatorExpression(3); + } + break; + case 2: + { + _localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState)); + ((ArithmeticBinaryContext)_localctx).left = _prevctx; + pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression); + setState(203); + if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)"); + setState(204); + ((ArithmeticBinaryContext)_localctx).operator = _input.LT(1); + _la = _input.LA(1); + if ( !(_la==PLUS || _la==MINUS) ) { + ((ArithmeticBinaryContext)_localctx).operator = (Token)_errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + setState(205); + ((ArithmeticBinaryContext)_localctx).right = operatorExpression(2); + } + break; + } + } + } + setState(210); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,15,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + unrollRecursionContexts(_parentctx); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class PrimaryExpressionContext extends ParserRuleContext { + public PrimaryExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_primaryExpression; } + + public PrimaryExpressionContext() { } + public void copyFrom(PrimaryExpressionContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class DereferenceContext extends PrimaryExpressionContext { + public QualifiedNameContext qualifiedName() { + return getRuleContext(QualifiedNameContext.class,0); + } + public DereferenceContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterDereference(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitDereference(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitDereference(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ConstantDefaultContext extends PrimaryExpressionContext { + public ConstantContext constant() { + return getRuleContext(ConstantContext.class,0); + } + public ConstantDefaultContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterConstantDefault(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitConstantDefault(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitConstantDefault(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ParenthesizedExpressionContext extends PrimaryExpressionContext { + public TerminalNode LP() { return getToken(EsqlBaseParser.LP, 0); } + public BooleanExpressionContext booleanExpression() { + return getRuleContext(BooleanExpressionContext.class,0); + } + public TerminalNode RP() { return getToken(EsqlBaseParser.RP, 0); } + public ParenthesizedExpressionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterParenthesizedExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitParenthesizedExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitParenthesizedExpression(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class FunctionExpressionContext extends PrimaryExpressionContext { + public IdentifierContext identifier() { + return getRuleContext(IdentifierContext.class,0); + } + public TerminalNode LP() { return getToken(EsqlBaseParser.LP, 0); } + public TerminalNode RP() { return getToken(EsqlBaseParser.RP, 0); } + public List booleanExpression() { + return getRuleContexts(BooleanExpressionContext.class); + } + public BooleanExpressionContext booleanExpression(int i) { + return getRuleContext(BooleanExpressionContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public FunctionExpressionContext(PrimaryExpressionContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFunctionExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFunctionExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitFunctionExpression(this); + else return visitor.visitChildren(this); + } + } + + public final PrimaryExpressionContext primaryExpression() throws RecognitionException { + PrimaryExpressionContext _localctx = new PrimaryExpressionContext(_ctx, getState()); + enterRule(_localctx, 18, RULE_primaryExpression); + int _la; + try { + setState(231); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,18,_ctx) ) { + case 1: + _localctx = new ConstantDefaultContext(_localctx); + enterOuterAlt(_localctx, 1); + { + setState(211); + constant(); + } + break; + case 2: + _localctx = new DereferenceContext(_localctx); + enterOuterAlt(_localctx, 2); + { + setState(212); + qualifiedName(); + } + break; + case 3: + _localctx = new ParenthesizedExpressionContext(_localctx); + enterOuterAlt(_localctx, 3); + { + setState(213); + match(LP); + setState(214); + booleanExpression(0); + setState(215); + match(RP); + } + break; + case 4: + _localctx = new FunctionExpressionContext(_localctx); + enterOuterAlt(_localctx, 4); + { + setState(217); + identifier(); + setState(218); + match(LP); + setState(227); + _errHandler.sync(this); + _la = _input.LA(1); + if ((((_la - 27)) & ~0x3f) == 0 && ((1L << (_la - 27)) & 3599201870855L) != 0) { + { + setState(219); + booleanExpression(0); + setState(224); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(220); + match(COMMA); + setState(221); + booleanExpression(0); + } + } + setState(226); + _errHandler.sync(this); + _la = _input.LA(1); + } + } + } + + setState(229); + match(RP); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class RowCommandContext extends ParserRuleContext { + public TerminalNode ROW() { return getToken(EsqlBaseParser.ROW, 0); } + public FieldsContext fields() { + return getRuleContext(FieldsContext.class,0); + } + public RowCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_rowCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterRowCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitRowCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitRowCommand(this); + else return visitor.visitChildren(this); + } + } + + public final RowCommandContext rowCommand() throws RecognitionException { + RowCommandContext _localctx = new RowCommandContext(_ctx, getState()); + enterRule(_localctx, 20, RULE_rowCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(233); + match(ROW); + setState(234); + fields(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class FieldsContext extends ParserRuleContext { + public List field() { + return getRuleContexts(FieldContext.class); + } + public FieldContext field(int i) { + return getRuleContext(FieldContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public FieldsContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_fields; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFields(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFields(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitFields(this); + else return visitor.visitChildren(this); + } + } + + public final FieldsContext fields() throws RecognitionException { + FieldsContext _localctx = new FieldsContext(_ctx, getState()); + enterRule(_localctx, 22, RULE_fields); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(236); + field(); + setState(241); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,19,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(237); + match(COMMA); + setState(238); + field(); + } + } + } + setState(243); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,19,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class FieldContext extends ParserRuleContext { + public BooleanExpressionContext booleanExpression() { + return getRuleContext(BooleanExpressionContext.class,0); + } + public QualifiedNameContext qualifiedName() { + return getRuleContext(QualifiedNameContext.class,0); + } + public TerminalNode ASSIGN() { return getToken(EsqlBaseParser.ASSIGN, 0); } + public FieldContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_field; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterField(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitField(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitField(this); + else return visitor.visitChildren(this); + } + } + + public final FieldContext field() throws RecognitionException { + FieldContext _localctx = new FieldContext(_ctx, getState()); + enterRule(_localctx, 24, RULE_field); + try { + setState(249); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,20,_ctx) ) { + case 1: + enterOuterAlt(_localctx, 1); + { + setState(244); + booleanExpression(0); + } + break; + case 2: + enterOuterAlt(_localctx, 2); + { + setState(245); + qualifiedName(); + setState(246); + match(ASSIGN); + setState(247); + booleanExpression(0); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class FromCommandContext extends ParserRuleContext { + public TerminalNode FROM() { return getToken(EsqlBaseParser.FROM, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public MetadataContext metadata() { + return getRuleContext(MetadataContext.class,0); + } + public FromCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_fromCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFromCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFromCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitFromCommand(this); + else return visitor.visitChildren(this); + } + } + + public final FromCommandContext fromCommand() throws RecognitionException { + FromCommandContext _localctx = new FromCommandContext(_ctx, getState()); + enterRule(_localctx, 26, RULE_fromCommand); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(251); + match(FROM); + setState(252); + sourceIdentifier(); + setState(257); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,21,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(253); + match(COMMA); + setState(254); + sourceIdentifier(); + } + } + } + setState(259); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,21,_ctx); + } + setState(261); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,22,_ctx) ) { + case 1: + { + setState(260); + metadata(); + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class MetadataContext extends ParserRuleContext { + public TerminalNode OPENING_BRACKET() { return getToken(EsqlBaseParser.OPENING_BRACKET, 0); } + public TerminalNode METADATA() { return getToken(EsqlBaseParser.METADATA, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public TerminalNode CLOSING_BRACKET() { return getToken(EsqlBaseParser.CLOSING_BRACKET, 0); } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public MetadataContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_metadata; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterMetadata(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitMetadata(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitMetadata(this); + else return visitor.visitChildren(this); + } + } + + public final MetadataContext metadata() throws RecognitionException { + MetadataContext _localctx = new MetadataContext(_ctx, getState()); + enterRule(_localctx, 28, RULE_metadata); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(263); + match(OPENING_BRACKET); + setState(264); + match(METADATA); + setState(265); + sourceIdentifier(); + setState(270); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(266); + match(COMMA); + setState(267); + sourceIdentifier(); + } + } + setState(272); + _errHandler.sync(this); + _la = _input.LA(1); + } + setState(273); + match(CLOSING_BRACKET); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class EvalCommandContext extends ParserRuleContext { + public TerminalNode EVAL() { return getToken(EsqlBaseParser.EVAL, 0); } + public FieldsContext fields() { + return getRuleContext(FieldsContext.class,0); + } + public EvalCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_evalCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterEvalCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitEvalCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitEvalCommand(this); + else return visitor.visitChildren(this); + } + } + + public final EvalCommandContext evalCommand() throws RecognitionException { + EvalCommandContext _localctx = new EvalCommandContext(_ctx, getState()); + enterRule(_localctx, 30, RULE_evalCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(275); + match(EVAL); + setState(276); + fields(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class StatsCommandContext extends ParserRuleContext { + public TerminalNode STATS() { return getToken(EsqlBaseParser.STATS, 0); } + public FieldsContext fields() { + return getRuleContext(FieldsContext.class,0); + } + public TerminalNode BY() { return getToken(EsqlBaseParser.BY, 0); } + public GroupingContext grouping() { + return getRuleContext(GroupingContext.class,0); + } + public StatsCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_statsCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterStatsCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitStatsCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitStatsCommand(this); + else return visitor.visitChildren(this); + } + } + + public final StatsCommandContext statsCommand() throws RecognitionException { + StatsCommandContext _localctx = new StatsCommandContext(_ctx, getState()); + enterRule(_localctx, 32, RULE_statsCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(278); + match(STATS); + setState(280); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { + case 1: + { + setState(279); + fields(); + } + break; + } + setState(284); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,25,_ctx) ) { + case 1: + { + setState(282); + match(BY); + setState(283); + grouping(); + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class InlinestatsCommandContext extends ParserRuleContext { + public TerminalNode INLINESTATS() { return getToken(EsqlBaseParser.INLINESTATS, 0); } + public FieldsContext fields() { + return getRuleContext(FieldsContext.class,0); + } + public TerminalNode BY() { return getToken(EsqlBaseParser.BY, 0); } + public GroupingContext grouping() { + return getRuleContext(GroupingContext.class,0); + } + public InlinestatsCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_inlinestatsCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterInlinestatsCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitInlinestatsCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitInlinestatsCommand(this); + else return visitor.visitChildren(this); + } + } + + public final InlinestatsCommandContext inlinestatsCommand() throws RecognitionException { + InlinestatsCommandContext _localctx = new InlinestatsCommandContext(_ctx, getState()); + enterRule(_localctx, 34, RULE_inlinestatsCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(286); + match(INLINESTATS); + setState(287); + fields(); + setState(290); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,26,_ctx) ) { + case 1: + { + setState(288); + match(BY); + setState(289); + grouping(); + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class GroupingContext extends ParserRuleContext { + public List qualifiedName() { + return getRuleContexts(QualifiedNameContext.class); + } + public QualifiedNameContext qualifiedName(int i) { + return getRuleContext(QualifiedNameContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public GroupingContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_grouping; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterGrouping(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitGrouping(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitGrouping(this); + else return visitor.visitChildren(this); + } + } + + public final GroupingContext grouping() throws RecognitionException { + GroupingContext _localctx = new GroupingContext(_ctx, getState()); + enterRule(_localctx, 36, RULE_grouping); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(292); + qualifiedName(); + setState(297); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,27,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(293); + match(COMMA); + setState(294); + qualifiedName(); + } + } + } + setState(299); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,27,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class SourceIdentifierContext extends ParserRuleContext { + public TerminalNode SRC_UNQUOTED_IDENTIFIER() { return getToken(EsqlBaseParser.SRC_UNQUOTED_IDENTIFIER, 0); } + public TerminalNode SRC_QUOTED_IDENTIFIER() { return getToken(EsqlBaseParser.SRC_QUOTED_IDENTIFIER, 0); } + public SourceIdentifierContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_sourceIdentifier; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSourceIdentifier(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSourceIdentifier(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSourceIdentifier(this); + else return visitor.visitChildren(this); + } + } + + public final SourceIdentifierContext sourceIdentifier() throws RecognitionException { + SourceIdentifierContext _localctx = new SourceIdentifierContext(_ctx, getState()); + enterRule(_localctx, 38, RULE_sourceIdentifier); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(300); + _la = _input.LA(1); + if ( !(_la==SRC_UNQUOTED_IDENTIFIER || _la==SRC_QUOTED_IDENTIFIER) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class QualifiedNameContext extends ParserRuleContext { + public List identifier() { + return getRuleContexts(IdentifierContext.class); + } + public IdentifierContext identifier(int i) { + return getRuleContext(IdentifierContext.class,i); + } + public List DOT() { return getTokens(EsqlBaseParser.DOT); } + public TerminalNode DOT(int i) { + return getToken(EsqlBaseParser.DOT, i); + } + public QualifiedNameContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_qualifiedName; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterQualifiedName(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitQualifiedName(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitQualifiedName(this); + else return visitor.visitChildren(this); + } + } + + public final QualifiedNameContext qualifiedName() throws RecognitionException { + QualifiedNameContext _localctx = new QualifiedNameContext(_ctx, getState()); + enterRule(_localctx, 40, RULE_qualifiedName); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(302); + identifier(); + setState(307); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,28,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(303); + match(DOT); + setState(304); + identifier(); + } + } + } + setState(309); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,28,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class IdentifierContext extends ParserRuleContext { + public TerminalNode UNQUOTED_IDENTIFIER() { return getToken(EsqlBaseParser.UNQUOTED_IDENTIFIER, 0); } + public TerminalNode QUOTED_IDENTIFIER() { return getToken(EsqlBaseParser.QUOTED_IDENTIFIER, 0); } + public IdentifierContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_identifier; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterIdentifier(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitIdentifier(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitIdentifier(this); + else return visitor.visitChildren(this); + } + } + + public final IdentifierContext identifier() throws RecognitionException { + IdentifierContext _localctx = new IdentifierContext(_ctx, getState()); + enterRule(_localctx, 42, RULE_identifier); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(310); + _la = _input.LA(1); + if ( !(_la==UNQUOTED_IDENTIFIER || _la==QUOTED_IDENTIFIER) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ConstantContext extends ParserRuleContext { + public ConstantContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_constant; } + + public ConstantContext() { } + public void copyFrom(ConstantContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class BooleanArrayLiteralContext extends ConstantContext { + public TerminalNode OPENING_BRACKET() { return getToken(EsqlBaseParser.OPENING_BRACKET, 0); } + public List booleanValue() { + return getRuleContexts(BooleanValueContext.class); + } + public BooleanValueContext booleanValue(int i) { + return getRuleContext(BooleanValueContext.class,i); + } + public TerminalNode CLOSING_BRACKET() { return getToken(EsqlBaseParser.CLOSING_BRACKET, 0); } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public BooleanArrayLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterBooleanArrayLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitBooleanArrayLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitBooleanArrayLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class DecimalLiteralContext extends ConstantContext { + public DecimalValueContext decimalValue() { + return getRuleContext(DecimalValueContext.class,0); + } + public DecimalLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterDecimalLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitDecimalLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitDecimalLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class NullLiteralContext extends ConstantContext { + public TerminalNode NULL() { return getToken(EsqlBaseParser.NULL, 0); } + public NullLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterNullLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitNullLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitNullLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class QualifiedIntegerLiteralContext extends ConstantContext { + public IntegerValueContext integerValue() { + return getRuleContext(IntegerValueContext.class,0); + } + public TerminalNode UNQUOTED_IDENTIFIER() { return getToken(EsqlBaseParser.UNQUOTED_IDENTIFIER, 0); } + public QualifiedIntegerLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterQualifiedIntegerLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitQualifiedIntegerLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitQualifiedIntegerLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class StringArrayLiteralContext extends ConstantContext { + public TerminalNode OPENING_BRACKET() { return getToken(EsqlBaseParser.OPENING_BRACKET, 0); } + public List string() { + return getRuleContexts(StringContext.class); + } + public StringContext string(int i) { + return getRuleContext(StringContext.class,i); + } + public TerminalNode CLOSING_BRACKET() { return getToken(EsqlBaseParser.CLOSING_BRACKET, 0); } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public StringArrayLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterStringArrayLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitStringArrayLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitStringArrayLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class StringLiteralContext extends ConstantContext { + public StringContext string() { + return getRuleContext(StringContext.class,0); + } + public StringLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterStringLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitStringLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitStringLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class NumericArrayLiteralContext extends ConstantContext { + public TerminalNode OPENING_BRACKET() { return getToken(EsqlBaseParser.OPENING_BRACKET, 0); } + public List numericValue() { + return getRuleContexts(NumericValueContext.class); + } + public NumericValueContext numericValue(int i) { + return getRuleContext(NumericValueContext.class,i); + } + public TerminalNode CLOSING_BRACKET() { return getToken(EsqlBaseParser.CLOSING_BRACKET, 0); } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public NumericArrayLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterNumericArrayLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitNumericArrayLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitNumericArrayLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class InputParamContext extends ConstantContext { + public TerminalNode PARAM() { return getToken(EsqlBaseParser.PARAM, 0); } + public InputParamContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterInputParam(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitInputParam(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitInputParam(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class IntegerLiteralContext extends ConstantContext { + public IntegerValueContext integerValue() { + return getRuleContext(IntegerValueContext.class,0); + } + public IntegerLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterIntegerLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitIntegerLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitIntegerLiteral(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class BooleanLiteralContext extends ConstantContext { + public BooleanValueContext booleanValue() { + return getRuleContext(BooleanValueContext.class,0); + } + public BooleanLiteralContext(ConstantContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterBooleanLiteral(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitBooleanLiteral(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitBooleanLiteral(this); + else return visitor.visitChildren(this); + } + } + + public final ConstantContext constant() throws RecognitionException { + ConstantContext _localctx = new ConstantContext(_ctx, getState()); + enterRule(_localctx, 44, RULE_constant); + int _la; + try { + setState(354); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,32,_ctx) ) { + case 1: + _localctx = new NullLiteralContext(_localctx); + enterOuterAlt(_localctx, 1); + { + setState(312); + match(NULL); + } + break; + case 2: + _localctx = new QualifiedIntegerLiteralContext(_localctx); + enterOuterAlt(_localctx, 2); + { + setState(313); + integerValue(); + setState(314); + match(UNQUOTED_IDENTIFIER); + } + break; + case 3: + _localctx = new DecimalLiteralContext(_localctx); + enterOuterAlt(_localctx, 3); + { + setState(316); + decimalValue(); + } + break; + case 4: + _localctx = new IntegerLiteralContext(_localctx); + enterOuterAlt(_localctx, 4); + { + setState(317); + integerValue(); + } + break; + case 5: + _localctx = new BooleanLiteralContext(_localctx); + enterOuterAlt(_localctx, 5); + { + setState(318); + booleanValue(); + } + break; + case 6: + _localctx = new InputParamContext(_localctx); + enterOuterAlt(_localctx, 6); + { + setState(319); + match(PARAM); + } + break; + case 7: + _localctx = new StringLiteralContext(_localctx); + enterOuterAlt(_localctx, 7); + { + setState(320); + string(); + } + break; + case 8: + _localctx = new NumericArrayLiteralContext(_localctx); + enterOuterAlt(_localctx, 8); + { + setState(321); + match(OPENING_BRACKET); + setState(322); + numericValue(); + setState(327); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(323); + match(COMMA); + setState(324); + numericValue(); + } + } + setState(329); + _errHandler.sync(this); + _la = _input.LA(1); + } + setState(330); + match(CLOSING_BRACKET); + } + break; + case 9: + _localctx = new BooleanArrayLiteralContext(_localctx); + enterOuterAlt(_localctx, 9); + { + setState(332); + match(OPENING_BRACKET); + setState(333); + booleanValue(); + setState(338); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(334); + match(COMMA); + setState(335); + booleanValue(); + } + } + setState(340); + _errHandler.sync(this); + _la = _input.LA(1); + } + setState(341); + match(CLOSING_BRACKET); + } + break; + case 10: + _localctx = new StringArrayLiteralContext(_localctx); + enterOuterAlt(_localctx, 10); + { + setState(343); + match(OPENING_BRACKET); + setState(344); + string(); + setState(349); + _errHandler.sync(this); + _la = _input.LA(1); + while (_la==COMMA) { + { + { + setState(345); + match(COMMA); + setState(346); + string(); + } + } + setState(351); + _errHandler.sync(this); + _la = _input.LA(1); + } + setState(352); + match(CLOSING_BRACKET); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class LimitCommandContext extends ParserRuleContext { + public TerminalNode LIMIT() { return getToken(EsqlBaseParser.LIMIT, 0); } + public TerminalNode INTEGER_LITERAL() { return getToken(EsqlBaseParser.INTEGER_LITERAL, 0); } + public LimitCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_limitCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterLimitCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitLimitCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitLimitCommand(this); + else return visitor.visitChildren(this); + } + } + + public final LimitCommandContext limitCommand() throws RecognitionException { + LimitCommandContext _localctx = new LimitCommandContext(_ctx, getState()); + enterRule(_localctx, 46, RULE_limitCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(356); + match(LIMIT); + setState(357); + match(INTEGER_LITERAL); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class SortCommandContext extends ParserRuleContext { + public TerminalNode SORT() { return getToken(EsqlBaseParser.SORT, 0); } + public List orderExpression() { + return getRuleContexts(OrderExpressionContext.class); + } + public OrderExpressionContext orderExpression(int i) { + return getRuleContext(OrderExpressionContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public SortCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_sortCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSortCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSortCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSortCommand(this); + else return visitor.visitChildren(this); + } + } + + public final SortCommandContext sortCommand() throws RecognitionException { + SortCommandContext _localctx = new SortCommandContext(_ctx, getState()); + enterRule(_localctx, 48, RULE_sortCommand); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(359); + match(SORT); + setState(360); + orderExpression(); + setState(365); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,33,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(361); + match(COMMA); + setState(362); + orderExpression(); + } + } + } + setState(367); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,33,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class OrderExpressionContext extends ParserRuleContext { + public Token ordering; + public Token nullOrdering; + public BooleanExpressionContext booleanExpression() { + return getRuleContext(BooleanExpressionContext.class,0); + } + public TerminalNode NULLS() { return getToken(EsqlBaseParser.NULLS, 0); } + public TerminalNode ASC() { return getToken(EsqlBaseParser.ASC, 0); } + public TerminalNode DESC() { return getToken(EsqlBaseParser.DESC, 0); } + public TerminalNode FIRST() { return getToken(EsqlBaseParser.FIRST, 0); } + public TerminalNode LAST() { return getToken(EsqlBaseParser.LAST, 0); } + public OrderExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_orderExpression; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterOrderExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitOrderExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitOrderExpression(this); + else return visitor.visitChildren(this); + } + } + + public final OrderExpressionContext orderExpression() throws RecognitionException { + OrderExpressionContext _localctx = new OrderExpressionContext(_ctx, getState()); + enterRule(_localctx, 50, RULE_orderExpression); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(368); + booleanExpression(0); + setState(370); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,34,_ctx) ) { + case 1: + { + setState(369); + ((OrderExpressionContext)_localctx).ordering = _input.LT(1); + _la = _input.LA(1); + if ( !(_la==ASC || _la==DESC) ) { + ((OrderExpressionContext)_localctx).ordering = (Token)_errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + break; + } + setState(374); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,35,_ctx) ) { + case 1: + { + setState(372); + match(NULLS); + setState(373); + ((OrderExpressionContext)_localctx).nullOrdering = _input.LT(1); + _la = _input.LA(1); + if ( !(_la==FIRST || _la==LAST) ) { + ((OrderExpressionContext)_localctx).nullOrdering = (Token)_errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class KeepCommandContext extends ParserRuleContext { + public TerminalNode KEEP() { return getToken(EsqlBaseParser.KEEP, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public TerminalNode PROJECT() { return getToken(EsqlBaseParser.PROJECT, 0); } + public KeepCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_keepCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterKeepCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitKeepCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitKeepCommand(this); + else return visitor.visitChildren(this); + } + } + + public final KeepCommandContext keepCommand() throws RecognitionException { + KeepCommandContext _localctx = new KeepCommandContext(_ctx, getState()); + enterRule(_localctx, 52, RULE_keepCommand); + try { + int _alt; + setState(394); + _errHandler.sync(this); + switch (_input.LA(1)) { + case KEEP: + enterOuterAlt(_localctx, 1); + { + setState(376); + match(KEEP); + setState(377); + sourceIdentifier(); + setState(382); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,36,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(378); + match(COMMA); + setState(379); + sourceIdentifier(); + } + } + } + setState(384); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,36,_ctx); + } + } + break; + case PROJECT: + enterOuterAlt(_localctx, 2); + { + setState(385); + match(PROJECT); + setState(386); + sourceIdentifier(); + setState(391); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,37,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(387); + match(COMMA); + setState(388); + sourceIdentifier(); + } + } + } + setState(393); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,37,_ctx); + } + } + break; + default: + throw new NoViableAltException(this); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class DropCommandContext extends ParserRuleContext { + public TerminalNode DROP() { return getToken(EsqlBaseParser.DROP, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public DropCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_dropCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterDropCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitDropCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitDropCommand(this); + else return visitor.visitChildren(this); + } + } + + public final DropCommandContext dropCommand() throws RecognitionException { + DropCommandContext _localctx = new DropCommandContext(_ctx, getState()); + enterRule(_localctx, 54, RULE_dropCommand); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(396); + match(DROP); + setState(397); + sourceIdentifier(); + setState(402); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,39,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(398); + match(COMMA); + setState(399); + sourceIdentifier(); + } + } + } + setState(404); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,39,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class RenameCommandContext extends ParserRuleContext { + public TerminalNode RENAME() { return getToken(EsqlBaseParser.RENAME, 0); } + public List renameClause() { + return getRuleContexts(RenameClauseContext.class); + } + public RenameClauseContext renameClause(int i) { + return getRuleContext(RenameClauseContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public RenameCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_renameCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterRenameCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitRenameCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitRenameCommand(this); + else return visitor.visitChildren(this); + } + } + + public final RenameCommandContext renameCommand() throws RecognitionException { + RenameCommandContext _localctx = new RenameCommandContext(_ctx, getState()); + enterRule(_localctx, 56, RULE_renameCommand); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(405); + match(RENAME); + setState(406); + renameClause(); + setState(411); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,40,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(407); + match(COMMA); + setState(408); + renameClause(); + } + } + } + setState(413); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,40,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class RenameClauseContext extends ParserRuleContext { + public SourceIdentifierContext oldName; + public SourceIdentifierContext newName; + public TerminalNode AS() { return getToken(EsqlBaseParser.AS, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public RenameClauseContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_renameClause; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterRenameClause(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitRenameClause(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitRenameClause(this); + else return visitor.visitChildren(this); + } + } + + public final RenameClauseContext renameClause() throws RecognitionException { + RenameClauseContext _localctx = new RenameClauseContext(_ctx, getState()); + enterRule(_localctx, 58, RULE_renameClause); + try { + enterOuterAlt(_localctx, 1); + { + setState(414); + ((RenameClauseContext)_localctx).oldName = sourceIdentifier(); + setState(415); + match(AS); + setState(416); + ((RenameClauseContext)_localctx).newName = sourceIdentifier(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class DissectCommandContext extends ParserRuleContext { + public TerminalNode DISSECT() { return getToken(EsqlBaseParser.DISSECT, 0); } + public PrimaryExpressionContext primaryExpression() { + return getRuleContext(PrimaryExpressionContext.class,0); + } + public StringContext string() { + return getRuleContext(StringContext.class,0); + } + public CommandOptionsContext commandOptions() { + return getRuleContext(CommandOptionsContext.class,0); + } + public DissectCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_dissectCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterDissectCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitDissectCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitDissectCommand(this); + else return visitor.visitChildren(this); + } + } + + public final DissectCommandContext dissectCommand() throws RecognitionException { + DissectCommandContext _localctx = new DissectCommandContext(_ctx, getState()); + enterRule(_localctx, 60, RULE_dissectCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(418); + match(DISSECT); + setState(419); + primaryExpression(); + setState(420); + string(); + setState(422); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,41,_ctx) ) { + case 1: + { + setState(421); + commandOptions(); + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class GrokCommandContext extends ParserRuleContext { + public TerminalNode GROK() { return getToken(EsqlBaseParser.GROK, 0); } + public PrimaryExpressionContext primaryExpression() { + return getRuleContext(PrimaryExpressionContext.class,0); + } + public StringContext string() { + return getRuleContext(StringContext.class,0); + } + public GrokCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_grokCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterGrokCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitGrokCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitGrokCommand(this); + else return visitor.visitChildren(this); + } + } + + public final GrokCommandContext grokCommand() throws RecognitionException { + GrokCommandContext _localctx = new GrokCommandContext(_ctx, getState()); + enterRule(_localctx, 62, RULE_grokCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(424); + match(GROK); + setState(425); + primaryExpression(); + setState(426); + string(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class MvExpandCommandContext extends ParserRuleContext { + public TerminalNode MV_EXPAND() { return getToken(EsqlBaseParser.MV_EXPAND, 0); } + public SourceIdentifierContext sourceIdentifier() { + return getRuleContext(SourceIdentifierContext.class,0); + } + public MvExpandCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_mvExpandCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterMvExpandCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitMvExpandCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitMvExpandCommand(this); + else return visitor.visitChildren(this); + } + } + + public final MvExpandCommandContext mvExpandCommand() throws RecognitionException { + MvExpandCommandContext _localctx = new MvExpandCommandContext(_ctx, getState()); + enterRule(_localctx, 64, RULE_mvExpandCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(428); + match(MV_EXPAND); + setState(429); + sourceIdentifier(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class CommandOptionsContext extends ParserRuleContext { + public List commandOption() { + return getRuleContexts(CommandOptionContext.class); + } + public CommandOptionContext commandOption(int i) { + return getRuleContext(CommandOptionContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public CommandOptionsContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_commandOptions; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterCommandOptions(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitCommandOptions(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitCommandOptions(this); + else return visitor.visitChildren(this); + } + } + + public final CommandOptionsContext commandOptions() throws RecognitionException { + CommandOptionsContext _localctx = new CommandOptionsContext(_ctx, getState()); + enterRule(_localctx, 66, RULE_commandOptions); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(431); + commandOption(); + setState(436); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,42,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(432); + match(COMMA); + setState(433); + commandOption(); + } + } + } + setState(438); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,42,_ctx); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class CommandOptionContext extends ParserRuleContext { + public IdentifierContext identifier() { + return getRuleContext(IdentifierContext.class,0); + } + public TerminalNode ASSIGN() { return getToken(EsqlBaseParser.ASSIGN, 0); } + public ConstantContext constant() { + return getRuleContext(ConstantContext.class,0); + } + public CommandOptionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_commandOption; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterCommandOption(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitCommandOption(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitCommandOption(this); + else return visitor.visitChildren(this); + } + } + + public final CommandOptionContext commandOption() throws RecognitionException { + CommandOptionContext _localctx = new CommandOptionContext(_ctx, getState()); + enterRule(_localctx, 68, RULE_commandOption); + try { + enterOuterAlt(_localctx, 1); + { + setState(439); + identifier(); + setState(440); + match(ASSIGN); + setState(441); + constant(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class BooleanValueContext extends ParserRuleContext { + public TerminalNode TRUE() { return getToken(EsqlBaseParser.TRUE, 0); } + public TerminalNode FALSE() { return getToken(EsqlBaseParser.FALSE, 0); } + public BooleanValueContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_booleanValue; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterBooleanValue(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitBooleanValue(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitBooleanValue(this); + else return visitor.visitChildren(this); + } + } + + public final BooleanValueContext booleanValue() throws RecognitionException { + BooleanValueContext _localctx = new BooleanValueContext(_ctx, getState()); + enterRule(_localctx, 70, RULE_booleanValue); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(443); + _la = _input.LA(1); + if ( !(_la==FALSE || _la==TRUE) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class NumericValueContext extends ParserRuleContext { + public DecimalValueContext decimalValue() { + return getRuleContext(DecimalValueContext.class,0); + } + public IntegerValueContext integerValue() { + return getRuleContext(IntegerValueContext.class,0); + } + public NumericValueContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_numericValue; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterNumericValue(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitNumericValue(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitNumericValue(this); + else return visitor.visitChildren(this); + } + } + + public final NumericValueContext numericValue() throws RecognitionException { + NumericValueContext _localctx = new NumericValueContext(_ctx, getState()); + enterRule(_localctx, 72, RULE_numericValue); + try { + setState(447); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,43,_ctx) ) { + case 1: + enterOuterAlt(_localctx, 1); + { + setState(445); + decimalValue(); + } + break; + case 2: + enterOuterAlt(_localctx, 2); + { + setState(446); + integerValue(); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class DecimalValueContext extends ParserRuleContext { + public TerminalNode DECIMAL_LITERAL() { return getToken(EsqlBaseParser.DECIMAL_LITERAL, 0); } + public TerminalNode PLUS() { return getToken(EsqlBaseParser.PLUS, 0); } + public TerminalNode MINUS() { return getToken(EsqlBaseParser.MINUS, 0); } + public DecimalValueContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_decimalValue; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterDecimalValue(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitDecimalValue(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitDecimalValue(this); + else return visitor.visitChildren(this); + } + } + + public final DecimalValueContext decimalValue() throws RecognitionException { + DecimalValueContext _localctx = new DecimalValueContext(_ctx, getState()); + enterRule(_localctx, 74, RULE_decimalValue); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(450); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==PLUS || _la==MINUS) { + { + setState(449); + _la = _input.LA(1); + if ( !(_la==PLUS || _la==MINUS) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + + setState(452); + match(DECIMAL_LITERAL); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class IntegerValueContext extends ParserRuleContext { + public TerminalNode INTEGER_LITERAL() { return getToken(EsqlBaseParser.INTEGER_LITERAL, 0); } + public TerminalNode PLUS() { return getToken(EsqlBaseParser.PLUS, 0); } + public TerminalNode MINUS() { return getToken(EsqlBaseParser.MINUS, 0); } + public IntegerValueContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_integerValue; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterIntegerValue(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitIntegerValue(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitIntegerValue(this); + else return visitor.visitChildren(this); + } + } + + public final IntegerValueContext integerValue() throws RecognitionException { + IntegerValueContext _localctx = new IntegerValueContext(_ctx, getState()); + enterRule(_localctx, 76, RULE_integerValue); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(455); + _errHandler.sync(this); + _la = _input.LA(1); + if (_la==PLUS || _la==MINUS) { + { + setState(454); + _la = _input.LA(1); + if ( !(_la==PLUS || _la==MINUS) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + + setState(457); + match(INTEGER_LITERAL); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class StringContext extends ParserRuleContext { + public TerminalNode STRING() { return getToken(EsqlBaseParser.STRING, 0); } + public StringContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_string; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterString(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitString(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitString(this); + else return visitor.visitChildren(this); + } + } + + public final StringContext string() throws RecognitionException { + StringContext _localctx = new StringContext(_ctx, getState()); + enterRule(_localctx, 78, RULE_string); + try { + enterOuterAlt(_localctx, 1); + { + setState(459); + match(STRING); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ComparisonOperatorContext extends ParserRuleContext { + public TerminalNode EQ() { return getToken(EsqlBaseParser.EQ, 0); } + public TerminalNode NEQ() { return getToken(EsqlBaseParser.NEQ, 0); } + public TerminalNode LT() { return getToken(EsqlBaseParser.LT, 0); } + public TerminalNode LTE() { return getToken(EsqlBaseParser.LTE, 0); } + public TerminalNode GT() { return getToken(EsqlBaseParser.GT, 0); } + public TerminalNode GTE() { return getToken(EsqlBaseParser.GTE, 0); } + public ComparisonOperatorContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_comparisonOperator; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterComparisonOperator(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitComparisonOperator(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitComparisonOperator(this); + else return visitor.visitChildren(this); + } + } + + public final ComparisonOperatorContext comparisonOperator() throws RecognitionException { + ComparisonOperatorContext _localctx = new ComparisonOperatorContext(_ctx, getState()); + enterRule(_localctx, 80, RULE_comparisonOperator); + int _la; + try { + enterOuterAlt(_localctx, 1); + { + setState(461); + _la = _input.LA(1); + if ( !(((_la) & ~0x3f) == 0 && ((1L << _la) & 1134907106097364992L) != 0) ) { + _errHandler.recoverInline(this); + } + else { + if ( _input.LA(1)==Token.EOF ) matchedEOF = true; + _errHandler.reportMatch(this); + consume(); + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ExplainCommandContext extends ParserRuleContext { + public TerminalNode EXPLAIN() { return getToken(EsqlBaseParser.EXPLAIN, 0); } + public SubqueryExpressionContext subqueryExpression() { + return getRuleContext(SubqueryExpressionContext.class,0); + } + public ExplainCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_explainCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterExplainCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitExplainCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitExplainCommand(this); + else return visitor.visitChildren(this); + } + } + + public final ExplainCommandContext explainCommand() throws RecognitionException { + ExplainCommandContext _localctx = new ExplainCommandContext(_ctx, getState()); + enterRule(_localctx, 82, RULE_explainCommand); + try { + enterOuterAlt(_localctx, 1); + { + setState(463); + match(EXPLAIN); + setState(464); + subqueryExpression(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class SubqueryExpressionContext extends ParserRuleContext { + public TerminalNode OPENING_BRACKET() { return getToken(EsqlBaseParser.OPENING_BRACKET, 0); } + public QueryContext query() { + return getRuleContext(QueryContext.class,0); + } + public TerminalNode CLOSING_BRACKET() { return getToken(EsqlBaseParser.CLOSING_BRACKET, 0); } + public SubqueryExpressionContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_subqueryExpression; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterSubqueryExpression(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitSubqueryExpression(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitSubqueryExpression(this); + else return visitor.visitChildren(this); + } + } + + public final SubqueryExpressionContext subqueryExpression() throws RecognitionException { + SubqueryExpressionContext _localctx = new SubqueryExpressionContext(_ctx, getState()); + enterRule(_localctx, 84, RULE_subqueryExpression); + try { + enterOuterAlt(_localctx, 1); + { + setState(466); + match(OPENING_BRACKET); + setState(467); + query(0); + setState(468); + match(CLOSING_BRACKET); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class ShowCommandContext extends ParserRuleContext { + public ShowCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_showCommand; } + + public ShowCommandContext() { } + public void copyFrom(ShowCommandContext ctx) { + super.copyFrom(ctx); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ShowInfoContext extends ShowCommandContext { + public TerminalNode SHOW() { return getToken(EsqlBaseParser.SHOW, 0); } + public TerminalNode INFO() { return getToken(EsqlBaseParser.INFO, 0); } + public ShowInfoContext(ShowCommandContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterShowInfo(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitShowInfo(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitShowInfo(this); + else return visitor.visitChildren(this); + } + } + @SuppressWarnings("CheckReturnValue") + public static class ShowFunctionsContext extends ShowCommandContext { + public TerminalNode SHOW() { return getToken(EsqlBaseParser.SHOW, 0); } + public TerminalNode FUNCTIONS() { return getToken(EsqlBaseParser.FUNCTIONS, 0); } + public ShowFunctionsContext(ShowCommandContext ctx) { copyFrom(ctx); } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterShowFunctions(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitShowFunctions(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitShowFunctions(this); + else return visitor.visitChildren(this); + } + } + + public final ShowCommandContext showCommand() throws RecognitionException { + ShowCommandContext _localctx = new ShowCommandContext(_ctx, getState()); + enterRule(_localctx, 86, RULE_showCommand); + try { + setState(474); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,46,_ctx) ) { + case 1: + _localctx = new ShowInfoContext(_localctx); + enterOuterAlt(_localctx, 1); + { + setState(470); + match(SHOW); + setState(471); + match(INFO); + } + break; + case 2: + _localctx = new ShowFunctionsContext(_localctx); + enterOuterAlt(_localctx, 2); + { + setState(472); + match(SHOW); + setState(473); + match(FUNCTIONS); + } + break; + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class EnrichCommandContext extends ParserRuleContext { + public SourceIdentifierContext policyName; + public SourceIdentifierContext matchField; + public TerminalNode ENRICH() { return getToken(EsqlBaseParser.ENRICH, 0); } + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public TerminalNode ON() { return getToken(EsqlBaseParser.ON, 0); } + public TerminalNode WITH() { return getToken(EsqlBaseParser.WITH, 0); } + public List enrichWithClause() { + return getRuleContexts(EnrichWithClauseContext.class); + } + public EnrichWithClauseContext enrichWithClause(int i) { + return getRuleContext(EnrichWithClauseContext.class,i); + } + public List COMMA() { return getTokens(EsqlBaseParser.COMMA); } + public TerminalNode COMMA(int i) { + return getToken(EsqlBaseParser.COMMA, i); + } + public EnrichCommandContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_enrichCommand; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterEnrichCommand(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitEnrichCommand(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitEnrichCommand(this); + else return visitor.visitChildren(this); + } + } + + public final EnrichCommandContext enrichCommand() throws RecognitionException { + EnrichCommandContext _localctx = new EnrichCommandContext(_ctx, getState()); + enterRule(_localctx, 88, RULE_enrichCommand); + try { + int _alt; + enterOuterAlt(_localctx, 1); + { + setState(476); + match(ENRICH); + setState(477); + ((EnrichCommandContext)_localctx).policyName = sourceIdentifier(); + setState(480); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,47,_ctx) ) { + case 1: + { + setState(478); + match(ON); + setState(479); + ((EnrichCommandContext)_localctx).matchField = sourceIdentifier(); + } + break; + } + setState(491); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,49,_ctx) ) { + case 1: + { + setState(482); + match(WITH); + setState(483); + enrichWithClause(); + setState(488); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,48,_ctx); + while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { + if ( _alt==1 ) { + { + { + setState(484); + match(COMMA); + setState(485); + enrichWithClause(); + } + } + } + setState(490); + _errHandler.sync(this); + _alt = getInterpreter().adaptivePredict(_input,48,_ctx); + } + } + break; + } + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + @SuppressWarnings("CheckReturnValue") + public static class EnrichWithClauseContext extends ParserRuleContext { + public SourceIdentifierContext newName; + public SourceIdentifierContext enrichField; + public List sourceIdentifier() { + return getRuleContexts(SourceIdentifierContext.class); + } + public SourceIdentifierContext sourceIdentifier(int i) { + return getRuleContext(SourceIdentifierContext.class,i); + } + public TerminalNode ASSIGN() { return getToken(EsqlBaseParser.ASSIGN, 0); } + public EnrichWithClauseContext(ParserRuleContext parent, int invokingState) { + super(parent, invokingState); + } + @Override public int getRuleIndex() { return RULE_enrichWithClause; } + @Override + public void enterRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterEnrichWithClause(this); + } + @Override + public void exitRule(ParseTreeListener listener) { + if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitEnrichWithClause(this); + } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor)visitor).visitEnrichWithClause(this); + else return visitor.visitChildren(this); + } + } + + public final EnrichWithClauseContext enrichWithClause() throws RecognitionException { + EnrichWithClauseContext _localctx = new EnrichWithClauseContext(_ctx, getState()); + enterRule(_localctx, 90, RULE_enrichWithClause); + try { + enterOuterAlt(_localctx, 1); + { + setState(496); + _errHandler.sync(this); + switch ( getInterpreter().adaptivePredict(_input,50,_ctx) ) { + case 1: + { + setState(493); + ((EnrichWithClauseContext)_localctx).newName = sourceIdentifier(); + setState(494); + match(ASSIGN); + } + break; + } + setState(498); + ((EnrichWithClauseContext)_localctx).enrichField = sourceIdentifier(); + } + } + catch (RecognitionException re) { + _localctx.exception = re; + _errHandler.reportError(this, re); + _errHandler.recover(this, re); + } + finally { + exitRule(); + } + return _localctx; + } + + public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) { + switch (ruleIndex) { + case 1: + return query_sempred((QueryContext)_localctx, predIndex); + case 5: + return booleanExpression_sempred((BooleanExpressionContext)_localctx, predIndex); + case 8: + return operatorExpression_sempred((OperatorExpressionContext)_localctx, predIndex); + } + return true; + } + private boolean query_sempred(QueryContext _localctx, int predIndex) { + switch (predIndex) { + case 0: + return precpred(_ctx, 1); + } + return true; + } + private boolean booleanExpression_sempred(BooleanExpressionContext _localctx, int predIndex) { + switch (predIndex) { + case 1: + return precpred(_ctx, 4); + case 2: + return precpred(_ctx, 3); + } + return true; + } + private boolean operatorExpression_sempred(OperatorExpressionContext _localctx, int predIndex) { + switch (predIndex) { + case 3: + return precpred(_ctx, 2); + case 4: + return precpred(_ctx, 1); + } + return true; + } + + public static final String _serializedATN = + "\u0004\u0001Q\u01f5\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001\u0002"+ + "\u0002\u0007\u0002\u0002\u0003\u0007\u0003\u0002\u0004\u0007\u0004\u0002"+ + "\u0005\u0007\u0005\u0002\u0006\u0007\u0006\u0002\u0007\u0007\u0007\u0002"+ + "\b\u0007\b\u0002\t\u0007\t\u0002\n\u0007\n\u0002\u000b\u0007\u000b\u0002"+ + "\f\u0007\f\u0002\r\u0007\r\u0002\u000e\u0007\u000e\u0002\u000f\u0007\u000f"+ + "\u0002\u0010\u0007\u0010\u0002\u0011\u0007\u0011\u0002\u0012\u0007\u0012"+ + "\u0002\u0013\u0007\u0013\u0002\u0014\u0007\u0014\u0002\u0015\u0007\u0015"+ + "\u0002\u0016\u0007\u0016\u0002\u0017\u0007\u0017\u0002\u0018\u0007\u0018"+ + "\u0002\u0019\u0007\u0019\u0002\u001a\u0007\u001a\u0002\u001b\u0007\u001b"+ + "\u0002\u001c\u0007\u001c\u0002\u001d\u0007\u001d\u0002\u001e\u0007\u001e"+ + "\u0002\u001f\u0007\u001f\u0002 \u0007 \u0002!\u0007!\u0002\"\u0007\"\u0002"+ + "#\u0007#\u0002$\u0007$\u0002%\u0007%\u0002&\u0007&\u0002\'\u0007\'\u0002"+ + "(\u0007(\u0002)\u0007)\u0002*\u0007*\u0002+\u0007+\u0002,\u0007,\u0002"+ + "-\u0007-\u0001\u0000\u0001\u0000\u0001\u0000\u0001\u0001\u0001\u0001\u0001"+ + "\u0001\u0001\u0001\u0001\u0001\u0001\u0001\u0005\u0001f\b\u0001\n\u0001"+ + "\f\u0001i\t\u0001\u0001\u0002\u0001\u0002\u0001\u0002\u0001\u0002\u0003"+ + "\u0002o\b\u0002\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001"+ + "\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001\u0003\u0001"+ + "\u0003\u0001\u0003\u0001\u0003\u0003\u0003~\b\u0003\u0001\u0004\u0001"+ + "\u0004\u0001\u0004\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+ + "\u0005\u0001\u0005\u0001\u0005\u0003\u0005\u008a\b\u0005\u0001\u0005\u0001"+ + "\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0005\u0005\u0091\b\u0005\n"+ + "\u0005\f\u0005\u0094\t\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+ + "\u0005\u0001\u0005\u0003\u0005\u009b\b\u0005\u0001\u0005\u0001\u0005\u0003"+ + "\u0005\u009f\b\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001\u0005\u0001"+ + "\u0005\u0001\u0005\u0005\u0005\u00a7\b\u0005\n\u0005\f\u0005\u00aa\t\u0005"+ + "\u0001\u0006\u0001\u0006\u0003\u0006\u00ae\b\u0006\u0001\u0006\u0001\u0006"+ + "\u0001\u0006\u0001\u0006\u0001\u0006\u0003\u0006\u00b5\b\u0006\u0001\u0006"+ + "\u0001\u0006\u0001\u0006\u0003\u0006\u00ba\b\u0006\u0001\u0007\u0001\u0007"+ + "\u0001\u0007\u0001\u0007\u0001\u0007\u0003\u0007\u00c1\b\u0007\u0001\b"+ + "\u0001\b\u0001\b\u0001\b\u0003\b\u00c7\b\b\u0001\b\u0001\b\u0001\b\u0001"+ + "\b\u0001\b\u0001\b\u0005\b\u00cf\b\b\n\b\f\b\u00d2\t\b\u0001\t\u0001\t"+ + "\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001\t\u0001"+ + "\t\u0005\t\u00df\b\t\n\t\f\t\u00e2\t\t\u0003\t\u00e4\b\t\u0001\t\u0001"+ + "\t\u0003\t\u00e8\b\t\u0001\n\u0001\n\u0001\n\u0001\u000b\u0001\u000b\u0001"+ + "\u000b\u0005\u000b\u00f0\b\u000b\n\u000b\f\u000b\u00f3\t\u000b\u0001\f"+ + "\u0001\f\u0001\f\u0001\f\u0001\f\u0003\f\u00fa\b\f\u0001\r\u0001\r\u0001"+ + "\r\u0001\r\u0005\r\u0100\b\r\n\r\f\r\u0103\t\r\u0001\r\u0003\r\u0106\b"+ + "\r\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0001\u000e\u0005\u000e"+ + "\u010d\b\u000e\n\u000e\f\u000e\u0110\t\u000e\u0001\u000e\u0001\u000e\u0001"+ + "\u000f\u0001\u000f\u0001\u000f\u0001\u0010\u0001\u0010\u0003\u0010\u0119"+ + "\b\u0010\u0001\u0010\u0001\u0010\u0003\u0010\u011d\b\u0010\u0001\u0011"+ + "\u0001\u0011\u0001\u0011\u0001\u0011\u0003\u0011\u0123\b\u0011\u0001\u0012"+ + "\u0001\u0012\u0001\u0012\u0005\u0012\u0128\b\u0012\n\u0012\f\u0012\u012b"+ + "\t\u0012\u0001\u0013\u0001\u0013\u0001\u0014\u0001\u0014\u0001\u0014\u0005"+ + "\u0014\u0132\b\u0014\n\u0014\f\u0014\u0135\t\u0014\u0001\u0015\u0001\u0015"+ + "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016"+ + "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016"+ + "\u0001\u0016\u0005\u0016\u0146\b\u0016\n\u0016\f\u0016\u0149\t\u0016\u0001"+ + "\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0005"+ + "\u0016\u0151\b\u0016\n\u0016\f\u0016\u0154\t\u0016\u0001\u0016\u0001\u0016"+ + "\u0001\u0016\u0001\u0016\u0001\u0016\u0001\u0016\u0005\u0016\u015c\b\u0016"+ + "\n\u0016\f\u0016\u015f\t\u0016\u0001\u0016\u0001\u0016\u0003\u0016\u0163"+ + "\b\u0016\u0001\u0017\u0001\u0017\u0001\u0017\u0001\u0018\u0001\u0018\u0001"+ + "\u0018\u0001\u0018\u0005\u0018\u016c\b\u0018\n\u0018\f\u0018\u016f\t\u0018"+ + "\u0001\u0019\u0001\u0019\u0003\u0019\u0173\b\u0019\u0001\u0019\u0001\u0019"+ + "\u0003\u0019\u0177\b\u0019\u0001\u001a\u0001\u001a\u0001\u001a\u0001\u001a"+ + "\u0005\u001a\u017d\b\u001a\n\u001a\f\u001a\u0180\t\u001a\u0001\u001a\u0001"+ + "\u001a\u0001\u001a\u0001\u001a\u0005\u001a\u0186\b\u001a\n\u001a\f\u001a"+ + "\u0189\t\u001a\u0003\u001a\u018b\b\u001a\u0001\u001b\u0001\u001b\u0001"+ + "\u001b\u0001\u001b\u0005\u001b\u0191\b\u001b\n\u001b\f\u001b\u0194\t\u001b"+ + "\u0001\u001c\u0001\u001c\u0001\u001c\u0001\u001c\u0005\u001c\u019a\b\u001c"+ + "\n\u001c\f\u001c\u019d\t\u001c\u0001\u001d\u0001\u001d\u0001\u001d\u0001"+ + "\u001d\u0001\u001e\u0001\u001e\u0001\u001e\u0001\u001e\u0003\u001e\u01a7"+ + "\b\u001e\u0001\u001f\u0001\u001f\u0001\u001f\u0001\u001f\u0001 \u0001"+ + " \u0001 \u0001!\u0001!\u0001!\u0005!\u01b3\b!\n!\f!\u01b6\t!\u0001\"\u0001"+ + "\"\u0001\"\u0001\"\u0001#\u0001#\u0001$\u0001$\u0003$\u01c0\b$\u0001%"+ + "\u0003%\u01c3\b%\u0001%\u0001%\u0001&\u0003&\u01c8\b&\u0001&\u0001&\u0001"+ + "\'\u0001\'\u0001(\u0001(\u0001)\u0001)\u0001)\u0001*\u0001*\u0001*\u0001"+ + "*\u0001+\u0001+\u0001+\u0001+\u0003+\u01db\b+\u0001,\u0001,\u0001,\u0001"+ + ",\u0003,\u01e1\b,\u0001,\u0001,\u0001,\u0001,\u0005,\u01e7\b,\n,\f,\u01ea"+ + "\t,\u0003,\u01ec\b,\u0001-\u0001-\u0001-\u0003-\u01f1\b-\u0001-\u0001"+ + "-\u0001-\u0000\u0003\u0002\n\u0010.\u0000\u0002\u0004\u0006\b\n\f\u000e"+ + "\u0010\u0012\u0014\u0016\u0018\u001a\u001c\u001e \"$&(*,.02468:<>@BDF"+ + "HJLNPRTVXZ\u0000\b\u0001\u0000<=\u0001\u0000>@\u0001\u0000LM\u0001\u0000"+ + "CD\u0002\u0000 ##\u0001\u0000&\'\u0002\u0000%%33\u0001\u00006;\u0213"+ + "\u0000\\\u0001\u0000\u0000\u0000\u0002_\u0001\u0000\u0000\u0000\u0004"+ + "n\u0001\u0000\u0000\u0000\u0006}\u0001\u0000\u0000\u0000\b\u007f\u0001"+ + "\u0000\u0000\u0000\n\u009e\u0001\u0000\u0000\u0000\f\u00b9\u0001\u0000"+ + "\u0000\u0000\u000e\u00c0\u0001\u0000\u0000\u0000\u0010\u00c6\u0001\u0000"+ + "\u0000\u0000\u0012\u00e7\u0001\u0000\u0000\u0000\u0014\u00e9\u0001\u0000"+ + "\u0000\u0000\u0016\u00ec\u0001\u0000\u0000\u0000\u0018\u00f9\u0001\u0000"+ + "\u0000\u0000\u001a\u00fb\u0001\u0000\u0000\u0000\u001c\u0107\u0001\u0000"+ + "\u0000\u0000\u001e\u0113\u0001\u0000\u0000\u0000 \u0116\u0001\u0000\u0000"+ + "\u0000\"\u011e\u0001\u0000\u0000\u0000$\u0124\u0001\u0000\u0000\u0000"+ + "&\u012c\u0001\u0000\u0000\u0000(\u012e\u0001\u0000\u0000\u0000*\u0136"+ + "\u0001\u0000\u0000\u0000,\u0162\u0001\u0000\u0000\u0000.\u0164\u0001\u0000"+ + "\u0000\u00000\u0167\u0001\u0000\u0000\u00002\u0170\u0001\u0000\u0000\u0000"+ + "4\u018a\u0001\u0000\u0000\u00006\u018c\u0001\u0000\u0000\u00008\u0195"+ + "\u0001\u0000\u0000\u0000:\u019e\u0001\u0000\u0000\u0000<\u01a2\u0001\u0000"+ + "\u0000\u0000>\u01a8\u0001\u0000\u0000\u0000@\u01ac\u0001\u0000\u0000\u0000"+ + "B\u01af\u0001\u0000\u0000\u0000D\u01b7\u0001\u0000\u0000\u0000F\u01bb"+ + "\u0001\u0000\u0000\u0000H\u01bf\u0001\u0000\u0000\u0000J\u01c2\u0001\u0000"+ + "\u0000\u0000L\u01c7\u0001\u0000\u0000\u0000N\u01cb\u0001\u0000\u0000\u0000"+ + "P\u01cd\u0001\u0000\u0000\u0000R\u01cf\u0001\u0000\u0000\u0000T\u01d2"+ + "\u0001\u0000\u0000\u0000V\u01da\u0001\u0000\u0000\u0000X\u01dc\u0001\u0000"+ + "\u0000\u0000Z\u01f0\u0001\u0000\u0000\u0000\\]\u0003\u0002\u0001\u0000"+ + "]^\u0005\u0000\u0000\u0001^\u0001\u0001\u0000\u0000\u0000_`\u0006\u0001"+ + "\uffff\uffff\u0000`a\u0003\u0004\u0002\u0000ag\u0001\u0000\u0000\u0000"+ + "bc\n\u0001\u0000\u0000cd\u0005\u001a\u0000\u0000df\u0003\u0006\u0003\u0000"+ + "eb\u0001\u0000\u0000\u0000fi\u0001\u0000\u0000\u0000ge\u0001\u0000\u0000"+ + "\u0000gh\u0001\u0000\u0000\u0000h\u0003\u0001\u0000\u0000\u0000ig\u0001"+ + "\u0000\u0000\u0000jo\u0003R)\u0000ko\u0003\u001a\r\u0000lo\u0003\u0014"+ + "\n\u0000mo\u0003V+\u0000nj\u0001\u0000\u0000\u0000nk\u0001\u0000\u0000"+ + "\u0000nl\u0001\u0000\u0000\u0000nm\u0001\u0000\u0000\u0000o\u0005\u0001"+ + "\u0000\u0000\u0000p~\u0003\u001e\u000f\u0000q~\u0003\"\u0011\u0000r~\u0003"+ + ".\u0017\u0000s~\u00034\u001a\u0000t~\u00030\u0018\u0000u~\u0003 \u0010"+ + "\u0000v~\u0003\b\u0004\u0000w~\u00036\u001b\u0000x~\u00038\u001c\u0000"+ + "y~\u0003<\u001e\u0000z~\u0003>\u001f\u0000{~\u0003X,\u0000|~\u0003@ \u0000"+ + "}p\u0001\u0000\u0000\u0000}q\u0001\u0000\u0000\u0000}r\u0001\u0000\u0000"+ + "\u0000}s\u0001\u0000\u0000\u0000}t\u0001\u0000\u0000\u0000}u\u0001\u0000"+ + "\u0000\u0000}v\u0001\u0000\u0000\u0000}w\u0001\u0000\u0000\u0000}x\u0001"+ + "\u0000\u0000\u0000}y\u0001\u0000\u0000\u0000}z\u0001\u0000\u0000\u0000"+ + "}{\u0001\u0000\u0000\u0000}|\u0001\u0000\u0000\u0000~\u0007\u0001\u0000"+ + "\u0000\u0000\u007f\u0080\u0005\u0012\u0000\u0000\u0080\u0081\u0003\n\u0005"+ + "\u0000\u0081\t\u0001\u0000\u0000\u0000\u0082\u0083\u0006\u0005\uffff\uffff"+ + "\u0000\u0083\u0084\u0005,\u0000\u0000\u0084\u009f\u0003\n\u0005\u0007"+ + "\u0085\u009f\u0003\u000e\u0007\u0000\u0086\u009f\u0003\f\u0006\u0000\u0087"+ + "\u0089\u0003\u000e\u0007\u0000\u0088\u008a\u0005,\u0000\u0000\u0089\u0088"+ + "\u0001\u0000\u0000\u0000\u0089\u008a\u0001\u0000\u0000\u0000\u008a\u008b"+ + "\u0001\u0000\u0000\u0000\u008b\u008c\u0005)\u0000\u0000\u008c\u008d\u0005"+ + "(\u0000\u0000\u008d\u0092\u0003\u000e\u0007\u0000\u008e\u008f\u0005\""+ + "\u0000\u0000\u008f\u0091\u0003\u000e\u0007\u0000\u0090\u008e\u0001\u0000"+ + "\u0000\u0000\u0091\u0094\u0001\u0000\u0000\u0000\u0092\u0090\u0001\u0000"+ + "\u0000\u0000\u0092\u0093\u0001\u0000\u0000\u0000\u0093\u0095\u0001\u0000"+ + "\u0000\u0000\u0094\u0092\u0001\u0000\u0000\u0000\u0095\u0096\u00052\u0000"+ + "\u0000\u0096\u009f\u0001\u0000\u0000\u0000\u0097\u0098\u0003\u000e\u0007"+ + "\u0000\u0098\u009a\u0005*\u0000\u0000\u0099\u009b\u0005,\u0000\u0000\u009a"+ + "\u0099\u0001\u0000\u0000\u0000\u009a\u009b\u0001\u0000\u0000\u0000\u009b"+ + "\u009c\u0001\u0000\u0000\u0000\u009c\u009d\u0005-\u0000\u0000\u009d\u009f"+ + "\u0001\u0000\u0000\u0000\u009e\u0082\u0001\u0000\u0000\u0000\u009e\u0085"+ + "\u0001\u0000\u0000\u0000\u009e\u0086\u0001\u0000\u0000\u0000\u009e\u0087"+ + "\u0001\u0000\u0000\u0000\u009e\u0097\u0001\u0000\u0000\u0000\u009f\u00a8"+ + "\u0001\u0000\u0000\u0000\u00a0\u00a1\n\u0004\u0000\u0000\u00a1\u00a2\u0005"+ + "\u001f\u0000\u0000\u00a2\u00a7\u0003\n\u0005\u0005\u00a3\u00a4\n\u0003"+ + "\u0000\u0000\u00a4\u00a5\u0005/\u0000\u0000\u00a5\u00a7\u0003\n\u0005"+ + "\u0004\u00a6\u00a0\u0001\u0000\u0000\u0000\u00a6\u00a3\u0001\u0000\u0000"+ + "\u0000\u00a7\u00aa\u0001\u0000\u0000\u0000\u00a8\u00a6\u0001\u0000\u0000"+ + "\u0000\u00a8\u00a9\u0001\u0000\u0000\u0000\u00a9\u000b\u0001\u0000\u0000"+ + "\u0000\u00aa\u00a8\u0001\u0000\u0000\u0000\u00ab\u00ad\u0003\u000e\u0007"+ + "\u0000\u00ac\u00ae\u0005,\u0000\u0000\u00ad\u00ac\u0001\u0000\u0000\u0000"+ + "\u00ad\u00ae\u0001\u0000\u0000\u0000\u00ae\u00af\u0001\u0000\u0000\u0000"+ + "\u00af\u00b0\u0005+\u0000\u0000\u00b0\u00b1\u0003N\'\u0000\u00b1\u00ba"+ + "\u0001\u0000\u0000\u0000\u00b2\u00b4\u0003\u000e\u0007\u0000\u00b3\u00b5"+ + "\u0005,\u0000\u0000\u00b4\u00b3\u0001\u0000\u0000\u0000\u00b4\u00b5\u0001"+ + "\u0000\u0000\u0000\u00b5\u00b6\u0001\u0000\u0000\u0000\u00b6\u00b7\u0005"+ + "1\u0000\u0000\u00b7\u00b8\u0003N\'\u0000\u00b8\u00ba\u0001\u0000\u0000"+ + "\u0000\u00b9\u00ab\u0001\u0000\u0000\u0000\u00b9\u00b2\u0001\u0000\u0000"+ + "\u0000\u00ba\r\u0001\u0000\u0000\u0000\u00bb\u00c1\u0003\u0010\b\u0000"+ + "\u00bc\u00bd\u0003\u0010\b\u0000\u00bd\u00be\u0003P(\u0000\u00be\u00bf"+ + "\u0003\u0010\b\u0000\u00bf\u00c1\u0001\u0000\u0000\u0000\u00c0\u00bb\u0001"+ + "\u0000\u0000\u0000\u00c0\u00bc\u0001\u0000\u0000\u0000\u00c1\u000f\u0001"+ + "\u0000\u0000\u0000\u00c2\u00c3\u0006\b\uffff\uffff\u0000\u00c3\u00c7\u0003"+ + "\u0012\t\u0000\u00c4\u00c5\u0007\u0000\u0000\u0000\u00c5\u00c7\u0003\u0010"+ + "\b\u0003\u00c6\u00c2\u0001\u0000\u0000\u0000\u00c6\u00c4\u0001\u0000\u0000"+ + "\u0000\u00c7\u00d0\u0001\u0000\u0000\u0000\u00c8\u00c9\n\u0002\u0000\u0000"+ + "\u00c9\u00ca\u0007\u0001\u0000\u0000\u00ca\u00cf\u0003\u0010\b\u0003\u00cb"+ + "\u00cc\n\u0001\u0000\u0000\u00cc\u00cd\u0007\u0000\u0000\u0000\u00cd\u00cf"+ + "\u0003\u0010\b\u0002\u00ce\u00c8\u0001\u0000\u0000\u0000\u00ce\u00cb\u0001"+ + "\u0000\u0000\u0000\u00cf\u00d2\u0001\u0000\u0000\u0000\u00d0\u00ce\u0001"+ + "\u0000\u0000\u0000\u00d0\u00d1\u0001\u0000\u0000\u0000\u00d1\u0011\u0001"+ + "\u0000\u0000\u0000\u00d2\u00d0\u0001\u0000\u0000\u0000\u00d3\u00e8\u0003"+ + ",\u0016\u0000\u00d4\u00e8\u0003(\u0014\u0000\u00d5\u00d6\u0005(\u0000"+ + "\u0000\u00d6\u00d7\u0003\n\u0005\u0000\u00d7\u00d8\u00052\u0000\u0000"+ + "\u00d8\u00e8\u0001\u0000\u0000\u0000\u00d9\u00da\u0003*\u0015\u0000\u00da"+ + "\u00e3\u0005(\u0000\u0000\u00db\u00e0\u0003\n\u0005\u0000\u00dc\u00dd"+ + "\u0005\"\u0000\u0000\u00dd\u00df\u0003\n\u0005\u0000\u00de\u00dc\u0001"+ + "\u0000\u0000\u0000\u00df\u00e2\u0001\u0000\u0000\u0000\u00e0\u00de\u0001"+ + "\u0000\u0000\u0000\u00e0\u00e1\u0001\u0000\u0000\u0000\u00e1\u00e4\u0001"+ + "\u0000\u0000\u0000\u00e2\u00e0\u0001\u0000\u0000\u0000\u00e3\u00db\u0001"+ + "\u0000\u0000\u0000\u00e3\u00e4\u0001\u0000\u0000\u0000\u00e4\u00e5\u0001"+ + "\u0000\u0000\u0000\u00e5\u00e6\u00052\u0000\u0000\u00e6\u00e8\u0001\u0000"+ + "\u0000\u0000\u00e7\u00d3\u0001\u0000\u0000\u0000\u00e7\u00d4\u0001\u0000"+ + "\u0000\u0000\u00e7\u00d5\u0001\u0000\u0000\u0000\u00e7\u00d9\u0001\u0000"+ + "\u0000\u0000\u00e8\u0013\u0001\u0000\u0000\u0000\u00e9\u00ea\u0005\u000e"+ + "\u0000\u0000\u00ea\u00eb\u0003\u0016\u000b\u0000\u00eb\u0015\u0001\u0000"+ + "\u0000\u0000\u00ec\u00f1\u0003\u0018\f\u0000\u00ed\u00ee\u0005\"\u0000"+ + "\u0000\u00ee\u00f0\u0003\u0018\f\u0000\u00ef\u00ed\u0001\u0000\u0000\u0000"+ + "\u00f0\u00f3\u0001\u0000\u0000\u0000\u00f1\u00ef\u0001\u0000\u0000\u0000"+ + "\u00f1\u00f2\u0001\u0000\u0000\u0000\u00f2\u0017\u0001\u0000\u0000\u0000"+ + "\u00f3\u00f1\u0001\u0000\u0000\u0000\u00f4\u00fa\u0003\n\u0005\u0000\u00f5"+ + "\u00f6\u0003(\u0014\u0000\u00f6\u00f7\u0005!\u0000\u0000\u00f7\u00f8\u0003"+ + "\n\u0005\u0000\u00f8\u00fa\u0001\u0000\u0000\u0000\u00f9\u00f4\u0001\u0000"+ + "\u0000\u0000\u00f9\u00f5\u0001\u0000\u0000\u0000\u00fa\u0019\u0001\u0000"+ + "\u0000\u0000\u00fb\u00fc\u0005\u0006\u0000\u0000\u00fc\u0101\u0003&\u0013"+ + "\u0000\u00fd\u00fe\u0005\"\u0000\u0000\u00fe\u0100\u0003&\u0013\u0000"+ + "\u00ff\u00fd\u0001\u0000\u0000\u0000\u0100\u0103\u0001\u0000\u0000\u0000"+ + "\u0101\u00ff\u0001\u0000\u0000\u0000\u0101\u0102\u0001\u0000\u0000\u0000"+ + "\u0102\u0105\u0001\u0000\u0000\u0000\u0103\u0101\u0001\u0000\u0000\u0000"+ + "\u0104\u0106\u0003\u001c\u000e\u0000\u0105\u0104\u0001\u0000\u0000\u0000"+ + "\u0105\u0106\u0001\u0000\u0000\u0000\u0106\u001b\u0001\u0000\u0000\u0000"+ + "\u0107\u0108\u0005A\u0000\u0000\u0108\u0109\u0005I\u0000\u0000\u0109\u010e"+ + "\u0003&\u0013\u0000\u010a\u010b\u0005\"\u0000\u0000\u010b\u010d\u0003"+ + "&\u0013\u0000\u010c\u010a\u0001\u0000\u0000\u0000\u010d\u0110\u0001\u0000"+ + "\u0000\u0000\u010e\u010c\u0001\u0000\u0000\u0000\u010e\u010f\u0001\u0000"+ + "\u0000\u0000\u010f\u0111\u0001\u0000\u0000\u0000\u0110\u010e\u0001\u0000"+ + "\u0000\u0000\u0111\u0112\u0005B\u0000\u0000\u0112\u001d\u0001\u0000\u0000"+ + "\u0000\u0113\u0114\u0005\u0004\u0000\u0000\u0114\u0115\u0003\u0016\u000b"+ + "\u0000\u0115\u001f\u0001\u0000\u0000\u0000\u0116\u0118\u0005\u0011\u0000"+ + "\u0000\u0117\u0119\u0003\u0016\u000b\u0000\u0118\u0117\u0001\u0000\u0000"+ + "\u0000\u0118\u0119\u0001\u0000\u0000\u0000\u0119\u011c\u0001\u0000\u0000"+ + "\u0000\u011a\u011b\u0005\u001e\u0000\u0000\u011b\u011d\u0003$\u0012\u0000"+ + "\u011c\u011a\u0001\u0000\u0000\u0000\u011c\u011d\u0001\u0000\u0000\u0000"+ + "\u011d!\u0001\u0000\u0000\u0000\u011e\u011f\u0005\b\u0000\u0000\u011f"+ + "\u0122\u0003\u0016\u000b\u0000\u0120\u0121\u0005\u001e\u0000\u0000\u0121"+ + "\u0123\u0003$\u0012\u0000\u0122\u0120\u0001\u0000\u0000\u0000\u0122\u0123"+ + "\u0001\u0000\u0000\u0000\u0123#\u0001\u0000\u0000\u0000\u0124\u0129\u0003"+ + "(\u0014\u0000\u0125\u0126\u0005\"\u0000\u0000\u0126\u0128\u0003(\u0014"+ + "\u0000\u0127\u0125\u0001\u0000\u0000\u0000\u0128\u012b\u0001\u0000\u0000"+ + "\u0000\u0129\u0127\u0001\u0000\u0000\u0000\u0129\u012a\u0001\u0000\u0000"+ + "\u0000\u012a%\u0001\u0000\u0000\u0000\u012b\u0129\u0001\u0000\u0000\u0000"+ + "\u012c\u012d\u0007\u0002\u0000\u0000\u012d\'\u0001\u0000\u0000\u0000\u012e"+ + "\u0133\u0003*\u0015\u0000\u012f\u0130\u0005$\u0000\u0000\u0130\u0132\u0003"+ + "*\u0015\u0000\u0131\u012f\u0001\u0000\u0000\u0000\u0132\u0135\u0001\u0000"+ + "\u0000\u0000\u0133\u0131\u0001\u0000\u0000\u0000\u0133\u0134\u0001\u0000"+ + "\u0000\u0000\u0134)\u0001\u0000\u0000\u0000\u0135\u0133\u0001\u0000\u0000"+ + "\u0000\u0136\u0137\u0007\u0003\u0000\u0000\u0137+\u0001\u0000\u0000\u0000"+ + "\u0138\u0163\u0005-\u0000\u0000\u0139\u013a\u0003L&\u0000\u013a\u013b"+ + "\u0005C\u0000\u0000\u013b\u0163\u0001\u0000\u0000\u0000\u013c\u0163\u0003"+ + "J%\u0000\u013d\u0163\u0003L&\u0000\u013e\u0163\u0003F#\u0000\u013f\u0163"+ + "\u00050\u0000\u0000\u0140\u0163\u0003N\'\u0000\u0141\u0142\u0005A\u0000"+ + "\u0000\u0142\u0147\u0003H$\u0000\u0143\u0144\u0005\"\u0000\u0000\u0144"+ + "\u0146\u0003H$\u0000\u0145\u0143\u0001\u0000\u0000\u0000\u0146\u0149\u0001"+ + "\u0000\u0000\u0000\u0147\u0145\u0001\u0000\u0000\u0000\u0147\u0148\u0001"+ + "\u0000\u0000\u0000\u0148\u014a\u0001\u0000\u0000\u0000\u0149\u0147\u0001"+ + "\u0000\u0000\u0000\u014a\u014b\u0005B\u0000\u0000\u014b\u0163\u0001\u0000"+ + "\u0000\u0000\u014c\u014d\u0005A\u0000\u0000\u014d\u0152\u0003F#\u0000"+ + "\u014e\u014f\u0005\"\u0000\u0000\u014f\u0151\u0003F#\u0000\u0150\u014e"+ + "\u0001\u0000\u0000\u0000\u0151\u0154\u0001\u0000\u0000\u0000\u0152\u0150"+ + "\u0001\u0000\u0000\u0000\u0152\u0153\u0001\u0000\u0000\u0000\u0153\u0155"+ + "\u0001\u0000\u0000\u0000\u0154\u0152\u0001\u0000\u0000\u0000\u0155\u0156"+ + "\u0005B\u0000\u0000\u0156\u0163\u0001\u0000\u0000\u0000\u0157\u0158\u0005"+ + "A\u0000\u0000\u0158\u015d\u0003N\'\u0000\u0159\u015a\u0005\"\u0000\u0000"+ + "\u015a\u015c\u0003N\'\u0000\u015b\u0159\u0001\u0000\u0000\u0000\u015c"+ + "\u015f\u0001\u0000\u0000\u0000\u015d\u015b\u0001\u0000\u0000\u0000\u015d"+ + "\u015e\u0001\u0000\u0000\u0000\u015e\u0160\u0001\u0000\u0000\u0000\u015f"+ + "\u015d\u0001\u0000\u0000\u0000\u0160\u0161\u0005B\u0000\u0000\u0161\u0163"+ + "\u0001\u0000\u0000\u0000\u0162\u0138\u0001\u0000\u0000\u0000\u0162\u0139"+ + "\u0001\u0000\u0000\u0000\u0162\u013c\u0001\u0000\u0000\u0000\u0162\u013d"+ + "\u0001\u0000\u0000\u0000\u0162\u013e\u0001\u0000\u0000\u0000\u0162\u013f"+ + "\u0001\u0000\u0000\u0000\u0162\u0140\u0001\u0000\u0000\u0000\u0162\u0141"+ + "\u0001\u0000\u0000\u0000\u0162\u014c\u0001\u0000\u0000\u0000\u0162\u0157"+ + "\u0001\u0000\u0000\u0000\u0163-\u0001\u0000\u0000\u0000\u0164\u0165\u0005"+ + "\n\u0000\u0000\u0165\u0166\u0005\u001c\u0000\u0000\u0166/\u0001\u0000"+ + "\u0000\u0000\u0167\u0168\u0005\u0010\u0000\u0000\u0168\u016d\u00032\u0019"+ + "\u0000\u0169\u016a\u0005\"\u0000\u0000\u016a\u016c\u00032\u0019\u0000"+ + "\u016b\u0169\u0001\u0000\u0000\u0000\u016c\u016f\u0001\u0000\u0000\u0000"+ + "\u016d\u016b\u0001\u0000\u0000\u0000\u016d\u016e\u0001\u0000\u0000\u0000"+ + "\u016e1\u0001\u0000\u0000\u0000\u016f\u016d\u0001\u0000\u0000\u0000\u0170"+ + "\u0172\u0003\n\u0005\u0000\u0171\u0173\u0007\u0004\u0000\u0000\u0172\u0171"+ + "\u0001\u0000\u0000\u0000\u0172\u0173\u0001\u0000\u0000\u0000\u0173\u0176"+ + "\u0001\u0000\u0000\u0000\u0174\u0175\u0005.\u0000\u0000\u0175\u0177\u0007"+ + "\u0005\u0000\u0000\u0176\u0174\u0001\u0000\u0000\u0000\u0176\u0177\u0001"+ + "\u0000\u0000\u0000\u01773\u0001\u0000\u0000\u0000\u0178\u0179\u0005\t"+ + "\u0000\u0000\u0179\u017e\u0003&\u0013\u0000\u017a\u017b\u0005\"\u0000"+ + "\u0000\u017b\u017d\u0003&\u0013\u0000\u017c\u017a\u0001\u0000\u0000\u0000"+ + "\u017d\u0180\u0001\u0000\u0000\u0000\u017e\u017c\u0001\u0000\u0000\u0000"+ + "\u017e\u017f\u0001\u0000\u0000\u0000\u017f\u018b\u0001\u0000\u0000\u0000"+ + "\u0180\u017e\u0001\u0000\u0000\u0000\u0181\u0182\u0005\f\u0000\u0000\u0182"+ + "\u0187\u0003&\u0013\u0000\u0183\u0184\u0005\"\u0000\u0000\u0184\u0186"+ + "\u0003&\u0013\u0000\u0185\u0183\u0001\u0000\u0000\u0000\u0186\u0189\u0001"+ + "\u0000\u0000\u0000\u0187\u0185\u0001\u0000\u0000\u0000\u0187\u0188\u0001"+ + "\u0000\u0000\u0000\u0188\u018b\u0001\u0000\u0000\u0000\u0189\u0187\u0001"+ + "\u0000\u0000\u0000\u018a\u0178\u0001\u0000\u0000\u0000\u018a\u0181\u0001"+ + "\u0000\u0000\u0000\u018b5\u0001\u0000\u0000\u0000\u018c\u018d\u0005\u0002"+ + "\u0000\u0000\u018d\u0192\u0003&\u0013\u0000\u018e\u018f\u0005\"\u0000"+ + "\u0000\u018f\u0191\u0003&\u0013\u0000\u0190\u018e\u0001\u0000\u0000\u0000"+ + "\u0191\u0194\u0001\u0000\u0000\u0000\u0192\u0190\u0001\u0000\u0000\u0000"+ + "\u0192\u0193\u0001\u0000\u0000\u0000\u01937\u0001\u0000\u0000\u0000\u0194"+ + "\u0192\u0001\u0000\u0000\u0000\u0195\u0196\u0005\r\u0000\u0000\u0196\u019b"+ + "\u0003:\u001d\u0000\u0197\u0198\u0005\"\u0000\u0000\u0198\u019a\u0003"+ + ":\u001d\u0000\u0199\u0197\u0001\u0000\u0000\u0000\u019a\u019d\u0001\u0000"+ + "\u0000\u0000\u019b\u0199\u0001\u0000\u0000\u0000\u019b\u019c\u0001\u0000"+ + "\u0000\u0000\u019c9\u0001\u0000\u0000\u0000\u019d\u019b\u0001\u0000\u0000"+ + "\u0000\u019e\u019f\u0003&\u0013\u0000\u019f\u01a0\u0005H\u0000\u0000\u01a0"+ + "\u01a1\u0003&\u0013\u0000\u01a1;\u0001\u0000\u0000\u0000\u01a2\u01a3\u0005"+ + "\u0001\u0000\u0000\u01a3\u01a4\u0003\u0012\t\u0000\u01a4\u01a6\u0003N"+ + "\'\u0000\u01a5\u01a7\u0003B!\u0000\u01a6\u01a5\u0001\u0000\u0000\u0000"+ + "\u01a6\u01a7\u0001\u0000\u0000\u0000\u01a7=\u0001\u0000\u0000\u0000\u01a8"+ + "\u01a9\u0005\u0007\u0000\u0000\u01a9\u01aa\u0003\u0012\t\u0000\u01aa\u01ab"+ + "\u0003N\'\u0000\u01ab?\u0001\u0000\u0000\u0000\u01ac\u01ad\u0005\u000b"+ + "\u0000\u0000\u01ad\u01ae\u0003&\u0013\u0000\u01aeA\u0001\u0000\u0000\u0000"+ + "\u01af\u01b4\u0003D\"\u0000\u01b0\u01b1\u0005\"\u0000\u0000\u01b1\u01b3"+ + "\u0003D\"\u0000\u01b2\u01b0\u0001\u0000\u0000\u0000\u01b3\u01b6\u0001"+ + "\u0000\u0000\u0000\u01b4\u01b2\u0001\u0000\u0000\u0000\u01b4\u01b5\u0001"+ + "\u0000\u0000\u0000\u01b5C\u0001\u0000\u0000\u0000\u01b6\u01b4\u0001\u0000"+ + "\u0000\u0000\u01b7\u01b8\u0003*\u0015\u0000\u01b8\u01b9\u0005!\u0000\u0000"+ + "\u01b9\u01ba\u0003,\u0016\u0000\u01baE\u0001\u0000\u0000\u0000\u01bb\u01bc"+ + "\u0007\u0006\u0000\u0000\u01bcG\u0001\u0000\u0000\u0000\u01bd\u01c0\u0003"+ + "J%\u0000\u01be\u01c0\u0003L&\u0000\u01bf\u01bd\u0001\u0000\u0000\u0000"+ + "\u01bf\u01be\u0001\u0000\u0000\u0000\u01c0I\u0001\u0000\u0000\u0000\u01c1"+ + "\u01c3\u0007\u0000\u0000\u0000\u01c2\u01c1\u0001\u0000\u0000\u0000\u01c2"+ + "\u01c3\u0001\u0000\u0000\u0000\u01c3\u01c4\u0001\u0000\u0000\u0000\u01c4"+ + "\u01c5\u0005\u001d\u0000\u0000\u01c5K\u0001\u0000\u0000\u0000\u01c6\u01c8"+ + "\u0007\u0000\u0000\u0000\u01c7\u01c6\u0001\u0000\u0000\u0000\u01c7\u01c8"+ + "\u0001\u0000\u0000\u0000\u01c8\u01c9\u0001\u0000\u0000\u0000\u01c9\u01ca"+ + "\u0005\u001c\u0000\u0000\u01caM\u0001\u0000\u0000\u0000\u01cb\u01cc\u0005"+ + "\u001b\u0000\u0000\u01ccO\u0001\u0000\u0000\u0000\u01cd\u01ce\u0007\u0007"+ + "\u0000\u0000\u01ceQ\u0001\u0000\u0000\u0000\u01cf\u01d0\u0005\u0005\u0000"+ + "\u0000\u01d0\u01d1\u0003T*\u0000\u01d1S\u0001\u0000\u0000\u0000\u01d2"+ + "\u01d3\u0005A\u0000\u0000\u01d3\u01d4\u0003\u0002\u0001\u0000\u01d4\u01d5"+ + "\u0005B\u0000\u0000\u01d5U\u0001\u0000\u0000\u0000\u01d6\u01d7\u0005\u000f"+ + "\u0000\u0000\u01d7\u01db\u00054\u0000\u0000\u01d8\u01d9\u0005\u000f\u0000"+ + "\u0000\u01d9\u01db\u00055\u0000\u0000\u01da\u01d6\u0001\u0000\u0000\u0000"+ + "\u01da\u01d8\u0001\u0000\u0000\u0000\u01dbW\u0001\u0000\u0000\u0000\u01dc"+ + "\u01dd\u0005\u0003\u0000\u0000\u01dd\u01e0\u0003&\u0013\u0000\u01de\u01df"+ + "\u0005J\u0000\u0000\u01df\u01e1\u0003&\u0013\u0000\u01e0\u01de\u0001\u0000"+ + "\u0000\u0000\u01e0\u01e1\u0001\u0000\u0000\u0000\u01e1\u01eb\u0001\u0000"+ + "\u0000\u0000\u01e2\u01e3\u0005K\u0000\u0000\u01e3\u01e8\u0003Z-\u0000"+ + "\u01e4\u01e5\u0005\"\u0000\u0000\u01e5\u01e7\u0003Z-\u0000\u01e6\u01e4"+ + "\u0001\u0000\u0000\u0000\u01e7\u01ea\u0001\u0000\u0000\u0000\u01e8\u01e6"+ + "\u0001\u0000\u0000\u0000\u01e8\u01e9\u0001\u0000\u0000\u0000\u01e9\u01ec"+ + "\u0001\u0000\u0000\u0000\u01ea\u01e8\u0001\u0000\u0000\u0000\u01eb\u01e2"+ + "\u0001\u0000\u0000\u0000\u01eb\u01ec\u0001\u0000\u0000\u0000\u01ecY\u0001"+ + "\u0000\u0000\u0000\u01ed\u01ee\u0003&\u0013\u0000\u01ee\u01ef\u0005!\u0000"+ + "\u0000\u01ef\u01f1\u0001\u0000\u0000\u0000\u01f0\u01ed\u0001\u0000\u0000"+ + "\u0000\u01f0\u01f1\u0001\u0000\u0000\u0000\u01f1\u01f2\u0001\u0000\u0000"+ + "\u0000\u01f2\u01f3\u0003&\u0013\u0000\u01f3[\u0001\u0000\u0000\u00003"+ + "gn}\u0089\u0092\u009a\u009e\u00a6\u00a8\u00ad\u00b4\u00b9\u00c0\u00c6"+ + "\u00ce\u00d0\u00e0\u00e3\u00e7\u00f1\u00f9\u0101\u0105\u010e\u0118\u011c"+ + "\u0122\u0129\u0133\u0147\u0152\u015d\u0162\u016d\u0172\u0176\u017e\u0187"+ + "\u018a\u0192\u019b\u01a6\u01b4\u01bf\u01c2\u01c7\u01da\u01e0\u01e8\u01eb"+ + "\u01f0"; + public static final ATN _ATN = + new ATNDeserializer().deserialize(_serializedATN.toCharArray()); + static { + _decisionToDFA = new DFA[_ATN.getNumberOfDecisions()]; + for (int i = 0; i < _ATN.getNumberOfDecisions(); i++) { + _decisionToDFA[i] = new DFA(_ATN.getDecisionState(i), i); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java new file mode 100644 index 0000000000000..ceef1b4e681a7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java @@ -0,0 +1,856 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.tree.ErrorNode; +import org.antlr.v4.runtime.tree.TerminalNode; + +/** + * This class provides an empty implementation of {@link EsqlBaseParserListener}, + * which can be extended to create a listener which only needs to handle a subset + * of the available methods. + */ +@SuppressWarnings("CheckReturnValue") +public class EsqlBaseParserBaseListener implements EsqlBaseParserListener { + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSingleStatement(EsqlBaseParser.SingleStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSingleStatement(EsqlBaseParser.SingleStatementContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSourceCommand(EsqlBaseParser.SourceCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSourceCommand(EsqlBaseParser.SourceCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterWhereCommand(EsqlBaseParser.WhereCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitWhereCommand(EsqlBaseParser.WhereCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalNot(EsqlBaseParser.LogicalNotContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalNot(EsqlBaseParser.LogicalNotContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIsNull(EsqlBaseParser.IsNullContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIsNull(EsqlBaseParser.IsNullContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterRegexExpression(EsqlBaseParser.RegexExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitRegexExpression(EsqlBaseParser.RegexExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalIn(EsqlBaseParser.LogicalInContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalIn(EsqlBaseParser.LogicalInContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparison(EsqlBaseParser.ComparisonContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparison(EsqlBaseParser.ComparisonContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDereference(EsqlBaseParser.DereferenceContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDereference(EsqlBaseParser.DereferenceContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterRowCommand(EsqlBaseParser.RowCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitRowCommand(EsqlBaseParser.RowCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterFields(EsqlBaseParser.FieldsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitFields(EsqlBaseParser.FieldsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterField(EsqlBaseParser.FieldContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitField(EsqlBaseParser.FieldContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterFromCommand(EsqlBaseParser.FromCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitFromCommand(EsqlBaseParser.FromCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterMetadata(EsqlBaseParser.MetadataContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitMetadata(EsqlBaseParser.MetadataContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterEvalCommand(EsqlBaseParser.EvalCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitEvalCommand(EsqlBaseParser.EvalCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterStatsCommand(EsqlBaseParser.StatsCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitStatsCommand(EsqlBaseParser.StatsCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterGrouping(EsqlBaseParser.GroupingContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitGrouping(EsqlBaseParser.GroupingContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterQualifiedName(EsqlBaseParser.QualifiedNameContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitQualifiedName(EsqlBaseParser.QualifiedNameContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIdentifier(EsqlBaseParser.IdentifierContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIdentifier(EsqlBaseParser.IdentifierContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNullLiteral(EsqlBaseParser.NullLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNullLiteral(EsqlBaseParser.NullLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterInputParam(EsqlBaseParser.InputParamContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitInputParam(EsqlBaseParser.InputParamContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterStringLiteral(EsqlBaseParser.StringLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitStringLiteral(EsqlBaseParser.StringLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterLimitCommand(EsqlBaseParser.LimitCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitLimitCommand(EsqlBaseParser.LimitCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSortCommand(EsqlBaseParser.SortCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSortCommand(EsqlBaseParser.SortCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterOrderExpression(EsqlBaseParser.OrderExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitOrderExpression(EsqlBaseParser.OrderExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterKeepCommand(EsqlBaseParser.KeepCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitKeepCommand(EsqlBaseParser.KeepCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDropCommand(EsqlBaseParser.DropCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDropCommand(EsqlBaseParser.DropCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterRenameCommand(EsqlBaseParser.RenameCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitRenameCommand(EsqlBaseParser.RenameCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterRenameClause(EsqlBaseParser.RenameClauseContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitRenameClause(EsqlBaseParser.RenameClauseContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDissectCommand(EsqlBaseParser.DissectCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDissectCommand(EsqlBaseParser.DissectCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterGrokCommand(EsqlBaseParser.GrokCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitGrokCommand(EsqlBaseParser.GrokCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterCommandOptions(EsqlBaseParser.CommandOptionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitCommandOptions(EsqlBaseParser.CommandOptionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterCommandOption(EsqlBaseParser.CommandOptionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitCommandOption(EsqlBaseParser.CommandOptionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterBooleanValue(EsqlBaseParser.BooleanValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitBooleanValue(EsqlBaseParser.BooleanValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterNumericValue(EsqlBaseParser.NumericValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitNumericValue(EsqlBaseParser.NumericValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterDecimalValue(EsqlBaseParser.DecimalValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitDecimalValue(EsqlBaseParser.DecimalValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterIntegerValue(EsqlBaseParser.IntegerValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitIntegerValue(EsqlBaseParser.IntegerValueContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterString(EsqlBaseParser.StringContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitString(EsqlBaseParser.StringContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterExplainCommand(EsqlBaseParser.ExplainCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitExplainCommand(EsqlBaseParser.ExplainCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterShowInfo(EsqlBaseParser.ShowInfoContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitShowInfo(EsqlBaseParser.ShowInfoContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx) { } + + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void enterEveryRule(ParserRuleContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void exitEveryRule(ParserRuleContext ctx) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void visitTerminal(TerminalNode node) { } + /** + * {@inheritDoc} + * + *

The default implementation does nothing.

+ */ + @Override public void visitErrorNode(ErrorNode node) { } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java new file mode 100644 index 0000000000000..48f5b33fcfec1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java @@ -0,0 +1,491 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; +import org.antlr.v4.runtime.tree.AbstractParseTreeVisitor; + +/** + * This class provides an empty implementation of {@link EsqlBaseParserVisitor}, + * which can be extended to create a visitor which only needs to handle a subset + * of the available methods. + * + * @param The return type of the visit operation. Use {@link Void} for + * operations with no return type. + */ +@SuppressWarnings("CheckReturnValue") +public class EsqlBaseParserBaseVisitor extends AbstractParseTreeVisitor implements EsqlBaseParserVisitor { + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSingleStatement(EsqlBaseParser.SingleStatementContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSourceCommand(EsqlBaseParser.SourceCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitWhereCommand(EsqlBaseParser.WhereCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitLogicalNot(EsqlBaseParser.LogicalNotContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitIsNull(EsqlBaseParser.IsNullContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitRegexExpression(EsqlBaseParser.RegexExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitLogicalIn(EsqlBaseParser.LogicalInContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitComparison(EsqlBaseParser.ComparisonContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitDereference(EsqlBaseParser.DereferenceContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitRowCommand(EsqlBaseParser.RowCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitFields(EsqlBaseParser.FieldsContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitField(EsqlBaseParser.FieldContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitFromCommand(EsqlBaseParser.FromCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitMetadata(EsqlBaseParser.MetadataContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitEvalCommand(EsqlBaseParser.EvalCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitStatsCommand(EsqlBaseParser.StatsCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitGrouping(EsqlBaseParser.GroupingContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitQualifiedName(EsqlBaseParser.QualifiedNameContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitIdentifier(EsqlBaseParser.IdentifierContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitNullLiteral(EsqlBaseParser.NullLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitInputParam(EsqlBaseParser.InputParamContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitStringLiteral(EsqlBaseParser.StringLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitLimitCommand(EsqlBaseParser.LimitCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSortCommand(EsqlBaseParser.SortCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitOrderExpression(EsqlBaseParser.OrderExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitKeepCommand(EsqlBaseParser.KeepCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitDropCommand(EsqlBaseParser.DropCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitRenameCommand(EsqlBaseParser.RenameCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitRenameClause(EsqlBaseParser.RenameClauseContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitDissectCommand(EsqlBaseParser.DissectCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitGrokCommand(EsqlBaseParser.GrokCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitCommandOptions(EsqlBaseParser.CommandOptionsContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitCommandOption(EsqlBaseParser.CommandOptionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitBooleanValue(EsqlBaseParser.BooleanValueContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitNumericValue(EsqlBaseParser.NumericValueContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitDecimalValue(EsqlBaseParser.DecimalValueContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitIntegerValue(EsqlBaseParser.IntegerValueContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitString(EsqlBaseParser.StringContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitExplainCommand(EsqlBaseParser.ExplainCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitShowInfo(EsqlBaseParser.ShowInfoContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

+ */ + @Override public T visitEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx) { return visitChildren(ctx); } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java new file mode 100644 index 0000000000000..04f0d6da3dbe4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java @@ -0,0 +1,748 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; +import org.antlr.v4.runtime.tree.ParseTreeListener; + +/** + * This interface defines a complete listener for a parse tree produced by + * {@link EsqlBaseParser}. + */ +public interface EsqlBaseParserListener extends ParseTreeListener { + /** + * Enter a parse tree produced by {@link EsqlBaseParser#singleStatement}. + * @param ctx the parse tree + */ + void enterSingleStatement(EsqlBaseParser.SingleStatementContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#singleStatement}. + * @param ctx the parse tree + */ + void exitSingleStatement(EsqlBaseParser.SingleStatementContext ctx); + /** + * Enter a parse tree produced by the {@code compositeQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + */ + void enterCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx); + /** + * Exit a parse tree produced by the {@code compositeQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + */ + void exitCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx); + /** + * Enter a parse tree produced by the {@code singleCommandQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + */ + void enterSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx); + /** + * Exit a parse tree produced by the {@code singleCommandQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + */ + void exitSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#sourceCommand}. + * @param ctx the parse tree + */ + void enterSourceCommand(EsqlBaseParser.SourceCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#sourceCommand}. + * @param ctx the parse tree + */ + void exitSourceCommand(EsqlBaseParser.SourceCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#processingCommand}. + * @param ctx the parse tree + */ + void enterProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#processingCommand}. + * @param ctx the parse tree + */ + void exitProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#whereCommand}. + * @param ctx the parse tree + */ + void enterWhereCommand(EsqlBaseParser.WhereCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#whereCommand}. + * @param ctx the parse tree + */ + void exitWhereCommand(EsqlBaseParser.WhereCommandContext ctx); + /** + * Enter a parse tree produced by the {@code logicalNot} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterLogicalNot(EsqlBaseParser.LogicalNotContext ctx); + /** + * Exit a parse tree produced by the {@code logicalNot} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitLogicalNot(EsqlBaseParser.LogicalNotContext ctx); + /** + * Enter a parse tree produced by the {@code booleanDefault} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx); + /** + * Exit a parse tree produced by the {@code booleanDefault} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx); + /** + * Enter a parse tree produced by the {@code isNull} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterIsNull(EsqlBaseParser.IsNullContext ctx); + /** + * Exit a parse tree produced by the {@code isNull} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitIsNull(EsqlBaseParser.IsNullContext ctx); + /** + * Enter a parse tree produced by the {@code regexExpression} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterRegexExpression(EsqlBaseParser.RegexExpressionContext ctx); + /** + * Exit a parse tree produced by the {@code regexExpression} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitRegexExpression(EsqlBaseParser.RegexExpressionContext ctx); + /** + * Enter a parse tree produced by the {@code logicalIn} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterLogicalIn(EsqlBaseParser.LogicalInContext ctx); + /** + * Exit a parse tree produced by the {@code logicalIn} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitLogicalIn(EsqlBaseParser.LogicalInContext ctx); + /** + * Enter a parse tree produced by the {@code logicalBinary} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void enterLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx); + /** + * Exit a parse tree produced by the {@code logicalBinary} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + */ + void exitLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#regexBooleanExpression}. + * @param ctx the parse tree + */ + void enterRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#regexBooleanExpression}. + * @param ctx the parse tree + */ + void exitRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx); + /** + * Enter a parse tree produced by the {@code valueExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + */ + void enterValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx); + /** + * Exit a parse tree produced by the {@code valueExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + */ + void exitValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx); + /** + * Enter a parse tree produced by the {@code comparison} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + */ + void enterComparison(EsqlBaseParser.ComparisonContext ctx); + /** + * Exit a parse tree produced by the {@code comparison} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + */ + void exitComparison(EsqlBaseParser.ComparisonContext ctx); + /** + * Enter a parse tree produced by the {@code operatorExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void enterOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx); + /** + * Exit a parse tree produced by the {@code operatorExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void exitOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx); + /** + * Enter a parse tree produced by the {@code arithmeticBinary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void enterArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx); + /** + * Exit a parse tree produced by the {@code arithmeticBinary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void exitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx); + /** + * Enter a parse tree produced by the {@code arithmeticUnary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void enterArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx); + /** + * Exit a parse tree produced by the {@code arithmeticUnary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + */ + void exitArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx); + /** + * Enter a parse tree produced by the {@code constantDefault} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void enterConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx); + /** + * Exit a parse tree produced by the {@code constantDefault} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void exitConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx); + /** + * Enter a parse tree produced by the {@code dereference} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void enterDereference(EsqlBaseParser.DereferenceContext ctx); + /** + * Exit a parse tree produced by the {@code dereference} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void exitDereference(EsqlBaseParser.DereferenceContext ctx); + /** + * Enter a parse tree produced by the {@code parenthesizedExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void enterParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx); + /** + * Exit a parse tree produced by the {@code parenthesizedExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void exitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx); + /** + * Enter a parse tree produced by the {@code functionExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void enterFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx); + /** + * Exit a parse tree produced by the {@code functionExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + */ + void exitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#rowCommand}. + * @param ctx the parse tree + */ + void enterRowCommand(EsqlBaseParser.RowCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#rowCommand}. + * @param ctx the parse tree + */ + void exitRowCommand(EsqlBaseParser.RowCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#fields}. + * @param ctx the parse tree + */ + void enterFields(EsqlBaseParser.FieldsContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#fields}. + * @param ctx the parse tree + */ + void exitFields(EsqlBaseParser.FieldsContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#field}. + * @param ctx the parse tree + */ + void enterField(EsqlBaseParser.FieldContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#field}. + * @param ctx the parse tree + */ + void exitField(EsqlBaseParser.FieldContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#fromCommand}. + * @param ctx the parse tree + */ + void enterFromCommand(EsqlBaseParser.FromCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#fromCommand}. + * @param ctx the parse tree + */ + void exitFromCommand(EsqlBaseParser.FromCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#metadata}. + * @param ctx the parse tree + */ + void enterMetadata(EsqlBaseParser.MetadataContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#metadata}. + * @param ctx the parse tree + */ + void exitMetadata(EsqlBaseParser.MetadataContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#evalCommand}. + * @param ctx the parse tree + */ + void enterEvalCommand(EsqlBaseParser.EvalCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#evalCommand}. + * @param ctx the parse tree + */ + void exitEvalCommand(EsqlBaseParser.EvalCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#statsCommand}. + * @param ctx the parse tree + */ + void enterStatsCommand(EsqlBaseParser.StatsCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#statsCommand}. + * @param ctx the parse tree + */ + void exitStatsCommand(EsqlBaseParser.StatsCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#inlinestatsCommand}. + * @param ctx the parse tree + */ + void enterInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#inlinestatsCommand}. + * @param ctx the parse tree + */ + void exitInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#grouping}. + * @param ctx the parse tree + */ + void enterGrouping(EsqlBaseParser.GroupingContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#grouping}. + * @param ctx the parse tree + */ + void exitGrouping(EsqlBaseParser.GroupingContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#sourceIdentifier}. + * @param ctx the parse tree + */ + void enterSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#sourceIdentifier}. + * @param ctx the parse tree + */ + void exitSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#qualifiedName}. + * @param ctx the parse tree + */ + void enterQualifiedName(EsqlBaseParser.QualifiedNameContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#qualifiedName}. + * @param ctx the parse tree + */ + void exitQualifiedName(EsqlBaseParser.QualifiedNameContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#identifier}. + * @param ctx the parse tree + */ + void enterIdentifier(EsqlBaseParser.IdentifierContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#identifier}. + * @param ctx the parse tree + */ + void exitIdentifier(EsqlBaseParser.IdentifierContext ctx); + /** + * Enter a parse tree produced by the {@code nullLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterNullLiteral(EsqlBaseParser.NullLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code nullLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitNullLiteral(EsqlBaseParser.NullLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code qualifiedIntegerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code qualifiedIntegerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code decimalLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code decimalLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code integerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code integerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code booleanLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code booleanLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code inputParam} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterInputParam(EsqlBaseParser.InputParamContext ctx); + /** + * Exit a parse tree produced by the {@code inputParam} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitInputParam(EsqlBaseParser.InputParamContext ctx); + /** + * Enter a parse tree produced by the {@code stringLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterStringLiteral(EsqlBaseParser.StringLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code stringLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitStringLiteral(EsqlBaseParser.StringLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code numericArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code numericArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code booleanArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code booleanArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx); + /** + * Enter a parse tree produced by the {@code stringArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void enterStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx); + /** + * Exit a parse tree produced by the {@code stringArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + */ + void exitStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#limitCommand}. + * @param ctx the parse tree + */ + void enterLimitCommand(EsqlBaseParser.LimitCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#limitCommand}. + * @param ctx the parse tree + */ + void exitLimitCommand(EsqlBaseParser.LimitCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#sortCommand}. + * @param ctx the parse tree + */ + void enterSortCommand(EsqlBaseParser.SortCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#sortCommand}. + * @param ctx the parse tree + */ + void exitSortCommand(EsqlBaseParser.SortCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#orderExpression}. + * @param ctx the parse tree + */ + void enterOrderExpression(EsqlBaseParser.OrderExpressionContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#orderExpression}. + * @param ctx the parse tree + */ + void exitOrderExpression(EsqlBaseParser.OrderExpressionContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#keepCommand}. + * @param ctx the parse tree + */ + void enterKeepCommand(EsqlBaseParser.KeepCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#keepCommand}. + * @param ctx the parse tree + */ + void exitKeepCommand(EsqlBaseParser.KeepCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#dropCommand}. + * @param ctx the parse tree + */ + void enterDropCommand(EsqlBaseParser.DropCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#dropCommand}. + * @param ctx the parse tree + */ + void exitDropCommand(EsqlBaseParser.DropCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#renameCommand}. + * @param ctx the parse tree + */ + void enterRenameCommand(EsqlBaseParser.RenameCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#renameCommand}. + * @param ctx the parse tree + */ + void exitRenameCommand(EsqlBaseParser.RenameCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#renameClause}. + * @param ctx the parse tree + */ + void enterRenameClause(EsqlBaseParser.RenameClauseContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#renameClause}. + * @param ctx the parse tree + */ + void exitRenameClause(EsqlBaseParser.RenameClauseContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#dissectCommand}. + * @param ctx the parse tree + */ + void enterDissectCommand(EsqlBaseParser.DissectCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#dissectCommand}. + * @param ctx the parse tree + */ + void exitDissectCommand(EsqlBaseParser.DissectCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#grokCommand}. + * @param ctx the parse tree + */ + void enterGrokCommand(EsqlBaseParser.GrokCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#grokCommand}. + * @param ctx the parse tree + */ + void exitGrokCommand(EsqlBaseParser.GrokCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#mvExpandCommand}. + * @param ctx the parse tree + */ + void enterMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#mvExpandCommand}. + * @param ctx the parse tree + */ + void exitMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#commandOptions}. + * @param ctx the parse tree + */ + void enterCommandOptions(EsqlBaseParser.CommandOptionsContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#commandOptions}. + * @param ctx the parse tree + */ + void exitCommandOptions(EsqlBaseParser.CommandOptionsContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#commandOption}. + * @param ctx the parse tree + */ + void enterCommandOption(EsqlBaseParser.CommandOptionContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#commandOption}. + * @param ctx the parse tree + */ + void exitCommandOption(EsqlBaseParser.CommandOptionContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#booleanValue}. + * @param ctx the parse tree + */ + void enterBooleanValue(EsqlBaseParser.BooleanValueContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#booleanValue}. + * @param ctx the parse tree + */ + void exitBooleanValue(EsqlBaseParser.BooleanValueContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#numericValue}. + * @param ctx the parse tree + */ + void enterNumericValue(EsqlBaseParser.NumericValueContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#numericValue}. + * @param ctx the parse tree + */ + void exitNumericValue(EsqlBaseParser.NumericValueContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#decimalValue}. + * @param ctx the parse tree + */ + void enterDecimalValue(EsqlBaseParser.DecimalValueContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#decimalValue}. + * @param ctx the parse tree + */ + void exitDecimalValue(EsqlBaseParser.DecimalValueContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#integerValue}. + * @param ctx the parse tree + */ + void enterIntegerValue(EsqlBaseParser.IntegerValueContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#integerValue}. + * @param ctx the parse tree + */ + void exitIntegerValue(EsqlBaseParser.IntegerValueContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#string}. + * @param ctx the parse tree + */ + void enterString(EsqlBaseParser.StringContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#string}. + * @param ctx the parse tree + */ + void exitString(EsqlBaseParser.StringContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#comparisonOperator}. + * @param ctx the parse tree + */ + void enterComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#comparisonOperator}. + * @param ctx the parse tree + */ + void exitComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#explainCommand}. + * @param ctx the parse tree + */ + void enterExplainCommand(EsqlBaseParser.ExplainCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#explainCommand}. + * @param ctx the parse tree + */ + void exitExplainCommand(EsqlBaseParser.ExplainCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#subqueryExpression}. + * @param ctx the parse tree + */ + void enterSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#subqueryExpression}. + * @param ctx the parse tree + */ + void exitSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx); + /** + * Enter a parse tree produced by the {@code showInfo} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + */ + void enterShowInfo(EsqlBaseParser.ShowInfoContext ctx); + /** + * Exit a parse tree produced by the {@code showInfo} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + */ + void exitShowInfo(EsqlBaseParser.ShowInfoContext ctx); + /** + * Enter a parse tree produced by the {@code showFunctions} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + */ + void enterShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx); + /** + * Exit a parse tree produced by the {@code showFunctions} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + */ + void exitShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#enrichCommand}. + * @param ctx the parse tree + */ + void enterEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#enrichCommand}. + * @param ctx the parse tree + */ + void exitEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx); + /** + * Enter a parse tree produced by {@link EsqlBaseParser#enrichWithClause}. + * @param ctx the parse tree + */ + void enterEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx); + /** + * Exit a parse tree produced by {@link EsqlBaseParser#enrichWithClause}. + * @param ctx the parse tree + */ + void exitEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java new file mode 100644 index 0000000000000..681de2590d575 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java @@ -0,0 +1,450 @@ +// ANTLR GENERATED CODE: DO NOT EDIT +package org.elasticsearch.xpack.esql.parser; +import org.antlr.v4.runtime.tree.ParseTreeVisitor; + +/** + * This interface defines a complete generic visitor for a parse tree produced + * by {@link EsqlBaseParser}. + * + * @param The return type of the visit operation. Use {@link Void} for + * operations with no return type. + */ +public interface EsqlBaseParserVisitor extends ParseTreeVisitor { + /** + * Visit a parse tree produced by {@link EsqlBaseParser#singleStatement}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSingleStatement(EsqlBaseParser.SingleStatementContext ctx); + /** + * Visit a parse tree produced by the {@code compositeQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx); + /** + * Visit a parse tree produced by the {@code singleCommandQuery} + * labeled alternative in {@link EsqlBaseParser#query}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSingleCommandQuery(EsqlBaseParser.SingleCommandQueryContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#sourceCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSourceCommand(EsqlBaseParser.SourceCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#processingCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitProcessingCommand(EsqlBaseParser.ProcessingCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#whereCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitWhereCommand(EsqlBaseParser.WhereCommandContext ctx); + /** + * Visit a parse tree produced by the {@code logicalNot} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitLogicalNot(EsqlBaseParser.LogicalNotContext ctx); + /** + * Visit a parse tree produced by the {@code booleanDefault} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitBooleanDefault(EsqlBaseParser.BooleanDefaultContext ctx); + /** + * Visit a parse tree produced by the {@code isNull} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitIsNull(EsqlBaseParser.IsNullContext ctx); + /** + * Visit a parse tree produced by the {@code regexExpression} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitRegexExpression(EsqlBaseParser.RegexExpressionContext ctx); + /** + * Visit a parse tree produced by the {@code logicalIn} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitLogicalIn(EsqlBaseParser.LogicalInContext ctx); + /** + * Visit a parse tree produced by the {@code logicalBinary} + * labeled alternative in {@link EsqlBaseParser#booleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#regexBooleanExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx); + /** + * Visit a parse tree produced by the {@code valueExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitValueExpressionDefault(EsqlBaseParser.ValueExpressionDefaultContext ctx); + /** + * Visit a parse tree produced by the {@code comparison} + * labeled alternative in {@link EsqlBaseParser#valueExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitComparison(EsqlBaseParser.ComparisonContext ctx); + /** + * Visit a parse tree produced by the {@code operatorExpressionDefault} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx); + /** + * Visit a parse tree produced by the {@code arithmeticBinary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx); + /** + * Visit a parse tree produced by the {@code arithmeticUnary} + * labeled alternative in {@link EsqlBaseParser#operatorExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx); + /** + * Visit a parse tree produced by the {@code constantDefault} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitConstantDefault(EsqlBaseParser.ConstantDefaultContext ctx); + /** + * Visit a parse tree produced by the {@code dereference} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitDereference(EsqlBaseParser.DereferenceContext ctx); + /** + * Visit a parse tree produced by the {@code parenthesizedExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx); + /** + * Visit a parse tree produced by the {@code functionExpression} + * labeled alternative in {@link EsqlBaseParser#primaryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#rowCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitRowCommand(EsqlBaseParser.RowCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#fields}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitFields(EsqlBaseParser.FieldsContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#field}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitField(EsqlBaseParser.FieldContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#fromCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitFromCommand(EsqlBaseParser.FromCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#metadata}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitMetadata(EsqlBaseParser.MetadataContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#evalCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitEvalCommand(EsqlBaseParser.EvalCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#statsCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitStatsCommand(EsqlBaseParser.StatsCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#inlinestatsCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#grouping}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitGrouping(EsqlBaseParser.GroupingContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#sourceIdentifier}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#qualifiedName}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitQualifiedName(EsqlBaseParser.QualifiedNameContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#identifier}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitIdentifier(EsqlBaseParser.IdentifierContext ctx); + /** + * Visit a parse tree produced by the {@code nullLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitNullLiteral(EsqlBaseParser.NullLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code qualifiedIntegerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code decimalLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitDecimalLiteral(EsqlBaseParser.DecimalLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code integerLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitIntegerLiteral(EsqlBaseParser.IntegerLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code booleanLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitBooleanLiteral(EsqlBaseParser.BooleanLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code inputParam} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitInputParam(EsqlBaseParser.InputParamContext ctx); + /** + * Visit a parse tree produced by the {@code stringLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitStringLiteral(EsqlBaseParser.StringLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code numericArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code booleanArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx); + /** + * Visit a parse tree produced by the {@code stringArrayLiteral} + * labeled alternative in {@link EsqlBaseParser#constant}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#limitCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitLimitCommand(EsqlBaseParser.LimitCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#sortCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSortCommand(EsqlBaseParser.SortCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#orderExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitOrderExpression(EsqlBaseParser.OrderExpressionContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#keepCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitKeepCommand(EsqlBaseParser.KeepCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#dropCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitDropCommand(EsqlBaseParser.DropCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#renameCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitRenameCommand(EsqlBaseParser.RenameCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#renameClause}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitRenameClause(EsqlBaseParser.RenameClauseContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#dissectCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitDissectCommand(EsqlBaseParser.DissectCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#grokCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitGrokCommand(EsqlBaseParser.GrokCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#mvExpandCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#commandOptions}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitCommandOptions(EsqlBaseParser.CommandOptionsContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#commandOption}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitCommandOption(EsqlBaseParser.CommandOptionContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#booleanValue}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitBooleanValue(EsqlBaseParser.BooleanValueContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#numericValue}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitNumericValue(EsqlBaseParser.NumericValueContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#decimalValue}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitDecimalValue(EsqlBaseParser.DecimalValueContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#integerValue}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitIntegerValue(EsqlBaseParser.IntegerValueContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#string}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitString(EsqlBaseParser.StringContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#comparisonOperator}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitComparisonOperator(EsqlBaseParser.ComparisonOperatorContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#explainCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitExplainCommand(EsqlBaseParser.ExplainCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#subqueryExpression}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitSubqueryExpression(EsqlBaseParser.SubqueryExpressionContext ctx); + /** + * Visit a parse tree produced by the {@code showInfo} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitShowInfo(EsqlBaseParser.ShowInfoContext ctx); + /** + * Visit a parse tree produced by the {@code showFunctions} + * labeled alternative in {@link EsqlBaseParser#showCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#enrichCommand}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx); + /** + * Visit a parse tree produced by {@link EsqlBaseParser#enrichWithClause}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java new file mode 100644 index 0000000000000..0f002ab8ef70f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java @@ -0,0 +1,176 @@ +/* + * 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.parser; + +import org.antlr.v4.runtime.BaseErrorListener; +import org.antlr.v4.runtime.CharStream; +import org.antlr.v4.runtime.CharStreams; +import org.antlr.v4.runtime.CommonTokenStream; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.RecognitionException; +import org.antlr.v4.runtime.Recognizer; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.TokenFactory; +import org.antlr.v4.runtime.TokenSource; +import org.antlr.v4.runtime.atn.PredictionMode; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.xpack.ql.parser.CaseChangingCharStream; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; + +import static org.elasticsearch.xpack.ql.parser.ParserUtils.source; + +public class EsqlParser { + + private static final Logger log = LogManager.getLogger(EsqlParser.class); + + public LogicalPlan createStatement(String query) { + return createStatement(query, List.of()); + } + + public LogicalPlan createStatement(String query, List params) { + if (log.isDebugEnabled()) { + log.debug("Parsing as statement: {}", query); + } + return invokeParser(query, params, EsqlBaseParser::singleStatement, AstBuilder::plan); + } + + private T invokeParser( + String query, + List params, + Function parseFunction, + BiFunction result + ) { + try { + EsqlBaseLexer lexer = new EsqlBaseLexer(new CaseChangingCharStream(CharStreams.fromString(query), false)); + + lexer.removeErrorListeners(); + lexer.addErrorListener(ERROR_LISTENER); + + Map paramTokens = new HashMap<>(); + TokenSource tokenSource = new ParametrizedTokenSource(lexer, paramTokens, params); + + CommonTokenStream tokenStream = new CommonTokenStream(tokenSource); + EsqlBaseParser parser = new EsqlBaseParser(tokenStream); + + parser.addParseListener(new PostProcessor()); + + parser.removeErrorListeners(); + parser.addErrorListener(ERROR_LISTENER); + + parser.getInterpreter().setPredictionMode(PredictionMode.SLL); + + ParserRuleContext tree = parseFunction.apply(parser); + + if (log.isDebugEnabled()) { + log.debug("Parse tree: {}", tree.toStringTree()); + } + + return result.apply(new AstBuilder(paramTokens), tree); + } catch (StackOverflowError e) { + throw new ParsingException("ESQL statement is too large, causing stack overflow when generating the parsing tree: [{}]", query); + } + } + + private class PostProcessor extends EsqlBaseParserBaseListener { + @Override + public void exitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) { + // TODO remove this at some point + EsqlBaseParser.IdentifierContext identifier = ctx.identifier(); + if (identifier.getText().equalsIgnoreCase("is_null")) { + throw new ParsingException( + source(ctx), + "is_null function is not supported anymore, please use 'is null'/'is not null' predicates instead" + ); + } + } + } + + private static final BaseErrorListener ERROR_LISTENER = new BaseErrorListener() { + @Override + public void syntaxError( + Recognizer recognizer, + Object offendingSymbol, + int line, + int charPositionInLine, + String message, + RecognitionException e + ) { + throw new ParsingException(message, e, line, charPositionInLine); + } + }; + + /** + * Finds all parameter tokens (?) and associates them with actual parameter values + *

+ * Parameters are positional and we know where parameters occurred in the original stream in order to associate them + * with actual values. + */ + private static class ParametrizedTokenSource implements TokenSource { + + private TokenSource delegate; + private Map paramTokens; + private int param; + private List params; + + ParametrizedTokenSource(TokenSource delegate, Map paramTokens, List params) { + this.delegate = delegate; + this.paramTokens = paramTokens; + this.params = params; + param = 0; + } + + @Override + public Token nextToken() { + Token token = delegate.nextToken(); + if (token.getType() == EsqlBaseLexer.PARAM) { + if (param >= params.size()) { + throw new ParsingException("Not enough actual parameters {} ", params.size()); + } + paramTokens.put(token, params.get(param)); + param++; + } + return token; + } + + @Override + public int getLine() { + return delegate.getLine(); + } + + @Override + public int getCharPositionInLine() { + return delegate.getCharPositionInLine(); + } + + @Override + public CharStream getInputStream() { + return delegate.getInputStream(); + } + + @Override + public String getSourceName() { + return delegate.getSourceName(); + } + + @Override + public void setTokenFactory(TokenFactory factory) { + delegate.setTokenFactory(factory); + } + + @Override + public TokenFactory getTokenFactory() { + return delegate.getTokenFactory(); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java new file mode 100644 index 0000000000000..efb27707a7c7a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java @@ -0,0 +1,467 @@ +/* + * 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.parser; + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.tree.ParseTree; +import org.antlr.v4.runtime.tree.TerminalNode; +import org.elasticsearch.common.Strings; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedStar; +import org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.expression.predicate.logical.And; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mod; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLikePattern; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardPattern; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypeConverter; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.DateUtils; +import org.elasticsearch.xpack.ql.util.StringUtils; + +import java.math.BigInteger; +import java.time.Duration; +import java.time.Period; +import java.time.ZoneId; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.BiFunction; + +import static java.util.Collections.emptyList; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.DATE_PERIOD; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.TIME_DURATION; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.source; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.typedParsing; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.visitList; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToInt; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.safeToLong; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsNumber; +import static org.elasticsearch.xpack.ql.util.StringUtils.WILDCARD; + +abstract class ExpressionBuilder extends IdentifierBuilder { + + private final Map params; + + ExpressionBuilder(Map params) { + this.params = params; + } + + protected Expression expression(ParseTree ctx) { + return typedParsing(this, ctx, Expression.class); + } + + protected List expressions(List contexts) { + return visitList(this, contexts, Expression.class); + } + + @Override + public Literal visitBooleanValue(EsqlBaseParser.BooleanValueContext ctx) { + Source source = source(ctx); + return new Literal(source, ctx.TRUE() != null, DataTypes.BOOLEAN); + } + + @Override + public Literal visitDecimalValue(EsqlBaseParser.DecimalValueContext ctx) { + Source source = source(ctx); + String text = ctx.getText(); + + try { + return new Literal(source, Double.valueOf(StringUtils.parseDouble(text)), DataTypes.DOUBLE); + } catch (QlIllegalArgumentException siae) { + throw new ParsingException(source, siae.getMessage()); + } + } + + @Override + public Literal visitIntegerValue(EsqlBaseParser.IntegerValueContext ctx) { + Source source = source(ctx); + String text = ctx.getText(); + Number number; + + try { + number = StringUtils.parseIntegral(text); + } catch (QlIllegalArgumentException siae) { + // if it's too large, then quietly try to parse as a float instead + try { + return new Literal(source, StringUtils.parseDouble(text), DataTypes.DOUBLE); + } catch (QlIllegalArgumentException ignored) {} + + throw new ParsingException(source, siae.getMessage()); + } + + Object val; + DataType type; + if (number instanceof BigInteger bi) { + val = asLongUnsigned(bi); + type = DataTypes.UNSIGNED_LONG; + } else if (number.intValue() == number.longValue()) { // try to downsize to int if possible (since that's the most common type) + val = number.intValue(); + type = DataTypes.INTEGER; + } else { + val = number.longValue(); + type = DataTypes.LONG; + } + return new Literal(source, val, type); + } + + @Override + public Object visitNumericArrayLiteral(EsqlBaseParser.NumericArrayLiteralContext ctx) { + Source source = source(ctx); + List numbers = visitList(this, ctx.numericValue(), Literal.class); + if (numbers.stream().anyMatch(l -> l.dataType() == DataTypes.DOUBLE)) { + return new Literal(source, mapNumbers(numbers, (no, dt) -> no.doubleValue()), DataTypes.DOUBLE); + } + if (numbers.stream().anyMatch(l -> l.dataType() == DataTypes.UNSIGNED_LONG)) { + return new Literal( + source, + mapNumbers( + numbers, + (no, dt) -> dt == DataTypes.UNSIGNED_LONG ? no.longValue() : asLongUnsigned(BigInteger.valueOf(no.longValue())) + ), + DataTypes.UNSIGNED_LONG + ); + } + if (numbers.stream().anyMatch(l -> l.dataType() == DataTypes.LONG)) { + return new Literal(source, mapNumbers(numbers, (no, dt) -> no.longValue()), DataTypes.LONG); + } + return new Literal(source, mapNumbers(numbers, (no, dt) -> no.intValue()), DataTypes.INTEGER); + } + + private List mapNumbers(List numbers, BiFunction map) { + return numbers.stream().map(l -> map.apply((Number) l.value(), l.dataType())).toList(); + } + + @Override + public Object visitBooleanArrayLiteral(EsqlBaseParser.BooleanArrayLiteralContext ctx) { + return visitArrayLiteral(ctx, ctx.booleanValue(), DataTypes.BOOLEAN); + } + + @Override + public Object visitStringArrayLiteral(EsqlBaseParser.StringArrayLiteralContext ctx) { + return visitArrayLiteral(ctx, ctx.string(), DataTypes.KEYWORD); + } + + private Object visitArrayLiteral(ParserRuleContext ctx, List contexts, DataType dataType) { + Source source = source(ctx); + List literals = visitList(this, contexts, Literal.class); + return new Literal(source, literals.stream().map(Literal::value).toList(), dataType); + } + + @Override + public Literal visitNullLiteral(EsqlBaseParser.NullLiteralContext ctx) { + Source source = source(ctx); + return new Literal(source, null, DataTypes.NULL); + } + + @Override + public Literal visitStringLiteral(EsqlBaseParser.StringLiteralContext ctx) { + return visitString(ctx.string()); + } + + @Override + public Literal visitString(EsqlBaseParser.StringContext ctx) { + Source source = source(ctx); + return new Literal(source, unquoteString(source), DataTypes.KEYWORD); + } + + @Override + public UnresolvedAttribute visitQualifiedName(EsqlBaseParser.QualifiedNameContext ctx) { + if (ctx == null) { + return null; + } + + return new UnresolvedAttribute( + source(ctx), + Strings.collectionToDelimitedString(visitList(this, ctx.identifier(), String.class), ".") + ); + } + + @Override + public Object visitQualifiedIntegerLiteral(EsqlBaseParser.QualifiedIntegerLiteralContext ctx) { + Source source = source(ctx); + Literal intLit = typedParsing(this, ctx.integerValue(), Literal.class); + Number value = (Number) intLit.value(); + if (intLit.dataType() == DataTypes.UNSIGNED_LONG) { + value = unsignedLongAsNumber(value.longValue()); + } + String qualifier = ctx.UNQUOTED_IDENTIFIER().getText().toLowerCase(Locale.ROOT); + + try { + Object quantity = switch (qualifier) { + case "millisecond", "milliseconds" -> Duration.ofMillis(safeToLong(value)); + case "second", "seconds" -> Duration.ofSeconds(safeToLong(value)); + case "minute", "minutes" -> Duration.ofMinutes(safeToLong(value)); + case "hour", "hours" -> Duration.ofHours(safeToLong(value)); + + case "day", "days" -> Period.ofDays(safeToInt(safeToLong(value))); + case "week", "weeks" -> Period.ofWeeks(safeToInt(safeToLong(value))); + case "month", "months" -> Period.ofMonths(safeToInt(safeToLong(value))); + case "year", "years" -> Period.ofYears(safeToInt(safeToLong(value))); + + default -> throw new ParsingException(source, "Unexpected numeric qualifier '{}'", qualifier); + }; + return new Literal(source, quantity, quantity instanceof Duration ? TIME_DURATION : DATE_PERIOD); + } catch (QlIllegalArgumentException | ArithmeticException e) { + // the range varies by unit: Duration#ofMinutes(), #ofHours() will Math#multiplyExact() to reduce the unit to seconds; + // and same for Period#ofWeeks() + throw new ParsingException(source, "Number [{}] outside of [{}] range", ctx.integerValue().getText(), qualifier); + } + } + + @Override + public Expression visitArithmeticUnary(EsqlBaseParser.ArithmeticUnaryContext ctx) { + Expression expr = expression(ctx.operatorExpression()); + Source source = source(ctx); + int type = ctx.operator.getType(); + + // TODO we could handle this a bit better (like ES SQL does it) so that -(-(-123)) results in the -123 the Literal + return type == EsqlBaseParser.MINUS ? new Neg(source, expr) : expr; + } + + @Override + public Expression visitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext ctx) { + Expression left = expression(ctx.left); + Expression right = expression(ctx.right); + Source source = source(ctx); + int type = ctx.operator.getType(); + + return switch (type) { + case EsqlBaseParser.ASTERISK -> new Mul(source, left, right); + case EsqlBaseParser.SLASH -> new Div(source, left, right); + case EsqlBaseParser.PERCENT -> new Mod(source, left, right); + case EsqlBaseParser.PLUS -> new Add(source, left, right); + case EsqlBaseParser.MINUS -> new Sub(source, left, right); + default -> throw new ParsingException(source, "Unknown arithmetic operator {}", source.text()); + }; + } + + @Override + public Expression visitComparison(EsqlBaseParser.ComparisonContext ctx) { + Expression left = expression(ctx.left); + Expression right = expression(ctx.right); + TerminalNode op = (TerminalNode) ctx.comparisonOperator().getChild(0); + + Source source = source(ctx); + ZoneId zoneId = DateUtils.UTC; + + return switch (op.getSymbol().getType()) { + case EsqlBaseParser.EQ -> new Equals(source, left, right, zoneId); + case EsqlBaseParser.NEQ -> new Not(source, new Equals(source, left, right, zoneId)); + case EsqlBaseParser.LT -> new LessThan(source, left, right, zoneId); + case EsqlBaseParser.LTE -> new LessThanOrEqual(source, left, right, zoneId); + case EsqlBaseParser.GT -> new GreaterThan(source, left, right, zoneId); + case EsqlBaseParser.GTE -> new GreaterThanOrEqual(source, left, right, zoneId); + default -> throw new ParsingException(source, "Unknown comparison operator {}", source.text()); + }; + } + + @Override + public Not visitLogicalNot(EsqlBaseParser.LogicalNotContext ctx) { + return new Not(source(ctx), expression(ctx.booleanExpression())); + } + + @Override + public Expression visitParenthesizedExpression(EsqlBaseParser.ParenthesizedExpressionContext ctx) { + return expression(ctx.booleanExpression()); + } + + @Override + public Expression visitOperatorExpressionDefault(EsqlBaseParser.OperatorExpressionDefaultContext ctx) { + return expression(ctx.primaryExpression()); + } + + @Override + public UnresolvedAttribute visitDereference(EsqlBaseParser.DereferenceContext ctx) { + return visitQualifiedName(ctx.qualifiedName()); + } + + @Override + public Expression visitFunctionExpression(EsqlBaseParser.FunctionExpressionContext ctx) { + return new UnresolvedFunction( + source(ctx), + visitIdentifier(ctx.identifier()), + FunctionResolutionStrategy.DEFAULT, + ctx.booleanExpression().stream().map(this::expression).toList() + ); + } + + @Override + public Expression visitLogicalBinary(EsqlBaseParser.LogicalBinaryContext ctx) { + int type = ctx.operator.getType(); + Source source = source(ctx); + Expression left = expression(ctx.left); + Expression right = expression(ctx.right); + + return type == EsqlBaseParser.AND ? new And(source, left, right) : new Or(source, left, right); + } + + @Override + public Expression visitLogicalIn(EsqlBaseParser.LogicalInContext ctx) { + List expressions = ctx.valueExpression().stream().map(this::expression).toList(); + Source source = source(ctx); + Expression e = expressions.size() == 2 + ? new Equals(source, expressions.get(0), expressions.get(1)) + : new In(source, expressions.get(0), expressions.subList(1, expressions.size())); + return ctx.NOT() == null ? e : new Not(source, e); + } + + @Override + public Object visitIsNull(EsqlBaseParser.IsNullContext ctx) { + Expression exp = expression(ctx.valueExpression()); + Source source = source(ctx.valueExpression(), ctx); + return ctx.NOT() != null ? new IsNotNull(source, exp) : new IsNull(source, exp); + } + + @Override + public Expression visitRegexBooleanExpression(EsqlBaseParser.RegexBooleanExpressionContext ctx) { + int type = ctx.kind.getType(); + Source source = source(ctx); + Expression left = expression(ctx.valueExpression()); + Literal pattern = visitString(ctx.pattern); + RegexMatch result = switch (type) { + case EsqlBaseParser.LIKE -> new WildcardLike(source, left, new WildcardPattern(pattern.fold().toString())); + case EsqlBaseParser.RLIKE -> new RLike(source, left, new RLikePattern(pattern.fold().toString())); + default -> throw new ParsingException("Invalid predicate type for [{}]", source.text()); + }; + return ctx.NOT() == null ? result : new Not(source, result); + } + + @Override + public Order visitOrderExpression(EsqlBaseParser.OrderExpressionContext ctx) { + return new Order( + source(ctx), + expression(ctx.booleanExpression()), + ctx.DESC() != null ? Order.OrderDirection.DESC : Order.OrderDirection.ASC, + (ctx.NULLS() != null && ctx.LAST() != null || ctx.NULLS() == null && ctx.DESC() == null) + ? Order.NullsPosition.LAST + : Order.NullsPosition.FIRST + ); + } + + public NamedExpression visitProjectExpression(EsqlBaseParser.SourceIdentifierContext ctx) { + Source src = source(ctx); + String identifier = visitSourceIdentifier(ctx); + return identifier.equals(WILDCARD) ? new UnresolvedStar(src, null) : new UnresolvedAttribute(src, identifier); + } + + @Override + public Alias visitRenameClause(EsqlBaseParser.RenameClauseContext ctx) { + Source src = source(ctx); + String newName = visitSourceIdentifier(ctx.newName); + String oldName = visitSourceIdentifier(ctx.oldName); + if (newName.contains(WILDCARD) || oldName.contains(WILDCARD)) { + throw new ParsingException(src, "Using wildcards (*) in renaming projections is not allowed [{}]", src.text()); + } + + return new Alias(src, newName, new UnresolvedAttribute(source(ctx.oldName), oldName)); + } + + @Override + public Alias visitField(EsqlBaseParser.FieldContext ctx) { + UnresolvedAttribute id = visitQualifiedName(ctx.qualifiedName()); + Expression value = expression(ctx.booleanExpression()); + String name = id == null ? ctx.getText() : id.qualifiedName(); + return new Alias(source(ctx), name, value); + } + + @Override + public List visitGrouping(EsqlBaseParser.GroupingContext ctx) { + return ctx != null ? visitList(this, ctx.qualifiedName(), NamedExpression.class) : emptyList(); + } + + @Override + public Object visitInputParam(EsqlBaseParser.InputParamContext ctx) { + TypedParamValue param = param(ctx.PARAM()); + DataType dataType = EsqlDataTypes.fromTypeName(param.type); + Source source = source(ctx); + if (dataType == null) { + throw new ParsingException(source, "Invalid parameter data type [{}]", param.type); + } + if (param.value == null) { + // no conversion is required for null values + return new Literal(source, null, dataType); + } + final DataType sourceType; + try { + sourceType = DataTypes.fromJava(param.value); + } catch (QlIllegalArgumentException ex) { + throw new ParsingException( + ex, + source, + "Unexpected actual parameter type [{}] for type [{}]", + param.value.getClass().getName(), + param.type + ); + } + if (sourceType == dataType) { + // no conversion is required if the value is already have correct type + return new Literal(source, param.value, dataType); + } + // otherwise we need to make sure that xcontent-serialized value is converted to the correct type + try { + + if (DataTypeConverter.canConvert(sourceType, dataType) == false) { + throw new ParsingException( + source, + "Cannot cast value [{}] of type [{}] to parameter type [{}]", + param.value, + sourceType, + dataType + ); + } + return new Literal(source, DataTypeConverter.converterFor(sourceType, dataType).convert(param.value), dataType); + } catch (QlIllegalArgumentException ex) { + throw new ParsingException(ex, source, "Unexpected actual parameter type [{}] for type [{}]", sourceType, param.type); + } + } + + private TypedParamValue param(TerminalNode node) { + if (node == null) { + return null; + } + + Token token = node.getSymbol(); + + if (params.containsKey(token) == false) { + throw new ParsingException(source(node), "Unexpected parameter"); + } + + return params.get(token); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java new file mode 100644 index 0000000000000..296206b1079b2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.parser; + +import org.antlr.v4.runtime.tree.TerminalNode; +import org.elasticsearch.common.Strings; + +import java.util.List; + +import static org.elasticsearch.xpack.ql.parser.ParserUtils.visitList; + +abstract class IdentifierBuilder extends AbstractBuilder { + + @Override + public String visitIdentifier(EsqlBaseParser.IdentifierContext ctx) { + return unquoteIdentifier(ctx.QUOTED_IDENTIFIER(), ctx.UNQUOTED_IDENTIFIER()); + } + + @Override + public String visitSourceIdentifier(EsqlBaseParser.SourceIdentifierContext ctx) { + return unquoteIdentifier(ctx.SRC_QUOTED_IDENTIFIER(), ctx.SRC_UNQUOTED_IDENTIFIER()); + } + + private static String unquoteIdentifier(TerminalNode quotedNode, TerminalNode unquotedNode) { + String result; + if (quotedNode != null) { + String identifier = quotedNode.getText(); + result = identifier.substring(1, identifier.length() - 1); + } else { + result = unquotedNode.getText(); + } + return result; + } + + public String visitSourceIdentifiers(List ctx) { + return Strings.collectionToDelimitedString(visitList(this, ctx, String.class), ","); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java new file mode 100644 index 0000000000000..2c3c12eb34257 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java @@ -0,0 +1,353 @@ +/* + * 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.parser; + +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.Token; +import org.antlr.v4.runtime.tree.ParseTree; +import org.elasticsearch.dissect.DissectException; +import org.elasticsearch.dissect.DissectParser; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Drop; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.EsqlUnresolvedRelation; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Explain; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.InlineStats; +import org.elasticsearch.xpack.esql.plan.logical.Keep; +import org.elasticsearch.xpack.esql.plan.logical.MvExpand; +import org.elasticsearch.xpack.esql.plan.logical.Rename; +import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.esql.plan.logical.show.ShowFunctions; +import org.elasticsearch.xpack.esql.plan.logical.show.ShowInfo; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.EmptyAttribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedStar; +import org.elasticsearch.xpack.ql.parser.ParserUtils; +import org.elasticsearch.xpack.ql.plan.TableIdentifier; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +import static org.elasticsearch.common.logging.HeaderWarning.addWarning; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.source; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.typedParsing; +import static org.elasticsearch.xpack.ql.parser.ParserUtils.visitList; +import static org.elasticsearch.xpack.ql.util.StringUtils.WILDCARD; + +public class LogicalPlanBuilder extends ExpressionBuilder { + + public LogicalPlanBuilder(Map params) { + super(params); + } + + protected LogicalPlan plan(ParseTree ctx) { + return ParserUtils.typedParsing(this, ctx, LogicalPlan.class); + } + + protected List plans(List ctxs) { + return ParserUtils.visitList(this, ctxs, LogicalPlan.class); + } + + @Override + public LogicalPlan visitSingleStatement(EsqlBaseParser.SingleStatementContext ctx) { + return plan(ctx.query()); + } + + @Override + public LogicalPlan visitCompositeQuery(EsqlBaseParser.CompositeQueryContext ctx) { + LogicalPlan input = plan(ctx.query()); + PlanFactory makePlan = typedParsing(this, ctx.processingCommand(), PlanFactory.class); + return makePlan.apply(input); + } + + @Override + public PlanFactory visitEvalCommand(EsqlBaseParser.EvalCommandContext ctx) { + return p -> new Eval(source(ctx), p, visitFields(ctx.fields())); + } + + @Override + public PlanFactory visitGrokCommand(EsqlBaseParser.GrokCommandContext ctx) { + return p -> { + String pattern = visitString(ctx.string()).fold().toString(); + Grok result = new Grok(source(ctx), p, expression(ctx.primaryExpression()), Grok.pattern(source(ctx), pattern)); + return result; + }; + } + + @Override + public PlanFactory visitDissectCommand(EsqlBaseParser.DissectCommandContext ctx) { + return p -> { + String pattern = visitString(ctx.string()).fold().toString(); + Map options = visitCommandOptions(ctx.commandOptions()); + String appendSeparator = ""; + for (Map.Entry item : options.entrySet()) { + if (item.getKey().equals("append_separator") == false) { + throw new ParsingException(source(ctx), "Invalid option for dissect: [{}]", item.getKey()); + } + if (item.getValue() instanceof String == false) { + throw new ParsingException( + source(ctx), + "Invalid value for dissect append_separator: expected a string, but was [{}]", + item.getValue() + ); + } + appendSeparator = (String) item.getValue(); + } + Source src = source(ctx); + + try { + DissectParser parser = new DissectParser(pattern, appendSeparator); + Set referenceKeys = parser.referenceKeys(); + if (referenceKeys.size() > 0) { + throw new ParsingException( + src, + "Reference keys not supported in dissect patterns: [%{*{}}]", + referenceKeys.iterator().next() + ); + } + List keys = parser.outputKeys() + .stream() + .map(x -> new ReferenceAttribute(src, x, DataTypes.KEYWORD)) + .map(Attribute.class::cast) + .toList(); + + return new Dissect(src, p, expression(ctx.primaryExpression()), new Dissect.Parser(pattern, appendSeparator, parser), keys); + } catch (DissectException e) { + throw new ParsingException(src, "Invalid pattern for dissect: [{}]", pattern); + } + }; + } + + @Override + public PlanFactory visitMvExpandCommand(EsqlBaseParser.MvExpandCommandContext ctx) { + String identifier = visitSourceIdentifier(ctx.sourceIdentifier()); + return child -> new MvExpand(source(ctx), child, new UnresolvedAttribute(source(ctx), identifier)); + + } + + @Override + public Map visitCommandOptions(EsqlBaseParser.CommandOptionsContext ctx) { + if (ctx == null) { + return Map.of(); + } + Map result = new HashMap<>(); + for (EsqlBaseParser.CommandOptionContext option : ctx.commandOption()) { + result.put(visitIdentifier(option.identifier()), expression(option.constant()).fold()); + } + return result; + } + + @Override + public LogicalPlan visitRowCommand(EsqlBaseParser.RowCommandContext ctx) { + return new Row(source(ctx), visitFields(ctx.fields())); + } + + @Override + public LogicalPlan visitFromCommand(EsqlBaseParser.FromCommandContext ctx) { + Source source = source(ctx); + TableIdentifier table = new TableIdentifier(source, null, visitSourceIdentifiers(ctx.sourceIdentifier())); + Map metadataMap = new LinkedHashMap<>(); + if (ctx.metadata() != null) { + for (var c : ctx.metadata().sourceIdentifier()) { + String id = visitSourceIdentifier(c); + Source src = source(c); + if (MetadataAttribute.isSupported(id) == false) { + throw new ParsingException(src, "unsupported metadata field [" + id + "]"); + } + Attribute a = metadataMap.put(id, MetadataAttribute.create(src, id)); + if (a != null) { + throw new ParsingException(src, "metadata field [" + id + "] already declared [" + a.source().source() + "]"); + } + } + } + return new EsqlUnresolvedRelation(source, table, Arrays.asList(metadataMap.values().toArray(Attribute[]::new))); + } + + @Override + public PlanFactory visitStatsCommand(EsqlBaseParser.StatsCommandContext ctx) { + List aggregates = visitFields(ctx.fields()); + List groupings = visitGrouping(ctx.grouping()); + if (aggregates.isEmpty() && groupings.isEmpty()) { + throw new ParsingException(source(ctx), "At least one aggregation or grouping expression required in [{}]", ctx.getText()); + } + // grouping keys are automatically added as aggregations however the user is not allowed to specify them + if (groupings.isEmpty() == false && aggregates.isEmpty() == false) { + var groupNames = Expressions.names(groupings); + + for (NamedExpression aggregate : aggregates) { + if (aggregate instanceof Alias a && a.child() instanceof UnresolvedAttribute ua && groupNames.contains(ua.name())) { + throw new ParsingException(ua.source(), "Cannot specify grouping expression [{}] as an aggregate", ua.name()); + } + } + } + aggregates.addAll(groupings); + return input -> new Aggregate(source(ctx), input, new ArrayList<>(groupings), aggregates); + } + + @Override + public PlanFactory visitInlinestatsCommand(EsqlBaseParser.InlinestatsCommandContext ctx) { + List aggregates = visitFields(ctx.fields()); + List groupings = visitGrouping(ctx.grouping()); + aggregates.addAll(groupings); + return input -> new InlineStats(source(ctx), input, new ArrayList<>(groupings), aggregates); + } + + @Override + public PlanFactory visitWhereCommand(EsqlBaseParser.WhereCommandContext ctx) { + Expression expression = expression(ctx.booleanExpression()); + return input -> new Filter(source(ctx), input, expression); + } + + @Override + public List visitFields(EsqlBaseParser.FieldsContext ctx) { + return ctx != null ? visitList(this, ctx.field(), NamedExpression.class) : new ArrayList<>(); + } + + @Override + public PlanFactory visitLimitCommand(EsqlBaseParser.LimitCommandContext ctx) { + Source source = source(ctx); + int limit = Integer.parseInt(ctx.INTEGER_LITERAL().getText()); + return input -> new Limit(source, new Literal(source, limit, DataTypes.INTEGER), input); + } + + @Override + public PlanFactory visitSortCommand(EsqlBaseParser.SortCommandContext ctx) { + List orders = visitList(this, ctx.orderExpression(), Order.class); + Source source = source(ctx); + return input -> new OrderBy(source, input, orders); + } + + @Override + public Explain visitExplainCommand(EsqlBaseParser.ExplainCommandContext ctx) { + return new Explain(source(ctx), plan(ctx.subqueryExpression().query())); + } + + @Override + public PlanFactory visitDropCommand(EsqlBaseParser.DropCommandContext ctx) { + var identifiers = ctx.sourceIdentifier(); + List removals = new ArrayList<>(identifiers.size()); + + for (EsqlBaseParser.SourceIdentifierContext idCtx : identifiers) { + Source src = source(idCtx); + String identifier = visitSourceIdentifier(idCtx); + if (identifier.equals(WILDCARD)) { + throw new ParsingException(src, "Removing all fields is not allowed [{}]", src.text()); + } + removals.add(new UnresolvedAttribute(src, identifier)); + } + + return child -> new Drop(source(ctx), child, removals); + } + + @Override + public PlanFactory visitRenameCommand(EsqlBaseParser.RenameCommandContext ctx) { + List renamings = ctx.renameClause().stream().map(this::visitRenameClause).toList(); + return child -> new Rename(source(ctx), child, renamings); + } + + @Override + public PlanFactory visitKeepCommand(EsqlBaseParser.KeepCommandContext ctx) { + if (ctx.PROJECT() != null) { + addWarning("PROJECT command is no longer supported, please use KEEP instead"); + } + List projections = new ArrayList<>(ctx.sourceIdentifier().size()); + boolean hasSeenStar = false; + for (var srcIdCtx : ctx.sourceIdentifier()) { + NamedExpression ne = visitProjectExpression(srcIdCtx); + if (ne instanceof UnresolvedStar) { + if (hasSeenStar) { + throw new ParsingException(ne.source(), "Cannot specify [*] more than once", ne.source().text()); + } else { + hasSeenStar = true; + } + } + projections.add(ne); + } + return child -> new Keep(source(ctx), child, projections); + } + + @Override + public LogicalPlan visitShowInfo(EsqlBaseParser.ShowInfoContext ctx) { + return new ShowInfo(source(ctx)); + } + + @Override + public LogicalPlan visitShowFunctions(EsqlBaseParser.ShowFunctionsContext ctx) { + return new ShowFunctions(source(ctx)); + } + + @Override + public PlanFactory visitEnrichCommand(EsqlBaseParser.EnrichCommandContext ctx) { + return p -> { + final String policyName = visitSourceIdentifier(ctx.policyName); + var source = source(ctx); + NamedExpression matchField = ctx.ON() != null + ? new UnresolvedAttribute(source(ctx.matchField), visitSourceIdentifier(ctx.matchField)) + : new EmptyAttribute(source); + if (matchField.name().contains("*")) { + throw new ParsingException( + source(ctx), + "Using wildcards (*) in ENRICH WITH projections is not allowed [{}]", + matchField.name() + ); + } + List keepClauses = visitList(this, ctx.enrichWithClause(), NamedExpression.class); + return new Enrich( + source, + p, + new Literal(source(ctx.policyName), policyName, DataTypes.KEYWORD), + matchField, + null, + keepClauses.isEmpty() ? List.of() : keepClauses + ); + }; + } + + @Override + public NamedExpression visitEnrichWithClause(EsqlBaseParser.EnrichWithClauseContext ctx) { + Source src = source(ctx); + String enrichField = enrichFieldName(ctx.enrichField); + String newName = enrichFieldName(ctx.newName); + UnresolvedAttribute enrichAttr = new UnresolvedAttribute(src, enrichField); + return newName == null ? enrichAttr : new Alias(src, newName, enrichAttr); + } + + private String enrichFieldName(EsqlBaseParser.SourceIdentifierContext ctx) { + String name = ctx == null ? null : visitSourceIdentifier(ctx); + if (name != null && name.contains(WILDCARD)) { + throw new ParsingException(source(ctx), "Using wildcards (*) in ENRICH WITH projections is not allowed [{}]", name); + } + return name; + } + + interface PlanFactory extends Function {} +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java new file mode 100644 index 0000000000000..1cb71d64d5548 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java @@ -0,0 +1,62 @@ +/* + * 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.parser; + +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.esql.EsqlClientException; +import org.elasticsearch.xpack.ql.tree.Source; + +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; + +public class ParsingException extends EsqlClientException { + private final int line; + private final int charPositionInLine; + + public ParsingException(String message, Exception cause, int line, int charPositionInLine) { + super(message, cause); + this.line = line; + this.charPositionInLine = charPositionInLine; + } + + ParsingException(String message, Object... args) { + this(Source.EMPTY, message, args); + } + + public ParsingException(Source source, String message, Object... args) { + super(message, args); + this.line = source.source().getLineNumber(); + this.charPositionInLine = source.source().getColumnNumber(); + } + + public ParsingException(Exception cause, Source source, String message, Object... args) { + super(cause, message, args); + this.line = source.source().getLineNumber(); + this.charPositionInLine = source.source().getColumnNumber(); + } + + public int getLineNumber() { + return line; + } + + public int getColumnNumber() { + return charPositionInLine + 1; + } + + public String getErrorMessage() { + return super.getMessage(); + } + + @Override + public RestStatus status() { + return RestStatus.BAD_REQUEST; + } + + @Override + public String getMessage() { + return format("line {}:{}: {}", getLineNumber(), getColumnNumber(), getErrorMessage()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/TypedParamValue.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/TypedParamValue.java new file mode 100644 index 0000000000000..74cc53e51b360 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/TypedParamValue.java @@ -0,0 +1,70 @@ +/* + * 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.parser; + +import java.util.Objects; + +/** + * Represent a strongly typed parameter value + */ +public class TypedParamValue { + + public final Object value; + public final String type; + private boolean hasExplicitType; // the type is explicitly set in the request or inferred by the parser + private ContentLocation tokenLocation; // location of the token failing the parsing rules + + public TypedParamValue(String type, Object value) { + this(type, value, true); + } + + public TypedParamValue(String type, Object value, boolean hasExplicitType) { + this.value = value; + this.type = type; + this.hasExplicitType = hasExplicitType; + } + + public boolean hasExplicitType() { + return hasExplicitType; + } + + public void hasExplicitType(boolean hasExplicitType) { + this.hasExplicitType = hasExplicitType; + } + + public ContentLocation tokenLocation() { + return tokenLocation; + } + + public void tokenLocation(ContentLocation tokenLocation) { + this.tokenLocation = tokenLocation; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TypedParamValue that = (TypedParamValue) o; + return Objects.equals(value, that.value) + && Objects.equals(type, that.type) + && Objects.equals(hasExplicitType, that.hasExplicitType); + } + + @Override + public int hashCode() { + return Objects.hash(value, type, hasExplicitType); + } + + @Override + public String toString() { + return String.valueOf(value) + " [" + type + "][" + hasExplicitType + "][" + tokenLocation + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Dissect.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Dissect.java new file mode 100644 index 0000000000000..49c87f2b4cc78 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Dissect.java @@ -0,0 +1,74 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.dissect.DissectParser; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class Dissect extends RegexExtract { + private final Parser parser; + + public record Parser(String pattern, String appendSeparator, DissectParser parser) { + + // Override hashCode and equals since the parser is considered equal if its pattern and + // appendSeparator are equal ( and DissectParser uses reference equality ) + @Override + public boolean equals(Object other) { + if (this == other) return true; + if (other == null || getClass() != other.getClass()) return false; + Parser that = (Parser) other; + return Objects.equals(this.pattern, that.pattern) && Objects.equals(this.appendSeparator, that.appendSeparator); + } + + @Override + public int hashCode() { + return Objects.hash(pattern, appendSeparator); + } + } + + public Dissect(Source source, LogicalPlan child, Expression input, Parser parser, List extracted) { + super(source, child, input, extracted); + this.parser = parser; + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Dissect(source(), newChild, input, parser, extractedFields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Dissect::new, child(), input, parser, extractedFields); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + Dissect dissect = (Dissect) o; + return Objects.equals(parser, dissect.parser); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), parser); + } + + public Parser parser() { + return parser; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Drop.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Drop.java new file mode 100644 index 0000000000000..d5ebc67388143 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Drop.java @@ -0,0 +1,59 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class Drop extends UnaryPlan { + private final List removals; + + public Drop(Source source, LogicalPlan child, List removals) { + super(source, child); + this.removals = removals; + } + + public List removals() { + return removals; + } + + @Override + public boolean expressionsResolved() { + return Resolvables.resolved(removals); + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Drop(source(), newChild, removals); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Drop::new, child(), removals); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), removals); + } + + @Override + public boolean equals(Object obj) { + if (false == super.equals(obj)) { + return false; + } + return Objects.equals(removals, ((Drop) obj).removals); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Enrich.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Enrich.java new file mode 100644 index 0000000000000..1ad73be7902f7 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Enrich.java @@ -0,0 +1,109 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.EmptyAttribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public class Enrich extends UnaryPlan { + private final Expression policyName; + private final NamedExpression matchField; + private final EnrichPolicyResolution policy; + private List enrichFields; + private List output; + + public Enrich( + Source source, + LogicalPlan child, + Expression policyName, + NamedExpression matchField, + EnrichPolicyResolution policy, + List enrichFields + ) { + super(source, child); + this.policyName = policyName; + this.matchField = matchField; + this.policy = policy; + this.enrichFields = enrichFields; + } + + public NamedExpression matchField() { + return matchField; + } + + public List enrichFields() { + return enrichFields; + } + + public EnrichPolicyResolution policy() { + return policy; + } + + public Expression policyName() { + return policyName; + } + + @Override + public boolean expressionsResolved() { + return policyName.resolved() + && matchField instanceof EmptyAttribute == false // matchField not defined in the query, needs to be resolved from the policy + && matchField.resolved() + && Resolvables.resolved(enrichFields()); + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Enrich(source(), newChild, policyName, matchField, policy, enrichFields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Enrich::new, child(), policyName, matchField, policy, enrichFields); + } + + @Override + public List output() { + if (enrichFields == null) { + return child().output(); + } + if (this.output == null) { + this.output = mergeOutputAttributes(enrichFields(), child().output()); + } + return output; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + Enrich enrich = (Enrich) o; + return Objects.equals(policyName, enrich.policyName) + && Objects.equals(matchField, enrich.matchField) + && Objects.equals(policy, enrich.policy) + && Objects.equals(enrichFields, enrich.enrichFields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), policyName, matchField, policy, enrichFields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/EsqlUnresolvedRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/EsqlUnresolvedRelation.java new file mode 100644 index 0000000000000..718c4e0049c9b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/EsqlUnresolvedRelation.java @@ -0,0 +1,39 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.plan.TableIdentifier; +import org.elasticsearch.xpack.ql.plan.logical.UnresolvedRelation; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +public class EsqlUnresolvedRelation extends UnresolvedRelation { + + private final List metadataFields; + + public EsqlUnresolvedRelation(Source source, TableIdentifier table, List metadataFields, String unresolvedMessage) { + super(source, table, "", false, unresolvedMessage); + this.metadataFields = metadataFields; + } + + public EsqlUnresolvedRelation(Source source, TableIdentifier table, List metadataFields) { + this(source, table, metadataFields, null); + } + + public List metadataFields() { + return metadataFields; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EsqlUnresolvedRelation::new, table(), metadataFields(), unresolvedMessage()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Eval.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Eval.java new file mode 100644 index 0000000000000..85db86596613c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Eval.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plan.logical; + +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public class Eval extends UnaryPlan { + + private final List fields; + + public Eval(Source source, LogicalPlan child, List fields) { + super(source, child); + this.fields = fields; + } + + public List fields() { + return fields; + } + + @Override + public List output() { + return mergeOutputAttributes(fields, child().output()); + } + + @Override + public boolean expressionsResolved() { + return Resolvables.resolved(fields); + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Eval(source(), newChild, fields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Eval::new, child(), fields); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Eval eval = (Eval) o; + return child().equals(eval.child()) && Objects.equals(fields, eval.fields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), fields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Explain.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Explain.java new file mode 100644 index 0000000000000..6a79616a8e15e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Explain.java @@ -0,0 +1,85 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.plan.logical.LeafPlan; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.Objects; + +public class Explain extends LeafPlan { + + public enum Type { + PARSED, + ANALYZED + } + + private final LogicalPlan query; + + public Explain(Source source, LogicalPlan query) { + super(source); + this.query = query; + } + + // TODO: implement again + // @Override + // public void execute(EsqlSession session, ActionListener listener) { + // ActionListener analyzedStringListener = listener.map( + // analyzed -> new Result( + // output(), + // List.of(List.of(query.toString(), Type.PARSED.toString()), List.of(analyzed, Type.ANALYZED.toString())) + // ) + // ); + // + // session.analyzedPlan( + // query, + // ActionListener.wrap( + // analyzed -> analyzedStringListener.onResponse(analyzed.toString()), + // e -> analyzedStringListener.onResponse(e.toString()) + // ) + // ); + // + // } + + @Override + public List output() { + return List.of( + new ReferenceAttribute(Source.EMPTY, "plan", DataTypes.KEYWORD), + new ReferenceAttribute(Source.EMPTY, "type", DataTypes.KEYWORD) + ); + } + + @Override + public boolean expressionsResolved() { + return true; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Explain::new, query); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Explain explain = (Explain) o; + return Objects.equals(query, explain.query); + } + + @Override + public int hashCode() { + return Objects.hash(query); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Grok.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Grok.java new file mode 100644 index 0000000000000..5106e0e27e52b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Grok.java @@ -0,0 +1,127 @@ +/* + * 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.plan.logical; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.grok.GrokBuiltinPatterns; +import org.elasticsearch.grok.GrokCaptureConfig; +import org.elasticsearch.grok.GrokCaptureType; +import org.elasticsearch.xpack.esql.expression.NamedExpressions; +import org.elasticsearch.xpack.esql.parser.ParsingException; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +public class Grok extends RegexExtract { + + public record Parser(String pattern, org.elasticsearch.grok.Grok grok) { + + private List extractedFields() { + return grok.captureConfig() + .stream() + .sorted(Comparator.comparing(GrokCaptureConfig::name)) + // promote small numeric types, since Grok can produce float values + .map(x -> new ReferenceAttribute(Source.EMPTY, x.name(), EsqlDataTypes.widenSmallNumericTypes(toDataType(x.type())))) + .collect(Collectors.toList()); + } + + private static DataType toDataType(GrokCaptureType type) { + return switch (type) { + case STRING -> DataTypes.KEYWORD; + case INTEGER -> DataTypes.INTEGER; + case LONG -> DataTypes.LONG; + case FLOAT -> DataTypes.FLOAT; + case DOUBLE -> DataTypes.DOUBLE; + case BOOLEAN -> DataTypes.BOOLEAN; + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Parser parser = (Parser) o; + return Objects.equals(pattern, parser.pattern); + } + + @Override + public int hashCode() { + return Objects.hash(pattern); + } + } + + public static Parser pattern(Source source, String pattern) { + try { + var builtinPatterns = GrokBuiltinPatterns.get(true); + org.elasticsearch.grok.Grok grok = new org.elasticsearch.grok.Grok(builtinPatterns, pattern, logger::warn); + return new Parser(pattern, grok); + } catch (IllegalArgumentException e) { + throw new ParsingException(source, "Invalid pattern [{}] for grok: {}", pattern, e.getMessage()); + } + } + + private static final Logger logger = LogManager.getLogger(Grok.class); + + private final Parser parser; + + public Grok(Source source, LogicalPlan child, Expression inputExpression, Parser parser) { + this(source, child, inputExpression, parser, parser.extractedFields()); + } + + public Grok(Source source, LogicalPlan child, Expression inputExpr, Parser parser, List extracted) { + super(source, child, inputExpr, extracted); + this.parser = parser; + + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Grok(source(), newChild, input, parser, extractedFields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Grok::new, child(), input, parser, extractedFields); + } + + @Override + public List output() { + return NamedExpressions.mergeOutputAttributes(extractedFields, child().output()); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + Grok grok = (Grok) o; + return Objects.equals(parser, grok.parser); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), parser); + } + + public Parser parser() { + return parser; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java new file mode 100644 index 0000000000000..9ad543fba4beb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java @@ -0,0 +1,82 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class InlineStats extends UnaryPlan { + + private final List groupings; + private final List aggregates; + + public InlineStats(Source source, LogicalPlan child, List groupings, List aggregates) { + super(source, child); + this.groupings = groupings; + this.aggregates = aggregates; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, InlineStats::new, child(), groupings, aggregates); + } + + @Override + public InlineStats replaceChild(LogicalPlan newChild) { + return new InlineStats(source(), newChild, groupings, aggregates); + } + + public List groupings() { + return groupings; + } + + public List aggregates() { + return aggregates; + } + + @Override + public boolean expressionsResolved() { + return Resolvables.resolved(groupings) && Resolvables.resolved(aggregates); + } + + @Override + public List output() { + return Expressions.asAttributes(aggregates); + } + + @Override + public int hashCode() { + return Objects.hash(groupings, aggregates, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + InlineStats other = (InlineStats) obj; + return Objects.equals(groupings, other.groupings) + && Objects.equals(aggregates, other.aggregates) + && Objects.equals(child(), other.child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Keep.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Keep.java new file mode 100644 index 0000000000000..de9d023bd9357 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Keep.java @@ -0,0 +1,49 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class Keep extends Project { + + public Keep(Source source, LogicalPlan child, List projections) { + super(source, child, projections); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Keep::new, child(), projections()); + } + + @Override + public Project replaceChild(LogicalPlan newChild) { + return new Keep(source(), newChild, projections()); + } + + @Override + public boolean expressionsResolved() { + return super.expressionsResolved(); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode()); + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/MvExpand.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/MvExpand.java new file mode 100644 index 0000000000000..6f7830a12c708 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/MvExpand.java @@ -0,0 +1,57 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.Objects; + +public class MvExpand extends UnaryPlan { + private final NamedExpression target; + + public MvExpand(Source source, LogicalPlan child, NamedExpression target) { + super(source, child); + this.target = target; + } + + public NamedExpression target() { + return target; + } + + @Override + public boolean expressionsResolved() { + return target.resolved(); + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new MvExpand(source(), newChild, target); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvExpand::new, child(), target); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), target); + } + + @Override + public boolean equals(Object obj) { + if (false == super.equals(obj)) { + return false; + } + return Objects.equals(target, ((MvExpand) obj).target); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/RegexExtract.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/RegexExtract.java new file mode 100644 index 0000000000000..7f8f5ea08aaf8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/RegexExtract.java @@ -0,0 +1,62 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public abstract class RegexExtract extends UnaryPlan { + protected final Expression input; + protected final List extractedFields; + + protected RegexExtract(Source source, LogicalPlan child, Expression input, List extracted) { + super(source, child); + this.input = input; + this.extractedFields = extracted; + } + + @Override + public boolean expressionsResolved() { + return input.resolved(); + } + + @Override + public List output() { + return mergeOutputAttributes(extractedFields, child().output()); + } + + public Expression input() { + return input; + } + + public List extractedFields() { + return extractedFields; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + RegexExtract that = (RegexExtract) o; + return Objects.equals(input, that.input) && Objects.equals(extractedFields, that.extractedFields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), input, extractedFields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Rename.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Rename.java new file mode 100644 index 0000000000000..393125a143a5a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Rename.java @@ -0,0 +1,66 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class Rename extends UnaryPlan { + + private final List renamings; + + public Rename(Source source, LogicalPlan child, List renamings) { + super(source, child); + this.renamings = renamings; + } + + public List renamings() { + return renamings; + } + + @Override + public boolean expressionsResolved() { + for (var alias : renamings) { + // don't call dataType() - it will fail on UnresolvedAttribute + if (alias.resolved() == false && alias.child() instanceof UnsupportedAttribute == false) { + return false; + } + } + return true; + } + + @Override + public UnaryPlan replaceChild(LogicalPlan newChild) { + return new Rename(source(), newChild, renamings); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Rename::new, child(), renamings); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), renamings); + } + + @Override + public boolean equals(Object obj) { + if (false == super.equals(obj)) { + return false; + } + return Objects.equals(renamings, ((Rename) obj).renamings); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Row.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Row.java new file mode 100644 index 0000000000000..143c291b82114 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Row.java @@ -0,0 +1,62 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LeafPlan; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class Row extends LeafPlan { + + private final List fields; + + public Row(Source source, List fields) { + super(source); + this.fields = fields; + } + + public List fields() { + return fields; + } + + @Override + public List output() { + return Expressions.asAttributes(fields); + } + + @Override + public boolean expressionsResolved() { + return Resolvables.resolved(fields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Row::new, fields); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Row constant = (Row) o; + return Objects.equals(fields, constant.fields); + } + + @Override + public int hashCode() { + return Objects.hash(fields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java new file mode 100644 index 0000000000000..99d75a13726a1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java @@ -0,0 +1,68 @@ +/* + * 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.plan.logical; + +import org.elasticsearch.xpack.ql.capabilities.Resolvables; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class TopN extends UnaryPlan { + + private final List order; + private final Expression limit; + + public TopN(Source source, LogicalPlan child, List order, Expression limit) { + super(source, child); + this.order = order; + this.limit = limit; + } + + @Override + public boolean expressionsResolved() { + return limit.resolved() && Resolvables.resolved(order); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, TopN::new, child(), order, limit); + } + + @Override + public TopN replaceChild(LogicalPlan newChild) { + return new TopN(source(), newChild, order, limit); + } + + public Expression limit() { + return limit; + } + + public List order() { + return order; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), order, limit); + } + + @Override + public boolean equals(Object obj) { + if (super.equals(obj)) { + var other = (TopN) obj; + return Objects.equals(order, other.order) && Objects.equals(limit, other.limit); + } + return false; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EsqlProject.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EsqlProject.java new file mode 100644 index 0000000000000..489a3787647b2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/EsqlProject.java @@ -0,0 +1,50 @@ +/* + * 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.plan.logical.local; + +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +public class EsqlProject extends Project { + + public EsqlProject(Source source, LogicalPlan child, List projections) { + super(source, child, projections); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EsqlProject::new, child(), projections()); + } + + @Override + public EsqlProject replaceChild(LogicalPlan newChild) { + return new EsqlProject(source(), newChild, projections()); + } + + @Override + public boolean expressionsResolved() { + for (NamedExpression projection : projections()) { + // don't call dataType() - it will fail on UnresolvedAttribute + if (projection.resolved() == false && projection instanceof UnsupportedAttribute == false) { + return false; + } + } + return true; + } + + @Override + public Project withProjections(List projections) { + return new EsqlProject(source(), child(), projections); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelation.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelation.java new file mode 100644 index 0000000000000..da73cb13a47fb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalRelation.java @@ -0,0 +1,66 @@ +/* + * 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.plan.logical.local; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.plan.logical.LeafPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class LocalRelation extends LeafPlan { + + private final List output; + private final LocalSupplier supplier; + + public LocalRelation(Source source, List output, LocalSupplier supplier) { + super(source); + this.output = output; + this.supplier = supplier; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, LocalRelation::new, output, supplier); + } + + public LocalSupplier supplier() { + return supplier; + } + + @Override + public boolean expressionsResolved() { + return true; + } + + @Override + public List output() { + return output; + } + + @Override + public int hashCode() { + return Objects.hash(output, supplier); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + LocalRelation other = (LocalRelation) obj; + return Objects.equals(supplier, other.supplier) && Objects.equals(output, other.output); + } + +} 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 new file mode 100644 index 0000000000000..7fa82359ffc45 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/local/LocalSupplier.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plan.logical.local; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; + +import java.util.Arrays; +import java.util.function.Supplier; + +public interface LocalSupplier extends Supplier { + + LocalSupplier EMPTY = new LocalSupplier() { + @Override + public Block[] get() { + return BlockUtils.NO_BLOCKS; + } + + @Override + public String toString() { + return "EMPTY"; + } + }; + + static LocalSupplier of(Block[] blocks) { + return new LocalSupplier() { + @Override + public Block[] get() { + return blocks; + } + + @Override + public String toString() { + return Arrays.toString(blocks); + } + }; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowFunctions.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowFunctions.java new file mode 100644 index 0000000000000..ae7f1db2bd463 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowFunctions.java @@ -0,0 +1,100 @@ +/* + * 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.plan.logical.show; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Strings; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.plan.logical.LeafPlan; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.session.Configuration; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; + +public class ShowFunctions extends LeafPlan { + + private final List attributes; + + public ShowFunctions(Source source) { + super(source); + + attributes = new ArrayList<>(); + for (var name : List.of("name", "synopsis")) { + attributes.add(new ReferenceAttribute(Source.EMPTY, name, KEYWORD)); + } + } + + @Override + public List output() { + return attributes; + } + + public List> values(FunctionRegistry functionRegistry) { + List> rows = new ArrayList<>(); + for (var def : functionRegistry.listFunctions(null)) { + List row = new ArrayList<>(); + row.add(asBytesRefOrNull(def.name())); + + var constructors = def.clazz().getConstructors(); + StringBuilder sb = new StringBuilder(def.name()); + sb.append('('); + if (constructors.length > 0) { + var params = constructors[0].getParameters(); // no multiple c'tors supported + for (int i = 1; i < params.length; i++) { // skipping 1st argument, the source + if (Configuration.class.isAssignableFrom(params[i].getType()) == false) { + if (i > 1) { + sb.append(", "); + } + sb.append(params[i].getName()); + if (List.class.isAssignableFrom(params[i].getType())) { + sb.append("..."); + } + } + } + } + sb.append(')'); + row.add(asBytesRefOrNull(sb.toString())); + + rows.add(row); + } + rows.sort(Comparator.comparing(x -> ((BytesRef) x.get(0)))); + return rows; + } + + private static BytesRef asBytesRefOrNull(String string) { + return Strings.hasText(string) ? new BytesRef(string) : null; + } + + @Override + public boolean expressionsResolved() { + return true; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public boolean equals(Object obj) { + return this == obj || obj != null && getClass() == obj.getClass(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowInfo.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowInfo.java new file mode 100644 index 0000000000000..b7fb35121f514 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/show/ShowInfo.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.plan.logical.show; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Build; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.plan.logical.LeafPlan; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; + +public class ShowInfo extends LeafPlan { + + private final List attributes; + + public ShowInfo(Source source) { + super(source); + + attributes = new ArrayList<>(); + for (var name : List.of("version", "date", "hash")) { + attributes.add(new ReferenceAttribute(Source.EMPTY, name, KEYWORD)); + } + } + + @Override + public List output() { + return attributes; + } + + public List> values() { + List row = new ArrayList<>(attributes.size()); + row.add(new BytesRef(Build.current().version())); + row.add(new BytesRef(Build.current().date())); + row.add(new BytesRef(Build.current().hash())); + return List.of(row); + } + + @Override + public boolean expressionsResolved() { + return true; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this); + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + + @Override + public boolean equals(Object obj) { + return this == obj || obj != null && getClass() == obj.getClass(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/AggregateExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/AggregateExec.java new file mode 100644 index 0000000000000..9feb5e9b009d1 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/AggregateExec.java @@ -0,0 +1,118 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class AggregateExec extends UnaryExec implements EstimatesRowSize { + private final List groupings; + private final List aggregates; + + private final Mode mode; + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + private final Integer estimatedRowSize; + + public enum Mode { + SINGLE, + PARTIAL, // maps raw inputs to intermediate outputs + FINAL, // maps intermediate inputs to final outputs + } + + public AggregateExec( + Source source, + PhysicalPlan child, + List groupings, + List aggregates, + Mode mode, + Integer estimatedRowSize + ) { + super(source, child); + this.groupings = groupings; + this.aggregates = aggregates; + this.mode = mode; + this.estimatedRowSize = estimatedRowSize; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, AggregateExec::new, child(), groupings, aggregates, mode, estimatedRowSize); + } + + @Override + public AggregateExec replaceChild(PhysicalPlan newChild) { + return new AggregateExec(source(), newChild, groupings, aggregates, mode, estimatedRowSize); + } + + public List groupings() { + return groupings; + } + + public List aggregates() { + return aggregates; + } + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + public Integer estimatedRowSize() { + return estimatedRowSize; + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + state.add(false, aggregates); // The groupings are contained within the aggregates + int size = state.consumeAllFields(true); + return Objects.equals(this.estimatedRowSize, size) ? this : new AggregateExec(source(), child(), groupings, aggregates, mode, size); + } + + public Mode getMode() { + return mode; + } + + @Override + public List output() { + return Expressions.asAttributes(aggregates); + } + + @Override + public int hashCode() { + return Objects.hash(groupings, aggregates, mode, estimatedRowSize, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + AggregateExec other = (AggregateExec) obj; + return Objects.equals(groupings, other.groupings) + && Objects.equals(aggregates, other.aggregates) + && Objects.equals(mode, other.mode) + && Objects.equals(estimatedRowSize, other.estimatedRowSize) + && Objects.equals(child(), other.child()); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/DissectExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/DissectExec.java new file mode 100644 index 0000000000000..3d15156ac2ee8 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/DissectExec.java @@ -0,0 +1,61 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class DissectExec extends RegexExtractExec { + + private final Dissect.Parser parser; + + public DissectExec( + Source source, + PhysicalPlan child, + Expression inputExpression, + Dissect.Parser parser, + List extractedAttributes + ) { + super(source, child, inputExpression, extractedAttributes); + this.parser = parser; + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new DissectExec(source(), newChild, inputExpression, parser, extractedFields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, DissectExec::new, child(), inputExpression, parser, extractedFields); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + DissectExec that = (DissectExec) o; + return Objects.equals(parser, that.parser); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), parser); + } + + public Dissect.Parser parser() { + return parser; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EnrichExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EnrichExec.java new file mode 100644 index 0000000000000..6f2b83ef0aa6f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EnrichExec.java @@ -0,0 +1,113 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public class EnrichExec extends UnaryExec implements EstimatesRowSize { + + private final NamedExpression matchField; + private final String policyName; + private final String policyMatchField; + private final EsIndex enrichIndex; + private final List enrichFields; + + /** + * + * @param source + * @param child + * @param matchField the match field in the source data + * @param policyName the enrich policy name + * @param policyMatchField the match field name in the policy + * @param enrichIndex the enricy policy index (the system index created by the policy execution, not the source index) + * @param enrichFields the enrich fields + */ + public EnrichExec( + Source source, + PhysicalPlan child, + NamedExpression matchField, + String policyName, + String policyMatchField, + EsIndex enrichIndex, + List enrichFields + ) { + super(source, child); + this.matchField = matchField; + this.policyName = policyName; + this.policyMatchField = policyMatchField; + this.enrichIndex = enrichIndex; + this.enrichFields = enrichFields; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EnrichExec::new, child(), matchField, policyName, policyMatchField, enrichIndex, enrichFields); + } + + @Override + public EnrichExec replaceChild(PhysicalPlan newChild) { + return new EnrichExec(source(), newChild, matchField, policyName, policyMatchField, enrichIndex, enrichFields); + } + + public NamedExpression matchField() { + return matchField; + } + + public EsIndex enrichIndex() { + return enrichIndex; + } + + public List enrichFields() { + return enrichFields; + } + + public String policyName() { + return policyName; + } + + public String policyMatchField() { + return policyMatchField; + } + + @Override + public List output() { + return mergeOutputAttributes(enrichFields, child().output()); + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + state.add(false, enrichFields); + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + EnrichExec that = (EnrichExec) o; + return Objects.equals(matchField, that.matchField) + && Objects.equals(policyName, that.policyName) + && Objects.equals(policyMatchField, that.policyMatchField) + && Objects.equals(enrichIndex, that.enrichIndex) + && Objects.equals(enrichFields, that.enrichFields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), matchField, policyName, policyMatchField, enrichIndex, enrichFields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java new file mode 100644 index 0000000000000..9add95c28f433 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java @@ -0,0 +1,184 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.querydsl.container.Sort; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.NodeUtils; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class EsQueryExec extends LeafExec implements EstimatesRowSize { + public static final DataType DOC_DATA_TYPE = new DataType("_doc", Integer.BYTES * 3, false, false, false); + + static final EsField DOC_ID_FIELD = new EsField("_doc", DOC_DATA_TYPE, Map.of(), false); + + public static boolean isSourceAttribute(Attribute attr) { + return "_doc".equals(attr.name()); + } + + private final EsIndex index; + private final QueryBuilder query; + private final Expression limit; + private final List sorts; + private final List attrs; + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + private final Integer estimatedRowSize; + + public record FieldSort(FieldAttribute field, Order.OrderDirection direction, Order.NullsPosition nulls) { + public FieldSortBuilder fieldSortBuilder() { + FieldSortBuilder builder = new FieldSortBuilder(field.name()); + builder.order(Sort.Direction.from(direction).asOrder()); + builder.missing(Sort.Missing.from(nulls).searchOrder()); + builder.unmappedType(field.dataType().esType()); + return builder; + } + } + + public EsQueryExec(Source source, EsIndex index, QueryBuilder query) { + this(source, index, List.of(new FieldAttribute(source, DOC_ID_FIELD.getName(), DOC_ID_FIELD)), query, null, null, null); + } + + public EsQueryExec( + Source source, + EsIndex index, + List attrs, + QueryBuilder query, + Expression limit, + List sorts, + Integer estimatedRowSize + ) { + super(source); + this.index = index; + this.query = query; + this.attrs = attrs; + this.limit = limit; + this.sorts = sorts; + this.estimatedRowSize = estimatedRowSize; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EsQueryExec::new, index, attrs, query, limit, sorts, estimatedRowSize); + } + + public EsIndex index() { + return index; + } + + public QueryBuilder query() { + return query; + } + + @Override + public List output() { + return attrs; + } + + public Expression limit() { + return limit; + } + + public List sorts() { + return sorts; + } + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + public Integer estimatedRowSize() { + return estimatedRowSize; + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + int size; + if (sorts == null || sorts.isEmpty()) { + // track doc ids + state.add(false, Integer.BYTES); + size = state.consumeAllFields(false); + } else { + // track doc ids and segment ids + state.add(false, Integer.BYTES * 2); + size = state.consumeAllFields(true); + } + return Objects.equals(this.estimatedRowSize, size) ? this : new EsQueryExec(source(), index, attrs, query, limit, sorts, size); + } + + public EsQueryExec withQuery(QueryBuilder query) { + return Objects.equals(this.query, query) ? this : new EsQueryExec(source(), index, attrs, query, limit, sorts, estimatedRowSize); + } + + public EsQueryExec withLimit(Expression limit) { + return Objects.equals(this.limit, limit) ? this : new EsQueryExec(source(), index, attrs, query, limit, sorts, estimatedRowSize); + } + + public EsQueryExec withSorts(List sorts) { + return Objects.equals(this.sorts, sorts) ? this : new EsQueryExec(source(), index, attrs, query, limit, sorts, estimatedRowSize); + } + + @Override + public int hashCode() { + return Objects.hash(index, attrs, query, limit, sorts); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + EsQueryExec other = (EsQueryExec) obj; + return Objects.equals(index, other.index) + && Objects.equals(attrs, other.attrs) + && Objects.equals(query, other.query) + && Objects.equals(limit, other.limit) + && Objects.equals(sorts, other.sorts) + && Objects.equals(estimatedRowSize, other.estimatedRowSize); + } + + @Override + public String nodeString() { + return nodeName() + + "[" + + index + + "], query[" + + (query != null ? Strings.toString(query, false, true) : "") + + "]" + + NodeUtils.limitedToString(attrs) + + ", limit[" + + (limit != null ? limit.toString() : "") + + "], sort[" + + (sorts != null ? sorts.toString() : "") + + "] estimatedRowSize[" + + estimatedRowSize + + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsSourceExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsSourceExec.java new file mode 100644 index 0000000000000..e7772ed14df34 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsSourceExec.java @@ -0,0 +1,79 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.NodeUtils; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class EsSourceExec extends LeafExec { + + private final EsIndex index; + private final List attributes; + private final QueryBuilder query; + + public EsSourceExec(EsRelation relation) { + this(relation.source(), relation.index(), relation.output(), null); + } + + public EsSourceExec(Source source, EsIndex index, List attributes, QueryBuilder query) { + super(source); + this.index = index; + this.attributes = attributes; + this.query = query; + } + + public EsIndex index() { + return index; + } + + public QueryBuilder query() { + return query; + } + + @Override + public List output() { + return attributes; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EsSourceExec::new, index, attributes, query); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + EsSourceExec other = (EsSourceExec) obj; + return Objects.equals(index, other.index) && Objects.equals(query, other.query); + } + + @Override + public String nodeString() { + return nodeName() + "[" + index + "]" + NodeUtils.limitedToString(attributes); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EstimatesRowSize.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EstimatesRowSize.java new file mode 100644 index 0000000000000..dc071785c234f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EstimatesRowSize.java @@ -0,0 +1,122 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; + +public interface EstimatesRowSize { + static PhysicalPlan estimateRowSize(int extraRowSize, PhysicalPlan plan) { + EstimatesRowSize.State state = new EstimatesRowSize.State(); + state.maxEstimatedRowSize = state.estimatedRowSize = extraRowSize; + return plan.transformDown(exec -> { + if (exec instanceof EstimatesRowSize r) { + return r.estimateRowSize(state); + } + return exec; + }); + } + + /** + * Estimate the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + * @return + */ + PhysicalPlan estimateRowSize(State state); + + final class State { + /** + * Estimated size of rows added by later operations. + */ + private int estimatedRowSize; + + /** + * Max value that {@link #estimatedRowSize} has had since the last + * call to {@link #consumeAllFields}. + */ + private int maxEstimatedRowSize; + + /** + * True if there is an operation that needs a sorted list of + * document ids (like {@link FieldExtractExec}) before the node + * being visited. That's used to add more bytes to any operation + * that loads documents out of order. + */ + private boolean needsSortedDocIds; + + /** + * Model an operator that has a fixed overhead. + */ + public void add(boolean needsSortedDocIds, int bytes) { + estimatedRowSize += bytes; + maxEstimatedRowSize = Math.max(estimatedRowSize, maxEstimatedRowSize); + this.needsSortedDocIds |= needsSortedDocIds; + } + + /** + * Model an operator that adds fields. + */ + public void add(boolean needsSortedDocIds, List expressions) { + expressions.stream().forEach(a -> estimatedRowSize += estimateSize(a.dataType())); + maxEstimatedRowSize = Math.max(estimatedRowSize, maxEstimatedRowSize); + this.needsSortedDocIds |= needsSortedDocIds; + } + + /** + * Model an operator that consumes all fields. + * @return the number of bytes added to pages emitted by the operator + * being modeled + */ + public int consumeAllFields(boolean producesUnsortedDocIds) { + int size = maxEstimatedRowSize; + if (producesUnsortedDocIds && needsSortedDocIds) { + size += DocVector.SHARD_SEGMENT_DOC_MAP_PER_ROW_OVERHEAD; + } + estimatedRowSize = maxEstimatedRowSize = 0; + needsSortedDocIds = false; + return size; + } + + @Override + public String toString() { + return "State{" + + "estimatedRowSize=" + + estimatedRowSize + + ", maxEstimatedRowSize=" + + maxEstimatedRowSize + + ", needsSortedDocIds=" + + needsSortedDocIds + + '}'; + } + } + + static int estimateSize(DataType dataType) { + ElementType elementType = LocalExecutionPlanner.toElementType(dataType); + return switch (elementType) { + case BOOLEAN -> 1; + case BYTES_REF -> { + if (dataType == DataTypes.IP) { + yield 16; + } + yield 50; // wild estimate for the size of a string. + } + case DOC -> throw new UnsupportedOperationException("can't load a doc with field extraction"); + case DOUBLE -> Double.BYTES; + case INT -> Integer.BYTES; + case LONG -> Long.BYTES; + case NULL -> 0; + case UNKNOWN -> throw new UnsupportedOperationException("unknown can't be the result of field extraction"); + }; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EvalExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EvalExec.java new file mode 100644 index 0000000000000..55049f752d7f3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EvalExec.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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public class EvalExec extends UnaryExec implements EstimatesRowSize { + private final List fields; + + public EvalExec(Source source, PhysicalPlan child, List fields) { + super(source, child); + this.fields = fields; + } + + public List fields() { + return fields; + } + + @Override + public List output() { + return mergeOutputAttributes(fields, child().output()); + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new EvalExec(source(), newChild, fields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, EvalExec::new, child(), fields); + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + state.add(false, fields); + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EvalExec eval = (EvalExec) o; + return child().equals(eval.child()) && Objects.equals(fields, eval.fields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), fields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeExec.java new file mode 100644 index 0000000000000..f1d215d352a50 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeExec.java @@ -0,0 +1,51 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +import static java.util.Collections.emptyList; + +public class ExchangeExec extends UnaryExec { + + private final List output; + private final boolean inBetweenAggs; + + public ExchangeExec(Source source, PhysicalPlan child) { + this(source, emptyList(), false, child); + } + + public ExchangeExec(Source source, List output, boolean inBetweenAggs, PhysicalPlan child) { + super(source, child); + this.output = output; + this.inBetweenAggs = inBetweenAggs; + } + + @Override + public List output() { + return output.isEmpty() ? super.output() : output; + } + + public boolean isInBetweenAggs() { + return inBetweenAggs; + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new ExchangeExec(source(), output, inBetweenAggs, newChild); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ExchangeExec::new, output, inBetweenAggs, child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSinkExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSinkExec.java new file mode 100644 index 0000000000000..365ddf9d889d9 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSinkExec.java @@ -0,0 +1,39 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +public class ExchangeSinkExec extends UnaryExec { + + private final List output; + + public ExchangeSinkExec(Source source, List output, PhysicalPlan child) { + super(source, child); + this.output = output; + } + + @Override + public List output() { + return output; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ExchangeSinkExec::new, output, child()); + } + + @Override + public ExchangeSinkExec replaceChild(PhysicalPlan newChild) { + return new ExchangeSinkExec(source(), output, newChild); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSourceExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSourceExec.java new file mode 100644 index 0000000000000..bc92cd7bd8a5c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ExchangeSourceExec.java @@ -0,0 +1,54 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class ExchangeSourceExec extends LeafExec { + + private final List output; + private final boolean intermediateAgg; + + public ExchangeSourceExec(Source source, List output, boolean intermediateAgg) { + super(source); + this.output = output; + this.intermediateAgg = intermediateAgg; + } + + @Override + public List output() { + return output; + } + + public boolean isIntermediateAgg() { + return intermediateAgg; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ExchangeSourceExec::new, output, intermediateAgg); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ExchangeSourceExec that = (ExchangeSourceExec) o; + return Objects.equals(output, that.output) && intermediateAgg == that.intermediateAgg; + } + + @Override + public int hashCode() { + return Objects.hash(output, intermediateAgg); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FieldExtractExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FieldExtractExec.java new file mode 100644 index 0000000000000..d252385acc89a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FieldExtractExec.java @@ -0,0 +1,87 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.NodeUtils; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +public class FieldExtractExec extends UnaryExec implements EstimatesRowSize { + private final List attributesToExtract; + private final Attribute sourceAttribute; + + public FieldExtractExec(Source source, PhysicalPlan child, List attributesToExtract) { + super(source, child); + this.attributesToExtract = attributesToExtract; + this.sourceAttribute = extractSourceAttributesFrom(child); + } + + public static Attribute extractSourceAttributesFrom(PhysicalPlan plan) { + return plan.outputSet().stream().filter(EsQueryExec::isSourceAttribute).findFirst().orElse(null); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, FieldExtractExec::new, child(), attributesToExtract); + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new FieldExtractExec(source(), newChild, attributesToExtract); + } + + public List attributesToExtract() { + return attributesToExtract; + } + + public Attribute sourceAttribute() { + return sourceAttribute; + } + + @Override + public List output() { + List output = new ArrayList<>(child().output()); + output.addAll(attributesToExtract); + return output; + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + state.add(true, attributesToExtract); + return this; + } + + @Override + public int hashCode() { + return Objects.hash(attributesToExtract, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + FieldExtractExec other = (FieldExtractExec) obj; + return Objects.equals(attributesToExtract, other.attributesToExtract) && Objects.equals(child(), other.child()); + } + + @Override + public String nodeString() { + return nodeName() + NodeUtils.limitedToString(attributesToExtract); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FilterExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FilterExec.java new file mode 100644 index 0000000000000..d1bc7396a1dbb --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FilterExec.java @@ -0,0 +1,63 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class FilterExec extends UnaryExec { + + private final Expression condition; + + public FilterExec(Source source, PhysicalPlan child, Expression condition) { + super(source, child); + this.condition = condition; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, FilterExec::new, child(), condition); + } + + @Override + public FilterExec replaceChild(PhysicalPlan newChild) { + return new FilterExec(source(), newChild, condition); + } + + public Expression condition() { + return condition; + } + + @Override + public List output() { + return child().output(); + } + + @Override + public int hashCode() { + return Objects.hash(condition, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + FilterExec other = (FilterExec) obj; + return Objects.equals(condition, other.condition) && Objects.equals(child(), other.child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java new file mode 100644 index 0000000000000..3a7d52e83df66 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/FragmentExec.java @@ -0,0 +1,105 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class FragmentExec extends LeafExec implements EstimatesRowSize { + + private final LogicalPlan fragment; + private final QueryBuilder esFilter; + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + private final Integer estimatedRowSize; + + public FragmentExec(LogicalPlan fragment) { + this(fragment.source(), fragment, null, null); + } + + public FragmentExec(Source source, LogicalPlan fragment, QueryBuilder esFilter, Integer estimatedRowSize) { + super(source); + this.fragment = fragment; + this.esFilter = esFilter; + this.estimatedRowSize = estimatedRowSize; + } + + public LogicalPlan fragment() { + return fragment; + } + + public QueryBuilder esFilter() { + return esFilter; + } + + public Integer estimatedRowSize() { + return estimatedRowSize; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, FragmentExec::new, fragment, esFilter, estimatedRowSize); + } + + @Override + public List output() { + return fragment.output(); + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + int estimatedRowSize = state.consumeAllFields(false); + return Objects.equals(estimatedRowSize, this.estimatedRowSize) + ? this + : new FragmentExec(source(), fragment, esFilter, estimatedRowSize); + } + + @Override + public int hashCode() { + return Objects.hash(fragment, esFilter, estimatedRowSize); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + FragmentExec other = (FragmentExec) obj; + return Objects.equals(fragment, other.fragment) + && Objects.equals(esFilter, other.esFilter) + && Objects.equals(estimatedRowSize, other.estimatedRowSize); + } + + @Override + public String nodeString() { + StringBuilder sb = new StringBuilder(); + sb.append(nodeName()); + sb.append("[filter="); + sb.append(esFilter); + sb.append(", estimatedRowSize="); + sb.append(estimatedRowSize); + sb.append(", fragment=[<>"); + sb.append(fragment.toString()); + sb.append("<>]]"); + return sb.toString(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/GrokExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/GrokExec.java new file mode 100644 index 0000000000000..3d36e787e1534 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/GrokExec.java @@ -0,0 +1,61 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class GrokExec extends RegexExtractExec { + + private final Grok.Parser parser; + + public GrokExec( + Source source, + PhysicalPlan child, + Expression inputExpression, + Grok.Parser parser, + List extractedAttributes + ) { + super(source, child, inputExpression, extractedAttributes); + this.parser = parser; + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new GrokExec(source(), newChild, inputExpression, parser, extractedFields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, GrokExec::new, child(), inputExpression, parser, extractedFields); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + GrokExec grokExec = (GrokExec) o; + return Objects.equals(parser, grokExec.parser); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), parser); + } + + public Grok.Parser pattern() { + return parser; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LeafExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LeafExec.java new file mode 100644 index 0000000000000..ecf3aed27d70e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LeafExec.java @@ -0,0 +1,25 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.Collections; +import java.util.List; + +public abstract class LeafExec extends PhysicalPlan { + + protected LeafExec(Source source) { + super(source, Collections.emptyList()); + } + + @Override + public final LeafExec replaceChildren(List newChildren) { + throw new UnsupportedOperationException("this type of node doesn't have any children to replace"); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LimitExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LimitExec.java new file mode 100644 index 0000000000000..36aa2ed733288 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LimitExec.java @@ -0,0 +1,57 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.Objects; + +public class LimitExec extends UnaryExec { + + private final Expression limit; + + public LimitExec(Source source, PhysicalPlan child, Expression limit) { + super(source, child); + this.limit = limit; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, LimitExec::new, child(), limit); + } + + @Override + public LimitExec replaceChild(PhysicalPlan newChild) { + return new LimitExec(source(), newChild, limit); + } + + public Expression limit() { + return limit; + } + + @Override + public int hashCode() { + return Objects.hash(limit, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + LimitExec other = (LimitExec) obj; + return Objects.equals(limit, other.limit) && Objects.equals(child(), other.child()); + } +} 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 new file mode 100644 index 0000000000000..9948eb2c76109 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LocalSourceExec.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plan.physical; + +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class LocalSourceExec extends LeafExec { + + private final List output; + private final LocalSupplier supplier; + + public LocalSourceExec(Source source, List output, LocalSupplier supplier) { + super(source); + this.output = output; + this.supplier = supplier; + } + + @Override + public List output() { + return output; + } + + public LocalSupplier supplier() { + return supplier; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, LocalSourceExec::new, output, supplier); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + var other = (LocalSourceExec) o; + return Objects.equals(supplier, other.supplier) && Objects.equals(output, other.output); + } + + @Override + public int hashCode() { + return Objects.hash(output, supplier); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/MvExpandExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/MvExpandExec.java new file mode 100644 index 0000000000000..4bbd4b8aae2e3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/MvExpandExec.java @@ -0,0 +1,56 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.Objects; + +public class MvExpandExec extends UnaryExec { + + private final NamedExpression target; + + public MvExpandExec(Source source, PhysicalPlan child, NamedExpression target) { + super(source, child); + this.target = target; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MvExpandExec::new, child(), target); + } + + @Override + public MvExpandExec replaceChild(PhysicalPlan newChild) { + return new MvExpandExec(source(), newChild, target); + } + + public NamedExpression target() { + return target; + } + + @Override + public int hashCode() { + return Objects.hash(target, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + MvExpandExec other = (MvExpandExec) obj; + + return Objects.equals(target, other.target) && Objects.equals(child(), other.child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OrderExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OrderExec.java new file mode 100644 index 0000000000000..7477bd331a66f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OrderExec.java @@ -0,0 +1,58 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class OrderExec extends UnaryExec { + + private final List order; + + public OrderExec(Source source, PhysicalPlan child, List order) { + super(source, child); + this.order = order; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, OrderExec::new, child(), order); + } + + @Override + public OrderExec replaceChild(PhysicalPlan newChild) { + return new OrderExec(source(), newChild, order); + } + + public List order() { + return order; + } + + @Override + public int hashCode() { + return Objects.hash(order, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + OrderExec other = (OrderExec) obj; + + return Objects.equals(order, other.order) && Objects.equals(child(), other.child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OutputExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OutputExec.java new file mode 100644 index 0000000000000..8d9118cb1e017 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/OutputExec.java @@ -0,0 +1,43 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plan.physical; + +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.function.Consumer; + +public class OutputExec extends UnaryExec { + + private final Consumer pageConsumer; + + public OutputExec(PhysicalPlan child, Consumer pageConsumer) { + super(null, child); + this.pageConsumer = pageConsumer; + } + + public OutputExec(Source source, PhysicalPlan child, Consumer pageConsumer) { + super(source, child); + this.pageConsumer = pageConsumer; + } + + public Consumer getPageConsumer() { + return pageConsumer; + } + + @Override + public UnaryExec replaceChild(PhysicalPlan newChild) { + return new OutputExec(source(), newChild, pageConsumer); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, OutputExec::new, child(), pageConsumer); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/PhysicalPlan.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/PhysicalPlan.java new file mode 100644 index 0000000000000..6e5c0d94ca450 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/PhysicalPlan.java @@ -0,0 +1,33 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.plan.QueryPlan; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; + +/** + * A PhysicalPlan is "how" a LogicalPlan (the "what") actually gets translated into one or more queries. + * + * LogicalPlan = I want to get from DEN to SFO + * PhysicalPlan = take Delta, DEN to SJC, then SJC to SFO + */ +public abstract class PhysicalPlan extends QueryPlan { + + public PhysicalPlan(Source source, List children) { + super(source, children); + } + + @Override + public abstract int hashCode(); + + @Override + public abstract boolean equals(Object obj); + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ProjectExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ProjectExec.java new file mode 100644 index 0000000000000..add2baf94d15e --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ProjectExec.java @@ -0,0 +1,64 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class ProjectExec extends UnaryExec { // TODO implement EstimatesRowSize *somehow* + + private final List projections; + + public ProjectExec(Source source, PhysicalPlan child, List projections) { + super(source, child); + this.projections = projections; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ProjectExec::new, child(), projections); + } + + @Override + public ProjectExec replaceChild(PhysicalPlan newChild) { + return new ProjectExec(source(), newChild, projections); + } + + public List projections() { + return projections; + } + + @Override + public List output() { + return Expressions.asAttributes(projections); + } + + @Override + public int hashCode() { + return Objects.hash(projections, child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + ProjectExec other = (ProjectExec) obj; + + return Objects.equals(projections, other.projections) && Objects.equals(child(), other.child()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RegexExtractExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RegexExtractExec.java new file mode 100644 index 0000000000000..689058d1ea646 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RegexExtractExec.java @@ -0,0 +1,62 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; + +public abstract class RegexExtractExec extends UnaryExec implements EstimatesRowSize { + + protected final Expression inputExpression; + protected final List extractedFields; + + protected RegexExtractExec(Source source, PhysicalPlan child, Expression inputExpression, List extractedFields) { + super(source, child); + this.inputExpression = inputExpression; + this.extractedFields = extractedFields; + } + + @Override + public List output() { + return mergeOutputAttributes(extractedFields, child().output()); + } + + public Expression inputExpression() { + return inputExpression; + } + + public List extractedFields() { + return extractedFields; + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + state.add(false, extractedFields); + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (super.equals(o) == false) return false; + RegexExtractExec that = (RegexExtractExec) o; + return Objects.equals(inputExpression, that.inputExpression) && Objects.equals(extractedFields, that.extractedFields); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), inputExpression, extractedFields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RowExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RowExec.java new file mode 100644 index 0000000000000..f59e2b3d7346a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/RowExec.java @@ -0,0 +1,53 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class RowExec extends LeafExec { + private final List fields; + + public RowExec(Source source, List fields) { + super(source); + this.fields = fields; + } + + public List fields() { + return fields; + } + + @Override + public List output() { + return Expressions.asAttributes(fields); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, RowExec::new, fields); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RowExec constant = (RowExec) o; + return Objects.equals(fields, constant.fields); + } + + @Override + public int hashCode() { + return Objects.hash(fields); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ShowExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ShowExec.java new file mode 100644 index 0000000000000..560d23753a498 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/ShowExec.java @@ -0,0 +1,54 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class ShowExec extends LeafExec { + + private final List attributes; + private final List> values; + + public ShowExec(Source source, List attributes, List> values) { + super(source); + this.attributes = attributes; + this.values = values; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ShowExec::new, attributes, values); + } + + @Override + public int hashCode() { + return Objects.hash(attributes, values); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + return obj instanceof ShowExec other && Objects.equals(attributes, other.attributes) && Objects.equals(values, other.values); + } + + @Override + public List output() { + return attributes; + } + + public List> values() { + return values; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java new file mode 100644 index 0000000000000..def6709e7a386 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java @@ -0,0 +1,83 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.List; +import java.util.Objects; + +public class TopNExec extends UnaryExec implements EstimatesRowSize { + private final Expression limit; + private final List order; + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + private final Integer estimatedRowSize; + + public TopNExec(Source source, PhysicalPlan child, List order, Expression limit, Integer estimatedRowSize) { + super(source, child); + this.order = order; + this.limit = limit; + this.estimatedRowSize = estimatedRowSize; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, TopNExec::new, child(), order, limit, estimatedRowSize); + } + + @Override + public TopNExec replaceChild(PhysicalPlan newChild) { + return new TopNExec(source(), newChild, order, limit, estimatedRowSize); + } + + public Expression limit() { + return limit; + } + + public List order() { + return order; + } + + /** + * Estimate of the number of bytes that'll be loaded per position before + * the stream of pages is consumed. + */ + public Integer estimatedRowSize() { + return estimatedRowSize; + } + + @Override + public PhysicalPlan estimateRowSize(State state) { + int size = state.consumeAllFields(true); + return Objects.equals(this.estimatedRowSize, size) ? this : new TopNExec(source(), child(), order, limit, size); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), order, limit, estimatedRowSize); + } + + @Override + public boolean equals(Object obj) { + boolean equals = super.equals(obj); + if (equals) { + var other = (TopNExec) obj; + equals = Objects.equals(order, other.order) + && Objects.equals(limit, other.limit) + && Objects.equals(estimatedRowSize, other.estimatedRowSize); + } + return equals; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/UnaryExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/UnaryExec.java new file mode 100644 index 0000000000000..0b25f90fd9444 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/UnaryExec.java @@ -0,0 +1,61 @@ +/* + * 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.plan.physical; + +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public abstract class UnaryExec extends PhysicalPlan { + + private final PhysicalPlan child; + + protected UnaryExec(Source source, PhysicalPlan child) { + super(source, Collections.singletonList(child)); + this.child = child; + } + + @Override + public final PhysicalPlan replaceChildren(List newChildren) { + return replaceChild(newChildren.get(0)); + } + + public abstract UnaryExec replaceChild(PhysicalPlan newChild); + + public PhysicalPlan child() { + return child; + } + + @Override + public List output() { + return child.output(); + } + + @Override + public int hashCode() { + return Objects.hashCode(child()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + UnaryExec other = (UnaryExec) obj; + + return Objects.equals(child, other.child); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java new file mode 100644 index 0000000000000..b8e2cb42ff5d2 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java @@ -0,0 +1,291 @@ +/* + * 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.planner; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.Aggregator; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.aggregation.GroupingAggregator; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.operator.AggregationOperator; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.HashAggregationOperator.HashAggregationOperatorFactory; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlannerContext; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.PhysicalOperation; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; + +abstract class AbstractPhysicalOperationProviders implements PhysicalOperationProviders { + + private final AggregateMapper aggregateMapper = new AggregateMapper(); + + @Override + public final PhysicalOperation groupingPhysicalOperation( + AggregateExec aggregateExec, + PhysicalOperation source, + LocalExecutionPlannerContext context + ) { + Layout.Builder layout = new Layout.Builder(); + Operator.OperatorFactory operatorFactory = null; + AggregateExec.Mode mode = aggregateExec.getMode(); + var aggregates = aggregateExec.aggregates(); + + var sourceLayout = source.layout; + + if (aggregateExec.groupings().isEmpty()) { + // not grouping + List aggregatorFactories = new ArrayList<>(); + + // append channels to the layout + if (mode == AggregateExec.Mode.FINAL) { + layout.appendChannels(aggregates); + } else { + layout.appendChannels(aggregateMapper.mapNonGrouping(aggregates)); + } + // create the agg factories + aggregatesToFactory( + aggregates, + mode, + sourceLayout, + context.bigArrays(), + false, // non-grouping + s -> aggregatorFactories.add(s.supplier.aggregatorFactory(s.mode)) + ); + + if (aggregatorFactories.isEmpty() == false) { + operatorFactory = new AggregationOperator.AggregationOperatorFactory( + aggregatorFactories, + mode == AggregateExec.Mode.FINAL ? AggregatorMode.FINAL : AggregatorMode.INITIAL + ); + } + } else { + // grouping + List aggregatorFactories = new ArrayList<>(); + List groupSpecs = new ArrayList<>(aggregateExec.groupings().size()); + for (Expression group : aggregateExec.groupings()) { + var groupAttribute = Expressions.attribute(group); + if (groupAttribute == null) { + throw new EsqlIllegalArgumentException("Unexpected non-named expression[{}] as grouping in [{}]", group, aggregateExec); + } + Set grpAttribIds = new HashSet<>(); + grpAttribIds.add(groupAttribute.id()); + + /* + * Check for aliasing in aggregates which occurs in two cases (due to combining project + stats): + * - before stats (keep x = a | stats by x) which requires the partial input to use a's channel + * - after stats (stats by a | keep x = a) which causes the output layout to refer to the follow-up alias + */ + for (NamedExpression agg : aggregates) { + if (agg instanceof Alias a) { + if (a.child() instanceof Attribute attr) { + if (groupAttribute.id().equals(attr.id())) { + grpAttribIds.add(a.id()); + // TODO: investigate whether a break could be used since it shouldn't be possible to have multiple + // attributes + // pointing to the same attribute + } + // partial mode only + // check if there's any alias used in grouping - no need for the final reduction since the intermediate data + // is in the output form + // if the group points to an alias declared in the aggregate, use the alias child as source + else if (mode == AggregateExec.Mode.PARTIAL) { + if (groupAttribute.semanticEquals(a.toAttribute())) { + groupAttribute = attr; + break; + } + } + } + } + } + layout.appendChannel(grpAttribIds); + groupSpecs.add(new GroupSpec(source.layout.getChannel(groupAttribute.id()), groupAttribute)); + } + + if (mode == AggregateExec.Mode.FINAL) { + for (var agg : aggregates) { + if (agg instanceof Alias alias && alias.child() instanceof AggregateFunction) { + layout.appendChannel(alias.id()); + } + } + } else { + layout.appendChannels(aggregateMapper.mapGrouping(aggregates)); + } + + // create the agg factories + aggregatesToFactory( + aggregates, + mode, + sourceLayout, + context.bigArrays(), + true, // grouping + s -> aggregatorFactories.add(s.supplier.groupingAggregatorFactory(s.mode)) + ); + + if (groupSpecs.size() == 1 && groupSpecs.get(0).channel == null) { + operatorFactory = ordinalGroupingOperatorFactory( + source, + aggregateExec, + aggregatorFactories, + groupSpecs.get(0).attribute, + groupSpecs.get(0).elementType(), + context + ); + } else { + operatorFactory = new HashAggregationOperatorFactory( + groupSpecs.stream().map(GroupSpec::toHashGroupSpec).toList(), + aggregatorFactories, + context.pageSize(aggregateExec.estimatedRowSize()), + context.bigArrays() + ); + } + } + if (operatorFactory != null) { + return source.with(operatorFactory, layout.build()); + } + throw new UnsupportedOperationException(); + } + + /*** + * Creates a standard layout for intermediate aggregations, typically used across exchanges. + * Puts the group first, followed by each aggregation. + * + * It's similar to the code above (groupingPhysicalOperation) but ignores the factory creation. + */ + public static List intermediateAttributes(List aggregates, List groupings) { + var aggregateMapper = new AggregateMapper(); + + List attrs = new ArrayList<>(); + + // no groups + if (groupings.isEmpty()) { + attrs = Expressions.asAttributes(aggregateMapper.mapNonGrouping(aggregates)); + } + // groups + else { + for (Expression group : groupings) { + var groupAttribute = Expressions.attribute(group); + if (groupAttribute == null) { + throw new EsqlIllegalArgumentException("Unexpected non-named expression[{}] as grouping", group); + } + Set grpAttribIds = new HashSet<>(); + grpAttribIds.add(groupAttribute.id()); + + /* + * Check for aliasing in aggregates which occurs in two cases (due to combining project + stats): + * - before stats (keep x = a | stats by x) which requires the partial input to use a's channel + * - after stats (stats by a | keep x = a) which causes the output layout to refer to the follow-up alias + */ + for (NamedExpression agg : aggregates) { + if (agg instanceof Alias a) { + if (a.child() instanceof Attribute attr) { + if (groupAttribute.id().equals(attr.id())) { + grpAttribIds.add(a.id()); + // TODO: investigate whether a break could be used since it shouldn't be possible to have multiple + // attributes + // pointing to the same attribute + } + } + } + } + attrs.add(groupAttribute); + } + + attrs.addAll(Expressions.asAttributes(aggregateMapper.mapGrouping(aggregates))); + } + return attrs; + } + + private record AggFunctionSupplierContext(AggregatorFunctionSupplier supplier, AggregatorMode mode) {} + + private void aggregatesToFactory( + List aggregates, + AggregateExec.Mode mode, + Layout layout, + BigArrays bigArrays, + boolean grouping, + Consumer consumer + ) { + for (NamedExpression ne : aggregates) { + if (ne instanceof Alias alias) { + var child = alias.child(); + if (child instanceof AggregateFunction aggregateFunction) { + AggregatorMode aggMode = null; + List sourceAttr; + + if (mode == AggregateExec.Mode.PARTIAL) { + aggMode = AggregatorMode.INITIAL; + // TODO: this needs to be made more reliable - use casting to blow up when dealing with expressions (e+1) + sourceAttr = List.of(Expressions.attribute(aggregateFunction.field())); + } else if (mode == AggregateExec.Mode.FINAL) { + aggMode = AggregatorMode.FINAL; + if (grouping) { + sourceAttr = aggregateMapper.mapGrouping(aggregateFunction); + } else { + sourceAttr = aggregateMapper.mapNonGrouping(aggregateFunction); + } + } else { + throw new UnsupportedOperationException(); + } + var aggParams = aggregateFunction.parameters(); + Object[] params = new Object[aggParams.size()]; + for (int i = 0; i < params.length; i++) { + params[i] = aggParams.get(i).fold(); + } + + List inputChannels = sourceAttr.stream().map(NamedExpression::id).map(layout::getChannel).toList(); + assert inputChannels != null && inputChannels.size() > 0 && inputChannels.stream().allMatch(i -> i >= 0); + if (aggregateFunction instanceof ToAggregator agg) { + consumer.accept(new AggFunctionSupplierContext(agg.supplier(bigArrays, inputChannels), aggMode)); + } else { + throw new UnsupportedOperationException("aggregate functions must extend ToAggregator"); + } + } + } + } + } + + private record GroupSpec(Integer channel, Attribute attribute) { + HashAggregationOperator.GroupSpec toHashGroupSpec() { + if (channel == null) { + throw new UnsupportedOperationException("planned to use ordinals but tried to use the hash instead"); + } + return new HashAggregationOperator.GroupSpec(channel, elementType()); + } + + ElementType elementType() { + return LocalExecutionPlanner.toElementType(attribute.dataType()); + } + } + + /** + * Build a grouping operator that operates on ordinals if possible. + */ + public abstract Operator.OperatorFactory ordinalGroupingOperatorFactory( + PhysicalOperation source, + AggregateExec aggregateExec, + List aggregatorFactories, + Attribute attrSource, + ElementType groupType, + LocalExecutionPlannerContext context + ); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java new file mode 100644 index 0000000000000..ef3fffe2e22be --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java @@ -0,0 +1,230 @@ +/* + * 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.planner; + +import org.elasticsearch.compute.aggregation.IntermediateStateDesc; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.xpack.esql.expression.SurrogateExpression; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.function.aggregate.CountDistinct; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Max; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Median; +import org.elasticsearch.xpack.esql.expression.function.aggregate.MedianAbsoluteDeviation; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Min; +import org.elasticsearch.xpack.esql.expression.function.aggregate.NumericAggregate; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Percentile; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.function.Function; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class AggregateMapper { + + static final List NUMERIC = List.of("Int", "Long", "Double"); + + /** List of all ESQL agg functions. */ + static final List> AGG_FUNCTIONS = List.of( + Count.class, + CountDistinct.class, + Max.class, + Median.class, + MedianAbsoluteDeviation.class, + Min.class, + Percentile.class, + Sum.class + ); + + /** Record of agg Class, type, and grouping (or non-grouping). */ + record AggDef(Class aggClazz, String type, boolean grouping) {} + + /** Map of AggDef types to intermediate named expressions. */ + private final Map> mapper; + + /** Cache of aggregates to intermediate expressions. */ + private final HashMap> cache = new HashMap<>(); + + AggregateMapper() { + this(AGG_FUNCTIONS.stream().filter(Predicate.not(SurrogateExpression.class::isAssignableFrom)).toList()); + } + + AggregateMapper(List> aggregateFunctionClasses) { + mapper = aggregateFunctionClasses.stream() + .flatMap(AggregateMapper::typeAndNames) + .flatMap(AggregateMapper::groupingAndNonGrouping) + .collect(Collectors.toUnmodifiableMap(aggDef -> aggDef, AggregateMapper::lookupIntermediateState)); + } + + public List mapNonGrouping(List aggregates) { + return aggregates.stream().flatMap(agg -> map(agg, false)).toList(); + } + + public List mapNonGrouping(Expression aggregate) { + return map(aggregate, false).toList(); + } + + public List mapGrouping(List aggregates) { + return aggregates.stream().flatMap(agg -> map(agg, true)).toList(); + } + + public List mapGrouping(Expression aggregate) { + return map(aggregate, true).toList(); + } + + private Stream map(Expression aggregate, boolean grouping) { + aggregate = unwrapAlias(aggregate); + return cache.computeIfAbsent(aggregate, aggKey -> computeEntryForAgg(aggKey, grouping)).stream(); + } + + private List computeEntryForAgg(Expression aggregate, boolean grouping) { + var aggDef = aggDefOrNull(aggregate, grouping); + if (aggDef != null) { + var is = getNonNull(aggDef); + var exp = isToNE(is).toList(); + return exp; + } + if (aggregate instanceof FieldAttribute || aggregate instanceof MetadataAttribute || aggregate instanceof ReferenceAttribute) { + // This condition is a little pedantic, but do we expected other expressions here? if so, then add them + return List.of(); + } else { + throw new UnsupportedOperationException("unknown: " + aggregate.getClass() + ": " + aggregate); + } + } + + /** Gets the agg from the mapper - wrapper around map::get for more informative failure.*/ + private List getNonNull(AggDef aggDef) { + var l = mapper.get(aggDef); + if (l == null) { + throw new AssertionError("Cannot find intermediate state for: " + aggDef); + } + return l; + } + + static Stream, String>> typeAndNames(Class clazz) { + List types; + if (NumericAggregate.class.isAssignableFrom(clazz)) { + types = NUMERIC; + } else if (clazz == Count.class) { + types = List.of(""); // no extra type distinction + } else { + assert clazz == CountDistinct.class : "Expected CountDistinct, got: " + clazz; + types = Stream.concat(NUMERIC.stream(), Stream.of("Boolean", "BytesRef")).toList(); + } + return types.stream().map(type -> new Tuple<>(clazz, type)); + } + + static Stream groupingAndNonGrouping(Tuple, String> tuple) { + return Stream.of(new AggDef(tuple.v1(), tuple.v2(), true), new AggDef(tuple.v1(), tuple.v2(), false)); + } + + static AggDef aggDefOrNull(Expression aggregate, boolean grouping) { + if (aggregate instanceof AggregateFunction aggregateFunction) { + return new AggDef( + aggregateFunction.getClass(), + dataTypeToString(aggregateFunction.field().dataType(), aggregateFunction.getClass()), + grouping + ); + } + return null; + } + + /** Retrieves the intermediate state description for a given class, type, and grouping. */ + static List lookupIntermediateState(AggDef aggDef) { + try { + return (List) lookup(aggDef.aggClazz(), aggDef.type(), aggDef.grouping()).invokeExact(); + } catch (Throwable t) { + throw new AssertionError(t); + } + } + + /** Looks up the intermediate state method for a given class, type, and grouping. */ + static MethodHandle lookup(Class clazz, String type, boolean grouping) { + try { + return MethodHandles.lookup() + .findStatic( + Class.forName(determineAggName(clazz, type, grouping)), + "intermediateStateDesc", + MethodType.methodType(List.class) + ); + } catch (IllegalAccessException | NoSuchMethodException | ClassNotFoundException e) { + throw new AssertionError(e); + } + } + + /** Determines the engines agg class name, for the given class, type, and grouping. */ + static String determineAggName(Class clazz, String type, boolean grouping) { + StringBuilder sb = new StringBuilder(); + sb.append("org.elasticsearch.compute.aggregation."); + sb.append(clazz.getSimpleName()); + sb.append(type); + sb.append(grouping ? "Grouping" : ""); + sb.append("AggregatorFunction"); + return sb.toString(); + } + + /** Maps intermediate state description to named expressions. */ + static Stream isToNE(List intermediateStateDescs) { + return intermediateStateDescs.stream().map(is -> new ReferenceAttribute(Source.EMPTY, is.name(), toDataType(is.type()))); + } + + /** Returns the data type for the engines element type. */ + // defaults to aggstate, but we'll eventually be able to remove this + static DataType toDataType(ElementType elementType) { + return switch (elementType) { + case BOOLEAN -> DataTypes.BOOLEAN; + case BYTES_REF -> DataTypes.KEYWORD; + case INT -> DataTypes.INTEGER; + case LONG -> DataTypes.LONG; + case DOUBLE -> DataTypes.DOUBLE; + default -> throw new UnsupportedOperationException("unsupported agg type: " + elementType); + }; + } + + /** Returns the string representation for the data type. This reflects the engine's aggs naming structure. */ + static String dataTypeToString(DataType type, Class aggClass) { + if (aggClass == Count.class) { + return ""; // no type distinction + } + if (type.equals(DataTypes.BOOLEAN)) { + return "Boolean"; + } else if (type.equals(DataTypes.INTEGER)) { + return "Int"; + } else if (type.equals(DataTypes.LONG) || type.equals(DataTypes.DATETIME)) { + return "Long"; + } else if (type.equals(DataTypes.DOUBLE)) { + return "Double"; + } else if (type.equals(DataTypes.KEYWORD) || type.equals(DataTypes.IP)) { + return "BytesRef"; + } else { + throw new UnsupportedOperationException("unsupported agg type: " + type); + } + } + + static Expression unwrapAlias(Expression expression) { + if (expression instanceof Alias alias) return alias.child(); + return expression; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ArithmeticMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ArithmeticMapper.java new file mode 100644 index 0000000000000..fd330e25b24b5 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ArithmeticMapper.java @@ -0,0 +1,122 @@ +/* + * 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.planner; + +import org.elasticsearch.common.TriFunction; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeRegistry; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.ArithmeticOperation; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mod; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.BiFunction; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.planner.ComparisonMapper.castToEvaluator; + +abstract class ArithmeticMapper extends EvalMapper.ExpressionMapper { + static final EvalMapper.ExpressionMapper ADD = new ArithmeticMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.AddIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.AddLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.AddUnsignedLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.AddDoublesEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper DIV = new ArithmeticMapper
( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.DivIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.DivLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.DivUnsignedLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.DivDoublesEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper MOD = new ArithmeticMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.ModIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.ModLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.ModUnsignedLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.ModDoublesEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper MUL = new ArithmeticMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.MulIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.MulLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.MulUnsignedLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.MulDoublesEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper SUB = new ArithmeticMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.SubIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.SubLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.SubUnsignedLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.SubDoublesEvaluator::new + ) { + }; + + private final TriFunction< + Source, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator> ints; + private final TriFunction< + Source, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator> longs; + private final TriFunction< + Source, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator> ulongs; + private final BiFunction doubles; + + private ArithmeticMapper( + TriFunction ints, + TriFunction longs, + TriFunction ulongs, + BiFunction doubles + ) { + this.ints = ints; + this.longs = longs; + this.ulongs = ulongs; + this.doubles = doubles; + } + + @Override + protected final Supplier map(ArithmeticOperation op, Layout layout) { + if (op.left().dataType().isNumeric()) { + DataType type = null; + if (op instanceof Div div) { + type = div.dataType(); + } else { + type = EsqlDataTypeRegistry.INSTANCE.commonType(op.left().dataType(), op.right().dataType()); + } + if (type == DataTypes.INTEGER) { + return castToEvaluator(op, layout, DataTypes.INTEGER, ints); + } + if (type == DataTypes.LONG) { + return castToEvaluator(op, layout, DataTypes.LONG, longs); + } + if (type == DataTypes.UNSIGNED_LONG) { + return castToEvaluator(op, layout, DataTypes.UNSIGNED_LONG, ulongs); + } + if (type == DataTypes.DOUBLE) { + return castToEvaluator(op, layout, DataTypes.DOUBLE, doubles); + } + } + throw new AssertionError("resolved type for [" + op + "] but didn't implement mapping"); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ComparisonMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ComparisonMapper.java new file mode 100644 index 0000000000000..eb2aa832d3ed3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ComparisonMapper.java @@ -0,0 +1,187 @@ +/* + * 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.planner; + +import org.elasticsearch.common.TriFunction; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Cast; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeRegistry; +import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.BiFunction; +import java.util.function.Supplier; + +abstract class ComparisonMapper extends EvalMapper.ExpressionMapper { + static final EvalMapper.ExpressionMapper EQUALS = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EqualsIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EqualsLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EqualsDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EqualsKeywordsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EqualsBoolsEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper NOT_EQUALS = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEqualsIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEqualsLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEqualsDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEqualsKeywordsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEqualsBoolsEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper GREATER_THAN = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanKeywordsEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper GREATER_THAN_OR_EQUAL = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqualIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqualLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqualDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqualKeywordsEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper LESS_THAN = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanKeywordsEvaluator::new + ) { + }; + + static final EvalMapper.ExpressionMapper LESS_THAN_OR_EQUAL = new ComparisonMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqualIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqualLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqualDoublesEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqualKeywordsEvaluator::new + ) { + }; + + private final BiFunction ints; + private final BiFunction longs; + private final BiFunction doubles; + private final BiFunction keywords; + private final BiFunction bools; + + private ComparisonMapper( + BiFunction ints, + BiFunction longs, + BiFunction doubles, + BiFunction keywords, + BiFunction bools + ) { + this.ints = ints; + this.longs = longs; + this.doubles = doubles; + this.keywords = keywords; + this.bools = bools; + } + + ComparisonMapper( + BiFunction ints, + BiFunction longs, + BiFunction doubles, + BiFunction keywords + ) { + this.ints = ints; + this.longs = longs; + this.doubles = doubles; + this.keywords = keywords; + this.bools = (lhs, rhs) -> { throw new AssertionError("bool unsupported"); }; + } + + @Override + protected final Supplier map(BinaryComparison bc, Layout layout) { + DataType leftType = bc.left().dataType(); + if (leftType.isNumeric()) { + DataType type = EsqlDataTypeRegistry.INSTANCE.commonType(leftType, bc.right().dataType()); + if (type == DataTypes.INTEGER) { + return castToEvaluator(bc, layout, DataTypes.INTEGER, ints); + } + if (type == DataTypes.LONG) { + return castToEvaluator(bc, layout, DataTypes.LONG, longs); + } + if (type == DataTypes.DOUBLE) { + return castToEvaluator(bc, layout, DataTypes.DOUBLE, doubles); + } + if (type == DataTypes.UNSIGNED_LONG) { + // using the long comparators will work on UL as well + return castToEvaluator(bc, layout, DataTypes.UNSIGNED_LONG, longs); + } + } + Supplier leftEval = EvalMapper.toEvaluator(bc.left(), layout); + Supplier rightEval = EvalMapper.toEvaluator(bc.right(), layout); + if (leftType == DataTypes.KEYWORD || leftType == DataTypes.TEXT || leftType == DataTypes.IP || leftType == DataTypes.VERSION) { + return () -> keywords.apply(leftEval.get(), rightEval.get()); + } + if (leftType == DataTypes.BOOLEAN) { + return () -> bools.apply(leftEval.get(), rightEval.get()); + } + if (leftType == DataTypes.DATETIME) { + return () -> longs.apply(leftEval.get(), rightEval.get()); + } + throw new AssertionError("resolved type for [" + bc + "] but didn't implement mapping"); + } + + static Supplier castToEvaluator( + BinaryOperator op, + Layout layout, + DataType required, + BiFunction buildEvaluator + ) { + Supplier lhs = Cast.cast( + op.left().dataType(), + required, + EvalMapper.toEvaluator(op.left(), layout) + ); + Supplier rhs = Cast.cast( + op.right().dataType(), + required, + EvalMapper.toEvaluator(op.right(), layout) + ); + return () -> buildEvaluator.apply(lhs.get(), rhs.get()); + } + + static Supplier castToEvaluator( + BinaryOperator op, + Layout layout, + DataType required, + TriFunction< + Source, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator, + EvalOperator.ExpressionEvaluator> buildEvaluator + ) { + Supplier lhs = Cast.cast( + op.left().dataType(), + required, + EvalMapper.toEvaluator(op.left(), layout) + ); + Supplier rhs = Cast.cast( + op.right().dataType(), + required, + EvalMapper.toEvaluator(op.right(), layout) + ); + return () -> buildEvaluator.apply(op.source(), lhs.get(), rhs.get()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java new file mode 100644 index 0000000000000..3a72bff0d0c82 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -0,0 +1,179 @@ +/* + * 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.planner; + +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.Query; +import org.elasticsearch.compute.aggregation.GroupingAggregator; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.lucene.LuceneOperator; +import org.elasticsearch.compute.lucene.LuceneSourceOperator.LuceneSourceOperatorFactory; +import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory; +import org.elasticsearch.compute.lucene.ValueSources; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.EmptySourceOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; +import org.elasticsearch.index.mapper.NestedLookup; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.search.NestedHelper; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.sort.SortBuilder; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec.FieldSort; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.DriverParallelism; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlannerContext; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.PhysicalOperation; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Attribute; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +import static org.elasticsearch.common.lucene.search.Queries.newNonNestedFilter; +import static org.elasticsearch.compute.lucene.LuceneSourceOperator.NO_LIMIT; + +public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProviders { + private static final Logger logger = LogManager.getLogger(EsPhysicalOperationProviders.class); + + private final List searchContexts; + + public EsPhysicalOperationProviders(List searchContexts) { + this.searchContexts = searchContexts; + } + + @Override + public final PhysicalOperation fieldExtractPhysicalOperation(FieldExtractExec fieldExtractExec, PhysicalOperation source) { + Layout.Builder layout = source.layout.builder(); + + var sourceAttr = fieldExtractExec.sourceAttribute(); + + PhysicalOperation op = source; + for (Attribute attr : fieldExtractExec.attributesToExtract()) { + layout.appendChannel(attr.id()); + Layout previousLayout = op.layout; + + var sources = ValueSources.sources( + searchContexts, + attr.name(), + EsqlDataTypes.isUnsupported(attr.dataType()), + LocalExecutionPlanner.toElementType(attr.dataType()) + ); + + int docChannel = previousLayout.getChannel(sourceAttr.id()); + + op = op.with( + new ValuesSourceReaderOperator.ValuesSourceReaderOperatorFactory(sources, docChannel, attr.name()), + layout.build() + ); + } + return op; + } + + @Override + public final PhysicalOperation sourcePhysicalOperation(EsQueryExec esQueryExec, LocalExecutionPlannerContext context) { + LuceneOperator.LuceneOperatorFactory operatorFactory = null; + Function querySupplier = searchContext -> { + SearchExecutionContext ctx = searchContext.getSearchExecutionContext(); + Query query = ctx.toQuery(esQueryExec.query()).query(); + NestedLookup nestedLookup = ctx.nestedLookup(); + if (nestedLookup != NestedLookup.EMPTY) { + NestedHelper nestedHelper = new NestedHelper(nestedLookup, ctx::isFieldMapped); + if (nestedHelper.mightMatchNestedDocs(query)) { + // filter out nested documents + query = new BooleanQuery.Builder().add(query, BooleanClause.Occur.MUST) + .add(newNonNestedFilter(ctx.indexVersionCreated()), BooleanClause.Occur.FILTER) + .build(); + } + } + AliasFilter aliasFilter = searchContext.request().getAliasFilter(); + if (aliasFilter != AliasFilter.EMPTY) { + Query filterQuery = ctx.toQuery(aliasFilter.getQueryBuilder()).query(); + query = new BooleanQuery.Builder().add(query, BooleanClause.Occur.MUST) + .add(filterQuery, BooleanClause.Occur.FILTER) + .build(); + } + return query; + }; + + List sorts = esQueryExec.sorts(); + List> fieldSorts = null; + assert esQueryExec.estimatedRowSize() != null : "estimated row size not initialized"; + int rowEstimatedSize = esQueryExec.estimatedRowSize(); + int limit = esQueryExec.limit() != null ? (Integer) esQueryExec.limit().fold() : NO_LIMIT; + if (sorts != null && sorts.isEmpty() == false) { + fieldSorts = new ArrayList<>(sorts.size()); + for (FieldSort sort : sorts) { + fieldSorts.add(sort.fieldSortBuilder()); + } + operatorFactory = new LuceneTopNSourceOperatorFactory( + searchContexts, + querySupplier, + context.dataPartitioning(), + context.taskConcurrency(), + context.pageSize(rowEstimatedSize), + limit, + fieldSorts + ); + } else { + operatorFactory = new LuceneSourceOperatorFactory( + searchContexts, + querySupplier, + context.dataPartitioning(), + context.taskConcurrency(), + context.pageSize(rowEstimatedSize), + limit + ); + } + Layout.Builder layout = new Layout.Builder(); + for (int i = 0; i < esQueryExec.output().size(); i++) { + layout.appendChannel(esQueryExec.output().get(i).id()); + } + if (operatorFactory.size() > 0) { + context.driverParallelism(new DriverParallelism(DriverParallelism.Type.DATA_PARALLELISM, operatorFactory.size())); + return PhysicalOperation.fromSource(operatorFactory, layout.build()); + } else { + return PhysicalOperation.fromSource(new EmptySourceOperator.Factory(), layout.build()); + } + } + + @Override + public final Operator.OperatorFactory ordinalGroupingOperatorFactory( + LocalExecutionPlanner.PhysicalOperation source, + AggregateExec aggregateExec, + List aggregatorFactories, + Attribute attrSource, + ElementType groupElementType, + LocalExecutionPlannerContext context + ) { + var sourceAttribute = FieldExtractExec.extractSourceAttributesFrom(aggregateExec.child()); + int docChannel = source.layout.getChannel(sourceAttribute.id()); + // The grouping-by values are ready, let's group on them directly. + // Costin: why are they ready and not already exposed in the layout? + return new OrdinalsGroupingOperator.OrdinalsGroupingOperatorFactory( + ValueSources.sources( + searchContexts, + attrSource.name(), + EsqlDataTypes.isUnsupported(attrSource.dataType()), + LocalExecutionPlanner.toElementType(attrSource.dataType()) + ), + docChannel, + attrSource.name(), + aggregatorFactories, + context.pageSize(aggregateExec.estimatedRowSize()), + context.bigArrays() + ); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EvalMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EvalMapper.java new file mode 100644 index 0000000000000..5f3ab86217381 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EvalMapper.java @@ -0,0 +1,283 @@ +/* + * 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.planner; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.ann.Evaluator; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.BytesRefBlock; +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.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator; +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull; +import org.elasticsearch.xpack.ql.util.ReflectionUtils; + +import java.util.List; +import java.util.function.IntFunction; +import java.util.function.Supplier; + +public final class EvalMapper { + abstract static class ExpressionMapper { + private final Class typeToken; + + protected ExpressionMapper() { + typeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); + } + + protected abstract Supplier map(E expression, Layout layout); + } + + private static final List> MAPPERS = List.of( + ArithmeticMapper.ADD, + ArithmeticMapper.DIV, + ArithmeticMapper.MOD, + ArithmeticMapper.MUL, + ArithmeticMapper.SUB, + ComparisonMapper.EQUALS, + ComparisonMapper.NOT_EQUALS, + ComparisonMapper.GREATER_THAN, + ComparisonMapper.GREATER_THAN_OR_EQUAL, + ComparisonMapper.LESS_THAN, + ComparisonMapper.LESS_THAN_OR_EQUAL, + InMapper.IN_MAPPER, + NegMapper.NEG_MAPPER, + RegexMapper.REGEX_MATCH, + new BooleanLogic(), + new Nots(), + new Attributes(), + new Literals(), + new IsNotNulls(), + new IsNulls() + ); + + private EvalMapper() {} + + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static Supplier toEvaluator(Expression exp, Layout layout) { + if (exp instanceof Mappable m) { + return m.toEvaluator(e -> toEvaluator(e, layout)); + } + for (ExpressionMapper em : MAPPERS) { + if (em.typeToken.isInstance(exp)) { + return em.map(exp, layout); + } + } + throw new QlIllegalArgumentException("Unsupported expression [{}]", exp); + } + + static class BooleanLogic extends ExpressionMapper { + @Override + protected Supplier map(BinaryLogic bc, Layout layout) { + Supplier leftEval = toEvaluator(bc.left(), layout); + Supplier rightEval = toEvaluator(bc.right(), layout); + /** + * Evaluator for the three-valued boolean expressions. + * We can't generate these with the {@link Evaluator} annotation because that + * always implements viral null. And three-valued boolean expressions don't. + * {@code false AND null} is {@code false} and {@code true OR null} is {@code true}. + */ + record BooleanLogicExpressionEvaluator(BinaryLogic bl, ExpressionEvaluator leftEval, ExpressionEvaluator rightEval) + implements + ExpressionEvaluator { + @Override + public Block eval(Page page) { + Block lhs = leftEval.eval(page); + Block rhs = rightEval.eval(page); + + Vector lhsVector = lhs.asVector(); + Vector rhsVector = rhs.asVector(); + if (lhsVector != null && rhsVector != null) { + return eval((BooleanVector) lhsVector, (BooleanVector) rhsVector); + } + return eval(lhs, rhs); + } + + /** + * Eval blocks, handling {@code null}. This takes {@link Block} instead of + * {@link BooleanBlock} because blocks that only contain + * {@code null} can't be cast to {@link BooleanBlock}. So we check for + * {@code null} first and don't cast at all if the value is {@code null}. + */ + private Block eval(Block lhs, Block rhs) { + int positionCount = lhs.getPositionCount(); + BooleanBlock.Builder result = BooleanBlock.newBlockBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + if (lhs.getValueCount(p) > 1) { + result.appendNull(); + continue; + } + if (rhs.getValueCount(p) > 1) { + result.appendNull(); + continue; + } + Boolean v = bl.function() + .apply( + lhs.isNull(p) ? null : ((BooleanBlock) lhs).getBoolean(lhs.getFirstValueIndex(p)), + rhs.isNull(p) ? null : ((BooleanBlock) rhs).getBoolean(rhs.getFirstValueIndex(p)) + ); + if (v == null) { + result.appendNull(); + continue; + } + result.appendBoolean(v); + } + return result.build(); + } + + private Block eval(BooleanVector lhs, BooleanVector rhs) { + int positionCount = lhs.getPositionCount(); + BooleanVector.Builder result = BooleanVector.newVectorBuilder(positionCount); + for (int p = 0; p < positionCount; p++) { + result.appendBoolean(bl.function().apply(lhs.getBoolean(p), rhs.getBoolean(p))); + } + return result.build().asBlock(); + } + + } + return () -> new BooleanLogicExpressionEvaluator(bc, leftEval.get(), rightEval.get()); + } + } + + static class Nots extends ExpressionMapper { + @Override + protected Supplier map(Not not, Layout layout) { + Supplier expEval = toEvaluator(not.field(), layout); + return () -> new org.elasticsearch.xpack.esql.expression.predicate.logical.NotEvaluator(expEval.get()); + } + } + + static class Attributes extends ExpressionMapper { + @Override + protected Supplier map(Attribute attr, Layout layout) { + record Attribute(int channel) implements ExpressionEvaluator { + @Override + public Block eval(Page page) { + return page.getBlock(channel); + } + } + int channel = layout.getChannel(attr.id()); + return () -> new Attribute(channel); + } + } + + static class Literals extends ExpressionMapper { + + @Override + protected Supplier map(Literal lit, Layout layout) { + record LiteralsEvaluator(IntFunction block) implements ExpressionEvaluator { + @Override + public Block eval(Page page) { + return block.apply(page.getPositionCount()); + } + } + IntFunction block = block(lit); + return () -> new LiteralsEvaluator(block); + } + + private IntFunction block(Literal lit) { + if (lit.value() == null) { + return Block::constantNullBlock; + } + return switch (LocalExecutionPlanner.toElementType(lit.dataType())) { + case BOOLEAN -> { + boolean v = (boolean) lit.value(); + yield positions -> BooleanBlock.newConstantBlockWith(v, positions); + } + case BYTES_REF -> { + BytesRef v = (BytesRef) lit.value(); + yield positions -> BytesRefBlock.newConstantBlockWith(v, positions); + } + case DOUBLE -> new IntFunction<>() { // TODO toString in the rest of these and tests for this + private final double v = (double) lit.value(); + + @Override + public Block apply(int positions) { + return DoubleBlock.newConstantBlockWith(v, positions); + } + + @Override + public String toString() { + return Double.toString(v); + } + }; + case INT -> { + int v = (int) lit.value(); + yield positions -> IntBlock.newConstantBlockWith(v, positions); + } + case LONG -> { + long v = (long) lit.value(); + yield positions -> LongBlock.newConstantBlockWith(v, positions); + } + case NULL -> Block::constantNullBlock; + case DOC, UNKNOWN -> throw new UnsupportedOperationException("can't eval to doc or unknown"); + }; + } + } + + static class IsNulls extends ExpressionMapper { + + @Override + protected Supplier map(IsNull isNull, Layout layout) { + Supplier field = toEvaluator(isNull.field(), layout); + return () -> new IsNullEvaluator(field.get()); + } + + record IsNullEvaluator(EvalOperator.ExpressionEvaluator field) implements EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + Block fieldBlock = field.eval(page); + if (fieldBlock.asVector() != null) { + return BooleanBlock.newConstantBlockWith(false, page.getPositionCount()); + } + boolean[] result = new boolean[page.getPositionCount()]; + for (int p = 0; p < page.getPositionCount(); p++) { + result[p] = fieldBlock.isNull(p); + } + return new BooleanArrayVector(result, result.length).asBlock(); + } + } + } + + static class IsNotNulls extends ExpressionMapper { + + @Override + protected Supplier map(IsNotNull isNotNull, Layout layout) { + Supplier field = toEvaluator(isNotNull.field(), layout); + return () -> new IsNotNullEvaluator(field.get()); + } + + record IsNotNullEvaluator(EvalOperator.ExpressionEvaluator field) implements EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + Block fieldBlock = field.eval(page); + if (fieldBlock.asVector() != null) { + return BooleanBlock.newConstantBlockWith(true, page.getPositionCount()); + } + boolean[] result = new boolean[page.getPositionCount()]; + for (int p = 0; p < page.getPositionCount(); p++) { + result[p] = fieldBlock.isNull(p) == false; + } + return new BooleanArrayVector(result, result.length).asBlock(); + } + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ExchangeLayout.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ExchangeLayout.java new file mode 100644 index 0000000000000..327dc588e09b6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ExchangeLayout.java @@ -0,0 +1,65 @@ +/* + * 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.planner; + +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.xpack.ql.expression.NameId; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptyMap; + +/** + * Decorating layout that creates the NameId -> Value lazily based on the calls made to its content. + * Essentially it maps the existing (old) NameIds to the new ones. + */ +class ExchangeLayout extends Layout { + + private final Map delegate; + private final Map> inverse; + private final Map mappingToOldLayout; + private int counter; + + ExchangeLayout(Layout layout) { + super(emptyMap(), 0); + this.delegate = layout.internalLayout(); + this.mappingToOldLayout = Maps.newMapWithExpectedSize(delegate.size()); + this.inverse = Maps.newMapWithExpectedSize(delegate.size()); + + for (Map.Entry entry : delegate.entrySet()) { + NameId key = entry.getKey(); + Integer value = entry.getValue(); + inverse.computeIfAbsent(value, k -> new HashSet<>()).add(key); + } + } + + @Override + public Integer getChannel(NameId id) { + var oldId = mappingToOldLayout.get(id); + if (oldId == null && counter < delegate.size()) { + var names = inverse.get(counter++); + for (var name : names) { + oldId = name; + mappingToOldLayout.put(id, oldId); + } + } + return delegate.get(oldId); + } + + @Override + public int numberOfIds() { + return delegate.size(); + } + + @Override + public int numberOfChannels() { + return inverse.size(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracter.java new file mode 100644 index 0000000000000..8ef4fd7d7bafe --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracter.java @@ -0,0 +1,220 @@ +/* + * 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.planner; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.operator.ColumnExtractOperator; +import org.elasticsearch.grok.FloatConsumer; +import org.elasticsearch.grok.Grok; +import org.elasticsearch.grok.GrokCaptureConfig; +import org.elasticsearch.grok.GrokCaptureExtracter; +import org.joni.Region; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.DoubleConsumer; +import java.util.function.Function; +import java.util.function.IntConsumer; +import java.util.function.LongConsumer; + +public class GrokEvaluatorExtracter implements ColumnExtractOperator.Evaluator, GrokCaptureExtracter { + + private final Grok parser; + private final String pattern; + + private final List fieldExtracters; + + private final boolean[] valuesSet; + private final Object[] firstValues; + private final ElementType[] positionToType; + private Block.Builder[] blocks; + + public GrokEvaluatorExtracter( + final Grok parser, + final String pattern, + final Map keyToBlock, + final Map types + ) { + this.parser = parser; + this.pattern = pattern; + this.valuesSet = new boolean[types.size()]; + this.firstValues = new Object[types.size()]; + this.positionToType = new ElementType[types.size()]; + + fieldExtracters = new ArrayList<>(parser.captureConfig().size()); + for (GrokCaptureConfig config : parser.captureConfig()) { + var key = config.name(); + ElementType type = types.get(key); + Integer blockIdx = keyToBlock.get(key); + positionToType[blockIdx] = type; + + fieldExtracters.add(config.nativeExtracter(new GrokCaptureConfig.NativeExtracterMap<>() { + @Override + public GrokCaptureExtracter forString(Function, GrokCaptureExtracter> buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + BytesRefBlock.Builder block = (BytesRefBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendBytesRef(new BytesRef((String) firstValues[blockIdx])); + valuesSet[blockIdx] = true; + } + block.appendBytesRef(new BytesRef(value)); + } + }); + } + + @Override + public GrokCaptureExtracter forInt(Function buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + IntBlock.Builder block = (IntBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendInt((int) firstValues[blockIdx]); + valuesSet[blockIdx] = true; + } + block.appendInt(value); + } + }); + } + + @Override + public GrokCaptureExtracter forLong(Function buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + LongBlock.Builder block = (LongBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendLong((long) firstValues[blockIdx]); + valuesSet[blockIdx] = true; + } + block.appendLong(value); + } + }); + } + + @Override + public GrokCaptureExtracter forFloat(Function buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + DoubleBlock.Builder block = (DoubleBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendDouble(((Float) firstValues[blockIdx]).doubleValue()); + valuesSet[blockIdx] = true; + } + block.appendDouble(value); + } + }); + } + + @Override + public GrokCaptureExtracter forDouble(Function buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + DoubleBlock.Builder block = (DoubleBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendDouble((double) firstValues[blockIdx]); + valuesSet[blockIdx] = true; + } + block.appendDouble(value); + } + }); + } + + @Override + public GrokCaptureExtracter forBoolean(Function, GrokCaptureExtracter> buildExtracter) { + return buildExtracter.apply(value -> { + if (firstValues[blockIdx] == null) { + firstValues[blockIdx] = value; + } else { + BooleanBlock.Builder block = (BooleanBlock.Builder) blocks()[blockIdx]; + if (valuesSet[blockIdx] == false) { + block.beginPositionEntry(); + block.appendBoolean((boolean) firstValues[blockIdx]); + valuesSet[blockIdx] = true; + } + block.appendBoolean(value); + } + }); + } + })); + } + + } + + private static void append(Object value, Block.Builder block, ElementType type) { + if (value instanceof Float f) { + // Grok patterns can produce float values (Eg. %{WORD:x:float}) + // Since ESQL does not support floats natively, but promotes them to Double, we are doing promotion here + // TODO remove when floats are supported + ((DoubleBlock.Builder) block).appendDouble(f.doubleValue()); + } else { + BlockUtils.appendValue(block, value, type); + } + } + + public Block.Builder[] blocks() { + return blocks; + } + + @Override + public void computeRow(BytesRefBlock inputBlock, int row, Block.Builder[] blocks, BytesRef spare) { + this.blocks = blocks; + int position = inputBlock.getFirstValueIndex(row); + int valueCount = inputBlock.getValueCount(row); + Arrays.fill(valuesSet, false); + Arrays.fill(firstValues, null); + for (int c = 0; c < valueCount; c++) { + BytesRef input = inputBlock.getBytesRef(position + c, spare); + parser.match(input.bytes, input.offset, input.length, this); + } + for (int i = 0; i < firstValues.length; i++) { + if (firstValues[i] == null) { + this.blocks[i].appendNull(); + } else if (valuesSet[i]) { + this.blocks[i].endPositionEntry(); + } else { + append(firstValues[i], blocks[i], positionToType[i]); + } + } + } + + @Override + public void extract(byte[] utf8Bytes, int offset, Region region) { + fieldExtracters.forEach(extracter -> extracter.extract(utf8Bytes, offset, region)); + } + + @Override + public String toString() { + return "GrokEvaluatorExtracter[pattern=" + pattern + "]"; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/InMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/InMapper.java new file mode 100644 index 0000000000000..03a92ddeb7bed --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/InMapper.java @@ -0,0 +1,112 @@ +/* + * 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.planner; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanArrayBlock; +import org.elasticsearch.compute.data.BooleanArrayVector; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BooleanVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.planner.ComparisonMapper.EQUALS; + +class InMapper extends EvalMapper.ExpressionMapper { + + public static final InMapper IN_MAPPER = new InMapper(); + + private InMapper() {} + + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Override + protected Supplier map(In in, Layout layout) { + List> listEvaluators = new ArrayList<>(in.list().size()); + in.list().forEach(e -> { + Equals eq = new Equals(in.source(), in.value(), e); + Supplier eqEvaluator = ((EvalMapper.ExpressionMapper) EQUALS).map(eq, layout); + listEvaluators.add(eqEvaluator); + }); + return () -> new InExpressionEvaluator(listEvaluators.stream().map(Supplier::get).toList()); + } + + record InExpressionEvaluator(List listEvaluators) implements EvalOperator.ExpressionEvaluator { + @Override + public Block eval(Page page) { + int positionCount = page.getPositionCount(); + boolean[] values = new boolean[positionCount]; + BitSet nulls = new BitSet(positionCount); // at least one evaluation resulted in NULL on a row + boolean nullInValues = false; // set when NULL's added in the values list: `field IN (valueA, null, valueB)` + + for (int i = 0; i < listEvaluators().size(); i++) { + var evaluator = listEvaluators.get(i); + Block block = evaluator.eval(page); + + Vector vector = block.asVector(); + if (vector != null) { + updateValues((BooleanVector) vector, values); + } else { + if (block.areAllValuesNull()) { + nullInValues = true; + } else { + updateValues((BooleanBlock) block, values, nulls); + } + } + } + + return evalWithNulls(values, nulls, nullInValues); + } + + private static void updateValues(BooleanVector vector, boolean[] values) { + for (int p = 0; p < values.length; p++) { + values[p] |= vector.getBoolean(p); + } + } + + private static void updateValues(BooleanBlock block, boolean[] values, BitSet nulls) { + for (int p = 0; p < values.length; p++) { + if (block.isNull(p)) { + nulls.set(p); + } else { + int start = block.getFirstValueIndex(p); + int end = start + block.getValueCount(p); + for (int i = start; i < end; i++) { // if MV_ANY is true, evaluation is true + if (block.getBoolean(i)) { + values[p] = true; + break; + } + } + } + } + } + + private static Block evalWithNulls(boolean[] values, BitSet nulls, boolean nullInValues) { + if (nulls.isEmpty() && nullInValues == false) { + return new BooleanArrayVector(values, values.length).asBlock(); + } else { + // 3VL: true trumps null; null trumps false. + for (int i = 0; i < values.length; i++) { + if (values[i]) { + nulls.clear(i); + } else if (nullInValues) { + nulls.set(i); + } // else: leave nulls as is + } + return new BooleanArrayBlock(values, values.length, null, nulls, Block.MvOrdering.UNORDERED); + } + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Layout.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Layout.java new file mode 100644 index 0000000000000..af7c94f45310f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Layout.java @@ -0,0 +1,136 @@ +/* + * 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.planner; + +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.NamedExpression; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Maintains the mapping from attribute ids to channels (block index). + * + * An attribute can only be mapped to exactly one channel but one channel can be mapped to multiple attributes. + */ +public class Layout { + + private final Map layout; + private final int numberOfChannels; + + Layout(Map layout, int numberOfChannels) { + this.layout = layout; + this.numberOfChannels = numberOfChannels; + } + + /** + * @param id the attribute id + * @return the channel to which the specific attribute id is mapped or `null` if the attribute id does not exist in the layout. + */ + public Integer getChannel(NameId id) { + return layout.get(id); + } + + /** + * @return the total number of ids in the layout. + */ + public int numberOfIds() { + return layout.size(); + } + + /** + * @return the total number of channels in the layout. + */ + public int numberOfChannels() { + return numberOfChannels; + } + + Map internalLayout() { + return layout; + } + + /** + * @return creates a builder to append to this layout. + */ + public Layout.Builder builder() { + return new Layout.Builder(this); + } + + @Override + public String toString() { + return "BlockLayout{" + "layout=" + layout + ", numberOfChannels=" + numberOfChannels + '}'; + } + + /** + * Builder class for Layout. The builder ensures that layouts cannot be altered after creation (through references to the underlying + * map). + */ + public static class Builder { + + private final List> channels; + + public Builder() { + this.channels = new ArrayList<>(); + } + + private Builder(Layout layout) { + channels = IntStream.range(0, layout.numberOfChannels).>mapToObj(i -> new HashSet<>()).collect(Collectors.toList()); + for (Map.Entry entry : layout.layout.entrySet()) { + channels.get(entry.getValue()).add(entry.getKey()); + } + } + + /** + * Appends a new channel to the layout. The channel is mapped to a single attribute id. + * @param id the attribute id + */ + public Builder appendChannel(NameId id) { + channels.add(Set.of(id)); + return this; + } + + /** + * Appends a new channel to the layout. The channel is mapped to one or more attribute ids. + * @param ids the attribute ids + */ + public Builder appendChannel(Set ids) { + if (ids.size() < 1) { + throw new IllegalArgumentException("Channel must be mapped to at least one id."); + } + channels.add(ids); + return this; + } + + public Builder appendChannels(Collection attributes) { + for (var attribute : attributes) { + appendChannel(attribute.id()); + } + return this; + } + + public Layout build() { + Map layout = new HashMap<>(); + int numberOfChannels = 0; + for (Set ids : this.channels) { + int channel = numberOfChannels++; + for (NameId id : ids) { + layout.putIfAbsent(id, channel); + } + } + return new Layout(Collections.unmodifiableMap(layout), numberOfChannels); + } + } +} 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 new file mode 100644 index 0000000000000..db7380019096d --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java @@ -0,0 +1,743 @@ +/* + * 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.planner; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.iterable.Iterables; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.DataPartitioning; +import org.elasticsearch.compute.operator.ColumnExtractOperator; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.EvalOperator.EvalOperatorFactory; +import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator; +import org.elasticsearch.compute.operator.FilterOperator.FilterOperatorFactory; +import org.elasticsearch.compute.operator.LocalSourceOperator; +import org.elasticsearch.compute.operator.LocalSourceOperator.LocalSourceFactory; +import org.elasticsearch.compute.operator.MvExpandOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.Operator.OperatorFactory; +import org.elasticsearch.compute.operator.OutputOperator.OutputOperatorFactory; +import org.elasticsearch.compute.operator.RowOperator.RowOperatorFactory; +import org.elasticsearch.compute.operator.ShowOperator; +import org.elasticsearch.compute.operator.SinkOperator; +import org.elasticsearch.compute.operator.SinkOperator.SinkOperatorFactory; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.SourceOperator.SourceOperatorFactory; +import org.elasticsearch.compute.operator.StringExtractOperator; +import org.elasticsearch.compute.operator.TopNOperator; +import org.elasticsearch.compute.operator.TopNOperator.TopNOperatorFactory; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkOperator.ExchangeSinkOperatorFactory; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceOperator.ExchangeSourceOperatorFactory; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupOperator; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupService; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.DissectExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.GrokExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.MvExpandExec; +import org.elasticsearch.xpack.esql.plan.physical.OutputExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.RowExec; +import org.elasticsearch.xpack.esql.plan.physical.ShowExec; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.util.Holder; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Stream; + +import static java.util.stream.Collectors.joining; +import static org.elasticsearch.compute.operator.LimitOperator.Factory; +import static org.elasticsearch.compute.operator.ProjectOperator.ProjectOperatorFactory; + +/** + * The local execution planner takes a plan (represented as PlanNode tree / digraph) as input and creates the corresponding + * drivers that are used to execute the given plan. + */ +public class LocalExecutionPlanner { + + private final String sessionId; + private final CancellableTask parentTask; + private final BigArrays bigArrays; + private final EsqlConfiguration configuration; + private final ExchangeSourceHandler exchangeSourceHandler; + private final ExchangeSinkHandler exchangeSinkHandler; + private final EnrichLookupService enrichLookupService; + private final PhysicalOperationProviders physicalOperationProviders; + + public LocalExecutionPlanner( + String sessionId, + CancellableTask parentTask, + BigArrays bigArrays, + EsqlConfiguration configuration, + ExchangeSourceHandler exchangeSourceHandler, + ExchangeSinkHandler exchangeSinkHandler, + EnrichLookupService enrichLookupService, + PhysicalOperationProviders physicalOperationProviders + ) { + this.sessionId = sessionId; + this.parentTask = parentTask; + this.bigArrays = bigArrays; + this.exchangeSourceHandler = exchangeSourceHandler; + this.exchangeSinkHandler = exchangeSinkHandler; + this.enrichLookupService = enrichLookupService; + this.physicalOperationProviders = physicalOperationProviders; + this.configuration = configuration; + } + + /** + * turn the given plan into a list of drivers to execute + */ + public LocalExecutionPlan plan(PhysicalPlan node) { + var context = new LocalExecutionPlannerContext( + new ArrayList<>(), + new Holder<>(DriverParallelism.SINGLE), + configuration.pragmas().taskConcurrency(), + configuration.pragmas().dataPartitioning(), + configuration.pragmas().pageSize(), + bigArrays + ); + + PhysicalOperation physicalOperation = plan(node, context); + + context.addDriverFactory( + new DriverFactory(new DriverSupplier(context.bigArrays, physicalOperation), context.driverParallelism().get()) + ); + + return new LocalExecutionPlan(context.driverFactories); + } + + private PhysicalOperation plan(PhysicalPlan node, LocalExecutionPlannerContext context) { + if (node instanceof AggregateExec aggregate) { + return planAggregation(aggregate, context); + } else if (node instanceof FieldExtractExec fieldExtractExec) { + return planFieldExtractNode(context, fieldExtractExec); + } else if (node instanceof ExchangeExec exchangeExec) { + return planExchange(exchangeExec, context); + } else if (node instanceof TopNExec topNExec) { + return planTopN(topNExec, context); + } else if (node instanceof EvalExec eval) { + return planEval(eval, context); + } else if (node instanceof DissectExec dissect) { + return planDissect(dissect, context); + } else if (node instanceof GrokExec grok) { + return planGrok(grok, context); + } else if (node instanceof ProjectExec project) { + return planProject(project, context); + } else if (node instanceof FilterExec filter) { + return planFilter(filter, context); + } else if (node instanceof LimitExec limit) { + return planLimit(limit, context); + } else if (node instanceof MvExpandExec mvExpand) { + return planMvExpand(mvExpand, context); + } + // source nodes + else if (node instanceof EsQueryExec esQuery) { + return planEsQueryNode(esQuery, context); + } else if (node instanceof RowExec row) { + return planRow(row, context); + } else if (node instanceof LocalSourceExec localSource) { + return planLocal(localSource, context); + } else if (node instanceof ShowExec show) { + return planShow(show); + } else if (node instanceof ExchangeSourceExec exchangeSource) { + return planExchangeSource(exchangeSource, context); + } + // lookups and joins + else if (node instanceof EnrichExec enrich) { + return planEnrich(enrich, context); + } + // output + else if (node instanceof OutputExec outputExec) { + return planOutput(outputExec, context); + } else if (node instanceof ExchangeSinkExec exchangeSink) { + return planExchangeSink(exchangeSink, context); + } + + throw new UnsupportedOperationException(node.nodeName()); + } + + private PhysicalOperation planAggregation(AggregateExec aggregate, LocalExecutionPlannerContext context) { + var source = plan(aggregate.child(), context); + return physicalOperationProviders.groupingPhysicalOperation(aggregate, source, context); + } + + private PhysicalOperation planEsQueryNode(EsQueryExec esQuery, LocalExecutionPlannerContext context) { + if (esQuery.query() == null) { + esQuery = new EsQueryExec( + esQuery.source(), + esQuery.index(), + esQuery.output(), + new MatchAllQueryBuilder(), + esQuery.limit(), + esQuery.sorts(), + esQuery.estimatedRowSize() + ); + } + return physicalOperationProviders.sourcePhysicalOperation(esQuery, context); + } + + private PhysicalOperation planFieldExtractNode(LocalExecutionPlannerContext context, FieldExtractExec fieldExtractExec) { + return physicalOperationProviders.fieldExtractPhysicalOperation(fieldExtractExec, plan(fieldExtractExec.child(), context)); + } + + /** + * Map QL's {@link DataType} to the compute engine's {@link ElementType}. + */ + public static ElementType toElementType(DataType dataType) { + if (dataType == DataTypes.LONG || dataType == DataTypes.DATETIME || dataType == DataTypes.UNSIGNED_LONG) { + return ElementType.LONG; + } + if (dataType == DataTypes.INTEGER) { + return ElementType.INT; + } + if (dataType == DataTypes.DOUBLE) { + return ElementType.DOUBLE; + } + // unsupported fields are passed through as a BytesRef + if (dataType == DataTypes.KEYWORD + || dataType == DataTypes.TEXT + || dataType == DataTypes.IP + || dataType == DataTypes.VERSION + || dataType == DataTypes.UNSUPPORTED) { + return ElementType.BYTES_REF; + } + if (dataType == DataTypes.NULL) { + return ElementType.NULL; + } + if (dataType == DataTypes.BOOLEAN) { + return ElementType.BOOLEAN; + } + throw new UnsupportedOperationException("unsupported data type [" + dataType + "]"); + } + + private PhysicalOperation planOutput(OutputExec outputExec, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(outputExec.child(), context); + var output = outputExec.output(); + + return source.withSink( + new OutputOperatorFactory( + Expressions.names(output), + alignPageToAttributes(output, source.layout), + outputExec.getPageConsumer() + ), + source.layout + ); + } + + private static Function alignPageToAttributes(List attrs, Layout layout) { + // align the page layout with the operator output + // extraction order - the list ordinal is the same as the column one + // while the value represents the position in the original page + final int[] mappedPosition = new int[attrs.size()]; + int index = -1; + boolean transformRequired = false; + for (var attribute : attrs) { + mappedPosition[++index] = layout.getChannel(attribute.id()); + transformRequired |= mappedPosition[index] != index; + } + Function transformer = transformRequired ? p -> { + var blocks = new Block[mappedPosition.length]; + for (int i = 0; i < blocks.length; i++) { + blocks[i] = p.getBlock(mappedPosition[i]); + } + return new Page(blocks); + } : Function.identity(); + + return transformer; + } + + private PhysicalOperation planExchange(ExchangeExec exchangeExec, LocalExecutionPlannerContext context) { + throw new EsqlIllegalArgumentException("Exchange needs to be replaced with a sink/source"); + } + + private PhysicalOperation planExchangeSink(ExchangeSinkExec exchangeSink, LocalExecutionPlannerContext context) { + Objects.requireNonNull(exchangeSinkHandler, "ExchangeSinkHandler wasn't provided"); + PhysicalOperation source = plan(exchangeSink.child(), context); + + Function transformer = exchangeSink.child() instanceof AggregateExec + ? Function.identity() + : alignPageToAttributes(exchangeSink.output(), source.layout); + + return source.withSink(new ExchangeSinkOperatorFactory(exchangeSinkHandler::createExchangeSink, transformer), source.layout); + } + + private PhysicalOperation planExchangeSource(ExchangeSourceExec exchangeSource, LocalExecutionPlannerContext context) { + Objects.requireNonNull(exchangeSourceHandler, "ExchangeSourceHandler wasn't provided"); + + var builder = new Layout.Builder(); + for (var attr : exchangeSource.output()) { + builder.appendChannel(attr.id()); + } + // decorate the layout + var l = builder.build(); + var layout = exchangeSource.isIntermediateAgg() ? new ExchangeLayout(l) : l; + + return PhysicalOperation.fromSource(new ExchangeSourceOperatorFactory(exchangeSourceHandler::createExchangeSource), layout); + } + + private PhysicalOperation planTopN(TopNExec topNExec, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(topNExec.child(), context); + + List orders = topNExec.order().stream().map(order -> { + int sortByChannel; + if (order.child() instanceof Attribute a) { + sortByChannel = source.layout.getChannel(a.id()); + } else { + throw new UnsupportedOperationException(); + } + + return new TopNOperator.SortOrder( + sortByChannel, + order.direction().equals(Order.OrderDirection.ASC), + order.nullsPosition().equals(Order.NullsPosition.FIRST) + ); + }).toList(); + + int limit; + if (topNExec.limit() instanceof Literal literal) { + limit = Integer.parseInt(literal.value().toString()); + } else { + throw new UnsupportedOperationException(); + } + + // TODO Replace page size with passing estimatedRowSize down + /* + * The 2000 below is a hack to account for incoming size and to make + * sure the estimated row size is never 0 which'd cause a divide by 0. + * But we should replace this with passing the estimate into the real + * topn and letting it actually measure the size of rows it produces. + * That'll be more accurate. And we don't have a path for estimating + * incoming rows. And we don't need one because we can estimate. + */ + return source.with(new TopNOperatorFactory(limit, orders, context.pageSize(2000 + topNExec.estimatedRowSize())), source.layout); + } + + private PhysicalOperation planEval(EvalExec eval, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(eval.child(), context); + + for (NamedExpression namedExpression : eval.fields()) { + Supplier evaluatorSupplier; + if (namedExpression instanceof Alias alias) { + evaluatorSupplier = EvalMapper.toEvaluator(alias.child(), source.layout); + } else { + throw new UnsupportedOperationException(); + } + Layout.Builder layout = source.layout.builder(); + layout.appendChannel(namedExpression.toAttribute().id()); + source = source.with(new EvalOperatorFactory(evaluatorSupplier), layout.build()); + } + return source; + } + + private PhysicalOperation planDissect(DissectExec dissect, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(dissect.child(), context); + Layout.Builder layoutBuilder = source.layout.builder(); + for (Attribute attr : dissect.extractedFields()) { + layoutBuilder.appendChannel(attr.id()); + } + final Expression expr = dissect.inputExpression(); + String[] attributeNames = Expressions.names(dissect.extractedFields()).toArray(new String[0]); + + Layout layout = layoutBuilder.build(); + source = source.with( + new StringExtractOperator.StringExtractOperatorFactory( + attributeNames, + EvalMapper.toEvaluator(expr, layout), + () -> (input) -> dissect.parser().parser().parse(input) + ), + layout + ); + return source; + } + + private PhysicalOperation planGrok(GrokExec grok, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(grok.child(), context); + Layout.Builder layoutBuilder = source.layout.builder(); + List extractedFields = grok.extractedFields(); + for (Attribute attr : extractedFields) { + layoutBuilder.appendChannel(attr.id()); + } + + Map fieldToPos = new HashMap<>(extractedFields.size()); + Map fieldToType = new HashMap<>(extractedFields.size()); + ElementType[] types = new ElementType[extractedFields.size()]; + for (int i = 0; i < extractedFields.size(); i++) { + Attribute extractedField = extractedFields.get(i); + ElementType type = toElementType(extractedField.dataType()); + fieldToPos.put(extractedField.name(), i); + fieldToType.put(extractedField.name(), type); + types[i] = type; + } + + Layout layout = layoutBuilder.build(); + source = source.with( + new ColumnExtractOperator.Factory( + types, + EvalMapper.toEvaluator(grok.inputExpression(), layout), + () -> new GrokEvaluatorExtracter(grok.pattern().grok(), grok.pattern().pattern(), fieldToPos, fieldToType) + ), + layout + ); + return source; + } + + private PhysicalOperation planEnrich(EnrichExec enrich, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(enrich.child(), context); + Layout.Builder layoutBuilder = source.layout.builder(); + List extractedFields = enrich.enrichFields(); + for (NamedExpression attr : extractedFields) { + layoutBuilder.appendChannel(attr.id()); + } + Layout layout = layoutBuilder.build(); + Set indices = enrich.enrichIndex().concreteIndices(); + if (indices.size() != 1) { + throw new EsqlIllegalArgumentException("Resolved enrich should have one concrete index; got " + indices); + } + String enrichIndex = Iterables.get(indices, 0); + return source.with( + new EnrichLookupOperator.Factory( + sessionId, + parentTask, + 1, // TODO: Add a concurrent setting for enrich - also support unordered mode + source.layout.getChannel(enrich.matchField().id()), + enrichLookupService, + enrichIndex, + "match", // TODO: enrich should also resolve the match_type + enrich.policyMatchField(), + enrich.enrichFields() + ), + layout + ); + } + + private Supplier toEvaluator(Expression exp, Layout layout) { + return EvalMapper.toEvaluator(exp, layout); + } + + private PhysicalOperation planRow(RowExec row, LocalExecutionPlannerContext context) { + List obj = row.fields().stream().map(f -> { + if (f instanceof Alias) { + return ((Alias) f).child().fold(); + } else { + return f.fold(); + } + }).toList(); + Layout.Builder layout = new Layout.Builder(); + var output = row.output(); + for (Attribute attribute : output) { + layout.appendChannel(attribute.id()); + } + return PhysicalOperation.fromSource(new RowOperatorFactory(obj), layout.build()); + } + + private PhysicalOperation planLocal(LocalSourceExec localSourceExec, LocalExecutionPlannerContext context) { + + Layout.Builder layout = new Layout.Builder(); + var output = localSourceExec.output(); + for (Attribute attribute : output) { + layout.appendChannel(attribute.id()); + } + LocalSourceOperator.BlockSupplier supplier = () -> localSourceExec.supplier().get(); + var operator = new LocalSourceOperator(supplier); + return PhysicalOperation.fromSource(new LocalSourceFactory(() -> operator), layout.build()); + } + + private PhysicalOperation planShow(ShowExec showExec) { + Layout.Builder layout = new Layout.Builder(); + for (var attribute : showExec.output()) { + layout.appendChannel(attribute.id()); + } + return PhysicalOperation.fromSource(new ShowOperator.ShowOperatorFactory(showExec.values()), layout.build()); + } + + private PhysicalOperation planProject(ProjectExec project, LocalExecutionPlannerContext context) { + var source = plan(project.child(), context); + + Map> inputChannelToOutputIds = new HashMap<>(); + for (NamedExpression ne : project.projections()) { + NameId inputId; + if (ne instanceof Alias a) { + inputId = ((NamedExpression) a.child()).id(); + } else { + inputId = ne.id(); + } + int inputChannel = source.layout.getChannel(inputId); + inputChannelToOutputIds.computeIfAbsent(inputChannel, ignore -> new HashSet<>()).add(ne.id()); + } + + BitSet mask = new BitSet(); + Layout.Builder layout = new Layout.Builder(); + + for (int inChannel = 0; inChannel < source.layout.numberOfChannels(); inChannel++) { + Set outputIds = inputChannelToOutputIds.get(inChannel); + + if (outputIds != null) { + mask.set(inChannel); + layout.appendChannel(outputIds); + } + } + + if (mask.cardinality() == source.layout.numberOfChannels()) { + // all columns are retained, project operator is not needed but the layout needs to be updated + return source.with(layout.build()); + } else { + return source.with(new ProjectOperatorFactory(mask), layout.build()); + } + } + + private PhysicalOperation planFilter(FilterExec filter, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(filter.child(), context); + // TODO: should this be extracted into a separate eval block? + return source.with(new FilterOperatorFactory(toEvaluator(filter.condition(), source.layout)), source.layout); + } + + private PhysicalOperation planLimit(LimitExec limit, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(limit.child(), context); + return source.with(new Factory((Integer) limit.limit().fold()), source.layout); + } + + private PhysicalOperation planMvExpand(MvExpandExec mvExpandExec, LocalExecutionPlannerContext context) { + PhysicalOperation source = plan(mvExpandExec.child(), context); + return source.with(new MvExpandOperator.Factory(source.layout.getChannel(mvExpandExec.target().id())), source.layout); + } + + /** + * Immutable physical operation. + */ + public static class PhysicalOperation implements Describable { + final SourceOperatorFactory sourceOperatorFactory; + final List intermediateOperatorFactories; + final SinkOperatorFactory sinkOperatorFactory; + + final Layout layout; // maps field names to channels + + /** Creates a new physical operation with the given source and layout. */ + static PhysicalOperation fromSource(SourceOperatorFactory sourceOperatorFactory, Layout layout) { + return new PhysicalOperation(sourceOperatorFactory, layout); + } + + /** Creates a new physical operation from this operation with the given layout. */ + PhysicalOperation with(Layout layout) { + return new PhysicalOperation(this, Optional.empty(), Optional.empty(), layout); + } + + /** Creates a new physical operation from this operation with the given intermediate operator and layout. */ + PhysicalOperation with(OperatorFactory operatorFactory, Layout layout) { + return new PhysicalOperation(this, Optional.of(operatorFactory), Optional.empty(), layout); + } + + /** Creates a new physical operation from this operation with the given sink and layout. */ + PhysicalOperation withSink(SinkOperatorFactory sink, Layout layout) { + return new PhysicalOperation(this, Optional.empty(), Optional.of(sink), layout); + } + + private PhysicalOperation(SourceOperatorFactory sourceOperatorFactory, Layout layout) { + this.sourceOperatorFactory = sourceOperatorFactory; + this.intermediateOperatorFactories = List.of(); + this.sinkOperatorFactory = null; + this.layout = layout; + } + + private PhysicalOperation( + PhysicalOperation physicalOperation, + Optional intermediateOperatorFactory, + Optional sinkOperatorFactory, + Layout layout + ) { + sourceOperatorFactory = physicalOperation.sourceOperatorFactory; + intermediateOperatorFactories = new ArrayList<>(); + intermediateOperatorFactories.addAll(physicalOperation.intermediateOperatorFactories); + intermediateOperatorFactory.ifPresent(intermediateOperatorFactories::add); + this.sinkOperatorFactory = sinkOperatorFactory.isPresent() ? sinkOperatorFactory.get() : null; + this.layout = layout; + } + + public SourceOperator source(DriverContext driverContext) { + return sourceOperatorFactory.get(driverContext); + } + + public void operators(List operators, DriverContext driverContext) { + intermediateOperatorFactories.stream().map(opFactory -> opFactory.get(driverContext)).forEach(operators::add); + } + + public SinkOperator sink(DriverContext driverContext) { + return sinkOperatorFactory.get(driverContext); + } + + @Override + public String describe() { + return Stream.concat( + Stream.concat(Stream.of(sourceOperatorFactory), intermediateOperatorFactories.stream()), + Stream.of(sinkOperatorFactory) + ).map(Describable::describe).collect(joining("\n\\_", "\\_", "")); + } + } + + /** + * The count and type of driver parallelism. + */ + record DriverParallelism(Type type, int instanceCount) { + + DriverParallelism { + if (instanceCount <= 0) { + throw new IllegalArgumentException("instance count must be greater than zero; got: " + instanceCount); + } + } + + static final DriverParallelism SINGLE = new DriverParallelism(Type.SINGLETON, 1); + + enum Type { + SINGLETON, + DATA_PARALLELISM, + TASK_LEVEL_PARALLELISM + } + } + + /** + * Context object used while generating a local plan. Currently only collects the driver factories as well as + * maintains information how many driver instances should be created for a given driver. + */ + public record LocalExecutionPlannerContext( + List driverFactories, + Holder driverParallelism, + int taskConcurrency, + DataPartitioning dataPartitioning, + int configuredPageSize, + BigArrays bigArrays + ) { + void addDriverFactory(DriverFactory driverFactory) { + driverFactories.add(driverFactory); + } + + void driverParallelism(DriverParallelism parallelism) { + driverParallelism.set(parallelism); + } + + int pageSize(Integer estimatedRowSize) { + if (estimatedRowSize == null) { + throw new IllegalStateException("estimated row size hasn't been set"); + } + if (estimatedRowSize == 0) { + throw new IllegalStateException("estimated row size can't be 0"); + } + if (configuredPageSize != 0) { + return configuredPageSize; + } + return Math.max(SourceOperator.MIN_TARGET_PAGE_SIZE, SourceOperator.TARGET_PAGE_SIZE / estimatedRowSize); + } + } + + record DriverSupplier(BigArrays bigArrays, PhysicalOperation physicalOperation) implements Function, Describable { + + @Override + public Driver apply(String sessionId) { + SourceOperator source = null; + List operators = new ArrayList<>(); + SinkOperator sink = null; + boolean success = false; + var driverContext = new DriverContext(); + try { + source = physicalOperation.source(driverContext); + physicalOperation.operators(operators, driverContext); + sink = physicalOperation.sink(driverContext); + success = true; + return new Driver(sessionId, driverContext, physicalOperation::describe, source, operators, sink, () -> {}); + } finally { + if (false == success) { + Releasables.close(source, () -> Releasables.close(operators), sink); + } + } + } + + @Override + public String describe() { + return physicalOperation.describe(); + } + } + + record DriverFactory(DriverSupplier driverSupplier, DriverParallelism driverParallelism) implements Describable { + @Override + public String describe() { + return "DriverFactory(instances = " + + driverParallelism.instanceCount() + + ", type = " + + driverParallelism.type() + + ")\n" + + driverSupplier.describe(); + } + } + + /** + * Plan representation that is geared towards execution on a single node + */ + public static class LocalExecutionPlan implements Describable { + final List driverFactories; + + LocalExecutionPlan(List driverFactories) { + this.driverFactories = driverFactories; + } + + public List createDrivers(String sessionId) { + List drivers = new ArrayList<>(); + for (DriverFactory df : driverFactories) { + for (int i = 0; i < df.driverParallelism.instanceCount; i++) { + drivers.add(df.driverSupplier.apply(sessionId)); + } + } + return drivers; + } + + @Override + public String describe() { + StringBuilder sb = new StringBuilder(); + sb.append(driverFactories.stream().map(DriverFactory::describe).collect(joining("\n"))); + return sb.toString(); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mappable.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mappable.java new file mode 100644 index 0000000000000..67c8c9a29ea7a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mappable.java @@ -0,0 +1,35 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner; + +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.Expression; + +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.fromArrayRow; +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; + +/** + * Expressions that have a mapping to an {@link EvalOperator.ExpressionEvaluator}. + */ +public interface Mappable { + Supplier toEvaluator(Function> toEvaluator); + + /** + * Fold using {@link #toEvaluator} so you don't need a "by hand" + * implementation of fold. The evaluator that it makes is "funny" + * in that it'll always call {@link Expression#fold}, but that's + * good enough. + */ + default Object fold() { + return toJavaObject(toEvaluator(e -> () -> p -> fromArrayRow(e.fold())[0]).get().eval(new Page(1)), 0); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mapper.java new file mode 100644 index 0000000000000..af3ba47e14bd4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/Mapper.java @@ -0,0 +1,233 @@ +/* + * 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.planner; + +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.MvExpand; +import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.esql.plan.logical.TopN; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.plan.logical.show.ShowFunctions; +import org.elasticsearch.xpack.esql.plan.logical.show.ShowInfo; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.DissectExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.GrokExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.MvExpandExec; +import org.elasticsearch.xpack.esql.plan.physical.OrderExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.RowExec; +import org.elasticsearch.xpack.esql.plan.physical.ShowExec; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan; + +import static org.elasticsearch.xpack.esql.plan.physical.AggregateExec.Mode; +import static org.elasticsearch.xpack.esql.plan.physical.AggregateExec.Mode.FINAL; +import static org.elasticsearch.xpack.esql.plan.physical.AggregateExec.Mode.PARTIAL; + +public class Mapper { + + private final FunctionRegistry functionRegistry; + private final boolean localMode; + + public Mapper(FunctionRegistry functionRegistry) { + this.functionRegistry = functionRegistry; + localMode = false; + } + + public Mapper(boolean localMode) { + this.functionRegistry = null; + this.localMode = localMode; + } + + public PhysicalPlan map(LogicalPlan p) { + // + // Leaf Node + // + + // Source + if (p instanceof EsRelation esRelation) { + return localMode ? new EsSourceExec(esRelation) : new FragmentExec(p); + } + + if (p instanceof Row row) { + return new RowExec(row.source(), row.fields()); + } + + if (p instanceof LocalRelation local) { + return new LocalSourceExec(local.source(), local.output(), local.supplier()); + } + + // Commands + if (p instanceof ShowFunctions showFunctions) { + return new ShowExec(showFunctions.source(), showFunctions.output(), showFunctions.values(functionRegistry)); + } + if (p instanceof ShowInfo showInfo) { + return new ShowExec(showInfo.source(), showInfo.output(), showInfo.values()); + } + + // + // Unary Plan + // + + if (p instanceof UnaryPlan ua) { + var child = map(ua.child()); + PhysicalPlan plan = null; + // in case of a fragment, push to it any current streaming operator + if (child instanceof FragmentExec && isPipelineBreaker(p) == false) { + plan = new FragmentExec(p); + } else { + plan = map(ua, child); + } + return plan; + } + + throw new UnsupportedOperationException(p.nodeName()); + } + + private static boolean isPipelineBreaker(LogicalPlan p) { + return p instanceof Aggregate || p instanceof TopN || p instanceof Limit || p instanceof OrderBy; + } + + private PhysicalPlan map(UnaryPlan p, PhysicalPlan child) { + // + // Pipeline operators + // + if (p instanceof Filter f) { + return new FilterExec(f.source(), child, f.condition()); + } + + if (p instanceof Project pj) { + return new ProjectExec(pj.source(), child, pj.projections()); + } + + if (p instanceof Eval eval) { + return new EvalExec(eval.source(), child, eval.fields()); + } + + if (p instanceof Dissect dissect) { + return new DissectExec(dissect.source(), child, dissect.input(), dissect.parser(), dissect.extractedFields()); + } + + if (p instanceof Grok grok) { + return new GrokExec(grok.source(), child, grok.input(), grok.parser(), grok.extractedFields()); + } + + if (p instanceof Enrich enrich) { + return new EnrichExec( + enrich.source(), + child, + enrich.matchField(), + enrich.policy().policyName(), + enrich.policy().policy().getMatchField(), + enrich.policy().index().get(), + enrich.enrichFields() + ); + } + + if (p instanceof MvExpand mvExpand) { + return new MvExpandExec(mvExpand.source(), map(mvExpand.child()), mvExpand.target()); + } + + // + // Pipeline breakers + // + if (p instanceof Limit limit) { + return map(limit, child); + } + + if (p instanceof OrderBy o) { + return map(o, child); + } + + if (p instanceof TopN topN) { + return map(topN, child); + } + + if (p instanceof Aggregate aggregate) { + return map(aggregate, child); + } + + throw new UnsupportedOperationException(p.nodeName()); + } + + private PhysicalPlan map(Aggregate aggregate, PhysicalPlan child) { + // in local mode the only aggregate that can appear is the partial side under an exchange + if (localMode) { + child = aggExec(aggregate, child, PARTIAL); + } + // otherwise create both sides of the aggregate (for parallelism purposes), if no fragment is present + // TODO: might be easier long term to end up with just one node and split if necessary instead of doing that always at this stage + else { + child = addExchangeForFragment(aggregate, child); + // exchange was added - use the intermediates for the output + if (child instanceof ExchangeExec exchange) { + var output = AbstractPhysicalOperationProviders.intermediateAttributes(aggregate.aggregates(), aggregate.groupings()); + child = new ExchangeExec(child.source(), output, true, exchange.child()); + } + // if no exchange was added, create the partial aggregate + else { + child = aggExec(aggregate, child, PARTIAL); + } + + // regardless, always add the final agg + child = aggExec(aggregate, child, FINAL); + } + + return child; + } + + private static AggregateExec aggExec(Aggregate aggregate, PhysicalPlan child, Mode aggMode) { + return new AggregateExec(aggregate.source(), child, aggregate.groupings(), aggregate.aggregates(), aggMode, null); + } + + private PhysicalPlan map(Limit limit, PhysicalPlan child) { + child = addExchangeForFragment(limit, child); + return new LimitExec(limit.source(), child, limit.limit()); + } + + private PhysicalPlan map(OrderBy o, PhysicalPlan child) { + child = addExchangeForFragment(o, child); + return new OrderExec(o.source(), child, o.order()); + } + + private PhysicalPlan map(TopN topN, PhysicalPlan child) { + child = addExchangeForFragment(topN, child); + return new TopNExec(topN.source(), child, topN.order(), topN.limit(), null); + } + + private PhysicalPlan addExchangeForFragment(LogicalPlan logical, PhysicalPlan child) { + // in case of fragment, preserve the streaming operator (order-by, limit or topN) for local replanning + // no need to do it for an aggregate since it gets split + // and clone it as a physical node along with the exchange + if (child instanceof FragmentExec) { + child = new FragmentExec(logical); + child = new ExchangeExec(child.source(), child); + } + return child; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/NegMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/NegMapper.java new file mode 100644 index 0000000000000..5fc46acc99312 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/NegMapper.java @@ -0,0 +1,62 @@ +/* + * 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.planner; + +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; + +abstract class NegMapper extends EvalMapper.ExpressionMapper { + static final EvalMapper.ExpressionMapper NEG_MAPPER = new NegMapper( + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.NegIntsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.NegLongsEvaluator::new, + org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.NegDoublesEvaluator::new + ) { + }; + + private final BiFunction ints; + + private final BiFunction longs; + private final Function doubles; + + private NegMapper( + BiFunction ints, + BiFunction longs, + Function doubles + ) { + this.ints = ints; + this.longs = longs; + this.doubles = doubles; + } + + @Override + protected final Supplier map(Neg neg, Layout layout) { + DataType type = neg.dataType(); + if (type.isNumeric()) { + var childEvaluator = EvalMapper.toEvaluator(neg.field(), layout).get(); + + if (type == DataTypes.INTEGER) { + return () -> ints.apply(neg.source(), childEvaluator); + } + // Unsigned longs are unsupported by choice; negating them would require implicitly converting to long. + if (type == DataTypes.LONG) { + return () -> longs.apply(neg.source(), childEvaluator); + } + if (type == DataTypes.DOUBLE) { + return () -> doubles.apply(childEvaluator); + } + } + throw new UnsupportedOperationException("arithmetic negation operator with unsupported data type [" + type.typeName() + "]"); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalOperationProviders.java new file mode 100644 index 0000000000000..6353005f44ace --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalOperationProviders.java @@ -0,0 +1,26 @@ +/* + * 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.planner; + +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlannerContext; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.PhysicalOperation; + +interface PhysicalOperationProviders { + PhysicalOperation fieldExtractPhysicalOperation(FieldExtractExec fieldExtractExec, PhysicalOperation source); + + PhysicalOperation sourcePhysicalOperation(EsQueryExec esQuery, LocalExecutionPlannerContext context); + + PhysicalOperation groupingPhysicalOperation( + AggregateExec aggregateExec, + PhysicalOperation source, + LocalExecutionPlannerContext context + ); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerificationException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerificationException.java new file mode 100644 index 0000000000000..f303fc5a7e047 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerificationException.java @@ -0,0 +1,26 @@ +/* + * 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.planner; + +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.esql.EsqlClientException; +import org.elasticsearch.xpack.ql.common.Failure; + +import java.util.Collection; + +public class PhysicalVerificationException extends EsqlClientException { + + public PhysicalVerificationException(Collection sources) { + super(Failure.failMessage(sources)); + } + + @Override + public RestStatus status() { + return RestStatus.BAD_REQUEST; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerifier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerifier.java new file mode 100644 index 0000000000000..063a662d23d24 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PhysicalVerifier.java @@ -0,0 +1,45 @@ +/* + * 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.planner; + +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; + +import java.util.Collection; +import java.util.LinkedHashSet; +import java.util.Set; + +import static org.elasticsearch.xpack.ql.common.Failure.fail; + +/** Physical plan verifier. */ +public final class PhysicalVerifier { + + /** Verifies the physical plan. */ + public Collection verify(PhysicalPlan plan) { + Set failures = new LinkedHashSet<>(); + + plan.forEachDown(FieldExtractExec.class, fieldExtractExec -> { + Attribute sourceAttribute = fieldExtractExec.sourceAttribute(); + if (sourceAttribute == null) { + failures.add( + fail( + fieldExtractExec, + "Need to add field extractor for [{}] but cannot detect source attributes from node [{}]", + Expressions.names(fieldExtractExec.attributesToExtract()), + fieldExtractExec.child() + ) + ); + } + }); + + return failures; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java new file mode 100644 index 0000000000000..bd0dc7d3f4e3f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java @@ -0,0 +1,116 @@ +/* + * 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.planner; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.xpack.esql.optimizer.LocalLogicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.util.Holder; + +import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; + +public class PlannerUtils { + + private static final Mapper mapper = new Mapper(true); + + public static Tuple breakPlanBetweenCoordinatorAndDataNode(PhysicalPlan plan, EsqlConfiguration config) { + var dataNodePlan = new Holder(); + + // split the given plan when encountering the exchange + PhysicalPlan coordinatorPlan = plan.transformUp(ExchangeExec.class, e -> { + // remember the datanode subplan and wire it to a sink + var subplan = e.child(); + dataNodePlan.set(new ExchangeSinkExec(e.source(), e.output(), subplan)); + + return new ExchangeSourceExec(e.source(), e.output(), e.isInBetweenAggs()); + }); + return new Tuple<>(coordinatorPlan, dataNodePlan.get()); + } + + /** + * Returns a set of concrete indices after resolving the original indices specified in the FROM command. + */ + public static Set planConcreteIndices(PhysicalPlan plan) { + if (plan == null) { + return Set.of(); + } + var indices = new LinkedHashSet(); + plan.forEachUp(FragmentExec.class, f -> f.fragment().forEachUp(EsRelation.class, r -> indices.addAll(r.index().concreteIndices()))); + return indices; + } + + /** + * Returns the original indices specified in the FROM command of the query. We need the original query to resolve alias filters. + */ + public static String[] planOriginalIndices(PhysicalPlan plan) { + if (plan == null) { + return Strings.EMPTY_ARRAY; + } + var indices = new LinkedHashSet(); + plan.forEachUp( + FragmentExec.class, + f -> f.fragment() + .forEachUp(EsRelation.class, r -> indices.addAll(Arrays.asList(Strings.commaDelimitedListToStringArray(r.index().name())))) + ); + return indices.toArray(String[]::new); + } + + public static PhysicalPlan localPlan(List searchContexts, EsqlConfiguration configuration, PhysicalPlan plan) { + return localPlan(configuration, plan, new SearchStats(searchContexts)); + } + + public static PhysicalPlan localPlan(EsqlConfiguration configuration, PhysicalPlan plan, SearchStats searchStats) { + var isCoordPlan = new Holder<>(Boolean.TRUE); + + final var localOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(configuration, searchStats)); + + var localPhysicalPlan = plan.transformUp(FragmentExec.class, f -> { + isCoordPlan.set(Boolean.FALSE); + var optimizedFragment = localOptimizer.localOptimize(f.fragment()); + var physicalFragment = mapper.map(optimizedFragment); + var filter = f.esFilter(); + if (filter != null) { + physicalFragment = physicalFragment.transformUp( + EsSourceExec.class, + query -> new EsSourceExec(Source.EMPTY, query.index(), query.output(), filter) + ); + } + var optimizer = new LocalPhysicalPlanOptimizer(new LocalPhysicalOptimizerContext(configuration)); + return EstimatesRowSize.estimateRowSize(f.estimatedRowSize(), optimizer.localOptimize(physicalFragment)); + }); + return isCoordPlan.get() ? plan : localPhysicalPlan; + } + + /** + * Extracts the ES query provided by the filter parameter + */ + public static QueryBuilder requestFilter(PhysicalPlan plan) { + var filter = new Holder(null); + plan.forEachDown(FragmentExec.class, es -> filter.set(es.esFilter())); + return filter.get(); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/RegexMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/RegexMapper.java new file mode 100644 index 0000000000000..5ae896562556f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/RegexMapper.java @@ -0,0 +1,27 @@ +/* + * 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.planner; + +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.ql.expression.predicate.regex.AbstractStringPattern; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch; + +import java.util.function.Supplier; + +public abstract class RegexMapper extends EvalMapper.ExpressionMapper> { + static final EvalMapper.ExpressionMapper REGEX_MATCH = new RegexMapper() { + @Override + protected Supplier map(RegexMatch expression, Layout layout) { + return () -> new org.elasticsearch.xpack.esql.expression.predicate.operator.regex.RegexMatchEvaluator( + EvalMapper.toEvaluator(expression.field(), layout).get(), + new CharacterRunAutomaton(((AbstractStringPattern) expression.pattern()).createAutomaton()) + ); + } + }; +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ToAggregator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ToAggregator.java new file mode 100644 index 0000000000000..0f4410e207b52 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/ToAggregator.java @@ -0,0 +1,20 @@ +/* + * 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.planner; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.aggregation.AggregatorFunctionSupplier; + +import java.util.List; + +/** + * Expressions that have a mapping to an {@link AggregatorFunctionSupplier}. + */ +public interface ToAggregator { + AggregatorFunctionSupplier supplier(BigArrays bigArrays, List inputChannels); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java new file mode 100644 index 0000000000000..0b9994b13de37 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -0,0 +1,438 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchShardsAction; +import org.elasticsearch.action.search.SearchShardsGroup; +import org.elasticsearch.action.search.SearchShardsRequest; +import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.ContextPreservingActionListener; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.action.support.RefCountingRunnable; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverTaskRunner; +import org.elasticsearch.compute.operator.exchange.ExchangeResponse; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardNotFoundException; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportRequestHandler; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.EsqlQueryAction; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupService; +import org.elasticsearch.xpack.esql.plan.physical.OutputExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.EsPhysicalOperationProviders; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.planner.PlannerUtils; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.plugin.EsqlPlugin.ESQL_THREAD_POOL_NAME; + +/** + * Computes the result of a {@link PhysicalPlan}. + */ +public class ComputeService { + private static final Logger LOGGER = LogManager.getLogger(ComputeService.class); + private final SearchService searchService; + private final BigArrays bigArrays; + private final TransportService transportService; + private final DriverTaskRunner driverRunner; + private final ExchangeService exchangeService; + private final EnrichLookupService enrichLookupService; + + public ComputeService( + SearchService searchService, + TransportService transportService, + ExchangeService exchangeService, + EnrichLookupService enrichLookupService, + ThreadPool threadPool, + BigArrays bigArrays + ) { + this.searchService = searchService; + this.transportService = transportService; + this.bigArrays = bigArrays.withCircuitBreaking(); + transportService.registerRequestHandler( + DATA_ACTION_NAME, + ESQL_THREAD_POOL_NAME, + DataNodeRequest::new, + new DataNodeRequestHandler() + ); + this.driverRunner = new DriverTaskRunner(transportService, threadPool.executor(ESQL_THREAD_POOL_NAME)); + this.exchangeService = exchangeService; + this.enrichLookupService = enrichLookupService; + } + + public void execute( + String sessionId, + CancellableTask rootTask, + PhysicalPlan physicalPlan, + EsqlConfiguration configuration, + ActionListener> listener + ) { + Tuple coordinatorAndDataNodePlan = PlannerUtils.breakPlanBetweenCoordinatorAndDataNode( + physicalPlan, + configuration + ); + final List collectedPages = Collections.synchronizedList(new ArrayList<>()); + PhysicalPlan coordinatorPlan = new OutputExec(coordinatorAndDataNodePlan.v1(), collectedPages::add); + PhysicalPlan dataNodePlan = coordinatorAndDataNodePlan.v2(); + + var concreteIndices = PlannerUtils.planConcreteIndices(physicalPlan); + + QueryPragmas queryPragmas = configuration.pragmas(); + + if (concreteIndices.isEmpty()) { + var computeContext = new ComputeContext(sessionId, List.of(), configuration, null, null); + runCompute(rootTask, computeContext, coordinatorPlan, listener.map(unused -> collectedPages)); + return; + } + QueryBuilder requestFilter = PlannerUtils.requestFilter(dataNodePlan); + String[] originalIndices = PlannerUtils.planOriginalIndices(physicalPlan); + computeTargetNodes(rootTask, requestFilter, concreteIndices, originalIndices, ActionListener.wrap(targetNodes -> { + final ExchangeSourceHandler exchangeSource = exchangeService.createSourceHandler( + sessionId, + queryPragmas.exchangeBufferSize(), + ESQL_THREAD_POOL_NAME + ); + try ( + Releasable ignored = exchangeSource::decRef; + RefCountingListener requestRefs = new RefCountingListener(listener.map(unused -> collectedPages)) + ) { + final AtomicBoolean cancelled = new AtomicBoolean(); + // wait until the source handler is completed + exchangeSource.addCompletionListener(requestRefs.acquire()); + // run compute on the coordinator + var computeContext = new ComputeContext(sessionId, List.of(), configuration, exchangeSource, null); + runCompute(rootTask, computeContext, coordinatorPlan, cancelOnFailure(rootTask, cancelled, requestRefs.acquire())); + // run compute on remote nodes + // TODO: This is wrong, we need to be able to cancel + runComputeOnRemoteNodes( + sessionId, + rootTask, + configuration, + dataNodePlan, + exchangeSource, + targetNodes, + () -> cancelOnFailure(rootTask, cancelled, requestRefs.acquire()).map(unused -> null) + ); + } + }, listener::onFailure)); + } + + private void runComputeOnRemoteNodes( + String sessionId, + CancellableTask rootTask, + EsqlConfiguration configuration, + PhysicalPlan dataNodePlan, + ExchangeSourceHandler exchangeSource, + List targetNodes, + Supplier> listener + ) { + // Do not complete the exchange sources until we have linked all remote sinks + final ListenableActionFuture blockingSinkFuture = new ListenableActionFuture<>(); + exchangeSource.addRemoteSink( + (sourceFinished, l) -> blockingSinkFuture.addListener(l.map(ignored -> new ExchangeResponse(null, true))), + 1 + ); + try (RefCountingRunnable exchangeRefs = new RefCountingRunnable(() -> blockingSinkFuture.onResponse(null))) { + // For each target node, first open a remote exchange on the remote node, then link the exchange source to + // the new remote exchange sink, and initialize the computation on the target node via data-node-request. + for (TargetNode targetNode : targetNodes) { + var targetNodeListener = ActionListener.releaseAfter(listener.get(), exchangeRefs.acquire()); + var queryPragmas = configuration.pragmas(); + ExchangeService.openExchange( + transportService, + targetNode.node(), + sessionId, + queryPragmas.exchangeBufferSize(), + ActionListener.wrap(unused -> { + var remoteSink = exchangeService.newRemoteSink(rootTask, sessionId, transportService, targetNode.node); + exchangeSource.addRemoteSink(remoteSink, queryPragmas.concurrentExchangeClients()); + transportService.sendChildRequest( + targetNode.node, + DATA_ACTION_NAME, + new DataNodeRequest(sessionId, configuration, targetNode.shardIds, targetNode.aliasFilters, dataNodePlan), + rootTask, + TransportRequestOptions.EMPTY, + new ActionListenerResponseHandler<>( + targetNodeListener, + DataNodeResponse::new, + TransportResponseHandler.TRANSPORT_WORKER + ) + ); + }, targetNodeListener::onFailure) + ); + } + } + } + + private ActionListener cancelOnFailure(CancellableTask task, AtomicBoolean cancelled, ActionListener listener) { + return listener.delegateResponse((l, e) -> { + l.onFailure(e); + if (cancelled.compareAndSet(false, true)) { + LOGGER.debug("cancelling ESQL task {} on failure", task); + transportService.getTaskManager().cancelTaskAndDescendants(task, "cancelled", false, ActionListener.noop()); + } + }); + } + + void runCompute(CancellableTask task, ComputeContext context, PhysicalPlan plan, ActionListener listener) { + listener = ActionListener.runAfter(listener, () -> Releasables.close(context.searchContexts)); + final List drivers; + try { + LocalExecutionPlanner planner = new LocalExecutionPlanner( + context.sessionId, + task, + bigArrays, + context.configuration, + context.exchangeSource(), + context.exchangeSink(), + enrichLookupService, + new EsPhysicalOperationProviders(context.searchContexts) + ); + + LOGGER.info("Received physical plan:\n{}", plan); + plan = PlannerUtils.localPlan(context.searchContexts, context.configuration, plan); + LocalExecutionPlanner.LocalExecutionPlan localExecutionPlan = planner.plan(plan); + + LOGGER.info("Local execution plan:\n{}", localExecutionPlan.describe()); + drivers = localExecutionPlan.createDrivers(context.sessionId); + if (drivers.isEmpty()) { + throw new IllegalStateException("no drivers created"); + } + LOGGER.info("using {} drivers", drivers.size()); + } catch (Exception e) { + listener.onFailure(e); + return; + } + driverRunner.executeDrivers(task, drivers, ActionListener.releaseAfter(listener, () -> Releasables.close(drivers))); + } + + private void acquireSearchContexts( + List shardIds, + Map aliasFilters, + ActionListener> listener + ) { + try { + List targetShards = new ArrayList<>(); + for (ShardId shardId : shardIds) { + var indexShard = searchService.getIndicesService().indexServiceSafe(shardId.getIndex()).getShard(shardId.id()); + targetShards.add(indexShard); + } + if (targetShards.isEmpty()) { + listener.onResponse(List.of()); + return; + } + CountDown countDown = new CountDown(targetShards.size()); + for (IndexShard targetShard : targetShards) { + targetShard.ensureShardSearchActive(ignored -> { + if (countDown.countDown()) { + ActionListener.completeWith(listener, () -> { + final List searchContexts = new ArrayList<>(targetShards.size()); + boolean success = false; + try { + for (IndexShard shard : targetShards) { + var aliasFilter = aliasFilters.getOrDefault(shard.shardId().getIndex(), AliasFilter.EMPTY); + ShardSearchRequest shardSearchLocalRequest = new ShardSearchRequest(shard.shardId(), 0, aliasFilter); + SearchContext context = searchService.createSearchContext( + shardSearchLocalRequest, + SearchService.NO_TIMEOUT + ); + searchContexts.add(context); + } + for (SearchContext searchContext : searchContexts) { + searchContext.preProcess(); + } + success = true; + return searchContexts; + } finally { + if (success == false) { + IOUtils.close(searchContexts); + } + } + }); + } + }); + } + } catch (Exception e) { + listener.onFailure(e); + } + } + + record TargetNode(DiscoveryNode node, List shardIds, Map aliasFilters) { + + } + + private void computeTargetNodes( + Task parentTask, + QueryBuilder filter, + Set concreteIndices, + String[] originalIndices, + ActionListener> listener + ) { + // Ideally, the search_shards API should be called before the field-caps API; however, this can lead + // to a situation where the column structure (i.e., matched data types) differs depending on the query. + ThreadContext threadContext = transportService.getThreadPool().getThreadContext(); + ActionListener preservingContextListener = ContextPreservingActionListener.wrapPreservingContext( + listener.map(resp -> { + Map nodes = new HashMap<>(); + for (DiscoveryNode node : resp.getNodes()) { + nodes.put(node.getId(), node); + } + Map> nodeToShards = new HashMap<>(); + Map> nodeToAliasFilters = new HashMap<>(); + for (SearchShardsGroup group : resp.getGroups()) { + var shardId = group.shardId(); + if (group.skipped()) { + continue; + } + if (group.allocatedNodes().isEmpty()) { + throw new ShardNotFoundException(group.shardId(), "no shard copies found {}", group.shardId()); + } + if (concreteIndices.contains(shardId.getIndexName()) == false) { + continue; + } + String targetNode = group.allocatedNodes().get(0); + nodeToShards.computeIfAbsent(targetNode, k -> new ArrayList<>()).add(shardId); + AliasFilter aliasFilter = resp.getAliasFilters().get(shardId.getIndex().getUUID()); + if (aliasFilter != null) { + nodeToAliasFilters.computeIfAbsent(targetNode, k -> new HashMap<>()).put(shardId.getIndex(), aliasFilter); + } + } + List targetNodes = new ArrayList<>(nodeToShards.size()); + for (Map.Entry> e : nodeToShards.entrySet()) { + DiscoveryNode node = nodes.get(e.getKey()); + Map aliasFilters = nodeToAliasFilters.getOrDefault(e.getKey(), Map.of()); + targetNodes.add(new TargetNode(node, e.getValue(), aliasFilters)); + } + return targetNodes; + }), + threadContext + ); + try (ThreadContext.StoredContext ignored = threadContext.newStoredContextPreservingResponseHeaders()) { + threadContext.markAsSystemContext(); + SearchShardsRequest searchShardsRequest = new SearchShardsRequest( + originalIndices, + SearchRequest.DEFAULT_INDICES_OPTIONS, + filter, + null, + null, + false, + null + ); + transportService.sendChildRequest( + transportService.getLocalNode(), + SearchShardsAction.NAME, + searchShardsRequest, + parentTask, + TransportRequestOptions.EMPTY, + new ActionListenerResponseHandler<>( + preservingContextListener, + SearchShardsResponse::new, + TransportResponseHandler.TRANSPORT_WORKER + ) + ); + } + } + + // TODO: To include stats/profiles + private static class DataNodeResponse extends TransportResponse { + DataNodeResponse() {} + + DataNodeResponse(StreamInput in) throws IOException { + super(in); + } + + @Override + public void writeTo(StreamOutput out) { + + } + } + + // TODO: Use an internal action here + public static final String DATA_ACTION_NAME = EsqlQueryAction.NAME + "/data"; + + private class DataNodeRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(DataNodeRequest request, TransportChannel channel, Task task) { + final var parentTask = (CancellableTask) task; + final var sessionId = request.sessionId(); + final var exchangeSink = exchangeService.getSinkHandler(sessionId); + parentTask.addListener(() -> exchangeService.finishSinkHandler(sessionId, new TaskCancelledException("task cancelled"))); + final ActionListener listener = new ChannelActionListener<>(channel).map(nullValue -> new DataNodeResponse()); + acquireSearchContexts(request.shardIds(), request.aliasFilters(), ActionListener.wrap(searchContexts -> { + var computeContext = new ComputeContext(sessionId, searchContexts, request.configuration(), null, exchangeSink); + runCompute(parentTask, computeContext, request.plan(), ActionListener.wrap(unused -> { + // don't return until all pages are fetched + exchangeSink.addCompletionListener( + ActionListener.releaseAfter(listener, () -> exchangeService.finishSinkHandler(sessionId, null)) + ); + }, e -> { + exchangeService.finishSinkHandler(sessionId, e); + listener.onFailure(e); + })); + }, e -> { + exchangeService.finishSinkHandler(sessionId, e); + listener.onFailure(e); + })); + } + } + + record ComputeContext( + String sessionId, + List searchContexts, + EsqlConfiguration configuration, + ExchangeSourceHandler exchangeSource, + ExchangeSinkHandler exchangeSink + ) {} +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequest.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequest.java new file mode 100644 index 0000000000000..0a2dfae30a1fc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequest.java @@ -0,0 +1,156 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +final class DataNodeRequest extends TransportRequest implements IndicesRequest { + private static final PlanNameRegistry planNameRegistry = new PlanNameRegistry(); + private final String sessionId; + private final EsqlConfiguration configuration; + private final List shardIds; + private final Map aliasFilters; + private final PhysicalPlan plan; + + private String[] indices; // lazily computed + + DataNodeRequest( + String sessionId, + EsqlConfiguration configuration, + List shardIds, + Map aliasFilters, + PhysicalPlan plan + ) { + this.sessionId = sessionId; + this.configuration = configuration; + this.shardIds = shardIds; + this.aliasFilters = aliasFilters; + this.plan = plan; + } + + DataNodeRequest(StreamInput in) throws IOException { + super(in); + this.sessionId = in.readString(); + this.configuration = new EsqlConfiguration(in); + this.shardIds = in.readList(ShardId::new); + this.aliasFilters = in.readMap(Index::new, AliasFilter::readFrom); + this.plan = new PlanStreamInput(in, planNameRegistry, in.namedWriteableRegistry(), configuration).readPhysicalPlanNode(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sessionId); + configuration.writeTo(out); + out.writeList(shardIds); + out.writeMap(aliasFilters); + new PlanStreamOutput(out, planNameRegistry).writePhysicalPlanNode(plan); + } + + @Override + public String[] indices() { + if (indices == null) { + indices = shardIds.stream().map(ShardId::getIndexName).distinct().toArray(String[]::new); + } + return indices; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + if (parentTaskId.isSet() == false) { + assert false : "DataNodeRequest must have a parent task"; + throw new IllegalStateException("DataNodeRequest must have a parent task"); + } + return new CancellableTask(id, type, action, "", parentTaskId, headers) { + @Override + public String getDescription() { + return DataNodeRequest.this.getDescription(); + } + }; + } + + String sessionId() { + return sessionId; + } + + EsqlConfiguration configuration() { + return configuration; + } + + QueryPragmas pragmas() { + return configuration.pragmas(); + } + + List shardIds() { + return shardIds; + } + + /** + * Returns a map from index UUID to alias filters + */ + Map aliasFilters() { + return aliasFilters; + } + + PhysicalPlan plan() { + return plan; + } + + @Override + public String getDescription() { + return "shards=" + shardIds + " plan=" + plan; + } + + @Override + public String toString() { + return "DataNodeRequest{" + getDescription() + "}"; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DataNodeRequest request = (DataNodeRequest) o; + return sessionId.equals(request.sessionId) + && configuration.equals(request.configuration) + && shardIds.equals(request.shardIds) + && aliasFilters.equals(request.aliasFilters) + && plan.equals(request.plan) + && getParentTask().equals(request.getParentTask()); + } + + @Override + public int hashCode() { + return Objects.hash(sessionId, configuration, shardIds, aliasFilters, plan); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParser.java new file mode 100644 index 0000000000000..9f522858358fc --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParser.java @@ -0,0 +1,80 @@ +/* + * 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.plugin; + +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.xcontent.MediaType; +import org.elasticsearch.xcontent.MediaTypeRegistry; +import org.elasticsearch.xcontent.ParsedMediaType; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.formatter.TextFormat; + +import java.util.Arrays; +import java.util.Locale; + +import static org.elasticsearch.xpack.esql.formatter.TextFormat.URL_PARAM_FORMAT; + +public class EsqlMediaTypeParser { + public static final MediaTypeRegistry MEDIA_TYPE_REGISTRY = new MediaTypeRegistry<>().register( + XContentType.values() + ).register(TextFormat.values()); + + /* + * Since we support {@link TextFormat} and + * {@link XContent} outputs we can't use {@link RestToXContentListener} + * like everything else. We want to stick as closely as possible to + * Elasticsearch's defaults though, while still layering in ways to + * control the output more easily. + * + * First we find the string that the user used to specify the response + * format. If there is a {@code format} parameter we use that. If there + * isn't but there is a {@code Accept} header then we use that. If there + * isn't then we use the {@code Content-Type} header which is required. + */ + public static MediaType getResponseMediaType(RestRequest request, EsqlQueryRequest esqlRequest) { + var mediaType = request.hasParam(URL_PARAM_FORMAT) ? mediaTypeFromParams(request) : mediaTypeFromHeaders(request); + return validateColumnarRequest(esqlRequest.columnar(), mediaType, request); + } + + private static MediaType mediaTypeFromHeaders(RestRequest request) { + ParsedMediaType acceptType = request.getParsedAccept(); + MediaType mediaType = acceptType != null ? acceptType.toMediaType(MEDIA_TYPE_REGISTRY) : request.getXContentType(); + return checkNonNullMediaType(mediaType, request); + } + + private static MediaType mediaTypeFromParams(RestRequest request) { + return MEDIA_TYPE_REGISTRY.queryParamToMediaType(request.param(URL_PARAM_FORMAT)); + } + + private static MediaType validateColumnarRequest(boolean requestIsColumnar, MediaType fromMediaType, RestRequest request) { + if (requestIsColumnar && fromMediaType instanceof TextFormat) { + throw new IllegalArgumentException( + "Invalid use of [columnar] argument: cannot be used in combination with " + + Arrays.stream(TextFormat.values()).map(MediaType::queryParameter).toList() + + " formats" + ); + } + return checkNonNullMediaType(fromMediaType, request); + } + + private static MediaType checkNonNullMediaType(MediaType mediaType, RestRequest request) { + if (mediaType == null) { + String msg = String.format( + Locale.ROOT, + "Invalid request content type: Accept=[%s], Content-Type=[%s], format=[%s]", + request.header("Accept"), + request.header("Content-Type"), + request.param("format") + ); + throw new IllegalArgumentException(msg); + } + + return mediaType; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java new file mode 100644 index 0000000000000..bbee23a7bcf17 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java @@ -0,0 +1,178 @@ +/* + * 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.plugin; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.compute.operator.AbstractPageMappingOperator; +import org.elasticsearch.compute.operator.DriverStatus; +import org.elasticsearch.compute.operator.LimitOperator; +import org.elasticsearch.compute.operator.MvExpandOperator; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkOperator; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceOperator; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.threadpool.ExecutorBuilder; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.tracing.Tracer; +import org.elasticsearch.watcher.ResourceWatcherService; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction; +import org.elasticsearch.xpack.esql.EsqlInfoTransportAction; +import org.elasticsearch.xpack.esql.EsqlUsageTransportAction; +import org.elasticsearch.xpack.esql.action.EsqlQueryAction; +import org.elasticsearch.xpack.esql.action.RestEsqlQueryAction; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolver; +import org.elasticsearch.xpack.esql.execution.PlanExecutor; +import org.elasticsearch.xpack.esql.querydsl.query.SingleValueQuery; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeRegistry; +import org.elasticsearch.xpack.ql.index.IndexResolver; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Stream; + +public class EsqlPlugin extends Plugin implements ActionPlugin { + /** + * The first version for ESQL. It's actual value is certainly wrong and will need to be + * updated when we merge. + */ + public static final TransportVersion TRANSPORT_MINIMUM_VERSION = TransportVersion.V_8_8_0; + + public static final String ESQL_THREAD_POOL_NAME = "esql"; + + public static final Setting QUERY_RESULT_TRUNCATION_MAX_SIZE = Setting.intSetting( + "esql.query.result_truncation_max_size", + 10000, + 1, + 1000000, + Setting.Property.NodeScope + ); + + @Override + public Collection createComponents( + Client client, + ClusterService clusterService, + ThreadPool threadPool, + ResourceWatcherService resourceWatcherService, + ScriptService scriptService, + NamedXContentRegistry xContentRegistry, + Environment environment, + NodeEnvironment nodeEnvironment, + NamedWriteableRegistry namedWriteableRegistry, + IndexNameExpressionResolver expressionResolver, + Supplier repositoriesServiceSupplier, + Tracer tracer, + AllocationService allocationService, + IndicesService indicesService + ) { + IndexResolver indexResolver = new IndexResolver( + client, + clusterService.getClusterName().value(), + EsqlDataTypeRegistry.INSTANCE, + Set::of + ); + return List.of( + new PlanExecutor(indexResolver, new EnrichPolicyResolver(clusterService, indexResolver, threadPool)), + new ExchangeService(clusterService.getSettings(), threadPool) + ); + } + + /** + * The settings defined by the ESQL plugin. + * + * @return the settings + */ + @Override + public List> getSettings() { + return List.of(QUERY_RESULT_TRUNCATION_MAX_SIZE); + } + + @Override + public List> getActions() { + return List.of( + new ActionHandler<>(EsqlQueryAction.INSTANCE, TransportEsqlQueryAction.class), + new ActionHandler<>(EsqlStatsAction.INSTANCE, TransportEsqlStatsAction.class), + new ActionHandler<>(XPackUsageFeatureAction.ESQL, EsqlUsageTransportAction.class), + new ActionHandler<>(XPackInfoFeatureAction.ESQL, EsqlInfoTransportAction.class) + ); + } + + @Override + public List getRestHandlers( + Settings settings, + RestController restController, + ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, + SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster + ) { + return List.of(new RestEsqlQueryAction()); + } + + @Override + public List getNamedWriteables() { + return Stream.concat( + List.of( + DriverStatus.ENTRY, + AbstractPageMappingOperator.Status.ENTRY, + ExchangeSinkOperator.Status.ENTRY, + ExchangeSourceOperator.Status.ENTRY, + LimitOperator.Status.ENTRY, + LuceneSourceOperator.Status.ENTRY, + MvExpandOperator.Status.ENTRY, + ValuesSourceReaderOperator.Status.ENTRY, + SingleValueQuery.ENTRY + ).stream(), + Block.getNamedWriteables().stream() + ).toList(); + } + + @Override + public List> getExecutorBuilders(Settings settings) { + final int allocatedProcessors = EsExecutors.allocatedProcessors(settings); + return List.of( + new FixedExecutorBuilder( + settings, + ESQL_THREAD_POOL_NAME, + ThreadPool.searchOrGetThreadPoolSize(allocatedProcessors), + 1000, + "esql", + EsExecutors.TaskTrackingConfig.DEFAULT + ) + ); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsAction.java new file mode 100644 index 0000000000000..b1e3ac1b0717c --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsAction.java @@ -0,0 +1,20 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.ActionType; + +public class EsqlStatsAction extends ActionType { + + public static final EsqlStatsAction INSTANCE = new EsqlStatsAction(); + public static final String NAME = "cluster:monitor/xpack/esql/stats/dist"; + + private EsqlStatsAction() { + super(NAME, EsqlStatsResponse::new); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsRequest.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsRequest.java new file mode 100644 index 0000000000000..47d0c6baa12b3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsRequest.java @@ -0,0 +1,74 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.support.nodes.BaseNodesRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.TransportRequest; + +import java.io.IOException; + +/** + * Request to gather usage statistics + */ +public class EsqlStatsRequest extends BaseNodesRequest { + + private boolean includeStats; + + public EsqlStatsRequest() { + super((String[]) null); + } + + public EsqlStatsRequest(StreamInput in) throws IOException { + super(in); + includeStats = in.readBoolean(); + } + + public boolean includeStats() { + return includeStats; + } + + public void includeStats(boolean includeStats) { + this.includeStats = includeStats; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(includeStats); + } + + @Override + public String toString() { + return "esql_stats"; + } + + static class NodeStatsRequest extends TransportRequest { + boolean includeStats; + + NodeStatsRequest(StreamInput in) throws IOException { + super(in); + includeStats = in.readBoolean(); + } + + NodeStatsRequest(EsqlStatsRequest request) { + includeStats = request.includeStats(); + } + + public boolean includeStats() { + return includeStats; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(includeStats); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsResponse.java new file mode 100644 index 0000000000000..d70416cdbda82 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlStatsResponse.java @@ -0,0 +1,102 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.support.nodes.BaseNodeResponse; +import org.elasticsearch.action.support.nodes.BaseNodesResponse; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.core.watcher.common.stats.Counters; + +import java.io.IOException; +import java.util.List; + +public class EsqlStatsResponse extends BaseNodesResponse implements ToXContentObject { + + public EsqlStatsResponse(StreamInput in) throws IOException { + super(in); + } + + public EsqlStatsResponse(ClusterName clusterName, List nodes, List failures) { + super(clusterName, nodes, failures); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return in.readList(NodeStatsResponse::readNodeResponse); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + out.writeList(nodes); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray("stats"); + for (NodeStatsResponse node : getNodes()) { + node.toXContent(builder, params); + } + builder.endArray(); + + return builder; + } + + public static class NodeStatsResponse extends BaseNodeResponse implements ToXContentObject { + + private Counters stats; + + public NodeStatsResponse(StreamInput in) throws IOException { + super(in); + if (in.readBoolean()) { + stats = new Counters(in); + } + } + + public NodeStatsResponse(DiscoveryNode node) { + super(node); + } + + public Counters getStats() { + return stats; + } + + public void setStats(Counters stats) { + this.stats = stats; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(stats != null); + if (stats != null) { + stats.writeTo(out); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (stats != null && stats.hasCounters()) { + builder.field("stats", stats.toNestedMap()); + } + builder.endObject(); + return builder; + } + + static NodeStatsResponse readNodeResponse(StreamInput in) throws IOException { + return new NodeStatsResponse(in); + } + + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java new file mode 100644 index 0000000000000..a5b8fb601e089 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java @@ -0,0 +1,102 @@ +/* + * 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.plugin; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.lucene.DataPartitioning; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.Objects; + +/** + * Holds the pragmas for an ESQL query. Just a wrapper of settings for now. + */ +public final class QueryPragmas implements Writeable { + public static final Setting EXCHANGE_BUFFER_SIZE = Setting.intSetting("exchange_buffer_size", 10); + public static final Setting EXCHANGE_CONCURRENT_CLIENTS = Setting.intSetting("exchange_concurrent_clients", 3); + private static final Setting TASK_CONCURRENCY = Setting.intSetting( + "task_concurrency", + ThreadPool.searchOrGetThreadPoolSize(EsExecutors.allocatedProcessors(Settings.EMPTY)) + ); + + public static final Setting DATA_PARTITIONING = Setting.enumSetting( + DataPartitioning.class, + "data_partitioning", + DataPartitioning.SEGMENT + ); + + /** + * Size of a page in entries with {@code 0} being a special value asking + * to adaptively size based on the number of columns in the page. + */ + public static final Setting PAGE_SIZE = Setting.intSetting("page_size", 0, 0); + + public static final QueryPragmas EMPTY = new QueryPragmas(Settings.EMPTY); + + private final Settings settings; + + public QueryPragmas(Settings settings) { + this.settings = settings; + } + + public QueryPragmas(StreamInput in) throws IOException { + this.settings = Settings.readSettingsFromStream(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + settings.writeTo(out); + } + + public int exchangeBufferSize() { + return EXCHANGE_BUFFER_SIZE.get(settings); + } + + public int concurrentExchangeClients() { + return EXCHANGE_CONCURRENT_CLIENTS.get(settings); + } + + public DataPartitioning dataPartitioning() { + return DATA_PARTITIONING.get(settings); + } + + public int taskConcurrency() { + return TASK_CONCURRENCY.get(settings); + } + + /** + * Size of a page in entries with {@code 0} being a special value asking + * to adaptively size based on the number of columns in the page. + */ + public int pageSize() { + return PAGE_SIZE.get(settings); + } + + public boolean isEmpty() { + return settings.isEmpty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + QueryPragmas pragmas = (QueryPragmas) o; + return settings.equals(pragmas.settings); + } + + @Override + public int hashCode() { + return Objects.hash(settings); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java new file mode 100644 index 0000000000000..dd9d9634701ea --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java @@ -0,0 +1,116 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.action.ColumnInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryAction; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupService; +import org.elasticsearch.xpack.esql.execution.PlanExecutor; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.Locale; + +import static org.elasticsearch.action.ActionListener.wrap; + +public class TransportEsqlQueryAction extends HandledTransportAction { + + private final PlanExecutor planExecutor; + private final ComputeService computeService; + private final ExchangeService exchangeService; + private final ClusterService clusterService; + private final EnrichLookupService enrichLookupService; + private final Settings settings; + + @Inject + public TransportEsqlQueryAction( + Settings settings, + TransportService transportService, + ActionFilters actionFilters, + PlanExecutor planExecutor, + SearchService searchService, + ExchangeService exchangeService, + ClusterService clusterService, + ThreadPool threadPool, + BigArrays bigArrays + ) { + super(EsqlQueryAction.NAME, transportService, actionFilters, EsqlQueryRequest::new); + this.planExecutor = planExecutor; + this.clusterService = clusterService; + exchangeService.registerTransportHandler(transportService); + this.exchangeService = exchangeService; + this.enrichLookupService = new EnrichLookupService(clusterService, searchService, transportService); + this.computeService = new ComputeService( + searchService, + transportService, + exchangeService, + enrichLookupService, + threadPool, + bigArrays + ); + this.settings = settings; + } + + @Override + protected void doExecute(Task task, EsqlQueryRequest request, ActionListener listener) { + EsqlConfiguration configuration = new EsqlConfiguration( + request.zoneId() != null ? request.zoneId() : ZoneOffset.UTC, + request.locale() != null ? request.locale() : Locale.US, + // TODO: plug-in security + null, + clusterService.getClusterName().value(), + request.pragmas(), + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.get(settings) + ); + String sessionId = sessionID(task); + planExecutor.esql(request, sessionId, configuration, wrap(r -> { + computeService.execute(sessionId, (CancellableTask) task, r, configuration, listener.map(pages -> { + List columns = r.output() + .stream() + .map(c -> new ColumnInfo(c.qualifiedName(), EsqlDataTypes.outputType(c.dataType()))) + .toList(); + return new EsqlQueryResponse(columns, pages, request.columnar()); + })); + }, listener::onFailure)); + } + + /** + * Returns the ID for this compute session. The ID is unique within the cluster, and is used + * to identify the compute-session across nodes. The ID is just the TaskID of the task that + * initiated the session. + */ + final String sessionID(Task task) { + return new TaskId(clusterService.localNode().getId(), task.getId()).toString(); + } + + public ExchangeService exchangeService() { + return exchangeService; + } + + public EnrichLookupService enrichLookupService() { + return enrichLookupService; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlStatsAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlStatsAction.java new file mode 100644 index 0000000000000..c59792968b274 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlStatsAction.java @@ -0,0 +1,82 @@ +/* + * 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.plugin; + +import org.elasticsearch.action.FailedNodeException; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.nodes.TransportNodesAction; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.execution.PlanExecutor; + +import java.io.IOException; +import java.util.List; + +/** + * Performs the stats operation. + */ +public class TransportEsqlStatsAction extends TransportNodesAction< + EsqlStatsRequest, + EsqlStatsResponse, + EsqlStatsRequest.NodeStatsRequest, + EsqlStatsResponse.NodeStatsResponse> { + + // the plan executor holds the metrics + private final PlanExecutor planExecutor; + + @Inject + public TransportEsqlStatsAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + PlanExecutor planExecutor + ) { + super( + EsqlStatsAction.NAME, + threadPool, + clusterService, + transportService, + actionFilters, + EsqlStatsRequest::new, + EsqlStatsRequest.NodeStatsRequest::new, + ThreadPool.Names.MANAGEMENT + ); + this.planExecutor = planExecutor; + } + + @Override + protected EsqlStatsResponse newResponse( + EsqlStatsRequest request, + List nodes, + List failures + ) { + return new EsqlStatsResponse(clusterService.getClusterName(), nodes, failures); + } + + @Override + protected EsqlStatsRequest.NodeStatsRequest newNodeRequest(EsqlStatsRequest request) { + return new EsqlStatsRequest.NodeStatsRequest(request); + } + + @Override + protected EsqlStatsResponse.NodeStatsResponse newNodeResponse(StreamInput in, DiscoveryNode node) throws IOException { + return new EsqlStatsResponse.NodeStatsResponse(in); + } + + @Override + protected EsqlStatsResponse.NodeStatsResponse nodeOperation(EsqlStatsRequest.NodeStatsRequest request, Task task) { + EsqlStatsResponse.NodeStatsResponse statsResponse = new EsqlStatsResponse.NodeStatsResponse(clusterService.localNode()); + statsResponse.setStats(planExecutor.metrics().stats()); + return statsResponse; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuery.java new file mode 100644 index 0000000000000..7adb87db03732 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuery.java @@ -0,0 +1,656 @@ +/* + * 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.querydsl.query; + +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.QueryVisitor; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.TwoPhaseIterator; +import org.apache.lucene.search.Weight; +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.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.index.fielddata.LeafNumericFieldData; +import org.elasticsearch.index.fielddata.LeafOrdinalsFieldData; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.search.sort.NestedSortBuilder; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.ql.querydsl.query.Query; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.io.IOException; +import java.util.Objects; + +/** + * Lucene query that wraps another query and only selects documents that match + * the wrapped query and have a single field value. + *

+ * This allows us to wrap regular lucene queries to have ESQL style semantics + * which will allow us to continue to push expressions to Lucene. + *

+ *

+ * We could have chosen not to wrap the lucene query and instead double check + * the results after they are loaded. That could be faster in some cases, but + * for now we're going to always wrap so we can always push. When we find cases + * where double checking is better we'll try that. + *

+ */ +public class SingleValueQuery extends Query { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + QueryBuilder.class, + "esql_single_value", + Builder::new + ); + + private final Query next; + private final String field; + + public SingleValueQuery(Query next, String field) { + super(next.source()); + this.next = next; + this.field = field; + } + + @Override + public boolean containsNestedField(String path, String field) { + return next.containsNestedField(path, field); + } + + @Override + public Query addNestedField(String path, String field, String format, boolean hasDocValues) { + return next.addNestedField(path, field, format, hasDocValues); + } + + @Override + public void enrichNestedSort(NestedSortBuilder sort) { + next.enrichNestedSort(sort); + } + + @Override + public Builder asBuilder() { + return new Builder(next.asBuilder(), field, new Stats()); + } + + @Override + protected String innerToString() { + return next.toString(); + } + + @Override + public SingleValueQuery negate(Source source) { + return new SingleValueQuery(next.negate(source), field); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass() || false == super.equals(o)) { + return false; + } + SingleValueQuery other = (SingleValueQuery) o; + return Objects.equals(next, other.next) && Objects.equals(field, other.field); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), next, field); + } + + public static class Builder extends AbstractQueryBuilder { + private final QueryBuilder next; + private final String field; + private final Stats stats; + + Builder(QueryBuilder next, String field, Stats stats) { + this.next = next; + this.field = field; + this.stats = stats; + } + + Builder(StreamInput in) throws IOException { + super(in); + this.next = in.readNamedWriteable(QueryBuilder.class); + this.field = in.readString(); + this.stats = new Stats(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(next); + out.writeString(field); + } + + public QueryBuilder next() { + return next; + } + + public String field() { + return field; + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + @Override + protected void doXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(ENTRY.name); + builder.field("field", field); + builder.field("next", next, params); + builder.endObject(); + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return EsqlPlugin.TRANSPORT_MINIMUM_VERSION; + } + + @Override + protected org.apache.lucene.search.Query doToQuery(SearchExecutionContext context) throws IOException { + MappedFieldType ft = context.getFieldType(field); + if (ft == null) { + stats.missingField++; + return new MatchNoDocsQuery("missing field [" + field + "]"); + } + return new LuceneQuery(next.toQuery(context), context.getForField(ft, MappedFieldType.FielddataOperation.SEARCH), stats); + } + + @Override + protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException { + QueryBuilder rewritten = next.rewrite(queryRewriteContext); + if (rewritten instanceof MatchNoneQueryBuilder) { + stats.rewrittenToMatchNone++; + return rewritten; + } + if (rewritten == next) { + return this; + } + return new Builder(rewritten, field, stats); + } + + @Override + protected boolean doEquals(Builder other) { + return next.equals(other.next) && field.equals(other.field); + } + + @Override + protected int doHashCode() { + return Objects.hash(next, field); + } + + Stats stats() { + return stats; + } + } + + private static class LuceneQuery extends org.apache.lucene.search.Query { + private final org.apache.lucene.search.Query next; + private final IndexFieldData fieldData; + private final Stats stats; + + LuceneQuery(org.apache.lucene.search.Query next, IndexFieldData fieldData, Stats stats) { + this.next = next; + this.fieldData = fieldData; + this.stats = stats; + } + + @Override + public void visit(QueryVisitor visitor) { + if (visitor.acceptField(fieldData.getFieldName())) { + visitor.visitLeaf(next); + } + } + + @Override + public org.apache.lucene.search.Query rewrite(IndexReader reader) throws IOException { + org.apache.lucene.search.Query rewritten = next.rewrite(reader); + if (rewritten instanceof MatchNoDocsQuery) { + stats.rewrittenToMatchNone++; + return rewritten; + } + if (rewritten == next) { + return this; + } + return new LuceneQuery(rewritten, fieldData, stats); + } + + @Override + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return new SingleValueWeight(this, next.createWeight(searcher, scoreMode, boost), fieldData); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj == null || obj.getClass() != getClass()) { + return false; + } + SingleValueQuery.LuceneQuery other = (SingleValueQuery.LuceneQuery) obj; + return next.equals(other.next) && fieldData.getFieldName().equals(other.fieldData.getFieldName()); + } + + @Override + public int hashCode() { + return Objects.hash(classHash(), next, fieldData); + } + + @Override + public String toString(String field) { + StringBuilder builder = new StringBuilder("single_value("); + if (false == this.fieldData.getFieldName().equals(field)) { + builder.append(this.fieldData.getFieldName()); + builder.append(":"); + } + builder.append(next); + return builder.append(")").toString(); + } + } + + private static class SingleValueWeight extends Weight { + private final Stats stats; + private final Weight next; + private final IndexFieldData fieldData; + + private SingleValueWeight(SingleValueQuery.LuceneQuery query, Weight next, IndexFieldData fieldData) { + super(query); + this.stats = query.stats; + this.next = next; + this.fieldData = fieldData; + } + + @Override + public Explanation explain(LeafReaderContext context, int doc) throws IOException { + Explanation nextExplanation = next.explain(context, doc); + if (false == nextExplanation.isMatch()) { + return Explanation.noMatch("next didn't match", nextExplanation); + } + LeafFieldData lfd = fieldData.load(context); + SortedBinaryDocValues values = lfd.getBytesValues(); + if (false == values.advanceExact(doc)) { + return Explanation.noMatch("no values in field", nextExplanation); + } + if (values.docValueCount() != 1) { + return Explanation.noMatch("field has too many values [" + values.docValueCount() + "]", nextExplanation); + } + return Explanation.match(nextExplanation.getValue(), "field has exactly 1 value", nextExplanation); + } + + @Override + public Scorer scorer(LeafReaderContext context) throws IOException { + Scorer nextScorer = next.scorer(context); + if (nextScorer == null) { + stats.noNextScorer++; + return null; + } + LeafFieldData lfd = fieldData.load(context); + /* + * SortedBinaryDocValues are available for most fields, but they + * are made available by eagerly converting non-bytes values to + * utf-8 strings. The eager conversion is quite expensive. So + * we specialize on numeric fields and fields with ordinals to + * avoid that expense in at least that case. + * + * Also! Lucene's FieldExistsQuery only needs one scorer that can + * use all the docs values iterators at DocIdSetIterators. We + * can't do that because we need the check the number of fields. + */ + if (lfd instanceof LeafNumericFieldData n) { + return scorer(nextScorer, n); + } + if (lfd instanceof LeafOrdinalsFieldData o) { + return scorer(nextScorer, o); + } + return scorer(nextScorer, lfd); + } + + private Scorer scorer(Scorer nextScorer, LeafNumericFieldData lfd) { + SortedNumericDocValues sortedNumerics = lfd.getLongValues(); + if (DocValues.unwrapSingleton(sortedNumerics) != null) { + // Segment contains only single valued fields. + stats.numericSingle++; + return nextScorer; + } + TwoPhaseIterator nextIterator = nextScorer.twoPhaseIterator(); + if (nextIterator == null) { + stats.numericMultiNoApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedNumericsAndSinglePhaseQueries(nextScorer.iterator(), sortedNumerics) + ); + } + stats.numericMultiApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedNumericsAndTwoPhaseQueries(nextIterator, sortedNumerics) + ); + } + + private Scorer scorer(Scorer nextScorer, LeafOrdinalsFieldData lfd) { + SortedSetDocValues sortedSet = lfd.getOrdinalsValues(); + if (DocValues.unwrapSingleton(sortedSet) != null) { + // Segment contains only single valued fields. + stats.ordinalsSingle++; + return nextScorer; + } + TwoPhaseIterator nextIterator = nextScorer.twoPhaseIterator(); + if (nextIterator == null) { + stats.ordinalsMultiNoApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedSetAndSinglePhaseQueries(nextScorer.iterator(), sortedSet) + ); + } + stats.ordinalsMultiApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedSetAndTwoPhaseQueries(nextIterator, sortedSet) + ); + } + + private Scorer scorer(Scorer nextScorer, LeafFieldData lfd) { + SortedBinaryDocValues sortedBinary = lfd.getBytesValues(); + TwoPhaseIterator nextIterator = nextScorer.twoPhaseIterator(); + if (nextIterator == null) { + stats.bytesNoApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedBinaryAndSinglePhaseQueries(nextScorer.iterator(), sortedBinary) + ); + } + stats.bytesApprox++; + return new SingleValueQueryScorer( + this, + nextScorer, + new TwoPhaseIteratorForSortedBinaryAndTwoPhaseQueries(nextIterator, sortedBinary) + ); + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return next.isCacheable(ctx); + } + } + + private static class SingleValueQueryScorer extends Scorer { + private final Scorer next; + private final TwoPhaseIterator iterator; + + private SingleValueQueryScorer(Weight weight, Scorer next, TwoPhaseIterator iterator) { + super(weight); + this.next = next; + this.iterator = iterator; + } + + @Override + public DocIdSetIterator iterator() { + return TwoPhaseIterator.asDocIdSetIterator(iterator); + } + + @Override + public TwoPhaseIterator twoPhaseIterator() { + return iterator; + } + + @Override + public float getMaxScore(int upTo) throws IOException { + return next.getMaxScore(upTo); + } + + @Override + public float score() throws IOException { + return next.score(); + } + + @Override + public int docID() { + return next.docID(); + } + } + + /** + * The estimated number of comparisons to check if a {@link SortedNumericDocValues} + * has more than one value. There isn't a good way to get that number out of + * {@link SortedNumericDocValues} so this is a guess. + */ + private static final int SORTED_NUMERIC_MATCH_COST = 10; + + private static class TwoPhaseIteratorForSortedNumericsAndSinglePhaseQueries extends TwoPhaseIterator { + private final SortedNumericDocValues sortedNumerics; + + private TwoPhaseIteratorForSortedNumericsAndSinglePhaseQueries( + DocIdSetIterator approximation, + SortedNumericDocValues sortedNumerics + ) { + super(approximation); + this.sortedNumerics = sortedNumerics; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedNumerics.advanceExact(approximation.docID())) { + return false; + } + return sortedNumerics.docValueCount() == 1; + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST; + } + } + + private static class TwoPhaseIteratorForSortedNumericsAndTwoPhaseQueries extends TwoPhaseIterator { + private final SortedNumericDocValues sortedNumerics; + private final TwoPhaseIterator next; + + private TwoPhaseIteratorForSortedNumericsAndTwoPhaseQueries(TwoPhaseIterator next, SortedNumericDocValues sortedNumerics) { + super(next.approximation()); + this.sortedNumerics = sortedNumerics; + this.next = next; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedNumerics.advanceExact(approximation.docID())) { + return false; + } + if (sortedNumerics.docValueCount() != 1) { + return false; + } + return next.matches(); + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST + next.matchCost(); + } + } + + private static class TwoPhaseIteratorForSortedBinaryAndSinglePhaseQueries extends TwoPhaseIterator { + private final SortedBinaryDocValues sortedBinary; + + private TwoPhaseIteratorForSortedBinaryAndSinglePhaseQueries(DocIdSetIterator approximation, SortedBinaryDocValues sortedBinary) { + super(approximation); + this.sortedBinary = sortedBinary; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedBinary.advanceExact(approximation.docID())) { + return false; + } + return sortedBinary.docValueCount() == 1; + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST; + } + } + + private static class TwoPhaseIteratorForSortedSetAndTwoPhaseQueries extends TwoPhaseIterator { + private final SortedSetDocValues sortedSet; + private final TwoPhaseIterator next; + + private TwoPhaseIteratorForSortedSetAndTwoPhaseQueries(TwoPhaseIterator next, SortedSetDocValues sortedSet) { + super(next.approximation()); + this.sortedSet = sortedSet; + this.next = next; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedSet.advanceExact(approximation.docID())) { + return false; + } + if (sortedSet.docValueCount() != 1) { + return false; + } + return next.matches(); + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST + next.matchCost(); + } + } + + private static class TwoPhaseIteratorForSortedSetAndSinglePhaseQueries extends TwoPhaseIterator { + private final SortedSetDocValues sortedSet; + + private TwoPhaseIteratorForSortedSetAndSinglePhaseQueries(DocIdSetIterator approximation, SortedSetDocValues sortedSet) { + super(approximation); + this.sortedSet = sortedSet; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedSet.advanceExact(approximation.docID())) { + return false; + } + return sortedSet.docValueCount() == 1; + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST; + } + } + + private static class TwoPhaseIteratorForSortedBinaryAndTwoPhaseQueries extends TwoPhaseIterator { + private final SortedBinaryDocValues sortedBinary; + private final TwoPhaseIterator next; + + private TwoPhaseIteratorForSortedBinaryAndTwoPhaseQueries(TwoPhaseIterator next, SortedBinaryDocValues sortedBinary) { + super(next.approximation()); + this.sortedBinary = sortedBinary; + this.next = next; + } + + @Override + public boolean matches() throws IOException { + if (false == sortedBinary.advanceExact(approximation.docID())) { + return false; + } + if (sortedBinary.docValueCount() != 1) { + return false; + } + return next.matches(); + } + + @Override + public float matchCost() { + return SORTED_NUMERIC_MATCH_COST + next.matchCost(); + } + } + + static class Stats { + // TODO expose stats somehow + private int missingField; + private int rewrittenToMatchNone; + private int noNextScorer; + private int numericSingle; + private int numericMultiNoApprox; + private int numericMultiApprox; + private int ordinalsSingle; + private int ordinalsMultiNoApprox; + private int ordinalsMultiApprox; + private int bytesNoApprox; + private int bytesApprox; + + int missingField() { + return missingField; + } + + int rewrittenToMatchNone() { + return rewrittenToMatchNone; + } + + int noNextScorer() { + return noNextScorer; + } + + int numericSingle() { + return numericSingle; + } + + int numericMultiNoApprox() { + return numericMultiNoApprox; + } + + int numericMultiApprox() { + return numericMultiApprox; + } + + int ordinalsSingle() { + return ordinalsSingle; + } + + int ordinalsMultiNoApprox() { + return ordinalsMultiNoApprox; + } + + int ordinalsMultiApprox() { + return ordinalsMultiApprox; + } + + int bytesNoApprox() { + return bytesNoApprox; + } + + int bytesApprox() { + return bytesApprox; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlConfiguration.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlConfiguration.java new file mode 100644 index 0000000000000..3d6f5ce18816f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlConfiguration.java @@ -0,0 +1,90 @@ +/* + * 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.session; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.ql.session.Configuration; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneId; +import java.util.Locale; +import java.util.Objects; + +public class EsqlConfiguration extends Configuration implements Writeable { + private final QueryPragmas pragmas; + + private final int resultTruncationMaxSize; + + private final Locale locale; + + public EsqlConfiguration( + ZoneId zi, + Locale locale, + String username, + String clusterName, + QueryPragmas pragmas, + int resultTruncationMaxSize + ) { + super(zi, username, clusterName); + this.locale = locale; + this.pragmas = pragmas; + this.resultTruncationMaxSize = resultTruncationMaxSize; + } + + public EsqlConfiguration(StreamInput in) throws IOException { + super(in.readZoneId(), Instant.ofEpochSecond(in.readVLong(), in.readVInt()), in.readOptionalString(), in.readOptionalString()); + locale = Locale.forLanguageTag(in.readString()); + this.pragmas = new QueryPragmas(in); + this.resultTruncationMaxSize = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeZoneId(zoneId); + var instant = now.toInstant(); + out.writeVLong(instant.getEpochSecond()); + out.writeVInt(instant.getNano()); + out.writeOptionalString(username); + out.writeOptionalString(clusterName); + out.writeString(locale.toLanguageTag()); + pragmas.writeTo(out); + out.writeVInt(resultTruncationMaxSize); + } + + public QueryPragmas pragmas() { + return pragmas; + } + + public int resultTruncationMaxSize() { + return resultTruncationMaxSize; + } + + public Locale locale() { + return locale; + } + + @Override + public boolean equals(Object o) { + if (super.equals(o)) { + EsqlConfiguration that = (EsqlConfiguration) o; + return resultTruncationMaxSize == that.resultTruncationMaxSize + && Objects.equals(pragmas, that.pragmas) + && Objects.equals(locale, that.locale); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), pragmas, resultTruncationMaxSize, locale); + } +} 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 new file mode 100644 index 0000000000000..59c0e9aa2b181 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -0,0 +1,196 @@ +/* + * 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.session; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.PreAnalyzer; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolver; +import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.parser.TypedParamValue; +import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.ql.analyzer.TableInfo; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.index.IndexResolver; +import org.elasticsearch.xpack.ql.index.MappingException; +import org.elasticsearch.xpack.ql.plan.TableIdentifier; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; + +import static org.elasticsearch.index.query.QueryBuilders.boolQuery; +import static org.elasticsearch.xpack.ql.util.ActionListeners.map; + +public class EsqlSession { + + private static final Logger LOGGER = LogManager.getLogger(EsqlSession.class); + + private final String sessionId; + private final EsqlConfiguration configuration; + private final IndexResolver indexResolver; + private final EnrichPolicyResolver enrichPolicyResolver; + + private final PreAnalyzer preAnalyzer; + private final Verifier verifier; + private final FunctionRegistry functionRegistry; + private final LogicalPlanOptimizer logicalPlanOptimizer; + + private final Mapper mapper; + private final PhysicalPlanOptimizer physicalPlanOptimizer; + + public EsqlSession( + String sessionId, + EsqlConfiguration configuration, + IndexResolver indexResolver, + EnrichPolicyResolver enrichPolicyResolver, + PreAnalyzer preAnalyzer, + FunctionRegistry functionRegistry, + LogicalPlanOptimizer logicalPlanOptimizer, + Mapper mapper, + Verifier verifier + ) { + this.sessionId = sessionId; + this.configuration = configuration; + this.indexResolver = indexResolver; + this.enrichPolicyResolver = enrichPolicyResolver; + this.preAnalyzer = preAnalyzer; + this.verifier = verifier; + this.functionRegistry = functionRegistry; + this.mapper = mapper; + this.logicalPlanOptimizer = logicalPlanOptimizer; + this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); + } + + public String sessionId() { + return sessionId; + } + + public void execute(EsqlQueryRequest request, ActionListener listener) { + LOGGER.debug("ESQL query:\n{}", request.query()); + optimizedPhysicalPlan( + parse(request.query(), request.params()), + listener.map(plan -> EstimatesRowSize.estimateRowSize(0, plan.transformUp(FragmentExec.class, f -> { + QueryBuilder filter = request.filter(); + if (filter != null) { + var fragmentFilter = f.esFilter(); + // TODO: have an ESFilter and push down to EsQueryExec / EsSource + // This is an ugly hack to push the filter parameter to Lucene + // TODO: filter integration testing + filter = fragmentFilter != null ? boolQuery().filter(fragmentFilter).must(filter) : filter; + LOGGER.debug("Fold filter {} to EsQueryExec", filter); + f = new FragmentExec(f.source(), f.fragment(), filter, f.estimatedRowSize()); + } + return f; + }))) + ); + } + + private LogicalPlan parse(String query, List params) { + var parsed = new EsqlParser().createStatement(query, params); + LOGGER.debug("Parsed logical plan:\n{}", parsed); + return parsed; + } + + public void analyzedPlan(LogicalPlan parsed, ActionListener listener) { + if (parsed.analyzed()) { + listener.onResponse(parsed); + return; + } + + preAnalyze(parsed, (indices, policies) -> { + Analyzer analyzer = new Analyzer(new AnalyzerContext(configuration, functionRegistry, indices, policies), verifier); + var plan = analyzer.analyze(parsed); + LOGGER.debug("Analyzed plan:\n{}", plan); + return plan; + }, listener); + } + + private void preAnalyze(LogicalPlan parsed, BiFunction action, ActionListener listener) { + PreAnalyzer.PreAnalysis preAnalysis = preAnalyzer.preAnalyze(parsed); + Set policyNames = new HashSet<>(preAnalysis.policyNames); + EnrichResolution resolution = new EnrichResolution(ConcurrentCollections.newConcurrentSet(), enrichPolicyResolver.allPolicyNames()); + AtomicReference resolvedIndex = new AtomicReference<>(); + ActionListener groupedListener = ActionListener.wrap(unused -> { + assert resolution.resolvedPolicies().size() == policyNames.size() + : resolution.resolvedPolicies().size() + " != " + policyNames.size(); + assert resolvedIndex.get() != null : "index wasn't resolved"; + listener.onResponse(action.apply(resolvedIndex.get(), resolution)); + }, listener::onFailure); + try (RefCountingListener refs = new RefCountingListener(groupedListener)) { + preAnalyzeIndices(parsed, refs.acquire(resolvedIndex::set)); + for (String policyName : policyNames) { + enrichPolicyResolver.resolvePolicy(policyName, refs.acquire(resolution.resolvedPolicies()::add)); + } + } + } + + private void preAnalyzeIndices(LogicalPlan parsed, ActionListener listener) { + PreAnalyzer.PreAnalysis preAnalysis = new PreAnalyzer().preAnalyze(parsed); + // TODO we plan to support joins in the future when possible, but for now we'll just fail early if we see one + if (preAnalysis.indices.size() > 1) { + // Note: JOINs are not supported but we detect them when + listener.onFailure(new MappingException("Queries with multiple indices are not supported")); + } else if (preAnalysis.indices.size() == 1) { + TableInfo tableInfo = preAnalysis.indices.get(0); + TableIdentifier table = tableInfo.id(); + indexResolver.resolveAsMergedMapping(table.index(), false, Map.of(), listener); + } else { + try { + // occurs when dealing with local relations (row a = 1) + listener.onResponse(IndexResolution.invalid("[none specified]")); + } catch (Exception ex) { + listener.onFailure(ex); + } + } + } + + public void optimizedPlan(LogicalPlan logicalPlan, ActionListener listener) { + analyzedPlan(logicalPlan, map(listener, p -> { + var plan = logicalPlanOptimizer.optimize(p); + LOGGER.debug("Optimized logicalPlan plan:\n{}", plan); + return plan; + })); + } + + public void physicalPlan(LogicalPlan optimized, ActionListener listener) { + optimizedPlan(optimized, map(listener, p -> { + var plan = mapper.map(p); + LOGGER.debug("Physical plan:\n{}", plan); + return plan; + })); + } + + public void optimizedPhysicalPlan(LogicalPlan logicalPlan, ActionListener listener) { + physicalPlan(logicalPlan, map(listener, p -> { + var plan = physicalPlanOptimizer.optimize(p); + LOGGER.debug("Optimized physical plan:\n{}", plan); + return plan; + })); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java new file mode 100644 index 0000000000000..275e154993700 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/Result.java @@ -0,0 +1,14 @@ +/* + * 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.session; + +import org.elasticsearch.xpack.ql.expression.Attribute; + +import java.util.List; + +public record Result(List columns, List> values) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/FeatureMetric.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/FeatureMetric.java new file mode 100644 index 0000000000000..d2680f30ea03f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/FeatureMetric.java @@ -0,0 +1,31 @@ +/* + * 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.stats; + +import java.util.Locale; + +public enum FeatureMetric { + + /** + * The order of these enum values is important, do not change it. + * For any new values added to it, they should go at the end of the list. + * see {@link org.elasticsearch.xpack.esql.analysis.Verifier#gatherMetrics} + */ + DISSECT, + EVAL, + GROK, + LIMIT, + SORT, + STATS, + WHERE; + + @Override + public String toString() { + return this.name().toLowerCase(Locale.ROOT); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/Metrics.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/Metrics.java new file mode 100644 index 0000000000000..6c5d9faf18ac4 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/Metrics.java @@ -0,0 +1,107 @@ +/* + * 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.stats; + +import org.elasticsearch.common.metrics.CounterMetric; +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.xpack.core.watcher.common.stats.Counters; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Class encapsulating the metrics collected for ESQL + */ +public class Metrics { + + private enum OperationType { + FAILED, + TOTAL; + + @Override + public String toString() { + return this.name().toLowerCase(Locale.ROOT); + } + } + + // map that holds total/failed counters for each client type (rest, kibana) + private final Map> opsByTypeMetrics; + // map that holds one counter per esql query "feature" (eval, sort, limit, where....) + private final Map featuresMetrics; + protected static String QPREFIX = "queries."; + protected static String FPREFIX = "features."; + + public Metrics() { + Map> qMap = new LinkedHashMap<>(); + for (QueryMetric metric : QueryMetric.values()) { + Map metricsMap = Maps.newLinkedHashMapWithExpectedSize(OperationType.values().length); + for (OperationType type : OperationType.values()) { + metricsMap.put(type, new CounterMetric()); + } + + qMap.put(metric, Collections.unmodifiableMap(metricsMap)); + } + opsByTypeMetrics = Collections.unmodifiableMap(qMap); + + Map fMap = Maps.newLinkedHashMapWithExpectedSize(FeatureMetric.values().length); + for (FeatureMetric featureMetric : FeatureMetric.values()) { + fMap.put(featureMetric, new CounterMetric()); + } + featuresMetrics = Collections.unmodifiableMap(fMap); + } + + /** + * Increments the "total" counter for a metric + * This method should be called only once per query. + */ + public void total(QueryMetric metric) { + inc(metric, OperationType.TOTAL); + } + + /** + * Increments the "failed" counter for a metric + */ + public void failed(QueryMetric metric) { + inc(metric, OperationType.FAILED); + } + + private void inc(QueryMetric metric, OperationType op) { + this.opsByTypeMetrics.get(metric).get(op).inc(); + } + + public void inc(FeatureMetric metric) { + this.featuresMetrics.get(metric).inc(); + } + + public Counters stats() { + Counters counters = new Counters(); + + // queries metrics + for (Entry> entry : opsByTypeMetrics.entrySet()) { + String metricName = entry.getKey().toString(); + + for (OperationType type : OperationType.values()) { + long metricCounter = entry.getValue().get(type).count(); + String operationTypeName = type.toString(); + + counters.inc(QPREFIX + metricName + "." + operationTypeName, metricCounter); + counters.inc(QPREFIX + "_all." + operationTypeName, metricCounter); + } + } + + // features metrics + for (Entry entry : featuresMetrics.entrySet()) { + counters.inc(FPREFIX + entry.getKey().toString(), entry.getValue().count()); + } + + return counters; + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/QueryMetric.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/QueryMetric.java new file mode 100644 index 0000000000000..e862006d058ac --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/QueryMetric.java @@ -0,0 +1,28 @@ +/* + * 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.stats; + +import java.util.Locale; + +public enum QueryMetric { + KIBANA, + REST; + + public static QueryMetric fromString(String metric) { + try { + return QueryMetric.valueOf(metric.toUpperCase(Locale.ROOT)); + } catch (Exception e) { + return REST; + } + } + + @Override + public String toString() { + return this.name().toLowerCase(Locale.ROOT); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/SearchStats.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/SearchStats.java new file mode 100644 index 0000000000000..9e6b55e6333ec --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/stats/SearchStats.java @@ -0,0 +1,194 @@ +/* + * 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.stats; + +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.index.Term; +import org.apache.lucene.index.Terms; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public class SearchStats { + + private final List contexts; + + private static class FieldStat { + private Long count; + private Boolean exists; + private Object min, max; + } + + private static final int CACHE_SIZE = 32; + + // simple non-thread-safe cache for avoiding unnecessary IO (which while fast it still I/O) + private final Map cache = new LinkedHashMap<>(CACHE_SIZE, 0.75f, true) { + @Override + protected boolean removeEldestEntry(Map.Entry eldest) { + return size() > CACHE_SIZE; + } + }; + + public SearchStats(List contexts) { + this.contexts = contexts; + } + + public long count() { + var count = new long[] { 0 }; + boolean completed = doWithContexts(r -> count[0] += r.numDocs(), false); + return completed ? count[0] : -1; + } + + public long count(String field) { + var stat = cache.computeIfAbsent(field, s -> new FieldStat()); + if (stat.count == null) { + var count = new long[] { 0 }; + boolean completed = doWithContexts(r -> count[0] += countEntries(r, field), false); + stat.count = completed ? count[0] : -1; + } + return stat.count; + } + + public long count(String field, BytesRef value) { + var count = new long[] { 0 }; + Term term = new Term(field, value); + boolean completed = doWithContexts(r -> count[0] += r.docFreq(term), false); + return completed ? count[0] : -1; + } + + public boolean exists(String field) { + var stat = cache.computeIfAbsent(field, s -> new FieldStat()); + if (stat.exists == null) { + stat.exists = false; + // even if there are deleted documents, check the existence of a field + // since if it's missing, deleted documents won't change that + for (SearchContext context : contexts) { + if (context.getSearchExecutionContext().isFieldMapped(field)) { + stat.exists = true; + break; + } + } + } + return stat.exists; + } + + public byte[] min(String field, DataType dataType) { + var stat = cache.computeIfAbsent(field, s -> new FieldStat()); + if (stat.min == null) { + var min = new byte[][] { null }; + doWithContexts(r -> { + byte[] localMin = PointValues.getMinPackedValue(r, field); + // TODO: how to compare with the previous min + if (localMin != null) { + if (min[0] == null) { + min[0] = localMin; + } else { + throw new EsqlIllegalArgumentException("Don't know how to compare with previous min"); + } + } + + }, true); + stat.min = min[0]; + } + // return stat.min; + return null; + } + + public byte[] max(String field, DataType dataType) { + var stat = cache.computeIfAbsent(field, s -> new FieldStat()); + if (stat.max == null) { + + var max = new byte[][] { null }; + doWithContexts(r -> { + byte[] localMax = PointValues.getMaxPackedValue(r, field); + // TODO: how to compare with the previous max + if (localMax != null) { + if (max[0] == null) { + max[0] = localMax; + } else { + throw new EsqlIllegalArgumentException("Don't know how to compare with previous max"); + } + } + }, true); + stat.max = max[0]; + } + // return stat.max; + return null; + } + + // + // @see org.elasticsearch.search.query.TopDocsCollectorManagerFactory#shortcutTotalHitCount(IndexReader, Query) + // + private static int countEntries(IndexReader indexReader, String field) { + int count = 0; + try { + for (LeafReaderContext context : indexReader.leaves()) { + LeafReader reader = context.reader(); + FieldInfos fieldInfos = reader.getFieldInfos(); + FieldInfo fieldInfo = fieldInfos.fieldInfo(field); + + if (fieldInfo != null) { + if (fieldInfo.getDocValuesType() == DocValuesType.NONE) { + // no shortcut possible: it's a text field, empty values are counted as no value. + return -1; + } + if (fieldInfo.getPointIndexDimensionCount() > 0) { + PointValues points = reader.getPointValues(field); + if (points != null) { + count += points.getDocCount(); + } + } else if (fieldInfo.getIndexOptions() != IndexOptions.NONE) { + Terms terms = reader.terms(field); + if (terms != null) { + count += terms.getDocCount(); + } + } else { + return -1; // no shortcut possible for fields that are not indexed + } + } + } + } catch (IOException ex) { + throw new EsqlIllegalArgumentException("Cannot access data storage", ex); + } + return count; + } + + private interface IndexReaderConsumer { + void consume(IndexReader reader) throws IOException; + } + + private boolean doWithContexts(IndexReaderConsumer consumer, boolean acceptsDeletions) { + try { + for (SearchContext context : contexts) { + for (LeafReaderContext leafContext : context.searcher().getLeafContexts()) { + var reader = leafContext.reader(); + if (acceptsDeletions == false && reader.hasDeletions()) { + return false; + } + consumer.consume(reader); + } + } + return true; + } catch (IOException ex) { + throw new EsqlIllegalArgumentException("Cannot access data storage", ex); + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistry.java new file mode 100644 index 0000000000000..f2b67196ac11f --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistry.java @@ -0,0 +1,56 @@ +/* + * 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.type; + +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypeConverter; +import org.elasticsearch.xpack.ql.type.DataTypeRegistry; + +import java.util.Collection; + +public class EsqlDataTypeRegistry implements DataTypeRegistry { + + public static final DataTypeRegistry INSTANCE = new EsqlDataTypeRegistry(); + + private EsqlDataTypeRegistry() {} + + @Override + public Collection dataTypes() { + return EsqlDataTypes.types(); + } + + @Override + public DataType fromEs(String typeName) { + return EsqlDataTypes.fromEs(typeName); + } + + @Override + public DataType fromJava(Object value) { + return EsqlDataTypes.fromJava(value); + } + + @Override + public boolean isUnsupported(DataType type) { + return EsqlDataTypes.isUnsupported(type); + } + + @Override + public boolean canConvert(DataType from, DataType to) { + return DataTypeConverter.canConvert(from, to); + } + + @Override + public Object convert(Object value, DataType type) { + return DataTypeConverter.convert(value, type); + } + + @Override + public DataType commonType(DataType left, DataType right) { + return DataTypeConverter.commonType(left, right); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypes.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypes.java new file mode 100644 index 0000000000000..982905ed56428 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypes.java @@ -0,0 +1,173 @@ +/* + * 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.type; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Stream; + +import static java.util.stream.Collectors.toMap; +import static java.util.stream.Collectors.toUnmodifiableMap; +import static org.elasticsearch.xpack.ql.type.DataTypes.BOOLEAN; +import static org.elasticsearch.xpack.ql.type.DataTypes.BYTE; +import static org.elasticsearch.xpack.ql.type.DataTypes.DATETIME; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.FLOAT; +import static org.elasticsearch.xpack.ql.type.DataTypes.HALF_FLOAT; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.IP; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.NESTED; +import static org.elasticsearch.xpack.ql.type.DataTypes.NULL; +import static org.elasticsearch.xpack.ql.type.DataTypes.OBJECT; +import static org.elasticsearch.xpack.ql.type.DataTypes.SCALED_FLOAT; +import static org.elasticsearch.xpack.ql.type.DataTypes.SHORT; +import static org.elasticsearch.xpack.ql.type.DataTypes.TEXT; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSUPPORTED; +import static org.elasticsearch.xpack.ql.type.DataTypes.VERSION; + +public final class EsqlDataTypes { + + public static final DataType DATE_PERIOD = new DataType("DATE_PERIOD", null, 3 * Integer.BYTES, false, false, false); + public static final DataType TIME_DURATION = new DataType("TIME_DURATION", null, Integer.BYTES + Long.BYTES, false, false, false); + + private static final Collection TYPES = Stream.of( + BOOLEAN, + UNSUPPORTED, + NULL, + BYTE, + SHORT, + INTEGER, + LONG, + DOUBLE, + FLOAT, + HALF_FLOAT, + KEYWORD, + TEXT, + DATETIME, + DATE_PERIOD, + TIME_DURATION, + IP, + OBJECT, + NESTED, + SCALED_FLOAT, + VERSION, + UNSIGNED_LONG + ).sorted(Comparator.comparing(DataType::typeName)).toList(); + + private static final Map NAME_TO_TYPE = TYPES.stream().collect(toUnmodifiableMap(DataType::typeName, t -> t)); + + private static final Map ES_TO_TYPE; + + static { + Map map = TYPES.stream().filter(e -> e.esType() != null).collect(toMap(DataType::esType, t -> t)); + ES_TO_TYPE = Collections.unmodifiableMap(map); + } + + private EsqlDataTypes() {} + + public static Collection types() { + return TYPES; + } + + public static DataType fromTypeName(String name) { + return NAME_TO_TYPE.get(name.toLowerCase(Locale.ROOT)); + } + + public static DataType fromEs(String name) { + DataType type = ES_TO_TYPE.get(name); + return type != null ? type : UNSUPPORTED; + } + + public static DataType fromJava(Object value) { + if (value == null) { + return NULL; + } + if (value instanceof Boolean) { + return BOOLEAN; + } + if (value instanceof Integer) { + return INTEGER; + } + if (value instanceof Long) { + return LONG; + } + if (value instanceof Double) { + return DOUBLE; + } + if (value instanceof Float) { + return FLOAT; + } + if (value instanceof String || value instanceof Character || value instanceof BytesRef) { + return KEYWORD; + } + + return null; + } + + public static boolean isUnsupported(DataType type) { + return DataTypes.isUnsupported(type); + } + + public static String outputType(DataType type) { + if (type != null && type.esType() != null) { + return type.esType(); + } + return "unsupported"; + } + + public static boolean isString(DataType t) { + return t == KEYWORD || t == TEXT; + } + + public static boolean isPrimitive(DataType t) { + return t != OBJECT && t != NESTED; + } + + /** + * Supported types that can be contained in a block. + */ + public static boolean isRepresentable(DataType t) { + return t != OBJECT + && t != NESTED + && t != UNSUPPORTED + && t != DATE_PERIOD + && t != TIME_DURATION + && t != BYTE + && t != SHORT + && t != FLOAT + && t != SCALED_FLOAT + && t != HALF_FLOAT; + } + + public static boolean areCompatible(DataType left, DataType right) { + if (left == right) { + return true; + } else { + return (left == NULL || right == NULL) || (isString(left) && isString(right)) || (left.isNumeric() && right.isNumeric()); + } + } + + public static DataType widenSmallNumericTypes(DataType type) { + if (type == BYTE || type == SHORT) { + return INTEGER; + } + if (type == HALF_FLOAT || type == FLOAT || type == SCALED_FLOAT) { + return DOUBLE; + } + return type; + } +} diff --git a/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy b/x-pack/plugin/esql/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java new file mode 100644 index 0000000000000..003dbe47486c9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -0,0 +1,458 @@ +/* + * 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; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.logging.HeaderWarning; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; +import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.CsvTestUtils.ActualResults; +import org.elasticsearch.xpack.esql.CsvTestUtils.Type; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichLookupService; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.optimizer.LocalLogicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.TestLocalPhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.TestPhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.OutputExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlan; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.esql.planner.TestPhysicalOperationProviders; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.DisabledSearchStats; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.CsvSpecReader; +import org.elasticsearch.xpack.ql.SpecReader; +import org.elasticsearch.xpack.ql.analyzer.PreAnalyzer; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.util.Holder; +import org.junit.After; +import org.junit.Before; +import org.mockito.Mockito; + +import java.io.IOException; +import java.net.URL; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.compute.operator.DriverRunner.runToCompletion; +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.elasticsearch.xpack.esql.CsvTestUtils.ExpectedResults; +import static org.elasticsearch.xpack.esql.CsvTestUtils.isEnabled; +import static org.elasticsearch.xpack.esql.CsvTestUtils.loadCsvSpecValues; +import static org.elasticsearch.xpack.esql.CsvTestUtils.loadPageFromCsv; +import static org.elasticsearch.xpack.esql.CsvTestsDataLoader.CSV_DATASET_MAP; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; +import static org.elasticsearch.xpack.esql.plugin.EsqlPlugin.ESQL_THREAD_POOL_NAME; +import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser; +import static org.elasticsearch.xpack.ql.TestUtils.classpathResources; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +/** + * CSV-based unit testing. + * + * Queries and their result live *.csv-spec files. + * The results used in these files were manually added by running the same query on a real (debug mode) ES node. CsvTestsDataLoader loads + * the test data helping to get the said results. + * + * CsvTestsDataLoader creates an index using the mapping in mapping-default.json. The same mapping file is also used to create the + * IndexResolver that helps validate the correctness of the query and the supported field data types. + * The created index and this class uses the data from employees.csv file as data. This class is creating one Page with Blocks in it using + * this file and the type of blocks matches the type of the schema specified on the first line of the csv file. These being said, the + * mapping in mapping-default.csv and employees.csv should be more or less in sync. An exception to this rule: + * + * languages:integer,languages.long:long. The mapping has "long" as a sub-field of "languages". ES knows what to do with sub-field, but + * employees.csv is specifically defining "languages.long" as "long" and also has duplicated columns for these two. + * + * ATM the first line from employees.csv file is not synchronized with the mapping itself. + * + * When we add support for more field types, CsvTests should change to support the new Block types. Same goes for employees.csv file + * (the schema needs adjustment) and the mapping-default.json file (to add or change an existing field). + * When we add more operators, optimization rules to the logical or physical plan optimizers, there may be the need to change the operators + * in TestPhysicalOperationProviders or adjust TestPhysicalPlanOptimizer. For example, the TestPhysicalPlanOptimizer is skipping any + * rules that push operations to ES itself (a Limit for example). The TestPhysicalOperationProviders is a bit more complicated than that: + * it’s creating its own Source physical operator, aggregation operator (just a tiny bit of it) and field extract operator. + * + * To log the results logResults() should return "true". + */ +// @TestLogging(value = "org.elasticsearch.xpack.esql:TRACE,org.elasticsearch.compute:TRACE", reason = "debug") +public class CsvTests extends ESTestCase { + + private static final Logger LOGGER = LogManager.getLogger(CsvTests.class); + + private final String fileName; + private final String groupName; + private final String testName; + private final Integer lineNumber; + private final CsvSpecReader.CsvTestCase testCase; + + private final EsqlConfiguration configuration = new EsqlConfiguration( + ZoneOffset.UTC, + Locale.US, + null, + null, + new QueryPragmas(Settings.builder().put("page_size", randomPageSize()).build()), + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(Settings.EMPTY) + ); + private final FunctionRegistry functionRegistry = new EsqlFunctionRegistry(); + private final EsqlParser parser = new EsqlParser(); + private final LogicalPlanOptimizer logicalPlanOptimizer = new LogicalPlanOptimizer(); + private final Mapper mapper = new Mapper(functionRegistry); + private final PhysicalPlanOptimizer physicalPlanOptimizer = new TestPhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); + private ThreadPool threadPool; + + @ParametersFactory(argumentFormatting = "%2$s.%3$s") + public static List readScriptSpec() throws Exception { + List urls = classpathResources("/*.csv-spec").stream().filter(x -> x.toString().contains("-ignoreCsvTests") == false).toList(); + assertTrue("Not enough specs found " + urls, urls.size() > 0); + return SpecReader.readScriptSpec(urls, specParser()); + } + + @Before + public void setUp() throws Exception { + super.setUp(); + int numThreads = randomBoolean() ? 1 : between(2, 16); + threadPool = new TestThreadPool( + "CsvTests", + new FixedExecutorBuilder( + Settings.EMPTY, + ESQL_THREAD_POOL_NAME, + numThreads, + 1024, + "esql", + EsExecutors.TaskTrackingConfig.DEFAULT + ) + ); + HeaderWarning.setThreadContext(threadPool.getThreadContext()); + } + + @After + public void teardown() { + HeaderWarning.removeThreadContext(threadPool.getThreadContext()); + } + + @After + public void tearDown() throws Exception { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + super.tearDown(); + } + + private int randomPageSize() { + if (randomBoolean()) { + return between(1, 16); + } else { + return between(1, 16 * 1024); + } + } + + public CsvTests(String fileName, String groupName, String testName, Integer lineNumber, CsvSpecReader.CsvTestCase testCase) { + this.fileName = fileName; + this.groupName = groupName; + this.testName = testName; + this.lineNumber = lineNumber; + this.testCase = testCase; + } + + public final void test() throws Throwable { + try { + assumeTrue("Test " + testName + " is not enabled", isEnabled(testName)); + doTest(); + } catch (Throwable th) { + throw reworkException(th); + } + } + + @Override + protected final boolean enableWarningsCheck() { + return false; // We use our own warnings check + } + + public boolean logResults() { + return false; + } + + private void doTest() throws Exception { + var actualResults = executePlan(); + var expected = loadCsvSpecValues(testCase.expectedResults); + + var log = logResults() ? LOGGER : null; + assertResults(expected, actualResults, log); + assertWarnings(actualResults.responseHeaders().getOrDefault("Warning", List.of())); + } + + protected void assertResults(ExpectedResults expected, ActualResults actual, Logger logger) { + CsvAssert.assertResults(expected, actual, logger); + /* + * Comment the assertion above and enable the next two lines to see the results returned by ES without any assertions being done. + * This is useful when creating a new test or trying to figure out what are the actual results. + */ + // CsvTestUtils.logMetaData(actual.columnNames(), actual.columnTypes(), LOGGER); + // CsvTestUtils.logData(actual.values(), LOGGER); + } + + private static IndexResolution loadIndexResolution(String mappingName, String indexName) { + var mapping = new TreeMap<>(loadMapping(mappingName)); + return IndexResolution.valid(new EsIndex(indexName, mapping, Set.of(indexName))); + } + + private static EnrichResolution loadEnrichPolicies() { + Set names = new HashSet<>(); + Set resolutions = new HashSet<>(); + for (CsvTestsDataLoader.EnrichConfig policyConfig : CsvTestsDataLoader.ENRICH_POLICIES) { + EnrichPolicy policy = loadEnrichPolicyMapping(policyConfig.policyFileName()); + CsvTestsDataLoader.TestsDataset sourceIndex = CSV_DATASET_MAP.get(policy.getIndices().get(0)); + // this could practically work, but it's wrong: + // EnrichPolicyResolution should contain the policy (system) index, not the source index + IndexResolution idxRes = loadIndexResolution(sourceIndex.mappingFileName(), sourceIndex.indexName()); + names.add(policyConfig.policyName()); + resolutions.add(new EnrichPolicyResolution(policyConfig.policyName(), policy, idxRes)); + } + return new EnrichResolution(resolutions, names); + } + + private static EnrichPolicy loadEnrichPolicyMapping(String policyFileName) { + URL policyMapping = CsvTestsDataLoader.class.getResource("/" + policyFileName); + assertThat(policyMapping, is(notNullValue())); + try { + String fileContent = CsvTestsDataLoader.readTextFile(policyMapping); + return EnrichPolicy.fromXContent(JsonXContent.jsonXContent.createParser(XContentParserConfiguration.EMPTY, fileContent)); + } catch (IOException e) { + throw new IllegalArgumentException("Cannot read resource " + policyFileName); + } + } + + private PhysicalPlan physicalPlan(LogicalPlan parsed, CsvTestsDataLoader.TestsDataset dataset) { + var indexResolution = loadIndexResolution(dataset.mappingFileName(), dataset.indexName()); + var enrichPolicies = loadEnrichPolicies(); + var analyzer = new Analyzer( + new AnalyzerContext(configuration, functionRegistry, indexResolution, enrichPolicies), + new Verifier(new Metrics()) + ); + var analyzed = analyzer.analyze(parsed); + var logicalOptimized = logicalPlanOptimizer.optimize(analyzed); + var physicalPlan = mapper.map(logicalOptimized); + var optimizedPlan = EstimatesRowSize.estimateRowSize(0, physicalPlanOptimizer.optimize(physicalPlan)); + opportunisticallyAssertPlanSerialization(physicalPlan, optimizedPlan); // comment out to disable serialization + return optimizedPlan; + } + + private static CsvTestsDataLoader.TestsDataset testsDataset(LogicalPlan parsed) { + var preAnalysis = new PreAnalyzer().preAnalyze(parsed); + var indices = preAnalysis.indices; + if (indices.size() == 0) { + return CSV_DATASET_MAP.values().iterator().next(); // default dataset for `row` source command + } else if (preAnalysis.indices.size() > 1) { + throw new IllegalArgumentException("unexpected index resolution to multiple entries [" + preAnalysis.indices.size() + "]"); + } + + String indexName = indices.get(0).id().index(); + var dataset = CSV_DATASET_MAP.get(indexName); + if (dataset == null) { + throw new IllegalArgumentException("unknown CSV dataset for table [" + indexName + "]"); + } + return dataset; + } + + private static TestPhysicalOperationProviders testOperationProviders(CsvTestsDataLoader.TestsDataset dataset) throws Exception { + var testData = loadPageFromCsv(CsvTests.class.getResource("/" + dataset.dataFileName())); + return new TestPhysicalOperationProviders(testData.v1(), testData.v2()); + } + + private ActualResults executePlan() throws Exception { + var parsed = parser.createStatement(testCase.query); + var testDataset = testsDataset(parsed); + + String sessionId = "csv-test"; + ExchangeSourceHandler exchangeSource = new ExchangeSourceHandler(between(1, 64), threadPool.executor(ESQL_THREAD_POOL_NAME)); + ExchangeSinkHandler exchangeSink = new ExchangeSinkHandler(between(1, 64), threadPool::relativeTimeInMillis); + LocalExecutionPlanner executionPlanner = new LocalExecutionPlanner( + sessionId, + new CancellableTask(1, "transport", "esql", null, TaskId.EMPTY_TASK_ID, Map.of()), + BigArrays.NON_RECYCLING_INSTANCE, + configuration, + exchangeSource, + exchangeSink, + Mockito.mock(EnrichLookupService.class), + testOperationProviders(testDataset) + ); + // + // Keep in sync with ComputeService#execute + // + var localTestOptimizer = new TestLocalPhysicalPlanOptimizer(new LocalPhysicalOptimizerContext(configuration)); + + PhysicalPlan physicalPlan = physicalPlan(parsed, testDataset); + Tuple coordinatorAndDataNodePlan = CSVbreakPlanBetweenCoordinatorAndDataNode( + physicalPlan, + localTestOptimizer + ); + PhysicalPlan coordinatorPlan = coordinatorAndDataNodePlan.v1(); + PhysicalPlan dataNodePlan = coordinatorAndDataNodePlan.v2(); + + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Coordinator plan\n" + coordinatorPlan); + LOGGER.trace("DataNode plan\n" + dataNodePlan); + } + + List columnNames = Expressions.names(coordinatorPlan.output()); + List dataTypes = new ArrayList<>(columnNames.size()); + List columnTypes = coordinatorPlan.output() + .stream() + .peek(o -> dataTypes.add(EsqlDataTypes.outputType(o.dataType()))) + .map(o -> Type.asType(o.dataType().name())) + .toList(); + + List drivers = new ArrayList<>(); + List collectedPages = Collections.synchronizedList(new ArrayList<>()); + Map> responseHeaders; + + // replace fragment inside the coordinator plan + try { + LocalExecutionPlan coordinatorNodeExecutionPlan = executionPlanner.plan(new OutputExec(coordinatorPlan, collectedPages::add)); + drivers.addAll(coordinatorNodeExecutionPlan.createDrivers(sessionId)); + if (dataNodePlan != null) { + var csvDataNodePhysicalPlan = CSVlocalPlan(List.of(), configuration, dataNodePlan, localTestOptimizer); + exchangeSource.addRemoteSink(exchangeSink::fetchPageAsync, randomIntBetween(1, 3)); + LocalExecutionPlan dataNodeExecutionPlan = executionPlanner.plan(csvDataNodePhysicalPlan); + drivers.addAll(dataNodeExecutionPlan.createDrivers(sessionId)); + Randomness.shuffle(drivers); + } + responseHeaders = runToCompletion(threadPool, between(1, 10_000), drivers); + } finally { + Releasables.close(() -> Releasables.close(drivers), exchangeSource::decRef); + } + return new ActualResults(columnNames, columnTypes, dataTypes, collectedPages, responseHeaders); + } + + // + // Clone of PlannerUtils + // + + // PlannerUtils#breakPlanBetweenCoordinatorAndDataNode + private static Tuple CSVbreakPlanBetweenCoordinatorAndDataNode( + PhysicalPlan plan, + LocalPhysicalPlanOptimizer optimizer + ) { + var dataNodePlan = new Holder(); + + // split the given plan when encountering the exchange + PhysicalPlan coordinatorPlan = plan.transformUp(ExchangeExec.class, e -> { + // remember the datanode subplan and wire it to a sink + var subplan = e.child(); + dataNodePlan.set(new ExchangeSinkExec(e.source(), e.output(), subplan)); + return new ExchangeSourceExec(e.source(), e.output(), e.isInBetweenAggs()); + }); + return new Tuple<>(coordinatorPlan, dataNodePlan.get()); + } + + private static PhysicalPlan CSVlocalPlan( + List searchContexts, + EsqlConfiguration configuration, + PhysicalPlan plan, + LocalPhysicalPlanOptimizer optimizer + ) { + final Mapper mapper = new Mapper(true); + + var localOptimizer = new LocalLogicalPlanOptimizer(new LocalLogicalOptimizerContext(configuration, new DisabledSearchStats())); + + var localPhysicalPlan = plan.transformUp(FragmentExec.class, f -> { + var optimizedFragment = localOptimizer.localOptimize(f.fragment()); + var physicalFragment = mapper.map(optimizedFragment); + return EstimatesRowSize.estimateRowSize(f.estimatedRowSize(), physicalFragment); + }); + return optimizer.localOptimize(localPhysicalPlan); + } + + private Throwable reworkException(Throwable th) { + StackTraceElement[] stackTrace = th.getStackTrace(); + StackTraceElement[] redone = new StackTraceElement[stackTrace.length + 1]; + System.arraycopy(stackTrace, 0, redone, 1, stackTrace.length); + redone[0] = new StackTraceElement(getClass().getName(), groupName + "." + testName, fileName, lineNumber); + + th.setStackTrace(redone); + return th; + } + + // Asserts that the serialization and deserialization of the plan creates an equivalent plan. + private static void opportunisticallyAssertPlanSerialization(PhysicalPlan... plans) { + for (var plan : plans) { + var tmp = plan; + do { + if (tmp instanceof LocalSourceExec) { + return; // skip plans with localSourceExec + } + } while (tmp.children().isEmpty() == false && (tmp = tmp.children().get(0)) != null); + + SerializationTestUtils.assertSerialization(plan); + } + } + + private void assertWarnings(List warnings) { + List normalized = new ArrayList<>(warnings.size()); + for (String w : warnings) { + normalized.add(HeaderWarning.extractWarningValueFromWarningHeader(w, false)); + } + assertMap(normalized, matchesList(testCase.expectedWarnings)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlInfoTransportActionTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlInfoTransportActionTests.java new file mode 100644 index 0000000000000..7f0bdbab6add3 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlInfoTransportActionTests.java @@ -0,0 +1,121 @@ +/* + * 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; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeUtils; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xcontent.ObjectPath; +import org.elasticsearch.xpack.core.action.XPackUsageFeatureResponse; +import org.elasticsearch.xpack.core.esql.EsqlFeatureSetUsage; +import org.elasticsearch.xpack.core.watcher.common.stats.Counters; +import org.elasticsearch.xpack.esql.plugin.EsqlStatsAction; +import org.elasticsearch.xpack.esql.plugin.EsqlStatsResponse; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.core.Is.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class EsqlInfoTransportActionTests extends ESTestCase { + + private ThreadPool threadPool; + private Client client; + + @Before + public void init() { + threadPool = new TestThreadPool(getTestName()); + client = mock(Client.class); + when(client.threadPool()).thenReturn(threadPool); + } + + @After + public void shutdown() { + threadPool.shutdown(); + } + + public void testAvailable() { + EsqlInfoTransportAction featureSet = new EsqlInfoTransportAction(mock(TransportService.class), mock(ActionFilters.class)); + assertThat(featureSet.available(), is(true)); + } + + public void testEnabled() { + EsqlInfoTransportAction featureSet = new EsqlInfoTransportAction(mock(TransportService.class), mock(ActionFilters.class)); + assertThat(featureSet.enabled(), is(true)); + } + + @SuppressWarnings("unchecked") + public void testUsageStats() throws Exception { + doAnswer(mock -> { + ActionListener listener = (ActionListener) mock.getArguments()[2]; + + List nodes = new ArrayList<>(); + DiscoveryNode first = DiscoveryNodeUtils.create("first"); + EsqlStatsResponse.NodeStatsResponse firstNode = new EsqlStatsResponse.NodeStatsResponse(first); + Counters firstCounters = new Counters(); + firstCounters.inc("foo.foo", 1); + firstCounters.inc("foo.bar.baz", 1); + firstNode.setStats(firstCounters); + nodes.add(firstNode); + + DiscoveryNode second = DiscoveryNodeUtils.create("second"); + EsqlStatsResponse.NodeStatsResponse secondNode = new EsqlStatsResponse.NodeStatsResponse(second); + Counters secondCounters = new Counters(); + secondCounters.inc("spam", 1); + secondCounters.inc("foo.bar.baz", 4); + secondNode.setStats(secondCounters); + nodes.add(secondNode); + + listener.onResponse(new EsqlStatsResponse(new ClusterName("whatever"), nodes, Collections.emptyList())); + return null; + }).when(client).execute(eq(EsqlStatsAction.INSTANCE), any(), any()); + ClusterService clusterService = mock(ClusterService.class); + final DiscoveryNode mockNode = mock(DiscoveryNode.class); + when(mockNode.getId()).thenReturn("mocknode"); + when(clusterService.localNode()).thenReturn(mockNode); + + var usageAction = new EsqlUsageTransportAction( + mock(TransportService.class), + clusterService, + threadPool, + mock(ActionFilters.class), + null, + client + ); + PlainActionFuture future = new PlainActionFuture<>(); + usageAction.masterOperation(mock(Task.class), null, null, future); + EsqlFeatureSetUsage esqlUsage = (EsqlFeatureSetUsage) future.get().getUsage(); + + long fooBarBaz = ObjectPath.eval("foo.bar.baz", esqlUsage.stats()); + long fooFoo = ObjectPath.eval("foo.foo", esqlUsage.stats()); + long spam = ObjectPath.eval("spam", esqlUsage.stats()); + + assertThat(esqlUsage.stats().keySet(), containsInAnyOrder("foo", "spam")); + assertThat(fooBarBaz, is(5L)); + assertThat(fooFoo, is(1L)); + assertThat(spam, is(1L)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java new file mode 100644 index 0000000000000..a9a5e411c0298 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -0,0 +1,127 @@ +/* + * 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; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeRegistry; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DateUtils; +import org.elasticsearch.xpack.ql.type.EsField; +import org.elasticsearch.xpack.ql.type.TypesTests; +import org.junit.Assert; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static org.elasticsearch.xpack.ql.TestUtils.of; +import static org.hamcrest.Matchers.instanceOf; + +public final class EsqlTestUtils { + + public static class TestSearchStats extends SearchStats { + + public TestSearchStats() { + super(emptyList()); + } + + @Override + public long count() { + return -1; + } + + @Override + public long count(String field) { + return exists(field) ? -1 : 0; + } + + @Override + public long count(String field, BytesRef value) { + return exists(field) ? -1 : 0; + } + + @Override + public boolean exists(String field) { + return true; + } + + @Override + public byte[] min(String field, DataType dataType) { + return null; + } + + @Override + public byte[] max(String field, DataType dataType) { + return null; + } + } + + public static final TestSearchStats TEST_SEARCH_STATS = new TestSearchStats(); + + public static final EsqlConfiguration TEST_CFG = new EsqlConfiguration( + DateUtils.UTC, + Locale.US, + null, + null, + new QueryPragmas(Settings.EMPTY), + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(Settings.EMPTY) + ); + + private EsqlTestUtils() {} + + public static Literal L(Object value) { + return of(value); + } + + public static LogicalPlan emptySource() { + return new LocalRelation(Source.EMPTY, emptyList(), LocalSupplier.EMPTY); + } + + public static LogicalPlan localSource(List fields, List row) { + return new LocalRelation(Source.EMPTY, fields, LocalSupplier.of(BlockUtils.fromListRow(row))); + } + + public static T as(Object node, Class type) { + Assert.assertThat(node, instanceOf(type)); + return type.cast(node); + } + + public static Map loadMapping(String name) { + return TypesTests.loadMapping(EsqlDataTypeRegistry.INSTANCE, name, true); + } + + public static EnrichResolution emptyPolicyResolution() { + return new EnrichResolution(Set.of(), Set.of()); + } + + public static SearchStats statsForMissingField(String... names) { + return new TestSearchStats() { + private final Set missingFields = Set.of(names); + + @Override + public boolean exists(String field) { + return missingFields.contains(field) == false; + } + }; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java new file mode 100644 index 0000000000000..41590112c10a9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java @@ -0,0 +1,87 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.RegexpQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.TermsQueryBuilder; +import org.elasticsearch.index.query.WildcardQueryBuilder; +import org.elasticsearch.test.EqualsHashCodeTestUtils; +import org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.querydsl.query.SingleValueQuery; +import org.elasticsearch.xpack.ql.expression.Expression; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; + +public class SerializationTestUtils { + + private static final PlanNameRegistry planNameRegistry = new PlanNameRegistry(); + + public static void assertSerialization(PhysicalPlan plan) { + var deserPlan = serializeDeserialize(plan, PlanStreamOutput::writePhysicalPlanNode, PlanStreamInput::readPhysicalPlanNode); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(plan, unused -> deserPlan); + } + + public static void assertSerialization(Expression expression) { + Expression deserExpression = serializeDeserialize(expression, PlanStreamOutput::writeExpression, PlanStreamInput::readExpression); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(expression, unused -> deserExpression); + } + + public static T serializeDeserialize(T orig, Serializer serializer, Deserializer deserializer) { + try (BytesStreamOutput out = new BytesStreamOutput()) { + PlanStreamOutput planStreamOutput = new PlanStreamOutput(out, planNameRegistry); + serializer.write(planStreamOutput, orig); + StreamInput in = new NamedWriteableAwareStreamInput( + ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), + writableRegistry() + ); + PlanStreamInput planStreamInput = new PlanStreamInput(in, planNameRegistry, writableRegistry(), EsqlTestUtils.TEST_CFG); + return deserializer.read(planStreamInput); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + public interface Serializer { + void write(PlanStreamOutput out, T object) throws IOException; + } + + public interface Deserializer { + T read(PlanStreamInput in) throws IOException; + } + + public static NamedWriteableRegistry writableRegistry() { + return new NamedWriteableRegistry( + List.of( + new NamedWriteableRegistry.Entry(QueryBuilder.class, TermQueryBuilder.NAME, TermQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, TermsQueryBuilder.NAME, TermsQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, MatchAllQueryBuilder.NAME, MatchAllQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, RangeQueryBuilder.NAME, RangeQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, BoolQueryBuilder.NAME, BoolQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, WildcardQueryBuilder.NAME, WildcardQueryBuilder::new), + new NamedWriteableRegistry.Entry(QueryBuilder.class, RegexpQueryBuilder.NAME, RegexpQueryBuilder::new), + SingleValueQuery.ENTRY + ) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestTests.java new file mode 100644 index 0000000000000..9fb12572fd7ce --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestTests.java @@ -0,0 +1,160 @@ +/* + * 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.action; + +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class EsqlQueryRequestTests extends ESTestCase { + + public void testParseFields() throws IOException { + String query = randomAlphaOfLengthBetween(1, 100); + boolean columnar = randomBoolean(); + ZoneId zoneId = randomZone(); + Locale locale = randomLocale(random()); + QueryBuilder filter = randomQueryBuilder(); + List params = randomList(5, () -> randomBoolean() ? randomInt(100) : randomAlphaOfLength(10)); + StringBuilder paramsString = new StringBuilder(); + paramsString.append("["); + boolean first = true; + for (Object param : params) { + if (first == false) { + paramsString.append(", "); + } + first = false; + if (param instanceof String) { + paramsString.append("\""); + paramsString.append(param); + paramsString.append("\""); + } else { + paramsString.append(param); + } + } + paramsString.append("]"); + String json = String.format(Locale.ROOT, """ + { + "query": "%s", + "columnar": %s, + "time_zone": "%s", + "locale": "%s", + "filter": %s, + "params": %s + }""", query, columnar, zoneId, locale.toLanguageTag(), filter, paramsString); + + EsqlQueryRequest request = parseEsqlQueryRequest(json); + + assertEquals(query, request.query()); + assertEquals(columnar, request.columnar()); + assertEquals(zoneId, request.zoneId()); + assertEquals(locale.toLanguageTag(), request.locale().toLanguageTag()); + assertEquals(locale, request.locale()); + assertEquals(filter, request.filter()); + + assertEquals(params.size(), request.params().size()); + for (int i = 0; i < params.size(); i++) { + assertEquals(params.get(i), request.params().get(i).value); + } + } + + public void testRejectUnknownFields() { + assertParserErrorMessage(""" + { + "query": "foo", + "time_z0ne": "Z" + }""", "unknown field [time_z0ne] did you mean [time_zone]?"); + + assertParserErrorMessage(""" + { + "query": "foo", + "asdf": "Z" + }""", "unknown field [asdf]"); + } + + public void testMissingQueryIsNotValidation() throws IOException { + EsqlQueryRequest request = parseEsqlQueryRequest(""" + { + "time_zone": "Z" + }"""); + assertNotNull(request.validate()); + assertThat(request.validate().getMessage(), containsString("[query] is required")); + } + + public void testTask() throws IOException { + String query = randomAlphaOfLength(10); + int id = randomInt(); + + EsqlQueryRequest request = parseEsqlQueryRequest(""" + { + "query": "QUERY" + }""".replace("QUERY", query)); + Task task = request.createTask(id, "transport", EsqlQueryAction.NAME, TaskId.EMPTY_TASK_ID, Map.of()); + assertThat(task.getDescription(), equalTo(query)); + + String localNode = randomAlphaOfLength(2); + TaskInfo taskInfo = task.taskInfo(localNode, true); + String json = taskInfo.toString(); + String expected = Streams.readFully(getClass().getClassLoader().getResourceAsStream("query_task.json")).utf8ToString(); + expected = expected.replaceAll("\r\n", "\n") + .replaceAll("\s*<\\d+>", "") + .replaceAll("FROM test \\| STATS MAX\\(d\\) by a, b", query) + .replaceAll("5326", Integer.toString(id)) + .replaceAll("2j8UKw1bRO283PMwDugNNg", localNode) + .replaceAll("2023-07-31T15:46:32\\.328Z", DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.formatMillis(taskInfo.startTime())) + .replaceAll("1690818392328", Long.toString(taskInfo.startTime())) + .replaceAll("41.7ms", TimeValue.timeValueNanos(taskInfo.runningTimeNanos()).toString()) + .replaceAll("41770830", Long.toString(taskInfo.runningTimeNanos())) + .trim(); + assertThat(json, equalTo(expected)); + } + + private static void assertParserErrorMessage(String json, String message) { + Exception e = expectThrows(IllegalArgumentException.class, () -> parseEsqlQueryRequest(json)); + assertThat(e.getMessage(), containsString(message)); + } + + private static EsqlQueryRequest parseEsqlQueryRequest(String json) throws IOException { + SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); + XContentParserConfiguration config = XContentParserConfiguration.EMPTY.withRegistry( + new NamedXContentRegistry(searchModule.getNamedXContents()) + ); + try (XContentParser parser = XContentType.JSON.xContent().createParser(config, json)) { + return EsqlQueryRequest.fromXContent(parser); + } + } + + private static QueryBuilder randomQueryBuilder() { + return randomFrom( + new TermQueryBuilder(randomAlphaOfLength(5), randomAlphaOfLengthBetween(1, 10)), + new RangeQueryBuilder(randomAlphaOfLength(5)).gt(randomIntBetween(0, 1000)) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java new file mode 100644 index 0000000000000..07899000cba4a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java @@ -0,0 +1,169 @@ +/* + * 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.action; + +import org.apache.lucene.document.InetAddressPoint; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.UnsupportedValueSource; +import org.elasticsearch.test.AbstractChunkedSerializingTestCase; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.versionfield.Version; + +import java.util.ArrayList; +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class EsqlQueryResponseTests extends AbstractChunkedSerializingTestCase { + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(Block.getNamedWriteables()); + } + + @Override + protected EsqlQueryResponse createXContextTestInstance(XContentType xContentType) { + // columnar param can't be different from the default value (false) since the EsqlQueryResponse will be serialized (by some random + // XContentType, not to a StreamOutput) and parsed back, which doesn't preserve columnar field's value. + return randomResponse(false); + } + + @Override + protected EsqlQueryResponse createTestInstance() { + return randomResponse(randomBoolean()); + } + + EsqlQueryResponse randomResponse(boolean columnar) { + int noCols = randomIntBetween(1, 10); + List columns = randomList(noCols, noCols, this::randomColumnInfo); + int noPages = randomIntBetween(1, 20); + List values = randomList(noPages, noPages, () -> randomPage(columns)); + return new EsqlQueryResponse(columns, values, columnar); + } + + private ColumnInfo randomColumnInfo() { + DataType type = randomValueOtherThanMany( + t -> false == DataTypes.isPrimitive(t) || t == EsqlDataTypes.DATE_PERIOD || t == EsqlDataTypes.TIME_DURATION, + () -> randomFrom(EsqlDataTypes.types()) + ); + type = EsqlDataTypes.widenSmallNumericTypes(type); + return new ColumnInfo(randomAlphaOfLength(10), type.esType()); + } + + private Page randomPage(List columns) { + return new Page(columns.stream().map(c -> { + Block.Builder builder = LocalExecutionPlanner.toElementType(EsqlDataTypes.fromEs(c.type())).newBlockBuilder(1); + switch (c.type()) { + case "unsigned_long", "long" -> ((LongBlock.Builder) builder).appendLong(randomLong()); + case "integer" -> ((IntBlock.Builder) builder).appendInt(randomInt()); + case "double" -> ((DoubleBlock.Builder) builder).appendDouble(randomDouble()); + case "keyword" -> ((BytesRefBlock.Builder) builder).appendBytesRef(new BytesRef(randomAlphaOfLength(10))); + case "text" -> ((BytesRefBlock.Builder) builder).appendBytesRef(new BytesRef(randomAlphaOfLength(10000))); + case "ip" -> ((BytesRefBlock.Builder) builder).appendBytesRef( + new BytesRef(InetAddressPoint.encode(randomIp(randomBoolean()))) + ); + case "date" -> ((LongBlock.Builder) builder).appendLong(randomInstant().toEpochMilli()); + case "boolean" -> ((BooleanBlock.Builder) builder).appendBoolean(randomBoolean()); + case "unsupported" -> ((BytesRefBlock.Builder) builder).appendBytesRef( + new BytesRef(UnsupportedValueSource.UNSUPPORTED_OUTPUT) + ); + case "version" -> ((BytesRefBlock.Builder) builder).appendBytesRef(new Version(randomIdentifier()).toBytesRef()); + case "null" -> builder.appendNull(); + default -> throw new UnsupportedOperationException("unsupported data type [" + c + "]"); + } + return builder.build(); + }).toArray(Block[]::new)); + } + + @Override + protected EsqlQueryResponse mutateInstance(EsqlQueryResponse instance) { + boolean allNull = true; + for (ColumnInfo info : instance.columns()) { + if (false == info.type().equals("null")) { + allNull = false; + } + } + return switch (allNull ? between(0, 1) : between(0, 2)) { + case 0 -> { + int mutCol = between(0, instance.columns().size() - 1); + List cols = new ArrayList<>(instance.columns()); + // keep the type the same so the values are still valid but change the name + cols.set(mutCol, new ColumnInfo(cols.get(mutCol).name() + "mut", cols.get(mutCol).type())); + yield new EsqlQueryResponse(cols, instance.pages(), instance.columnar()); + } + case 1 -> new EsqlQueryResponse(instance.columns(), instance.pages(), false == instance.columnar()); + case 2 -> { + int noPages = instance.pages().size(); + yield new EsqlQueryResponse( + instance.columns(), + randomValueOtherThan(instance.pages(), () -> randomList(noPages, noPages, () -> randomPage(instance.columns()))), + instance.columnar() + ); + } + default -> throw new IllegalArgumentException(); + }; + } + + @Override + protected Writeable.Reader instanceReader() { + return EsqlQueryResponse::new; + } + + @Override + protected EsqlQueryResponse doParseInstance(XContentParser parser) { + return EsqlQueryResponse.fromXContent(parser); + } + + public void testChunkResponseSizeColumnar() { + EsqlQueryResponse resp = randomResponse(true); + int columnCount = resp.pages().get(0).getBlockCount(); + int bodySize = resp.pages().stream().mapToInt(p -> p.getPositionCount() * p.getBlockCount()).sum() + columnCount * 2; + assertChunkCount(resp, r -> 5 + bodySize); + } + + public void testChunkResponseSizeRows() { + EsqlQueryResponse resp = randomResponse(false); + int bodySize = resp.pages().stream().mapToInt(p -> p.getPositionCount()).sum(); + assertChunkCount(resp, r -> 5 + bodySize); + } + + public void testSimpleXContentColumnar() { + EsqlQueryResponse response = simple(true); + assertThat(Strings.toString(response), equalTo(""" + {"columns":[{"name":"foo","type":"integer"}],"values":[[40,80]]}""")); + } + + public void testSimpleXContentRows() { + EsqlQueryResponse response = simple(false); + assertThat(Strings.toString(response), equalTo(""" + {"columns":[{"name":"foo","type":"integer"}],"values":[[40],[80]]}""")); + } + + private EsqlQueryResponse simple(boolean columnar) { + return new EsqlQueryResponse( + List.of(new ColumnInfo("foo", "integer")), + List.of(new Page(new IntArrayVector(new int[] { 40, 80 }, 2).asBlock())), + columnar + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java new file mode 100644 index 0000000000000..e6c3b152849a8 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -0,0 +1,108 @@ +/* + * 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.analysis; + +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +public final class AnalyzerTestUtils { + + private AnalyzerTestUtils() {} + + public static Analyzer defaultAnalyzer() { + return analyzer(analyzerDefaultMapping()); + } + + public static Analyzer expandedDefaultAnalyzer() { + return analyzer(analyzerExpandedDefaultMapping()); + } + + public static Analyzer analyzer(IndexResolution indexResolution) { + return analyzer(indexResolution, new Verifier(new Metrics())); + } + + public static Analyzer analyzer(IndexResolution indexResolution, Verifier verifier) { + return new Analyzer( + new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), indexResolution, defaultEnrichResolution()), + verifier + ); + } + + public static Analyzer analyzer(Verifier verifier) { + return new Analyzer( + new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), analyzerDefaultMapping(), defaultEnrichResolution()), + verifier + ); + } + + public static LogicalPlan analyze(String query) { + return analyze(query, "mapping-basic.json"); + } + + public static LogicalPlan analyze(String query, String mapping) { + return analyze(query, analyzer(loadMapping(mapping, "test"))); + } + + public static LogicalPlan analyze(String query, Analyzer analyzer) { + var plan = new EsqlParser().createStatement(query); + // System.out.println(plan); + var analyzed = analyzer.analyze(plan); + // System.out.println(analyzed); + return analyzed; + } + + public static IndexResolution loadMapping(String resource, String indexName) { + EsIndex test = new EsIndex(indexName, EsqlTestUtils.loadMapping(resource)); + return IndexResolution.valid(test); + } + + public static IndexResolution analyzerDefaultMapping() { + return loadMapping("mapping-basic.json", "test"); + } + + public static IndexResolution analyzerExpandedDefaultMapping() { + return loadMapping("mapping-default.json", "test"); + } + + public static EnrichResolution defaultEnrichResolution() { + EnrichPolicyResolution policyRes = loadEnrichPolicyResolution( + "languages", + "language_code", + "languages_idx", + "mapping-languages.json" + ); + return new EnrichResolution(Set.of(policyRes), Set.of("languages")); + } + + public static EnrichPolicyResolution loadEnrichPolicyResolution( + String policyName, + String matchField, + String idxName, + String mappingFile + ) { + IndexResolution mapping = loadMapping(mappingFile, idxName); + List enrichFields = new ArrayList<>(mapping.get().mapping().keySet()); + enrichFields.remove(matchField); + return new EnrichPolicyResolution( + policyName, + new EnrichPolicy(EnrichPolicy.MATCH_TYPE, null, List.of(idxName), matchField, enrichFields), + mapping + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java new file mode 100644 index 0000000000000..5074b45f02428 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -0,0 +1,1274 @@ +/* + * 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.analysis; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Max; +import org.elasticsearch.xpack.esql.plan.logical.EsqlUnresolvedRelation; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.TableIdentifier; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.TypesTests; + +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyze; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyzer; +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.loadMapping; +import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; + +//@TestLogging(value = "org.elasticsearch.xpack.esql.analysis:TRACE", reason = "debug") +public class AnalyzerTests extends ESTestCase { + + private static final EsqlUnresolvedRelation UNRESOLVED_RELATION = new EsqlUnresolvedRelation( + EMPTY, + new TableIdentifier(EMPTY, null, "idx"), + List.of() + ); + + public void testIndexResolution() { + EsIndex idx = new EsIndex("idx", Map.of()); + Analyzer analyzer = analyzer(IndexResolution.valid(idx)); + var plan = analyzer.analyze(UNRESOLVED_RELATION); + var limit = as(plan, Limit.class); + + assertEquals(new EsRelation(EMPTY, idx, false), limit.child()); + } + + public void testFailOnUnresolvedIndex() { + Analyzer analyzer = analyzer(IndexResolution.invalid("Unknown index [idx]")); + + VerificationException e = expectThrows(VerificationException.class, () -> analyzer.analyze(UNRESOLVED_RELATION)); + + assertThat(e.getMessage(), containsString("Unknown index [idx]")); + } + + public void testIndexWithClusterResolution() { + EsIndex idx = new EsIndex("cluster:idx", Map.of()); + Analyzer analyzer = analyzer(IndexResolution.valid(idx)); + + var plan = analyzer.analyze(UNRESOLVED_RELATION); + var limit = as(plan, Limit.class); + + assertEquals(new EsRelation(EMPTY, idx, false), limit.child()); + } + + public void testAttributeResolution() { + EsIndex idx = new EsIndex("idx", TypesTests.loadMapping("mapping-one-field.json")); + Analyzer analyzer = analyzer(IndexResolution.valid(idx)); + + var plan = analyzer.analyze( + new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no")))) + ); + + var limit = as(plan, Limit.class); + var eval = as(limit.child(), Eval.class); + assertEquals(1, eval.fields().size()); + assertEquals(new Alias(EMPTY, "e", new FieldAttribute(EMPTY, "emp_no", idx.mapping().get("emp_no"))), eval.fields().get(0)); + + assertEquals(2, eval.output().size()); + Attribute empNo = eval.output().get(0); + assertEquals("emp_no", empNo.name()); + assertThat(empNo, instanceOf(FieldAttribute.class)); + Attribute e = eval.output().get(1); + assertEquals("e", e.name()); + assertThat(e, instanceOf(ReferenceAttribute.class)); + } + + public void testAttributeResolutionOfChainedReferences() { + Analyzer analyzer = analyzer(loadMapping("mapping-one-field.json", "idx")); + + var plan = analyzer.analyze( + new Eval( + EMPTY, + new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no")))), + List.of(new Alias(EMPTY, "ee", new UnresolvedAttribute(EMPTY, "e"))) + ) + ); + + var limit = as(plan, Limit.class); + var eval = as(limit.child(), Eval.class); + + assertEquals(1, eval.fields().size()); + Alias eeField = (Alias) eval.fields().get(0); + assertEquals("ee", eeField.name()); + assertEquals("e", ((ReferenceAttribute) eeField.child()).name()); + + assertEquals(3, eval.output().size()); + Attribute empNo = eval.output().get(0); + assertEquals("emp_no", empNo.name()); + assertThat(empNo, instanceOf(FieldAttribute.class)); + Attribute e = eval.output().get(1); + assertEquals("e", e.name()); + assertThat(e, instanceOf(ReferenceAttribute.class)); + Attribute ee = eval.output().get(2); + assertEquals("ee", ee.name()); + assertThat(ee, instanceOf(ReferenceAttribute.class)); + } + + public void testRowAttributeResolution() { + EsIndex idx = new EsIndex("idx", Map.of()); + Analyzer analyzer = analyzer(IndexResolution.valid(idx)); + + var plan = analyzer.analyze( + new Eval( + EMPTY, + new Row(EMPTY, List.of(new Alias(EMPTY, "emp_no", new Literal(EMPTY, 1, DataTypes.INTEGER)))), + List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_no"))) + ) + ); + + var limit = as(plan, Limit.class); + var eval = as(limit.child(), Eval.class); + assertEquals(1, eval.fields().size()); + assertEquals(new Alias(EMPTY, "e", new ReferenceAttribute(EMPTY, "emp_no", DataTypes.INTEGER)), eval.fields().get(0)); + + assertEquals(2, eval.output().size()); + Attribute empNo = eval.output().get(0); + assertEquals("emp_no", empNo.name()); + assertThat(empNo, instanceOf(ReferenceAttribute.class)); + Attribute e = eval.output().get(1); + assertEquals("e", e.name()); + assertThat(e, instanceOf(ReferenceAttribute.class)); + + Row row = (Row) eval.child(); + ReferenceAttribute rowEmpNo = (ReferenceAttribute) row.output().get(0); + assertEquals(rowEmpNo.id(), empNo.id()); + } + + public void testUnresolvableAttribute() { + Analyzer analyzer = analyzer(loadMapping("mapping-one-field.json", "idx")); + + VerificationException ve = expectThrows( + VerificationException.class, + () -> analyzer.analyze( + new Eval(EMPTY, UNRESOLVED_RELATION, List.of(new Alias(EMPTY, "e", new UnresolvedAttribute(EMPTY, "emp_nos")))) + ) + ); + + assertThat(ve.getMessage(), containsString("Unknown column [emp_nos], did you mean [emp_no]?")); + } + + public void testProjectBasic() { + assertProjection(""" + from test + | keep first_name + """, "first_name"); + } + + public void testProjectBasicPattern() { + assertProjection(""" + from test + | keep first*name + """, "first_name"); + assertProjectionTypes(""" + from test + | keep first*name + """, DataTypes.KEYWORD); + } + + public void testProjectIncludePattern() { + assertProjection(""" + from test + | keep *name + """, "first_name", "last_name"); + } + + public void testProjectIncludeMultiStarPattern() { + assertProjection(""" + from test + | keep *t*name + """, "first_name", "last_name"); + } + + public void testProjectStar() { + assertProjection(""" + from test + | keep * + """, "_meta_field", "emp_no", "first_name", "gender", "languages", "last_name", "salary"); + } + + public void testNoProjection() { + assertProjection(""" + from test + """, "_meta_field", "emp_no", "first_name", "gender", "languages", "last_name", "salary"); + assertProjectionTypes( + """ + from test + """, + DataTypes.KEYWORD, + DataTypes.INTEGER, + DataTypes.KEYWORD, + DataTypes.TEXT, + DataTypes.INTEGER, + DataTypes.KEYWORD, + DataTypes.INTEGER + ); + } + + public void testProjectOrder() { + assertProjection(""" + from test + | keep first_name, *, last_name + """, "first_name", "_meta_field", "emp_no", "gender", "languages", "salary", "last_name"); + } + + public void testProjectThenDropName() { + assertProjection(""" + from test + | keep *name + | drop first_name + """, "last_name"); + } + + public void testProjectAfterDropName() { + assertProjection(""" + from test + | drop first_name + | keep *name + """, "last_name"); + } + + public void testProjectKeepAndDropName() { + assertProjection(""" + from test + | drop first_name + | keep last_name + """, "last_name"); + } + + public void testProjectDropPattern() { + assertProjection(""" + from test + | keep * + | drop *_name + """, "_meta_field", "emp_no", "gender", "languages", "salary"); + } + + public void testProjectDropNoStarPattern() { + assertProjection(""" + from test + | drop *_name + """, "_meta_field", "emp_no", "gender", "languages", "salary"); + } + + public void testProjectOrderPatternWithRest() { + assertProjection(""" + from test + | keep *name, *, emp_no + """, "first_name", "last_name", "_meta_field", "gender", "languages", "salary", "emp_no"); + } + + public void testProjectDropPatternAndKeepOthers() { + assertProjection(""" + from test + | drop l* + | keep first_name, salary + """, "first_name", "salary"); + } + + public void testErrorOnNoMatchingPatternInclusion() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | keep *nonExisting + """)); + assertThat(e.getMessage(), containsString("No match found for [*nonExisting]")); + } + + public void testErrorOnNoMatchingPatternExclusion() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | drop *nonExisting + """)); + assertThat(e.getMessage(), containsString("No match found for [*nonExisting]")); + } + + // + // Unsupported field + // + + public void testIncludeUnsupportedFieldExplicit() { + assertProjectionWithMapping(""" + from test + | keep unsupported + """, "mapping-multi-field-variation.json", "unsupported"); + } + + public void testUnsupportedFieldAfterProject() { + var errorMessage = "Cannot use field [unsupported] with unsupported type [ip_range]"; + + verifyUnsupported(""" + from test + | keep unsupported + | eval x = unsupported + """, errorMessage); + } + + public void testUnsupportedFieldEvalAfterProject() { + var errorMessage = "Cannot use field [unsupported] with unsupported type [ip_range]"; + + verifyUnsupported(""" + from test + | keep unsupported + | eval x = unsupported + 1 + """, errorMessage); + } + + public void testUnsupportedFieldFilterAfterProject() { + var errorMessage = "Cannot use field [unsupported] with unsupported type [ip_range]"; + + verifyUnsupported(""" + from test + | keep unsupported + | where unsupported == null + """, errorMessage); + } + + public void testUnsupportedFieldFunctionAfterProject() { + var errorMessage = "Cannot use field [unsupported] with unsupported type [ip_range]"; + + verifyUnsupported(""" + from test + | keep unsupported + | where length(unsupported) > 0 + """, errorMessage); + } + + public void testUnsupportedFieldSortAfterProject() { + var errorMessage = "Cannot use field [unsupported] with unsupported type [ip_range]"; + + verifyUnsupported(""" + from test + | keep unsupported + | sort unsupported + """, errorMessage); + } + + public void testIncludeUnsupportedFieldPattern() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | keep un* + """)); + assertThat(e.getMessage(), containsString("No match found for [un*]")); + } + + public void testDropUnsupportedFieldExplicit() { + assertProjectionWithMapping( + """ + from test + | drop unsupported + """, + "mapping-multi-field-variation.json", + "bool", + "date", + "date_nanos", + "float", + "foo_type", + "int", + "keyword", + "point", + "shape", + "some.ambiguous", + "some.ambiguous.normalized", + "some.ambiguous.one", + "some.ambiguous.two", + "some.dotted.field", + "some.string", + "some.string.normalized", + "some.string.typical", + "text", + "unsigned_long", + "version" + ); + } + + public void testDropMultipleUnsupportedFieldsExplicitly() { + verifyUnsupported(""" + from test + | drop languages, gender + """, "Unknown column [languages]"); + } + + public void testDropPatternUnsupportedFields() { + assertProjection(""" + from test + | drop *ala* + """, "_meta_field", "emp_no", "first_name", "gender", "languages", "last_name"); + } + + public void testDropUnsupportedPattern() { + assertProjectionWithMapping( + """ + from test + | drop un* + """, + "mapping-multi-field-variation.json", + "bool", + "date", + "date_nanos", + "float", + "foo_type", + "int", + "keyword", + "point", + "shape", + "some.ambiguous", + "some.ambiguous.normalized", + "some.ambiguous.one", + "some.ambiguous.two", + "some.dotted.field", + "some.string", + "some.string.normalized", + "some.string.typical", + "text", + "version" + ); + } + + public void testRename() { + assertProjection(""" + from test + | rename emp_no as e + | keep first_name, e + """, "first_name", "e"); + } + + public void testChainedRename() { + assertProjection(""" + from test + | rename emp_no as r1, r1 as r2, r2 as r3 + | keep first_name, r3 + """, "first_name", "r3"); + } + + public void testChainedRenameReuse() { + assertProjection(""" + from test + | rename emp_no as r1, r1 as r2, r2 as r3, first_name as r1 + | keep r1, r3 + """, "r1", "r3"); + } + + public void testRenameBackAndForth() { + assertProjection(""" + from test + | rename emp_no as r1, r1 as emp_no + | keep emp_no + """, "emp_no"); + } + + public void testRenameReuseAlias() { + assertProjection(""" + from test + | rename emp_no as e, first_name as e + """, "_meta_field", "e", "gender", "languages", "last_name", "salary"); + } + + public void testRenameUnsupportedField() { + assertProjectionWithMapping(""" + from test + | rename unsupported as u + | keep int, u, float + """, "mapping-multi-field-variation.json", "int", "u", "float"); + } + + public void testRenameUnsupportedFieldChained() { + assertProjectionWithMapping(""" + from test + | rename unsupported as u1, u1 as u2 + | keep int, u2, float + """, "mapping-multi-field-variation.json", "int", "u2", "float"); + } + + public void testRenameUnsupportedAndResolved() { + assertProjectionWithMapping(""" + from test + | rename unsupported as u, float as f + | keep int, u, f + """, "mapping-multi-field-variation.json", "int", "u", "f"); + } + + public void testRenameUnsupportedSubFieldAndResolved() { + assertProjectionWithMapping(""" + from test + | rename some.string as ss, float as f + | keep int, ss, f + """, "mapping-multi-field-variation.json", "int", "ss", "f"); + } + + public void testRenameUnsupportedAndUnknown() { + verifyUnsupported(""" + from test + | rename text as t, doesnotexist as d + """, "Found 1 problem\n" + "line 2:21: Unknown column [doesnotexist]"); + } + + public void testRenameResolvedAndUnknown() { + verifyUnsupported(""" + from test + | rename int as i, doesnotexist as d + """, "Found 1 problem\n" + "line 2:20: Unknown column [doesnotexist]"); + } + + public void testUnsupportedFieldUsedExplicitly() { + assertProjectionWithMapping(""" + from test + | keep foo_type + """, "mapping-multi-field-variation.json", "foo_type"); + } + + public void testUnsupportedFieldTypes() { + assertProjectionWithMapping(""" + from test + | keep unsigned_long, date, date_nanos, unsupported, point, version + """, "mapping-multi-field-variation.json", "unsigned_long", "date", "date_nanos", "unsupported", "point", "version"); + } + + public void testUnsupportedDottedFieldUsedExplicitly() { + assertProjectionWithMapping(""" + from test + | keep some.string + """, "mapping-multi-field-variation.json", "some.string"); + } + + public void testUnsupportedParentField() { + verifyUnsupported( + """ + from test + | keep text, text.keyword + """, + "Found 1 problem\n" + "line 2:14: Unknown column [text.keyword], did you mean any of [text.wildcard, text.raw]?", + "mapping-multi-field.json" + ); + } + + public void testUnsupportedParentFieldAndItsSubField() { + assertProjectionWithMapping(""" + from test + | keep text, text.english + """, "mapping-multi-field.json", "text", "text.english"); + } + + public void testUnsupportedDeepHierarchy() { + assertProjectionWithMapping(""" + from test + | keep x.y.z.w, x.y.z, x.y, x + """, "mapping-multi-field-with-nested.json", "x.y.z.w", "x.y.z", "x.y", "x"); + } + + /** + * Here x.y.z.v is of type "keyword" but its parent is of unsupported type "foobar". + */ + public void testUnsupportedValidFieldTypeInDeepHierarchy() { + assertProjectionWithMapping(""" + from test + | keep x.y.z.v + """, "mapping-multi-field-with-nested.json", "x.y.z.v"); + } + + public void testUnsupportedValidFieldTypeInNestedParentField() { + verifyUnsupported(""" + from test + | keep dep.dep_id.keyword + """, "Found 1 problem\n" + "line 2:8: Unknown column [dep.dep_id.keyword]", "mapping-multi-field-with-nested.json"); + } + + public void testUnsupportedObjectAndNested() { + verifyUnsupported( + """ + from test + | keep dep, some + """, + "Found 2 problems\n" + "line 2:8: Unknown column [dep]\n" + "line 2:13: Unknown column [some]", + "mapping-multi-field-with-nested.json" + ); + } + + public void testDropNestedField() { + verifyUnsupported( + """ + from test + | drop dep, dep.dep_id.keyword + """, + "Found 2 problems\n" + "line 2:8: Unknown column [dep]\n" + "line 2:13: Unknown column [dep.dep_id.keyword]", + "mapping-multi-field-with-nested.json" + ); + } + + public void testDropNestedWildcardField() { + verifyUnsupported(""" + from test + | drop dep.* + """, "Found 1 problem\n" + "line 2:8: No match found for [dep.*]", "mapping-multi-field-with-nested.json"); + } + + public void testSupportedDeepHierarchy() { + assertProjectionWithMapping(""" + from test + | keep some.dotted.field, some.string.normalized + """, "mapping-multi-field-with-nested.json", "some.dotted.field", "some.string.normalized"); + } + + public void testDropSupportedDottedField() { + assertProjectionWithMapping( + """ + from test + | drop some.dotted.field + """, + "mapping-multi-field-variation.json", + "bool", + "date", + "date_nanos", + "float", + "foo_type", + "int", + "keyword", + "point", + "shape", + "some.ambiguous", + "some.ambiguous.normalized", + "some.ambiguous.one", + "some.ambiguous.two", + "some.string", + "some.string.normalized", + "some.string.typical", + "text", + "unsigned_long", + "unsupported", + "version" + ); + } + + public void testImplicitProjectionOfDeeplyComplexMapping() { + assertProjectionWithMapping( + "from test", + "mapping-multi-field-with-nested.json", + "binary", + "binary_stored", + "bool", + "date", + "date_nanos", + "geo_shape", + "int", + "keyword", + "shape", + "some.ambiguous", + "some.ambiguous.normalized", + "some.ambiguous.one", + "some.ambiguous.two", + "some.dotted.field", + "some.string", + "some.string.normalized", + "some.string.typical", + "text", + "unsigned_long", + "unsupported", + "x", + "x.y", + "x.y.z", + "x.y.z.v", + "x.y.z.w" + ); + } + + public void testDropWildcardDottedField() { + assertProjectionWithMapping( + """ + from test + | drop some.ambiguous.* + """, + "mapping-multi-field-with-nested.json", + "binary", + "binary_stored", + "bool", + "date", + "date_nanos", + "geo_shape", + "int", + "keyword", + "shape", + "some.ambiguous", + "some.dotted.field", + "some.string", + "some.string.normalized", + "some.string.typical", + "text", + "unsigned_long", + "unsupported", + "x", + "x.y", + "x.y.z", + "x.y.z.v", + "x.y.z.w" + ); + } + + public void testDropWildcardDottedField2() { + assertProjectionWithMapping( + """ + from test + | drop some.* + """, + "mapping-multi-field-with-nested.json", + "binary", + "binary_stored", + "bool", + "date", + "date_nanos", + "geo_shape", + "int", + "keyword", + "shape", + "text", + "unsigned_long", + "unsupported", + "x", + "x.y", + "x.y.z", + "x.y.z.v", + "x.y.z.w" + ); + } + + public void testProjectOrderPatternWithDottedFields() { + assertProjectionWithMapping( + """ + from test + | keep *some.string*, *, some.ambiguous.two, keyword + """, + "mapping-multi-field-with-nested.json", + "some.string", + "some.string.normalized", + "some.string.typical", + "binary", + "binary_stored", + "bool", + "date", + "date_nanos", + "geo_shape", + "int", + "shape", + "some.ambiguous", + "some.ambiguous.normalized", + "some.ambiguous.one", + "some.dotted.field", + "text", + "unsigned_long", + "unsupported", + "x", + "x.y", + "x.y.z", + "x.y.z.v", + "x.y.z.w", + "some.ambiguous.two", + "keyword" + ); + } + + public void testUnsupportedFieldUsedExplicitly2() { + assertProjectionWithMapping(""" + from test + | keep keyword, point + """, "mapping-multi-field-variation.json", "keyword", "point"); + } + + public void testCantFilterAfterDrop() { + verifyUnsupported(""" + from test + | stats c = avg(float) by int + | drop int + | where int > 0 + """, "Unknown column [int]"); + } + + public void testProjectAggGroupsRefs() { + assertProjection(""" + from test + | stats c = count(salary) by last_name + | eval d = c + 1 + | keep d, last_name + """, "d", "last_name"); + } + + public void testExplicitProjectAndLimit() { + var plan = analyze(""" + from test + """); + var limit = as(plan, Limit.class); + as(limit.child(), EsRelation.class); + } + + private static final String[] COMPARISONS = new String[] { "==", "!=", "<", "<=", ">", ">=" }; + + public void testCompareIntToString() { + for (String comparison : COMPARISONS) { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | where emp_no COMPARISON "foo" + """.replace("COMPARISON", comparison))); + assertThat( + e.getMessage(), + containsString( + "first argument of [emp_no COMPARISON \"foo\"] is [numeric] so second argument must also be [numeric] but was [keyword]" + .replace("COMPARISON", comparison) + ) + ); + } + } + + public void testCompareStringToInt() { + for (String comparison : COMPARISONS) { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | where "foo" COMPARISON emp_no + """.replace("COMPARISON", comparison))); + assertThat( + e.getMessage(), + containsString( + "first argument of [\"foo\" COMPARISON emp_no] is [keyword] so second argument must also be [keyword] but was [integer]" + .replace("COMPARISON", comparison) + ) + ); + } + } + + public void testCompareDateToString() { + for (String comparison : COMPARISONS) { + assertProjectionWithMapping(""" + from test + | where date COMPARISON "1985-01-01T00:00:00Z" + | keep date + """.replace("COMPARISON", comparison), "mapping-multi-field-variation.json", "date"); + } + } + + public void testCompareStringToDate() { + for (String comparison : COMPARISONS) { + assertProjectionWithMapping(""" + from test + | where "1985-01-01T00:00:00Z" COMPARISON date + | keep date + """.replace("COMPARISON", comparison), "mapping-multi-field-variation.json", "date"); + } + } + + public void testCompareDateToStringFails() { + for (String comparison : COMPARISONS) { + verifyUnsupported(""" + from test + | where date COMPARISON "not-a-date" + | keep date + """.replace("COMPARISON", comparison), "Invalid date [not-a-date]", "mapping-multi-field-variation.json"); + } + } + + public void testDateFormatOnInt() { + verifyUnsupported(""" + from test + | eval date_format(int) + """, "first argument of [date_format(int)] must be [datetime], found value [int] type [integer]"); + } + + public void testDateFormatOnFloat() { + verifyUnsupported(""" + from test + | eval date_format(float) + """, "first argument of [date_format(float)] must be [datetime], found value [float] type [double]"); + } + + public void testDateFormatOnText() { + verifyUnsupported(""" + from test + | eval date_format(keyword) + """, "first argument of [date_format(keyword)] must be [datetime], found value [keyword] type [keyword]"); + } + + public void testDateFormatWithNumericFormat() { + verifyUnsupported(""" + from test + | eval date_format(date, 1) + """, "second argument of [date_format(date, 1)] must be [string], found value [1] type [integer]"); + } + + public void testDateFormatWithDateFormat() { + verifyUnsupported(""" + from test + | eval date_format(date, date) + """, "second argument of [date_format(date, date)] must be [string], found value [date] type [datetime]"); + } + + public void testDateParseOnInt() { + verifyUnsupported(""" + from test + | eval date_parse(int, keyword) + """, "first argument of [date_parse(int, keyword)] must be [string], found value [int] type [integer]"); + } + + public void testDateParseOnDate() { + verifyUnsupported(""" + from test + | eval date_parse(date, keyword) + """, "first argument of [date_parse(date, keyword)] must be [string], found value [date] type [datetime]"); + } + + public void testDateParseOnIntPattern() { + verifyUnsupported(""" + from test + | eval date_parse(keyword, int) + """, "second argument of [date_parse(keyword, int)] must be [string], found value [int] type [integer]"); + } + + public void testDateTruncOnInt() { + verifyUnsupported(""" + from test + | eval date_trunc(int, "1M") + """, "first argument of [date_trunc(int, \"1M\")] must be [datetime], found value [int] type [integer]"); + } + + public void testDateTruncOnFloat() { + verifyUnsupported(""" + from test + | eval date_trunc(float, "1M") + """, "first argument of [date_trunc(float, \"1M\")] must be [datetime], found value [float] type [double]"); + } + + public void testDateTruncOnText() { + verifyUnsupported(""" + from test + | eval date_trunc(keyword, "1M") + """, "first argument of [date_trunc(keyword, \"1M\")] must be [datetime], found value [keyword] type [keyword]"); + } + + public void testDateTruncWithNumericInterval() { + verifyUnsupported(""" + from test + | eval date_trunc(date, 1) + """, "second argument of [date_trunc(date, 1)] must be [dateperiod or timeduration], found value [1] type [integer]"); + } + + public void testDateTruncWithDateInterval() { + verifyUnsupported(""" + from test + | eval date_trunc(date, date) + """, "second argument of [date_trunc(date, date)] must be [dateperiod or timeduration], found value [date] type [datetime]"); + } + + // check field declaration is validated even across duplicated declarations + public void testAggsWithDuplicatesAndNonExistingFunction() throws Exception { + verifyUnsupported(""" + row a = 1, b = 2 + | stats x = non_existing(a), x = count(a) by b + """, "Unknown function [non_existing]"); + } + + // check field declaration is validated even across duplicated declarations + public void testAggsWithDuplicatesAndNonExistingField() throws Exception { + verifyUnsupported(""" + row a = 1, b = 2 + | stats x = max(non_existing), x = count(a) by b + """, "Unknown column [non_existing]"); + } + + // duplicates get merged after stats and do not prevent following commands to blow up + // due to ambiguity + public void testAggsWithDuplicates() throws Exception { + var plan = analyze(""" + row a = 1, b = 2 + | stats x = count(a), x = min(a), x = max(a) by b + | sort x + """); + + var limit = as(plan, Limit.class); + var order = as(limit.child(), OrderBy.class); + var agg = as(order.child(), Aggregate.class); + var aggregates = agg.aggregates(); + assertThat(aggregates, hasSize(2)); + assertThat(Expressions.names(aggregates), contains("x", "b")); + var alias = as(aggregates.get(0), Alias.class); + var max = as(alias.child(), Max.class); + } + + // expected stats b by b (grouping overrides the rest of the aggs) + public void testAggsWithOverridingInputAndGrouping() throws Exception { + var plan = analyze(""" + row a = 1, b = 2 + | stats b = count(a), b = max(a) by b + | sort b + """); + var limit = as(plan, Limit.class); + var order = as(limit.child(), OrderBy.class); + var agg = as(order.child(), Aggregate.class); + var aggregates = agg.aggregates(); + assertThat(aggregates, hasSize(1)); + assertThat(Expressions.names(aggregates), contains("b")); + } + + public void testAggsWithoutAgg() throws Exception { + var plan = analyze(""" + row a = 1, b = 2 + | stats by a + """); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + var aggregates = agg.aggregates(); + assertThat(aggregates, hasSize(1)); + assertThat(Expressions.names(aggregates), contains("a")); + assertThat(Expressions.names(agg.groupings()), contains("a")); + assertEquals(agg.groupings(), agg.aggregates()); + } + + public void testAggsWithoutAggAndFollowingCommand() throws Exception { + var plan = analyze(""" + row a = 1, b = 2 + | stats by a + | sort a + """); + + var limit = as(plan, Limit.class); + var order = as(limit.child(), OrderBy.class); + var agg = as(order.child(), Aggregate.class); + var aggregates = agg.aggregates(); + assertThat(aggregates, hasSize(1)); + assertThat(Expressions.names(aggregates), contains("a")); + assertThat(Expressions.names(agg.groupings()), contains("a")); + assertEquals(agg.groupings(), agg.aggregates()); + } + + public void testUnsupportedFieldsInStats() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + + verifyUnsupported(""" + from test + | stats max(point) + """, errorMsg); + verifyUnsupported(""" + from test + | stats max(int) by point + """, errorMsg); + verifyUnsupported(""" + from test + | stats max(int) by bool, point + """, errorMsg); + } + + public void testUnsupportedFieldsInEval() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + + verifyUnsupported(""" + from test + | eval x = point + """, errorMsg); + verifyUnsupported(""" + from test + | eval foo = 1, x = point + """, errorMsg); + verifyUnsupported(""" + from test + | eval x = 1 + point + """, errorMsg); + } + + public void testUnsupportedFieldsInWhere() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + + verifyUnsupported(""" + from test + | where point == "[1.0, 1.0]" + """, errorMsg); + verifyUnsupported(""" + from test + | where int > 2 and point == "[1.0, 1.0]" + """, errorMsg); + } + + public void testUnsupportedFieldsInSort() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + + verifyUnsupported(""" + from test + | sort point + """, errorMsg); + verifyUnsupported(""" + from test + | sort int, point + """, errorMsg); + } + + public void testUnsupportedFieldsInDissect() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + verifyUnsupported(""" + from test + | dissect point \"%{foo}\" + """, errorMsg); + } + + public void testUnsupportedFieldsInGrok() { + var errorMsg = "Cannot use field [point] with unsupported type [geo_point]"; + verifyUnsupported(""" + from test + | grok point \"%{WORD:foo}\" + """, errorMsg); + } + + public void testRegexOnInt() { + for (String op : new String[] { "like", "rlike" }) { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | where emp_no COMPARISON "foo" + """.replace("COMPARISON", op))); + assertThat( + e.getMessage(), + containsString( + "argument of [emp_no COMPARISON \"foo\"] must be [string], found value [emp_no] type [integer]".replace( + "COMPARISON", + op + ) + ) + ); + } + } + + public void testUnsupportedTypesWithToString() { + // DATE_PERIOD and TIME_DURATION types have been added, but not really patched through the engine; i.e. supported. + final String supportedTypes = "boolean, datetime, double, integer, ip, keyword, long, unsigned_long or version"; + verifyUnsupported( + "row period = 1 year | eval to_string(period)", + "line 1:28: argument of [to_string(period)] must be [" + supportedTypes + "], found value [period] type [date_period]" + ); + verifyUnsupported( + "row duration = 1 hour | eval to_string(duration)", + "line 1:30: argument of [to_string(duration)] must be [" + supportedTypes + "], found value [duration] type [time_duration]" + ); + verifyUnsupported("from test | eval to_string(point)", "line 1:28: Cannot use field [point] with unsupported type [geo_point]"); + } + + public void testNonExistingEnrichPolicy() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | enrich foo on bar + """)); + assertThat(e.getMessage(), containsString("unresolved enrich policy [foo]")); + } + + public void testNonExistingEnrichPolicyWithSimilarName() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | enrich language on bar + """)); + assertThat(e.getMessage(), containsString("unresolved enrich policy [language], did you mean [languages]")); + } + + public void testEnrichPolicyMatchFieldName() { + verifyUnsupported("from test | enrich languages on bar", "Unknown column [bar]"); + verifyUnsupported("from test | enrich languages on keywords", "Unknown column [keywords], did you mean [keyword]?"); + verifyUnsupported("from test | enrich languages on keyword with foo", "Enrich field [foo] not found in enrich policy [languages]"); + verifyUnsupported( + "from test | enrich languages on keyword with language_namez", + "Enrich field [language_namez] not found in enrich policy [languages], did you mean [language_name]" + ); + verifyUnsupported( + "from test | enrich languages on keyword with x = language_namez", + "Enrich field [language_namez] not found in enrich policy [languages], did you mean [language_name]" + ); + } + + public void testEnrichWrongMatchFieldType() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | enrich languages on languages + | keep first_name, language_name, id + """)); + assertThat( + e.getMessage(), + containsString("Unsupported type [INTEGER] for enrich matching field [languages]; only KEYWORD allowed") + ); + } + + public void testValidEnrich() { + assertProjection(""" + from test + | eval x = to_string(languages) + | enrich languages on x + | keep first_name, language_name + """, "first_name", "language_name"); + + assertProjection(""" + from test + | eval x = to_string(languages) + | enrich languages on x with language_name + | keep first_name, language_name + """, "first_name", "language_name"); + + assertProjection(""" + from test + | eval x = to_string(languages) + | enrich languages on x with y = language_name + | keep first_name, y + """, "first_name", "y"); + } + + public void testEnrichExcludesPolicyKey() { + var e = expectThrows(VerificationException.class, () -> analyze(""" + from test + | eval x = to_string(languages) + | enrich languages on x + | keep first_name, language_name, id + """)); + assertThat(e.getMessage(), containsString("Unknown column [id]")); + } + + private void verifyUnsupported(String query, String errorMessage) { + verifyUnsupported(query, errorMessage, "mapping-multi-field-variation.json"); + } + + private void verifyUnsupported(String query, String errorMessage, String mappingFileName) { + var e = expectThrows(VerificationException.class, () -> analyze(query, mappingFileName)); + assertThat(e.getMessage(), containsString(errorMessage)); + } + + private void assertProjection(String query, String... names) { + var plan = analyze(query); + var limit = as(plan, Limit.class); + assertThat(Expressions.names(limit.output()), contains(names)); + } + + private void assertProjectionTypes(String query, DataType... types) { + var plan = analyze(query); + var limit = as(plan, Limit.class); + assertThat(limit.output().stream().map(NamedExpression::dataType).toList(), contains(types)); + } + + private void assertProjectionWithMapping(String query, String mapping, String... names) { + var plan = analyze(query, mapping.toString()); + var limit = as(plan, Limit.class); + assertThat(Expressions.names(limit.output()), contains(names)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java new file mode 100644 index 0000000000000..d3d764c40af1f --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java @@ -0,0 +1,48 @@ +/* + * 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.analysis; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.ql.ParsingException; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.type.TypesTests; + +import static org.elasticsearch.xpack.esql.EsqlTestUtils.TEST_CFG; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyPolicyResolution; + +public class ParsingTests extends ESTestCase { + private static final String INDEX_NAME = "test"; + private static final EsqlParser parser = new EsqlParser(); + + private final IndexResolution defaultIndex = loadIndexResolution("mapping-basic.json"); + private final Analyzer defaultAnalyzer = new Analyzer( + new AnalyzerContext(TEST_CFG, new EsqlFunctionRegistry(), defaultIndex, emptyPolicyResolution()), + new Verifier(new Metrics()) + ); + + public void testConcatFunctionInvalidInputs() { + assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat()")); + assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(a)")); + assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(123)")); + } + + private String error(String query) { + ParsingException e = expectThrows(ParsingException.class, () -> defaultAnalyzer.analyze(parser.createStatement(query))); + String message = e.getMessage(); + assertTrue(message.startsWith("line ")); + return message.substring("line ".length()); + } + + private static IndexResolution loadIndexResolution(String name) { + return IndexResolution.valid(new EsIndex(INDEX_NAME, TypesTests.loadMapping(name))); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java new file mode 100644 index 0000000000000..1dd7661a1b74b --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -0,0 +1,285 @@ +/* + * 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.analysis; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.parser.TypedParamValue; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG; +import static org.hamcrest.Matchers.containsString; + +public class VerifierTests extends ESTestCase { + + private static final EsqlParser parser = new EsqlParser(); + private final Analyzer defaultAnalyzer = AnalyzerTestUtils.expandedDefaultAnalyzer(); + + public void testIncompatibleTypesInMathOperation() { + assertEquals( + "1:40: second argument of [a + c] must be [numeric], found value [c] type [keyword]", + error("row a = 1, b = 2, c = \"xxx\" | eval y = a + c") + ); + assertEquals( + "1:40: second argument of [a - c] must be [numeric], found value [c] type [keyword]", + error("row a = 1, b = 2, c = \"xxx\" | eval y = a - c") + ); + } + + public void testRoundFunctionInvalidInputs() { + assertEquals( + "1:31: first argument of [round(b, 3)] must be [numeric], found value [b] type [keyword]", + error("row a = 1, b = \"c\" | eval x = round(b, 3)") + ); + assertEquals( + "1:31: first argument of [round(b)] must be [numeric], found value [b] type [keyword]", + error("row a = 1, b = \"c\" | eval x = round(b)") + ); + assertEquals( + "1:31: second argument of [round(a, b)] must be [integer], found value [b] type [keyword]", + error("row a = 1, b = \"c\" | eval x = round(a, b)") + ); + assertEquals( + "1:31: second argument of [round(a, 3.5)] must be [integer], found value [3.5] type [double]", + error("row a = 1, b = \"c\" | eval x = round(a, 3.5)") + ); + assertEquals( + "1:9: second argument of [round(123.45, \"1\")] must be [integer], found value [\"1\"] type [keyword]", + error("row a = round(123.45, \"1\")") + ); + } + + public void testAggsExpressionsInStatsAggs() { + assertEquals( + "1:44: expected an aggregate function or group but got [salary] of type [FieldAttribute]", + error("from test | eval z = 2 | stats x = avg(z), salary by emp_no") + ); + assertEquals( + "1:19: expected an aggregate function or group but got [length(first_name)] of type [Length]", + error("from test | stats length(first_name), count(1) by first_name") + ); + assertEquals( + "1:19: aggregate function's parameters must be an attribute or literal; found [emp_no / 2] of type [Div]", + error("from test | stats x = avg(emp_no / 2) by emp_no") + ); + assertEquals( + "1:25: argument of [avg(first_name)] must be [numeric], found value [first_name] type [keyword]", + error("from test | stats count(avg(first_name)) by first_name") + ); + assertEquals( + "1:19: aggregate function's parameters must be an attribute or literal; found [length(first_name)] of type [Length]", + error("from test | stats count(length(first_name)) by first_name") + ); + assertEquals( + "1:23: expected an aggregate function or group but got [emp_no + avg(emp_no)] of type [Add]", + error("from test | stats x = emp_no + avg(emp_no) by emp_no") + ); + } + + public void testDoubleRenamingField() { + assertEquals( + "1:44: Column [emp_no] renamed to [r1] and is no longer available [emp_no as r3]", + error("from test | rename emp_no as r1, r1 as r2, emp_no as r3 | keep r3") + ); + } + + public void testDuplicateRenaming() { + assertEquals( + "1:34: Column [emp_no] renamed to [r1] and is no longer available [emp_no as r1]", + error("from test | rename emp_no as r1, emp_no as r1 | keep r1") + ); + } + + public void testDoubleRenamingReference() { + assertEquals( + "1:61: Column [r1] renamed to [r2] and is no longer available [r1 as r3]", + error("from test | rename emp_no as r1, r1 as r2, first_name as x, r1 as r3 | keep r3") + ); + } + + public void testDropAfterRenaming() { + assertEquals("1:40: Unknown column [emp_no]", error("from test | rename emp_no as r1 | drop emp_no")); + } + + public void testNonStringFieldsInDissect() { + assertEquals( + "1:21: Dissect only supports KEYWORD or TEXT values, found expression [emp_no] type [INTEGER]", + error("from test | dissect emp_no \"%{foo}\"") + ); + } + + public void testNonStringFieldsInGrok() { + assertEquals( + "1:18: Grok only supports KEYWORD or TEXT values, found expression [emp_no] type [INTEGER]", + error("from test | grok emp_no \"%{WORD:foo}\"") + ); + } + + public void testMixedNonConvertibleTypesInIn() { + assertEquals( + "1:19: 2nd argument of [emp_no in (1, \"two\")] must be [integer], found value [\"two\"] type [keyword]", + error("from test | where emp_no in (1, \"two\")") + ); + } + + public void testMixedNumericalNonConvertibleTypesInIn() { + assertEquals( + "1:19: 2nd argument of [3 in (1, to_ul(3))] must be [integer], found value [to_ul(3)] type [unsigned_long]", + error("from test | where 3 in (1, to_ul(3))") + ); + assertEquals( + "1:19: 1st argument of [to_ul(3) in (1, 3)] must be [unsigned_long], found value [1] type [integer]", + error("from test | where to_ul(3) in (1, 3)") + ); + } + + public void testUnsignedLongTypeMixInComparisons() { + List types = EsqlDataTypes.types() + .stream() + .filter(dt -> dt.isNumeric() && EsqlDataTypes.isRepresentable(dt) && dt != UNSIGNED_LONG) + .map(DataType::typeName) + .toList(); + for (var type : types) { + for (var comp : List.of("==", "!=", ">", ">=", "<=", "<")) { + String left, right, leftType, rightType; + if (randomBoolean()) { + left = "ul"; + leftType = "unsigned_long"; + right = "n"; + rightType = type; + } else { + left = "n"; + leftType = type; + right = "ul"; + rightType = "unsigned_long"; + } + var operation = left + " " + comp + " " + right; + assertThat( + error("row n = to_" + type + "(1), ul = to_ul(1) | where " + operation), + containsString( + "first argument of [" + + operation + + "] is [" + + leftType + + "] and second is [" + + rightType + + "]." + + " [unsigned_long] can only be operated on together with another [unsigned_long]" + ) + ); + } + } + } + + public void testUnsignedLongTypeMixInArithmetics() { + List types = EsqlDataTypes.types() + .stream() + .filter(dt -> dt.isNumeric() && EsqlDataTypes.isRepresentable(dt) && dt != UNSIGNED_LONG) + .map(DataType::typeName) + .toList(); + for (var type : types) { + for (var operation : List.of("+", "-", "*", "/", "%")) { + String left, right, leftType, rightType; + if (randomBoolean()) { + left = "ul"; + leftType = "unsigned_long"; + right = "n"; + rightType = type; + } else { + left = "n"; + leftType = type; + right = "ul"; + rightType = "unsigned_long"; + } + var op = left + " " + operation + " " + right; + assertThat( + error("row n = to_" + type + "(1), ul = to_ul(1) | eval " + op), + containsString( + "first argument of [" + + op + + "] is [" + + leftType + + "] and second is [" + + rightType + + "]." + + " [unsigned_long] can only be operated on together with another [unsigned_long]" + ) + ); + } + } + } + + public void testUnsignedLongNegation() { + assertEquals( + "1:29: negation unsupported for arguments of type [unsigned_long] in expression [-x]", + error("row x = to_ul(1) | eval y = -x") + ); + } + + public void testSumOnDate() { + assertEquals( + "1:19: argument of [sum(hire_date)] must be [numeric], found value [hire_date] type [datetime]", + error("from test | stats sum(hire_date)") + ); + } + + public void testWrongInputParam() { + assertEquals( + "1:19: first argument of [emp_no == ?] is [numeric] so second argument must also be [numeric] but was [keyword]", + error("from test | where emp_no == ?", "foo") + ); + + assertEquals( + "1:19: first argument of [emp_no == ?] is [numeric] so second argument must also be [numeric] but was [null]", + error("from test | where emp_no == ?", new Object[] { null }) + ); + } + + public void testPeriodAndDurationInRowAssignment() { + for (var unit : List.of("millisecond", "second", "minute", "hour", "day", "week", "month", "year")) { + assertEquals("1:5: cannot use [1 " + unit + "] directly in a row assignment", error("row a = 1 " + unit)); + } + } + + private String error(String query) { + return error(query, defaultAnalyzer); + + } + + private String error(String query, Object... params) { + return error(query, defaultAnalyzer, params); + } + + private String error(String query, Analyzer analyzer, Object... params) { + List parameters = new ArrayList<>(); + for (Object param : params) { + if (param == null) { + parameters.add(new TypedParamValue("null", null)); + } else if (param instanceof String) { + parameters.add(new TypedParamValue("keyword", param)); + } else if (param instanceof Number) { + parameters.add(new TypedParamValue("param", param)); + } else { + throw new IllegalArgumentException("VerifierTests don't support params of type " + param.getClass()); + } + } + VerificationException e = expectThrows( + VerificationException.class, + () -> analyzer.analyze(parser.createStatement(query, parameters)) + ); + String message = e.getMessage(); + assertTrue(message.startsWith("Found ")); + String pattern = "\nline "; + int index = message.indexOf(pattern); + return message.substring(index + pattern.length()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperatorTests.java new file mode 100644 index 0000000000000..107f749aefa0f --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichQuerySourceOperatorTests.java @@ -0,0 +1,214 @@ +/* + * 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.enrich; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.TermInSetQuery; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.tests.store.MockDirectoryWrapper; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.test.ESTestCase; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; + +public class EnrichQuerySourceOperatorTests extends ESTestCase { + + public void testQueries() throws Exception { + MockDirectoryWrapper dir = newMockDirectory(); + IndexWriterConfig iwc = new IndexWriterConfig(); + iwc.setMergePolicy(NoMergePolicy.INSTANCE); + IndexWriter writer = new IndexWriter(dir, iwc); + List> terms = List.of( + List.of("a2"), + List.of("a1", "c1", "b2"), + List.of("a2"), + List.of("a3"), + List.of("b2", "b1", "a1") + ); + for (List ts : terms) { + Document doc = new Document(); + for (String t : ts) { + doc.add(new StringField("uid", t, Field.Store.NO)); + } + writer.addDocument(doc); + } + writer.commit(); + DirectoryReader reader = DirectoryReader.open(writer); + writer.close(); + + BytesRefBlock inputTerms = BytesRefBlock.newBlockBuilder(6) + .appendBytesRef(new BytesRef("b2")) + .beginPositionEntry() + .appendBytesRef(new BytesRef("c1")) + .appendBytesRef(new BytesRef("a2")) + .endPositionEntry() + .appendBytesRef(new BytesRef("z2")) + .appendNull() + .appendBytesRef(new BytesRef("a3")) + .appendNull() + .build(); + + MappedFieldType uidField = new KeywordFieldMapper.KeywordFieldType("uid"); + QueryList queryList = QueryList.termQueryList(uidField, mock(SearchExecutionContext.class), inputTerms); + assertThat(queryList.getPositionCount(), equalTo(6)); + assertThat(queryList.getQuery(0), equalTo(new TermQuery(new Term("uid", new BytesRef("b2"))))); + assertThat(queryList.getQuery(1), equalTo(new TermInSetQuery("uid", new BytesRef("c1"), new BytesRef("a2")))); + assertThat(queryList.getQuery(2), equalTo(new TermQuery(new Term("uid", new BytesRef("z2"))))); + assertNull(queryList.getQuery(3)); + assertThat(queryList.getQuery(4), equalTo(new TermQuery(new Term("uid", new BytesRef("a3"))))); + assertNull(queryList.getQuery(5)); + // pos -> terms -> docs + // ----------------------------- + // 0 -> [b2] -> [1, 4] + // 1 -> [c1, a2] -> [1, 0, 2] + // 2 -> [z2] -> [] + // 3 -> [] -> [] + // 4 -> [a1] -> [3] + // 5 -> [] -> [] + EnrichQuerySourceOperator queryOperator = new EnrichQuerySourceOperator(queryList, reader); + { + Page p0 = queryOperator.getOutput(); + assertNotNull(p0); + assertThat(p0.getPositionCount(), equalTo(2)); + IntVector docs = getDocVector(p0, 0); + assertThat(docs.getInt(0), equalTo(1)); + assertThat(docs.getInt(1), equalTo(4)); + Block positions = p0.getBlock(1); + assertThat(BlockUtils.toJavaObject(positions, 0), equalTo(0)); + assertThat(BlockUtils.toJavaObject(positions, 1), equalTo(0)); + } + { + Page p1 = queryOperator.getOutput(); + assertNotNull(p1); + assertThat(p1.getPositionCount(), equalTo(3)); + IntVector docs = getDocVector(p1, 0); + assertThat(docs.getInt(0), equalTo(0)); + assertThat(docs.getInt(1), equalTo(1)); + assertThat(docs.getInt(2), equalTo(2)); + Block positions = p1.getBlock(1); + assertThat(BlockUtils.toJavaObject(positions, 0), equalTo(1)); + assertThat(BlockUtils.toJavaObject(positions, 1), equalTo(1)); + assertThat(BlockUtils.toJavaObject(positions, 2), equalTo(1)); + } + { + Page p2 = queryOperator.getOutput(); + assertNull(p2); + } + { + Page p3 = queryOperator.getOutput(); + assertNull(p3); + } + { + Page p4 = queryOperator.getOutput(); + assertNotNull(p4); + assertThat(p4.getPositionCount(), equalTo(1)); + IntVector docs = getDocVector(p4, 0); + assertThat(docs.getInt(0), equalTo(3)); + Block positions = p4.getBlock(1); + assertThat(BlockUtils.toJavaObject(positions, 0), equalTo(4)); + } + { + Page p5 = queryOperator.getOutput(); + assertNull(p5); + } + { + assertFalse(queryOperator.isFinished()); + Page p6 = queryOperator.getOutput(); + assertNull(p6); + } + assertTrue(queryOperator.isFinished()); + IOUtils.close(reader, dir); + } + + public void testRandomMatchQueries() throws Exception { + MockDirectoryWrapper dir = newMockDirectory(); + IndexWriterConfig iwc = new IndexWriterConfig(); + iwc.setMergePolicy(NoMergePolicy.INSTANCE); + IndexWriter writer = new IndexWriter(dir, iwc); + int numTerms = randomIntBetween(10, 1000); + Map terms = new HashMap<>(); + for (int i = 0; i < numTerms; i++) { + Document doc = new Document(); + String term = "term-" + i; + terms.put(term, i); + doc.add(new StringField("uid", term, Field.Store.NO)); + writer.addDocument(doc); + } + writer.forceMerge(1); + writer.commit(); + DirectoryReader reader = DirectoryReader.open(writer); + writer.close(); + + Map> expectedPositions = new HashMap<>(); + int numPositions = randomIntBetween(1, 1000); + BytesRefBlock.Builder inputTerms = BytesRefBlock.newBlockBuilder(numPositions); + for (int i = 0; i < numPositions; i++) { + if (randomBoolean()) { + String term = randomFrom(terms.keySet()); + inputTerms.appendBytesRef(new BytesRef(term)); + Integer position = terms.get(term); + expectedPositions.put(i, Set.of(position)); + } else { + if (randomBoolean()) { + inputTerms.appendNull(); + } else { + String term = "other-" + randomIntBetween(1, 100); + inputTerms.appendBytesRef(new BytesRef(term)); + } + } + } + MappedFieldType uidField = new KeywordFieldMapper.KeywordFieldType("uid"); + QueryList queryList = QueryList.termQueryList(uidField, mock(SearchExecutionContext.class), inputTerms.build()); + EnrichQuerySourceOperator queryOperator = new EnrichQuerySourceOperator(queryList, reader); + Map> actualPositions = new HashMap<>(); + while (queryOperator.isFinished() == false) { + Page page = queryOperator.getOutput(); + if (page != null) { + IntVector docs = getDocVector(page, 0); + IntBlock positions = page.getBlock(1); + for (int i = 0; i < page.getPositionCount(); i++) { + int doc = docs.getInt(i); + int position = positions.getInt(i); + actualPositions.computeIfAbsent(position, k -> new HashSet<>()).add(doc); + } + } + } + assertThat(actualPositions, equalTo(expectedPositions)); + IOUtils.close(reader, dir); + } + + private static IntVector getDocVector(Page page, int blockIndex) { + DocBlock doc = page.getBlock(blockIndex); + return doc.asVector().docs(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperatorTests.java new file mode 100644 index 0000000000000..0a0f6e5217044 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/MergePositionsOperatorTests.java @@ -0,0 +1,102 @@ +/* + * 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.enrich; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.ConstantIntVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.test.ESTestCase; + +import java.util.List; + +import static org.hamcrest.Matchers.equalTo; + +public class MergePositionsOperatorTests extends ESTestCase { + + public void testSimple() { + MergePositionsOperator mergeOperator = new MergePositionsOperator( + randomBoolean(), + 7, + 0, + new int[] { 1, 2 }, + new ElementType[] { ElementType.BYTES_REF, ElementType.INT } + ); + mergeOperator.addInput( + new Page( + new ConstantIntVector(1, 1).asBlock(), + BytesRefBlock.newBlockBuilder(1).appendBytesRef(new BytesRef("w0")).build(), + IntBlock.newBlockBuilder(1).appendNull().build() + ) + ); + mergeOperator.addInput( + new Page( + new ConstantIntVector(2, 1).asBlock(), + BytesRefBlock.newBlockBuilder(1) + .beginPositionEntry() + .appendBytesRef(new BytesRef("a1")) + .appendBytesRef(new BytesRef("c1")) + .endPositionEntry() + .build(), + IntBlock.newBlockBuilder(1).appendNull().build() + ) + ); + mergeOperator.addInput( + new Page( + new ConstantIntVector(3, 2).asBlock(), + BytesRefBlock.newBlockBuilder(1) + .appendBytesRef(new BytesRef("f5")) + .beginPositionEntry() + .appendBytesRef(new BytesRef("k1")) + .appendBytesRef(new BytesRef("k2")) + .endPositionEntry() + .build(), + IntBlock.newBlockBuilder(1).appendInt(2020).appendInt(2021).build() + ) + ); + mergeOperator.addInput( + new Page( + new ConstantIntVector(5, 1).asBlock(), + BytesRefBlock.newBlockBuilder(1) + .beginPositionEntry() + .appendBytesRef(new BytesRef("r2")) + .appendBytesRef(new BytesRef("k2")) + .endPositionEntry() + .build(), + IntBlock.newBlockBuilder(1).appendInt(2023).build() + ) + ); + mergeOperator.finish(); + Page out = mergeOperator.getOutput(); + assertTrue(mergeOperator.isFinished()); + assertNotNull(out); + assertThat(out.getPositionCount(), equalTo(7)); + assertThat(out.getBlockCount(), equalTo(2)); + BytesRefBlock f1 = out.getBlock(0); + IntBlock f2 = out.getBlock(1); + + assertTrue(f1.isNull(0)); + assertThat(BlockUtils.toJavaObject(f1, 1), equalTo(new BytesRef("w0"))); + assertThat(BlockUtils.toJavaObject(f1, 2), equalTo(List.of(new BytesRef("a1"), new BytesRef("c1")))); + assertThat(BlockUtils.toJavaObject(f1, 3), equalTo(List.of(new BytesRef("f5"), new BytesRef("k1"), new BytesRef("k2")))); + assertTrue(f1.isNull(4)); + assertThat(BlockUtils.toJavaObject(f1, 5), equalTo(List.of(new BytesRef("r2"), new BytesRef("k2")))); + assertTrue(f1.isNull(6)); + + assertTrue(f2.isNull(0)); + assertTrue(f2.isNull(1)); + assertTrue(f2.isNull(2)); + assertThat(BlockUtils.toJavaObject(f2, 3), equalTo(List.of(2020, 2021))); + assertTrue(f2.isNull(4)); + assertThat(BlockUtils.toJavaObject(f2, 5), equalTo(2023)); + assertTrue(f2.isNull(6)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java new file mode 100644 index 0000000000000..60da1359ebc7e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java @@ -0,0 +1,301 @@ +/* + * 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.expression.function; + +import org.apache.lucene.document.InetAddressPoint; +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.planner.EvalMapper; +import org.elasticsearch.xpack.esql.planner.Layout; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.EsField; +import org.elasticsearch.xpack.versionfield.Version; +import org.hamcrest.Matcher; + +import java.time.Duration; +import java.time.Period; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; +import static org.hamcrest.Matchers.equalTo; + +/** + * Base class for function tests. Tests based on this class will generally build out a single example evaluation, + * which can be automatically tested against several scenarios (null handling, concurrency, etc). + */ +public abstract class AbstractFunctionTestCase extends ESTestCase { + + /** + * Holds a data value and the intended parse type of that value + * @param data - value to test against + * @param type - type of the value, for building expressions + */ + public record TypedData(Object data, DataType type, String name) { + public TypedData(Object data, String name) { + this(data, EsqlDataTypes.fromJava(data), name); + } + } + + public static class TestCase { + /** + * The {@link Source} this test case should be run with + */ + private Source source; + /** + * The parameter values and types to pass into the function for this test run + */ + private List data; + + /** + * The expected toString output for the evaluator this fuction invocation should generate + */ + String evaluatorToString; + /** + * The expected output type for the case being tested + */ + DataType expectedType; + /** + * A matcher to validate the output of the function run on the given input data + */ + private Matcher matcher; + + public TestCase(List data, String evaluatorToString, DataType expectedType, Matcher matcher) { + this.source = Source.EMPTY; + this.data = data; + this.evaluatorToString = evaluatorToString; + this.expectedType = expectedType; + this.matcher = matcher; + } + + public Source getSource() { + return source; + } + + public List getData() { + return data; + } + + public List getDataAsFields() { + return data.stream().map(t -> field(t.name(), t.type())).collect(Collectors.toList()); + } + + public List getDataAsLiterals() { + return data.stream().map(t -> new Literal(source, t.data(), t.type())).collect(Collectors.toList()); + } + + public List getDataValues() { + return data.stream().map(t -> t.data()).collect(Collectors.toList()); + } + + public Matcher getMatcher() { + return matcher; + } + } + + /** + * This class exists to give a human-readable string representation of the test case. + */ + protected static class TestCaseSupplier implements Supplier { + + private String name; + private final Supplier wrapped; + + public TestCaseSupplier(String name, Supplier wrapped) { + this.name = name; + this.wrapped = wrapped; + } + + @Override + public TestCase get() { + return wrapped.get(); + } + + @Override + public String toString() { + return name; + } + } + + /** + * Generate a random value of the appropriate type to fit into blocks of {@code e}. + */ + public static Literal randomLiteral(DataType type) { + return new Literal(Source.EMPTY, switch (type.typeName()) { + case "boolean" -> randomBoolean(); + case "byte" -> randomByte(); + case "short" -> randomShort(); + case "integer" -> randomInt(); + case "unsigned_long", "long" -> randomLong(); + case "date_period" -> Period.ofDays(randomInt(10)); + case "datetime" -> randomMillisUpToYear9999(); + case "double", "scaled_float" -> randomDouble(); + case "float" -> randomFloat(); + case "half_float" -> HalfFloatPoint.sortableShortToHalfFloat(HalfFloatPoint.halfFloatToSortableShort(randomFloat())); + case "keyword" -> new BytesRef(randomAlphaOfLength(5)); + case "ip" -> new BytesRef(InetAddressPoint.encode(randomIp(randomBoolean()))); + case "time_duration" -> Duration.ofMillis(randomNonNegativeLong()); + case "text" -> new BytesRef(randomAlphaOfLength(50)); + case "version" -> new Version(randomIdentifier()).toBytesRef(); + case "null" -> null; + default -> throw new IllegalArgumentException("can't make random values for [" + type.typeName() + "]"); + }, type); + } + + protected TestCase testCase; + + protected static Iterable parameterSuppliersFromTypedData(List cases) { + List parameters = new ArrayList<>(cases.size()); + for (TestCaseSupplier element : cases) { + parameters.add(new Object[] { element }); + } + return parameters; + } + + protected static FieldAttribute field(String name, DataType type) { + return new FieldAttribute(Source.EMPTY, name, new EsField(name, type, Map.of(), true)); + } + + /** + * Build the expression being tested, for the given source and list of arguments. Test classes need to implement this + * to have something to test. + * @param source the source + * @param args arg list from the test case, should match the length expected + * @return an expression for evaluating the function being tested on the given arguments + */ + protected abstract Expression build(Source source, List args); + + protected final Expression buildFieldExpression(TestCase testCase) { + return build(testCase.getSource(), testCase.getDataAsFields()); + } + + protected final Expression buildLiteralExpression(TestCase testCase) { + return build(testCase.getSource(), testCase.getDataAsLiterals()); + } + + protected final Supplier evaluator(Expression e) { + if (e.foldable()) { + e = new Literal(e.source(), e.fold(), e.dataType()); + } + Layout.Builder builder = new Layout.Builder(); + // Hack together a layout by scanning for Fields. + // Those will show up in the layout in whatever order a depth first traversal finds them. + buildLayout(builder, e); + return EvalMapper.toEvaluator(e, builder.build()); + } + + protected final Page row(List values) { + return new Page(BlockUtils.fromListRow(values)); + } + + private void buildLayout(Layout.Builder builder, Expression e) { + if (e instanceof FieldAttribute f) { + builder.appendChannel(f.id()); + return; + } + for (Expression c : e.children()) { + buildLayout(builder, c); + } + } + + protected final void assertResolveTypeValid(Expression expression, DataType expectedType) { + assertTrue(expression.typeResolved().resolved()); + assertThat(expression.dataType(), equalTo(expectedType)); + } + + public final void testSimple() { + Expression expression = buildFieldExpression(testCase); + assertThat(expression.dataType(), equalTo(testCase.expectedType)); + // TODO should we convert unsigned_long into BigDecimal so it's easier to assert? + Object result = toJavaObject(evaluator(expression).get().eval(row(testCase.getDataValues())), 0); + assertThat(result, testCase.getMatcher()); + } + + public final void testSimpleWithNulls() { + List simpleData = testCase.getDataValues(); + EvalOperator.ExpressionEvaluator eval = evaluator(buildFieldExpression(testCase)).get(); + Block[] orig = BlockUtils.fromListRow(simpleData); + for (int i = 0; i < orig.length; i++) { + List data = new ArrayList<>(); + Block[] blocks = new Block[orig.length]; + for (int b = 0; b < blocks.length; b++) { + if (b == i) { + blocks[b] = orig[b].elementType().newBlockBuilder(1).appendNull().build(); + data.add(null); + } else { + blocks[b] = orig[b]; + data.add(simpleData.get(b)); + } + } + assertSimpleWithNulls(data, eval.eval(new Page(blocks)), i); + } + } + + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertTrue("argument " + nullBlock + " is null", value.isNull(0)); + } + + public final void testSimpleInManyThreads() throws ExecutionException, InterruptedException { + int count = 10_000; + int threads = 5; + Supplier evalSupplier = evaluator(buildFieldExpression(testCase)); + ExecutorService exec = Executors.newFixedThreadPool(threads); + try { + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + List simpleData = testCase.getDataValues(); + Page page = row(simpleData); + + futures.add(exec.submit(() -> { + EvalOperator.ExpressionEvaluator eval = evalSupplier.get(); + for (int c = 0; c < count; c++) { + assertThat(toJavaObject(eval.eval(page), 0), testCase.getMatcher()); + } + })); + } + for (Future f : futures) { + f.get(); + } + } finally { + exec.shutdown(); + } + } + + public final void testEvaluatorSimpleToString() { + assertThat(evaluator(buildFieldExpression(testCase)).get().toString(), equalTo(testCase.evaluatorToString)); + } + + public final void testSimpleConstantFolding() { + Expression e = buildLiteralExpression(testCase); + assertThat(e.dataType(), equalTo(testCase.expectedType)); + assertTrue(e.foldable()); + assertThat(e.fold(), testCase.getMatcher()); + } + + public void testSerializationOfSimple() { + assertSerialization(buildFieldExpression(testCase)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java new file mode 100644 index 0000000000000..73babc87e81ed --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java @@ -0,0 +1,107 @@ +/* + * 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.expression.function; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ql.ParsingException; +import org.elasticsearch.xpack.ql.QlIllegalArgumentException; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.function.FunctionDefinition; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistryTests; +import org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.tree.SourceTests; + +import java.util.Arrays; + +import static org.elasticsearch.xpack.ql.TestUtils.randomConfiguration; +import static org.elasticsearch.xpack.ql.expression.function.FunctionRegistry.def; +import static org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy.DEFAULT; +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; + +public class EsqlFunctionRegistryTests extends ESTestCase { + + public void testFunctionResolving() { + UnresolvedFunction ur = uf(DEFAULT, mock(Expression.class)); + FunctionRegistry r = new EsqlFunctionRegistry(defineDummyFunction(ur, "dummyfunction", "dummyfunc")); + + // Resolve by primary name + FunctionDefinition def; + String functionName; + for (int i = 0; i < 10; i++) { + functionName = randomCapitalizedString("dummyfunction"); + def = r.resolveFunction(r.resolveAlias(functionName)); + assertEquals(ur.source(), ur.buildResolved(randomConfiguration(), def).source()); + } + + // Resolve by alias + for (int i = 0; i < 10; i++) { + functionName = randomCapitalizedString("dummyfunc"); + def = r.resolveFunction(r.resolveAlias(functionName)); + assertEquals(ur.source(), ur.buildResolved(randomConfiguration(), def).source()); + } + + // Not resolved + QlIllegalArgumentException e = expectThrows( + QlIllegalArgumentException.class, + () -> r.resolveFunction(r.resolveAlias("Dummy_Function")) + ); + assertThat(e.getMessage(), is("Cannot find function dummy_function; this should have been caught during analysis")); + + e = expectThrows(QlIllegalArgumentException.class, () -> r.resolveFunction(r.resolveAlias("dummy_Function"))); + assertThat(e.getMessage(), is("Cannot find function dummy_function; this should have been caught during analysis")); + } + + public void testUnaryFunction() { + UnresolvedFunction ur = uf(DEFAULT, mock(Expression.class)); + FunctionRegistry r = new EsqlFunctionRegistry(defineDummyUnaryFunction(ur)); + FunctionDefinition def = r.resolveFunction(ur.name()); + + // No children aren't supported + ParsingException e = expectThrows(ParsingException.class, () -> uf(DEFAULT).buildResolved(randomConfiguration(), def)); + assertThat(e.getMessage(), endsWith("expects exactly one argument")); + + // Multiple children aren't supported + e = expectThrows( + ParsingException.class, + () -> uf(DEFAULT, mock(Expression.class), mock(Expression.class)).buildResolved(randomConfiguration(), def) + ); + assertThat(e.getMessage(), endsWith("expects exactly one argument")); + } + + private static UnresolvedFunction uf(FunctionResolutionStrategy resolutionStrategy, Expression... children) { + return new UnresolvedFunction(SourceTests.randomSource(), "dummyFunction", resolutionStrategy, Arrays.asList(children)); + } + + private static FunctionDefinition defineDummyFunction(UnresolvedFunction ur, String... names) { + return def(FunctionRegistryTests.DummyFunction.class, (Source l, Expression e) -> { + assertSame(e, ur.children().get(0)); + return new FunctionRegistryTests.DummyFunction(l); + }, names); + } + + private static FunctionDefinition defineDummyUnaryFunction(UnresolvedFunction ur) { + return defineDummyFunction(ur, "dummyFunction"); + } + + private String randomCapitalizedString(String input) { + StringBuilder output = new StringBuilder(); + for (char c : input.toCharArray()) { + if (randomBoolean()) { + output.append(Character.toUpperCase(c)); + } else { + output.append(c); + } + } + return output.toString(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/WarningsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/WarningsTests.java new file mode 100644 index 0000000000000..4b191dcd47981 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/WarningsTests.java @@ -0,0 +1,37 @@ +/* + * 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.expression.function; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ql.tree.Source; + +public class WarningsTests extends ESTestCase { + public void testRegister() { + Warnings warnings = new Warnings(new Source(1, 1, "foo")); + warnings.registerException(new IllegalArgumentException()); + assertCriticalWarnings( + "Line 1:2: evaluation of [foo] failed, treating result as null. Only first 20 failures recorded.", + "java.lang.IllegalArgumentException: null" + ); + } + + public void testRegisterFilled() { + Warnings warnings = new Warnings(new Source(1, 1, "foo")); + for (int i = 0; i < Warnings.MAX_ADDED_WARNINGS + 1000; i++) { + warnings.registerException(new IllegalArgumentException(Integer.toString(i))); + } + + String[] expected = new String[21]; + expected[0] = "Line 1:2: evaluation of [foo] failed, treating result as null. Only first 20 failures recorded."; + for (int i = 0; i < Warnings.MAX_ADDED_WARNINGS; i++) { + expected[i + 1] = "java.lang.IllegalArgumentException: " + i; + } + + assertCriticalWarnings(expected); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/AbstractScalarFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/AbstractScalarFunctionTestCase.java new file mode 100644 index 0000000000000..6de410042cc0b --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/AbstractScalarFunctionTestCase.java @@ -0,0 +1,190 @@ +/* + * 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.expression.function.scalar; + +import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.TypeResolutions; +import org.elasticsearch.xpack.ql.tree.Location; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; + +/** + * Base class for function tests. + */ +public abstract class AbstractScalarFunctionTestCase extends AbstractFunctionTestCase { + /** + * Describe supported arguments. Build each argument with + * {@link #required} or {@link #optional}. + */ + protected abstract List argSpec(); + + /** + * The data type that applying this function to arguments of this type should produce. + */ + protected abstract DataType expectedType(List argTypes); + + /** + * Define a required argument. + */ + protected final ArgumentSpec required(DataType... validTypes) { + return new ArgumentSpec(false, withNullAndSorted(validTypes)); + } + + /** + * Define an optional argument. + */ + protected final ArgumentSpec optional(DataType... validTypes) { + return new ArgumentSpec(true, withNullAndSorted(validTypes)); + } + + private Set withNullAndSorted(DataType[] validTypes) { + Set realValidTypes = new LinkedHashSet<>(); + Arrays.stream(validTypes).sorted(Comparator.comparing(DataType::name)).forEach(realValidTypes::add); + realValidTypes.add(DataTypes.NULL); + return realValidTypes; + } + + /** + * All string types (keyword, text, match_only_text, etc). For passing to {@link #required} or {@link #optional}. + */ + protected static DataType[] strings() { + return EsqlDataTypes.types().stream().filter(DataTypes::isString).toArray(DataType[]::new); + } + + /** + * All integer types (long, int, short, byte). For passing to {@link #required} or {@link #optional}. + */ + protected static DataType[] integers() { + return EsqlDataTypes.types().stream().filter(DataType::isInteger).toArray(DataType[]::new); + } + + /** + * All rational types (double, float, whatever). For passing to {@link #required} or {@link #optional}. + */ + protected static DataType[] rationals() { + return EsqlDataTypes.types().stream().filter(DataType::isRational).toArray(DataType[]::new); + } + + /** + * All numeric types (integers and rationals.) For passing to {@link #required} or {@link #optional}. + */ + protected static DataType[] numerics() { + return EsqlDataTypes.types().stream().filter(DataType::isNumeric).toArray(DataType[]::new); + } + + protected final DataType[] representableNumerics() { + // TODO numeric should only include representable numbers but that is a change for a followup + return EsqlDataTypes.types().stream().filter(DataType::isNumeric).filter(EsqlDataTypes::isRepresentable).toArray(DataType[]::new); + } + + protected final DataType[] representable() { + return EsqlDataTypes.types().stream().filter(EsqlDataTypes::isRepresentable).toArray(DataType[]::new); + } + + protected record ArgumentSpec(boolean optional, Set validTypes) {} + + public final void testSimpleResolveTypeValid() { + // TODO: The expected output type should probably be on the TestCase + Expression expression = buildFieldExpression(testCase); + assertResolveTypeValid(expression, expectedType(expression.children().stream().map(e -> e.dataType()).toList())); + } + + public final void testResolveType() { + List specs = argSpec(); + for (int mutArg = 0; mutArg < specs.size(); mutArg++) { + for (DataType mutArgType : EsqlDataTypes.types()) { + List args = new ArrayList<>(specs.size()); + for (int arg = 0; arg < specs.size(); arg++) { + if (mutArg == arg) { + args.add(new Literal(new Source(Location.EMPTY, "arg" + arg), "", mutArgType)); + } else { + args.add(new Literal(new Source(Location.EMPTY, "arg" + arg), "", specs.get(arg).validTypes.iterator().next())); + } + } + assertResolution(specs, args, mutArg, mutArgType, specs.get(mutArg).validTypes.contains(mutArgType)); + int optionalIdx = specs.size() - 1; + while (optionalIdx > 0 && specs.get(optionalIdx).optional()) { + args.remove(optionalIdx--); + assertResolution( + specs, + args, + mutArg, + mutArgType, + args.size() <= mutArg || specs.get(mutArg).validTypes.contains(mutArgType) + ); + } + } + } + } + + private void assertResolution(List specs, List args, int mutArg, DataType mutArgType, boolean shouldBeValid) { + Expression exp = build(new Source(Location.EMPTY, "exp"), args); + logger.info("checking {} is {}", exp.nodeString(), shouldBeValid ? "valid" : "invalid"); + if (shouldBeValid) { + assertResolveTypeValid(exp, expectedType(args.stream().map(Expression::dataType).toList())); + return; + } + Expression.TypeResolution resolution = exp.typeResolved(); + assertFalse(exp.nodeString(), resolution.resolved()); + assertThat(exp.nodeString(), resolution.message(), badTypeError(specs, mutArg, mutArgType)); + } + + protected Matcher badTypeError(List spec, int badArgPosition, DataType badArgType) { + String ordinal = spec.size() == 1 + ? "" + : TypeResolutions.ParamOrdinal.fromIndex(badArgPosition).name().toLowerCase(Locale.ROOT) + " "; + return equalTo( + ordinal + + "argument of [exp] must be [" + + expectedTypeName(spec.get(badArgPosition).validTypes()) + + "], found value [arg" + + badArgPosition + + "] type [" + + badArgType.typeName() + + "]" + ); + } + + private String expectedTypeName(Set validTypes) { + List withoutNull = validTypes.stream().filter(t -> t != DataTypes.NULL).toList(); + if (withoutNull.equals(Arrays.asList(strings()))) { + return "string"; + } + if (withoutNull.equals(Arrays.asList(integers()))) { + return "integer"; + } + if (withoutNull.equals(Arrays.asList(rationals()))) { + return "double"; + } + if (withoutNull.equals(Arrays.asList(numerics())) || withoutNull.equals(Arrays.asList(representableNumerics()))) { + return "numeric"; + } + if (withoutNull.equals(List.of(DataTypes.DATETIME))) { + return "datetime"; + } + if (validTypes.equals(Set.copyOf(Arrays.asList(representable())))) { + return "representable"; + } + throw new IllegalArgumentException("can't guess expected type for " + validTypes); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/CaseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/CaseTests.java new file mode 100644 index 0000000000000..e0ea9865d0256 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/CaseTests.java @@ -0,0 +1,176 @@ +/* + * 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.expression.function.scalar.conditional; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expression.TypeResolution; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Stream; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class CaseTests extends AbstractFunctionTestCase { + + public CaseTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + /** + * Generate the test cases for this test + */ + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("basics", () -> { + List typedData = List.of( + new TypedData(true, DataTypes.BOOLEAN, "cond"), + new TypedData(new BytesRef("a"), DataTypes.KEYWORD, "a"), + new TypedData(new BytesRef("b"), DataTypes.KEYWORD, "b") + ); + return new TestCase( + typedData, + "CaseEvaluator[resultType=BYTES_REF, conditions=[ConditionEvaluator[condition=Attribute[channel=0], " + + "value=Attribute[channel=1]]], elseVal=Attribute[channel=2]]", + DataTypes.KEYWORD, + equalTo(new BytesRef("a")) + ); + }))); + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + if (nullBlock == 0) { + assertThat(toJavaObject(value, 0), equalTo(data.get(2))); + return; + } + if (((Boolean) data.get(0)).booleanValue()) { + if (nullBlock == 1) { + super.assertSimpleWithNulls(data, value, nullBlock); + } else { + assertThat(toJavaObject(value, 0), equalTo(data.get(1))); + } + return; + } + if (nullBlock == 2) { + super.assertSimpleWithNulls(data, value, nullBlock); + } else { + assertThat(toJavaObject(value, 0), equalTo(data.get(2))); + } + } + + @Override + protected Expression build(Source source, List args) { + return new Case(Source.EMPTY, args.stream().toList()); + } + + public void testEvalCase() { + testCase( + caseExpr -> toJavaObject( + caseExpr.toEvaluator(child -> evaluator(child)).get().eval(new Page(IntBlock.newConstantBlockWith(0, 1))), + 0 + ) + ); + } + + public void testFoldCase() { + testCase(caseExpr -> { + assertTrue(caseExpr.foldable()); + return caseExpr.fold(); + }); + } + + public void testCase(Function toValue) { + assertEquals(1, toValue.apply(caseExpr(true, 1))); + assertNull(toValue.apply(caseExpr(false, 1))); + assertEquals(2, toValue.apply(caseExpr(false, 1, 2))); + assertEquals(1, toValue.apply(caseExpr(true, 1, true, 2))); + assertEquals(2, toValue.apply(caseExpr(false, 1, true, 2))); + assertNull(toValue.apply(caseExpr(false, 1, false, 2))); + assertEquals(3, toValue.apply(caseExpr(false, 1, false, 2, 3))); + assertNull(toValue.apply(caseExpr(true, null, 1))); + assertEquals(1, toValue.apply(caseExpr(false, null, 1))); + assertEquals(1, toValue.apply(caseExpr(false, field("ignored", DataTypes.INTEGER), 1))); + assertEquals(1, toValue.apply(caseExpr(true, 1, field("ignored", DataTypes.INTEGER)))); + } + + public void testIgnoreLeadingNulls() { + assertEquals(DataTypes.INTEGER, resolveType(false, null, 1)); + assertEquals(DataTypes.INTEGER, resolveType(false, null, false, null, false, 2, null)); + assertEquals(DataTypes.NULL, resolveType(false, null, null)); + assertEquals(DataTypes.BOOLEAN, resolveType(false, null, field("bool", DataTypes.BOOLEAN))); + } + + public void testCaseWithInvalidCondition() { + assertEquals("expected at least two arguments in [] but got 0", resolveCase().message()); + assertEquals("expected at least two arguments in [] but got 1", resolveCase(1).message()); + assertEquals("first argument of [] must be [boolean], found value [1] type [integer]", resolveCase(1, 2).message()); + assertEquals( + "third argument of [] must be [boolean], found value [3] type [integer]", + resolveCase(true, 2, 3, 4, 5).message() + ); + } + + public void testCaseWithIncompatibleTypes() { + assertEquals("third argument of [] must be [integer], found value [hi] type [keyword]", resolveCase(true, 1, "hi").message()); + assertEquals( + "fourth argument of [] must be [integer], found value [hi] type [keyword]", + resolveCase(true, 1, false, "hi", 5).message() + ); + assertEquals( + "argument of [] must be [integer], found value [hi] type [keyword]", + resolveCase(true, 1, false, 2, true, 5, "hi").message() + ); + } + + public void testCaseIsLazy() { + Case caseExpr = caseExpr(true, 1, true, 2); + assertEquals(1, toJavaObject(caseExpr.toEvaluator(child -> { + Object value = child.fold(); + if (value != null && value.equals(2)) { + return () -> page -> { + fail("Unexpected evaluation of 4th argument"); + return null; + }; + } + return evaluator(child); + }).get().eval(new Page(IntBlock.newConstantBlockWith(0, 1))), 0)); + } + + private static Case caseExpr(Object... args) { + return new Case(Source.synthetic(""), Stream.of(args).map(arg -> { + if (arg instanceof Expression e) { + return e; + } + return new Literal(Source.synthetic(arg == null ? "null" : arg.toString()), arg, EsqlDataTypes.fromJava(arg)); + }).toList()); + } + + private static TypeResolution resolveCase(Object... args) { + return caseExpr(args).resolveType(); + } + + private static DataType resolveType(Object... args) { + return caseExpr(args).dataType(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNotNullTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNotNullTests.java new file mode 100644 index 0000000000000..cf12f3f130c9e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNotNullTests.java @@ -0,0 +1,77 @@ +/* + * 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.expression.function.scalar.conditional; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class IsNotNullTests extends AbstractScalarFunctionTestCase { + public IsNotNullTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Keyword Not Null", () -> { + return new TestCase( + List.of(new TypedData(new BytesRef("cat"), DataTypes.KEYWORD, "exp")), + "IsNotNullEvaluator[field=Attribute[channel=0]]", + DataTypes.BOOLEAN, + equalTo(true) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.BOOLEAN; + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertFalse(((BooleanBlock) value).asVector().getBoolean(0)); + } + + @Override + protected List argSpec() { + return List.of(required(EsqlDataTypes.types().toArray(DataType[]::new))); + } + + @Override + protected Expression build(Source source, List args) { + return new IsNotNull(Source.EMPTY, args.get(0)); + } + + public void testAllTypes() { + for (DataType type : EsqlDataTypes.types()) { + if (DataTypes.isPrimitive(type) == false) { + continue; + } + Literal lit = randomLiteral(EsqlDataTypes.widenSmallNumericTypes(type)); + assertThat(new IsNotNull(Source.EMPTY, lit).fold(), equalTo(lit.value() != null)); + assertThat(new IsNotNull(Source.EMPTY, new Literal(Source.EMPTY, null, type)).fold(), equalTo(false)); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNullTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNullTests.java new file mode 100644 index 0000000000000..e8d218119d3c1 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/IsNullTests.java @@ -0,0 +1,77 @@ +/* + * 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.expression.function.scalar.conditional; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNull; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class IsNullTests extends AbstractScalarFunctionTestCase { + public IsNullTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Keyword is Null", () -> { + return new TestCase( + List.of(new TypedData(new BytesRef("cat"), DataTypes.KEYWORD, "exp")), + "IsNullEvaluator[field=Attribute[channel=0]]", + DataTypes.BOOLEAN, + equalTo(false) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.BOOLEAN; + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertTrue(((BooleanBlock) value).asVector().getBoolean(0)); + } + + @Override + protected List argSpec() { + return List.of(required(EsqlDataTypes.types().toArray(DataType[]::new))); + } + + @Override + protected Expression build(Source source, List args) { + return new IsNull(Source.EMPTY, args.get(0)); + } + + public void testAllTypes() { + for (DataType type : EsqlDataTypes.types()) { + if (DataTypes.isPrimitive(type) == false) { + continue; + } + Literal lit = randomLiteral(EsqlDataTypes.widenSmallNumericTypes(type)); + assertThat(new IsNull(Source.EMPTY, lit).fold(), equalTo(lit.value() == null)); + assertThat(new IsNull(Source.EMPTY, new Literal(Source.EMPTY, null, type)).fold(), equalTo(true)); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractTests.java new file mode 100644 index 0000000000000..f56af90bafe84 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractTests.java @@ -0,0 +1,84 @@ +/* + * 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.expression.function.scalar.date; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.time.Instant; +import java.time.ZonedDateTime; +import java.time.temporal.ChronoField; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class DateExtractTests extends AbstractScalarFunctionTestCase { + public DateExtractTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Date Extract Year", () -> { + return new TestCase( + List.of( + new TypedData(1687944333000L, DataTypes.DATETIME, "date"), + new TypedData(new BytesRef("YEAR"), DataTypes.KEYWORD, "field") + ), + "DateExtractEvaluator[value=Attribute[channel=0], chronoField=Attribute[channel=1], zone=Z]", + DataTypes.LONG, + equalTo(2023L) + ); + }))); + } + + public void testAllChronoFields() { + long epochMilli = 1687944333123L; + ZonedDateTime date = Instant.ofEpochMilli(epochMilli).atZone(EsqlTestUtils.TEST_CFG.zoneId()); + for (ChronoField value : ChronoField.values()) { + DateExtract instance = new DateExtract( + Source.EMPTY, + new Literal(Source.EMPTY, epochMilli, DataTypes.DATETIME), + new Literal(Source.EMPTY, new BytesRef(value.name()), DataTypes.KEYWORD), + EsqlTestUtils.TEST_CFG + ); + + assertThat(instance.fold(), is(date.getLong(value))); + assertThat( + DateExtract.process(epochMilli, new BytesRef(value.name()), EsqlTestUtils.TEST_CFG.zoneId()), + is(date.getLong(value)) + ); + } + } + + @Override + protected Expression build(Source source, List args) { + return new DateExtract(source, args.get(0), args.get(1), EsqlTestUtils.TEST_CFG); + } + + @Override + protected List argSpec() { + return List.of(required(DataTypes.DATETIME), required(strings())); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.LONG; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseTests.java new file mode 100644 index 0000000000000..d2ecc980596ed --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.function.scalar.date; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class DateParseTests extends AbstractScalarFunctionTestCase { + public DateParseTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Basic Case", () -> { + return new TestCase( + List.of( + new TypedData(new BytesRef("2023-05-05"), DataTypes.KEYWORD, "first"), + new TypedData(new BytesRef("yyyy-MM-dd"), DataTypes.KEYWORD, "second") + ), + "DateParseEvaluator[val=Attribute[channel=0], formatter=Attribute[channel=1], zoneId=Z]", + DataTypes.DATETIME, + equalTo(1683244800000L) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new DateParse(source, args.get(0), args.size() > 1 ? args.get(1) : null); + } + + @Override + protected List argSpec() { + return List.of(required(strings()), optional(strings())); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DATETIME; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java new file mode 100644 index 0000000000000..7488d4020b32d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java @@ -0,0 +1,164 @@ +/* + * 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.expression.function.scalar.date; + +import org.elasticsearch.common.Rounding; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.SerializationTestUtils; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.DateEsField; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.time.Duration; +import java.time.Instant; +import java.time.Period; +import java.util.Collections; +import java.util.Map; + +import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.createRounding; +import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.process; +import static org.hamcrest.Matchers.containsString; + +public class DateTruncTests extends ESTestCase { + + public void testCreateRoundingDuration() { + Rounding.Prepared rounding; + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(0))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(-10))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + rounding = createRounding(Duration.ofHours(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); + + rounding = createRounding(Duration.ofHours(10)); + assertEquals(10, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); + + rounding = createRounding(Duration.ofMinutes(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + + rounding = createRounding(Duration.ofMinutes(100)); + assertEquals(100, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + + rounding = createRounding(Duration.ofSeconds(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); + + rounding = createRounding(Duration.ofSeconds(120)); + assertEquals(120, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); + + rounding = createRounding(Duration.ofSeconds(60).plusMinutes(5).plusHours(1)); + assertEquals(1 + 5 + 60, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + } + + public void testCreateRoundingPeriod() { + Rounding.Prepared rounding; + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofMonths(0))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(-10))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.of(0, 1, 1))); + assertThat(e.getMessage(), containsString("Time interval is not supported")); + + rounding = createRounding(Period.ofDays(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); + + rounding = createRounding(Period.ofDays(4)); + assertEquals(4, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); + + rounding = createRounding(Period.ofDays(7)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR), 0d); + + rounding = createRounding(Period.ofMonths(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MONTH_OF_YEAR), 0d); + + rounding = createRounding(Period.ofMonths(3)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.QUARTER_OF_YEAR), 0d); + + rounding = createRounding(Period.ofYears(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.YEAR_OF_CENTURY), 0d); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(3))); + assertThat(e.getMessage(), containsString("Time interval is not supported")); + } + + public void testCreateRoundingNullInterval() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(null)); + assertThat(e.getMessage(), containsString("Time interval is not supported")); + } + + public void testDateTruncFunction() { + long ts = toMillis("2023-02-17T10:25:33.38Z"); + + assertEquals(toMillis("2023-02-17T00:00:00.00Z"), process(ts, createRounding(Period.ofDays(1)))); + assertEquals(toMillis("2023-02-01T00:00:00.00Z"), process(ts, createRounding(Period.ofMonths(1)))); + assertEquals(toMillis("2023-01-01T00:00:00.00Z"), process(ts, createRounding(Period.ofYears(1)))); + + assertEquals(toMillis("2023-02-12T00:00:00.00Z"), process(ts, createRounding(Period.ofDays(10)))); + // 7 days period should return weekly rounding + assertEquals(toMillis("2023-02-13T00:00:00.00Z"), process(ts, createRounding(Period.ofDays(7)))); + // 3 months period should return quarterly + assertEquals(toMillis("2023-01-01T00:00:00.00Z"), process(ts, createRounding(Period.ofMonths(3)))); + + assertEquals(toMillis("2023-02-17T10:00:00.00Z"), process(ts, createRounding(Duration.ofHours(1)))); + assertEquals(toMillis("2023-02-17T10:25:00.00Z"), process(ts, createRounding(Duration.ofMinutes(1)))); + assertEquals(toMillis("2023-02-17T10:25:33.00Z"), process(ts, createRounding(Duration.ofSeconds(1)))); + + assertEquals(toMillis("2023-02-17T09:00:00.00Z"), process(ts, createRounding(Duration.ofHours(3)))); + assertEquals(toMillis("2023-02-17T10:15:00.00Z"), process(ts, createRounding(Duration.ofMinutes(15)))); + assertEquals(toMillis("2023-02-17T10:25:30.00Z"), process(ts, createRounding(Duration.ofSeconds(30)))); + assertEquals(toMillis("2023-02-17T10:25:30.00Z"), process(ts, createRounding(Duration.ofSeconds(30)))); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Period.ofDays(-1)))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Duration.ofHours(-1)))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + } + + private static long toMillis(String timestamp) { + return Instant.parse(timestamp).toEpochMilli(); + } + + public void testSerialization() { + var dateTrunc = new DateTrunc(Source.EMPTY, randomDateField(), randomDateIntervalLiteral()); + SerializationTestUtils.assertSerialization(dateTrunc); + } + + private static FieldAttribute randomDateField() { + String fieldName = randomAlphaOfLength(randomIntBetween(1, 25)); + String dateName = randomAlphaOfLength(randomIntBetween(1, 25)); + boolean hasDocValues = randomBoolean(); + if (randomBoolean()) { + return new FieldAttribute(Source.EMPTY, fieldName, new EsField(dateName, DataTypes.DATETIME, Map.of(), hasDocValues)); + } else { + return new FieldAttribute(Source.EMPTY, fieldName, DateEsField.dateEsField(dateName, Collections.emptyMap(), hasDocValues)); + } + } + + private static Literal randomDateIntervalLiteral() { + Duration duration = switch (randomInt(5)) { + case 0 -> Duration.ofNanos(randomIntBetween(1, 100000)); + case 1 -> Duration.ofMillis(randomIntBetween(1, 1000)); + case 2 -> Duration.ofSeconds(randomIntBetween(1, 1000)); + case 3 -> Duration.ofMinutes(randomIntBetween(1, 1000)); + case 4 -> Duration.ofHours(randomIntBetween(1, 100)); + case 5 -> Duration.ofDays(randomIntBetween(1, 60)); + default -> throw new AssertionError(); + }; + return new Literal(Source.EMPTY, duration, EsqlDataTypes.TIME_DURATION); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsTests.java new file mode 100644 index 0000000000000..6b4fcdfea2032 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsTests.java @@ -0,0 +1,80 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class AbsTests extends AbstractScalarFunctionTestCase { + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Integer", () -> { + int arg = randomInt(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.INTEGER, "arg")), + "AbsIntEvaluator[fieldVal=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(Math.abs(arg)) + ); + }), new TestCaseSupplier("UnsignedLong", () -> { + long arg = randomLong(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.UNSIGNED_LONG, "arg")), + "Attribute[channel=0]", + DataTypes.UNSIGNED_LONG, + equalTo(arg) + ); + }), new TestCaseSupplier("Long", () -> { + long arg = randomLong(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.LONG, "arg")), + "AbsLongEvaluator[fieldVal=Attribute[channel=0]]", + DataTypes.LONG, + equalTo(Math.abs(arg)) + ); + }), new TestCaseSupplier("Double", () -> { + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "AbsDoubleEvaluator[fieldVal=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.abs(arg)) + ); + }))); + } + + public AbsTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @Override + protected Expression build(Source source, List args) { + return new Abs(source, args.get(0)); + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractRationalUnaryPredicateTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractRationalUnaryPredicateTests.java new file mode 100644 index 0000000000000..e23956dd36715 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbstractRationalUnaryPredicateTests.java @@ -0,0 +1,38 @@ +/* + * 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.expression.function.scalar.math; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; + +public abstract class AbstractRationalUnaryPredicateTests extends AbstractScalarFunctionTestCase { + protected abstract RationalUnaryPredicate build(Source source, Expression value); + + protected abstract Matcher resultMatcher(double d); + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.BOOLEAN; + } + + @Override + protected final List argSpec() { + return List.of(required(rationals())); + } + + @Override + protected Expression build(Source source, List args) { + return build(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosTests.java new file mode 100644 index 0000000000000..7900672ac5876 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class AcosTests extends AbstractScalarFunctionTestCase { + public AcosTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("double", () -> { + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "AcosEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.acos(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Acos(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinTests.java new file mode 100644 index 0000000000000..5671d4259beb2 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class AsinTests extends AbstractScalarFunctionTestCase { + public AsinTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("double", () -> { + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "AsinEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.asin(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Asin(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Tests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Tests.java new file mode 100644 index 0000000000000..ade7c6a9307ba --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Tests.java @@ -0,0 +1,57 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class Atan2Tests extends AbstractScalarFunctionTestCase { + public Atan2Tests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("double", () -> { + double y = randomDoubleBetween(Double.MIN_VALUE, Double.MAX_VALUE, true); + double x = randomDoubleBetween(Double.MIN_VALUE, Double.MAX_VALUE, true); + return new TestCase( + List.of(new TypedData(y, DataTypes.DOUBLE, "y"), new TypedData(x, DataTypes.DOUBLE, "x")), + "Atan2Evaluator[y=Attribute[channel=0], x=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(Math.atan2(y, x)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics()), required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Atan2(source, args.get(0), args.get(1)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanTests.java new file mode 100644 index 0000000000000..3075edbf0f8d8 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class AtanTests extends AbstractScalarFunctionTestCase { + public AtanTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("double", () -> { + double arg = randomDoubleBetween(Double.MIN_VALUE, Double.MAX_VALUE, true); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "AtanEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.atan(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Atan(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucketTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucketTests.java new file mode 100644 index 0000000000000..d2f5e8f7fef5d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/AutoBucketTests.java @@ -0,0 +1,93 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class AutoBucketTests extends AbstractScalarFunctionTestCase { + public AutoBucketTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Autobucket Single date", () -> { + List args = List.of( + new TypedData(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.parseMillis("2023-02-17T09:00:00.00Z"), DataTypes.DATETIME, "arg") + ); + return new TestCase( + args, + "DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding[DAY_OF_MONTH in Z][fixed to midnight]]", + DataTypes.DATETIME, + resultsMatcher(args) + ); + }))); + } + + private Expression build(Source source, Expression arg) { + Literal from; + Literal to; + if (arg.dataType() == DataTypes.DATETIME) { + from = new Literal(Source.EMPTY, new BytesRef("2023-02-01T00:00:00.00Z"), DataTypes.KEYWORD); + to = new Literal(Source.EMPTY, new BytesRef("2023-03-01T00:00:00.00Z"), DataTypes.KEYWORD); + } else { + from = new Literal(Source.EMPTY, 0, DataTypes.DOUBLE); + to = new Literal(Source.EMPTY, 1000, DataTypes.DOUBLE); + } + return new AutoBucket(source, arg, new Literal(Source.EMPTY, 50, DataTypes.INTEGER), from, to); + } + + @Override + protected DataType expectedType(List argTypes) { + if (argTypes.get(0).isNumeric()) { + return DataTypes.DOUBLE; + } + return argTypes.get(0); + } + + private static Matcher resultsMatcher(List typedData) { + long millis = ((Number) typedData.get(0).data()).longValue(); + return equalTo(Rounding.builder(Rounding.DateTimeUnit.DAY_OF_MONTH).build().prepareForUnknown().round(millis)); + } + + @Override + protected List argSpec() { + DataType[] numerics = numerics(); + DataType[] all = new DataType[numerics.length + 1]; + all[0] = DataTypes.DATETIME; + System.arraycopy(numerics, 0, all, 1, numerics.length); + return List.of(required(all)); + } + + @Override + protected Expression build(Source source, List args) { + return build(source, args.get(0)); + } + + @Override + protected Matcher badTypeError(List spec, int badArgPosition, DataType badArgType) { + return equalTo("first argument of [exp] must be [datetime or numeric], found value [arg0] type [" + badArgType.typeName() + "]"); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosTests.java new file mode 100644 index 0000000000000..9e8369be6d6e9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class CosTests extends AbstractScalarFunctionTestCase { + public CosTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "CosEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.cos(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Cos(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshTests.java new file mode 100644 index 0000000000000..2d53d35fb2252 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class CoshTests extends AbstractScalarFunctionTestCase { + public CoshTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "CoshEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.cosh(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Cosh(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/ETests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/ETests.java new file mode 100644 index 0000000000000..8947ad78e0356 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/ETests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class ETests extends AbstractScalarFunctionTestCase { + public ETests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("E Test", () -> { + return new TestCase( + List.of(new TypedData(1, DataTypes.INTEGER, "foo")), + "LiteralsEvaluator[block=2.718281828459045]", + DataTypes.DOUBLE, + equalTo(Math.E) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new E(Source.EMPTY); + } + + @Override + protected List argSpec() { + return List.of(); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertThat(((DoubleBlock) value).asVector().getDouble(0), equalTo(Math.E)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorTests.java new file mode 100644 index 0000000000000..845d74cbeed84 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class FloorTests extends AbstractScalarFunctionTestCase { + public FloorTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "FloorDoubleEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.floor(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Floor(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteTests.java new file mode 100644 index 0000000000000..ea342a1b173f9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteTests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class IsFiniteTests extends AbstractRationalUnaryPredicateTests { + public IsFiniteTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData( + List.of( + new TestCaseSupplier("NaN", () -> makeTestCase(Double.NaN, false)), + new TestCaseSupplier("positive Infinity", () -> makeTestCase(Double.POSITIVE_INFINITY, false)), + new TestCaseSupplier("negative Infinity", () -> makeTestCase(Double.NEGATIVE_INFINITY, false)), + new TestCaseSupplier("positive small double", () -> makeTestCase(randomDouble(), true)), + new TestCaseSupplier("negative small double", () -> makeTestCase(-randomDouble(), true)), + new TestCaseSupplier("positive large double", () -> makeTestCase(1 / randomDouble(), true)), + new TestCaseSupplier("negative large double", () -> makeTestCase(-1 / randomDouble(), true)) + ) + ); + } + + private static TestCase makeTestCase(double val, boolean expected) { + return new TestCase( + List.of(new TypedData(val, DataTypes.DOUBLE, "arg")), + "IsFiniteEvaluator[val=Attribute[channel=0]]", + DataTypes.BOOLEAN, + equalTo(expected) + ); + } + + @Override + protected RationalUnaryPredicate build(Source source, Expression value) { + return new IsFinite(source, value); + } + + @Override + protected Matcher resultMatcher(double d) { + return equalTo(Double.isNaN(d) == false && Double.isInfinite(d) == false); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteTests.java new file mode 100644 index 0000000000000..b74d60a2a2e77 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteTests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class IsInfiniteTests extends AbstractRationalUnaryPredicateTests { + public IsInfiniteTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData( + List.of( + new TestCaseSupplier("NaN", () -> makeTestCase(Double.NaN, false)), + new TestCaseSupplier("positive Infinity", () -> makeTestCase(Double.POSITIVE_INFINITY, true)), + new TestCaseSupplier("negative Infinity", () -> makeTestCase(Double.NEGATIVE_INFINITY, true)), + new TestCaseSupplier("positive small double", () -> makeTestCase(randomDouble(), false)), + new TestCaseSupplier("negative small double", () -> makeTestCase(-randomDouble(), false)), + new TestCaseSupplier("positive large double", () -> makeTestCase(1 / randomDouble(), false)), + new TestCaseSupplier("negative large double", () -> makeTestCase(-1 / randomDouble(), false)) + ) + ); + } + + private static TestCase makeTestCase(double val, boolean expected) { + return new TestCase( + List.of(new TypedData(val, DataTypes.DOUBLE, "arg")), + "IsInfiniteEvaluator[val=Attribute[channel=0]]", + DataTypes.BOOLEAN, + equalTo(expected) + ); + } + + @Override + protected RationalUnaryPredicate build(Source source, Expression value) { + return new IsInfinite(source, value); + } + + @Override + protected Matcher resultMatcher(double d) { + return equalTo(Double.isInfinite(d)); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNTests.java new file mode 100644 index 0000000000000..79120081815b0 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNTests.java @@ -0,0 +1,63 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class IsNaNTests extends AbstractRationalUnaryPredicateTests { + public IsNaNTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData( + List.of( + new TestCaseSupplier("NaN", () -> makeTestCase(Double.NaN, true)), + new TestCaseSupplier("positive Infinity", () -> makeTestCase(Double.POSITIVE_INFINITY, false)), + new TestCaseSupplier("negative Infinity", () -> makeTestCase(Double.NEGATIVE_INFINITY, false)), + new TestCaseSupplier("positive small double", () -> makeTestCase(randomDouble(), false)), + new TestCaseSupplier("negative small double", () -> makeTestCase(-randomDouble(), false)), + new TestCaseSupplier("positive large double", () -> makeTestCase(1 / randomDouble(), false)), + new TestCaseSupplier("negative large double", () -> makeTestCase(-1 / randomDouble(), false)) + + ) + ); + } + + private static TestCase makeTestCase(double val, boolean expected) { + return new TestCase( + List.of(new TypedData(val, DataTypes.DOUBLE, "arg")), + "IsNaNEvaluator[val=Attribute[channel=0]]", + DataTypes.BOOLEAN, + equalTo(expected) + ); + } + + @Override + protected RationalUnaryPredicate build(Source source, Expression value) { + return new IsNaN(source, value); + } + + @Override + protected Matcher resultMatcher(double d) { + return equalTo(Double.isNaN(d)); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10Tests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10Tests.java new file mode 100644 index 0000000000000..400c46502cbc8 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10Tests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class Log10Tests extends AbstractScalarFunctionTestCase { + public Log10Tests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Log10 of Double", () -> { + // TODO: include larger values here + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "Log10DoubleEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.log10(arg)) + ); + }))); + } + + private Matcher resultsMatcher(List typedData) { + return equalTo(Math.log10((Double) typedData.get(0).data())); + } + + @Override + protected Expression build(Source source, List args) { + return new Log10(source, args.get(0)); + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PiTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PiTests.java new file mode 100644 index 0000000000000..12dc65da440f7 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PiTests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class PiTests extends AbstractScalarFunctionTestCase { + public PiTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Pi Test", () -> { + return new TestCase( + List.of(new TypedData(1, DataTypes.INTEGER, "foo")), + "LiteralsEvaluator[block=3.141592653589793]", + DataTypes.DOUBLE, + equalTo(Math.PI) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new Pi(Source.EMPTY); + } + + @Override + protected List argSpec() { + return List.of(); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertThat(((DoubleBlock) value).asVector().getDouble(0), equalTo(Math.PI)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowTests.java new file mode 100644 index 0000000000000..c0df800dd26bc --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowTests.java @@ -0,0 +1,155 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class PowTests extends AbstractScalarFunctionTestCase { + public PowTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("pow(, )", () -> { + double base = 1 / randomDouble(); + int exponent = between(-30, 30); + return new TestCase( + List.of(new TypedData(base, DataTypes.DOUBLE, "arg"), new TypedData(exponent, DataTypes.INTEGER, "exp")), + "PowDoubleEvaluator[base=Attribute[channel=0], exponent=CastIntToDoubleEvaluator[v=Attribute[channel=1]]]", + DataTypes.DOUBLE, + equalTo(Math.pow(base, exponent)) + ); + }))); + } + + public void testExamples() { + // Test NaN + assertEquals(null, process(Double.NaN, 1)); + assertEquals(null, process(1, Double.NaN)); + + // Test with Integers + assertEquals(1, process(1, 1)); + assertEquals(1, process(randomIntBetween(-1000, 1000), 0)); + int baseInt = randomIntBetween(-1000, 1000); + assertEquals(baseInt, process(baseInt, 1)); + assertEquals((int) Math.pow(baseInt, 2), process(baseInt, 2)); + assertEquals(0, process(123, -1)); + double exponentDouble = randomDoubleBetween(-10.0, 10.0, true); + assertWithNanCheck(Math.pow(baseInt, exponentDouble), baseInt, exponentDouble); + + // Test with Longs + assertEquals(1L, process(1L, 1)); + assertEquals(1L, process(randomLongBetween(-1000, 1000), 0)); + long baseLong = randomLongBetween(-1000, 1000); + assertEquals(baseLong, process(baseLong, 1)); + assertEquals((long) Math.pow(baseLong, 2), process(baseLong, 2)); + assertEquals(0, process(123, -1)); + assertWithNanCheck(Math.pow(baseLong, exponentDouble), baseLong, exponentDouble); + + // Test with Doubles + assertEquals(1.0, process(1.0, 1)); + assertEquals(1.0, process(randomDoubleBetween(-1000.0, 1000.0, true), 0)); + double baseDouble = randomDoubleBetween(-1000.0, 1000.0, true); + assertEquals(baseDouble, process(baseDouble, 1)); + assertEquals(Math.pow(baseDouble, 2), process(baseDouble, 2)); + assertEquals(0, process(123, -1)); + assertWithNanCheck(Math.pow(baseDouble, exponentDouble), baseDouble, exponentDouble); + } + + private void assertWithNanCheck(double expected, Number base, double exponent) { + if (Double.isNaN(expected)) { + ignoreWarning("java.lang.ArithmeticException: invalid result: pow(" + base.doubleValue() + ", " + exponent + ")"); + assertNull("pow(" + base + "," + exponent + ") yields NaN, so we expect NULL", process(base, exponent)); + } else { + assertEquals("pow(" + base + "," + exponent + ")", expected, process(base, exponent)); + } + } + + private Object process(Number base, Number exponent) { + return toJavaObject( + evaluator(new Pow(Source.EMPTY, field("base", typeOf(base)), field("exponent", typeOf(exponent)))).get() + .eval(row(List.of(base, exponent))), + 0 + ); + } + + private DataType typeOf(Number val) { + if (val instanceof Integer) { + return DataTypes.INTEGER; + } + if (val instanceof Long) { + return DataTypes.LONG; + } + if (val instanceof Double) { + return DataTypes.DOUBLE; + } + throw new UnsupportedOperationException("unsupported type [" + val.getClass() + "]"); + } + + @Override + protected DataType expectedType(List argTypes) { + var base = argTypes.get(0); + var exp = argTypes.get(1); + if (base.isRational() || exp.isRational()) { + return DataTypes.DOUBLE; + } else if (base.size() == Long.BYTES || exp.size() == Long.BYTES) { + return DataTypes.LONG; + } else { + return DataTypes.INTEGER; + } + } + + @Override + protected List argSpec() { + return List.of(required(numerics()), required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Pow(source, args.get(0), args.get(1)); + } + + private List ignoreWarnings = new ArrayList<>(); + + private void ignoreWarning(String warning) { + ignoreWarnings.add(warning); + } + + @Override + public void ensureNoWarnings() { + super.ensureNoWarnings(); + ignoreWarnings.clear(); + } + + @Override + protected List filteredWarnings() { + // TODO: This avoids failing the tests for ArithmeticExceptions, but it would be better to assert on the expected warnings + // That would involve overriding ensureWarnings() and getting access to the threadContext + List filteredWarnings = super.filteredWarnings(); + filteredWarnings.add("Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded."); + filteredWarnings.add("java.lang.ArithmeticException: invalid result: pow(NaN, 1.0)"); + filteredWarnings.add("java.lang.ArithmeticException: invalid result: pow(1.0, NaN)"); + filteredWarnings.addAll(ignoreWarnings); + return filteredWarnings; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundTests.java new file mode 100644 index 0000000000000..385b42d23a177 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundTests.java @@ -0,0 +1,128 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.math.Maths; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class RoundTests extends AbstractScalarFunctionTestCase { + public RoundTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("round(, )", () -> { + double number = 1 / randomDouble(); + int precision = between(-30, 30); + return new TestCase( + List.of(new TypedData(number, DataTypes.DOUBLE, "number"), new TypedData(precision, DataTypes.INTEGER, "precision")), + "RoundDoubleEvaluator[val=Attribute[channel=0], decimals=CastIntToLongEvaluator[v=Attribute[channel=1]]]", + DataTypes.DOUBLE, + equalTo(Maths.round(number, precision)) + ); + }))); + } + + public void testExamples() { + assertEquals(123, process(123)); + assertEquals(123, process(123, randomIntBetween(0, 1024))); + assertEquals(120, process(123, -1)); + assertEquals(123.5, process(123.45, 1)); + assertEquals(123.0, process(123.45, 0)); + assertEquals(123.0, process(123.45)); + assertEquals(123L, process(123L, 0)); + assertEquals(123L, process(123L, 5)); + assertEquals(120L, process(123L, -1)); + assertEquals(100L, process(123L, -2)); + assertEquals(0L, process(123L, -3)); + assertEquals(0L, process(123L, -100)); + assertEquals(1000L, process(999L, -1)); + assertEquals(1000.0, process(999.0, -1)); + assertEquals(130L, process(125L, -1)); + assertEquals(12400L, process(12350L, -2)); + assertEquals(12400.0, process(12350.0, -2)); + assertEquals(12300.0, process(12349.0, -2)); + assertEquals(-12300L, process(-12349L, -2)); + assertEquals(-12400L, process(-12350L, -2)); + assertEquals(-12400.0, process(-12350.0, -2)); + assertEquals(-100L, process(-123L, -2)); + assertEquals(-120.0, process(-123.45, -1)); + assertEquals(-123.5, process(-123.45, 1)); + assertEquals(-124.0, process(-123.5, 0)); + assertEquals(-123.0, process(-123.45)); + assertEquals(123.456, process(123.456, Integer.MAX_VALUE)); + assertEquals(0.0, process(123.456, Integer.MIN_VALUE)); + assertEquals(0L, process(0L, 0)); + assertEquals(0, process(0, 0)); + assertEquals(Long.MAX_VALUE, process(Long.MAX_VALUE)); + assertEquals(Long.MAX_VALUE, process(Long.MAX_VALUE, 5)); + assertEquals(Long.MIN_VALUE, process(Long.MIN_VALUE)); + assertEquals(Long.MIN_VALUE, process(Long.MIN_VALUE, 5)); + } + + private Object process(Number val) { + return toJavaObject(evaluator(new Round(Source.EMPTY, field("val", typeOf(val)), null)).get().eval(row(List.of(val))), 0); + } + + private Object process(Number val, int decimals) { + return toJavaObject( + evaluator(new Round(Source.EMPTY, field("val", typeOf(val)), field("decimals", DataTypes.INTEGER))).get() + .eval(row(List.of(val, decimals))), + 0 + ); + } + + private DataType typeOf(Number val) { + if (val instanceof Integer) { + return DataTypes.INTEGER; + } + if (val instanceof Long) { + return DataTypes.LONG; + } + if (val instanceof Double) { + return DataTypes.DOUBLE; + } + throw new UnsupportedOperationException("unsupported type [" + val.getClass() + "]"); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } + + public void testNoDecimalsToString() { + assertThat( + evaluator(new Round(Source.EMPTY, field("val", DataTypes.DOUBLE), null)).get().toString(), + equalTo("RoundDoubleNoDecimalsEvaluator[val=Attribute[channel=0]]") + ); + } + + @Override + protected List argSpec() { + return List.of(required(numerics()), optional(integers())); + } + + @Override + protected Expression build(Source source, List args) { + return new Round(source, args.get(0), args.size() < 2 ? null : args.get(1)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinTests.java new file mode 100644 index 0000000000000..9514b9cfe5c39 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class SinTests extends AbstractScalarFunctionTestCase { + public SinTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "SinEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.sin(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Sin(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhTests.java new file mode 100644 index 0000000000000..a41f7a1ddf317 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class SinhTests extends AbstractScalarFunctionTestCase { + public SinhTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "SinhEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.sinh(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Sinh(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtTests.java new file mode 100644 index 0000000000000..c6549443ad880 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtTests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class SqrtTests extends AbstractScalarFunctionTestCase { + public SqrtTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Sqrt of Double", () -> { + // TODO: include larger values here + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "SqrtDoubleEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.sqrt(arg)) + ); + }))); + } + + private Matcher resultsMatcher(List typedData) { + return equalTo(Math.sqrt((Double) typedData.get(0).data())); + } + + @Override + protected Expression build(Source source, List args) { + return new Sqrt(source, args.get(0)); + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanTests.java new file mode 100644 index 0000000000000..d84734ff0b3de --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class TanTests extends AbstractScalarFunctionTestCase { + public TanTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "TanEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.tan(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Tan(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhTests.java new file mode 100644 index 0000000000000..6713bc5bee5e0 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TanhTests.java @@ -0,0 +1,56 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class TanhTests extends AbstractScalarFunctionTestCase { + public TanhTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("large double value", () -> { + double arg = 1 / randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "TanhEvaluator[val=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(Math.tanh(arg)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected List argSpec() { + return List.of(required(numerics())); + } + + @Override + protected Expression build(Source source, List args) { + return new Tanh(source, args.get(0)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TauTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TauTests.java new file mode 100644 index 0000000000000..25560533898fe --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/math/TauTests.java @@ -0,0 +1,62 @@ +/* + * 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.expression.function.scalar.math; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class TauTests extends AbstractScalarFunctionTestCase { + public TauTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Tau Test", () -> { + return new TestCase( + List.of(new TypedData(1, DataTypes.INTEGER, "foo")), + "LiteralsEvaluator[block=6.283185307179586]", + DataTypes.DOUBLE, + equalTo(Tau.TAU) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new Tau(Source.EMPTY); + } + + @Override + protected List argSpec() { + return List.of(); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; + } + + @Override + protected void assertSimpleWithNulls(List data, Block value, int nullBlock) { + assertThat(((DoubleBlock) value).asVector().getDouble(0), equalTo(Tau.TAU)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunctionTestCase.java new file mode 100644 index 0000000000000..4d4d658727d70 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/AbstractMultivalueFunctionTestCase.java @@ -0,0 +1,452 @@ +/* + * 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.expression.function.scalar.multivalue; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.util.NumericUtils; +import org.hamcrest.Matcher; + +import java.math.BigInteger; +import java.util.Collections; +import java.util.List; +import java.util.function.BiFunction; +import java.util.stream.DoubleStream; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public abstract class AbstractMultivalueFunctionTestCase extends AbstractScalarFunctionTestCase { + /** + * Build a test case with {@code boolean} values. + */ + protected static void booleans( + List cases, + String name, + String evaluatorName, + BiFunction, Matcher> matcher + ) { + booleans(cases, name, evaluatorName, DataTypes.BOOLEAN, matcher); + } + + /** + * Build a test case with {@code boolean} values. + */ + protected static void booleans( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction, Matcher> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(false)", + () -> new TestCase( + List.of(new TypedData(List.of(false), DataTypes.BOOLEAN, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(false)) + ) + ) + ); + cases.add( + new TestCaseSupplier( + name + "(true)", + () -> new TestCase( + List.of(new TypedData(List.of(true), DataTypes.BOOLEAN, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(true)) + ) + ) + ); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(2, 100, ESTestCase::randomBoolean); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.BOOLEAN, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream()) + ); + })); + } + } + + /** + * Build a test case with {@link BytesRef} values. + */ + protected static void bytesRefs( + List cases, + String name, + String evaluatorName, + BiFunction, Matcher> matcher + ) { + bytesRefs(cases, name, evaluatorName, DataTypes.KEYWORD, matcher); + } + + /** + * Build a test case with {@link BytesRef} values. + */ + protected static void bytesRefs( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction, Matcher> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(\"\")", + () -> new TestCase( + List.of(new TypedData(List.of(new BytesRef("")), DataTypes.KEYWORD, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(new BytesRef(""))) + ) + ) + ); + cases.add(new TestCaseSupplier(name + "(BytesRef)", () -> { + BytesRef data = new BytesRef(randomAlphaOfLength(10)); + return new TestCase( + List.of(new TypedData(List.of(data), DataTypes.KEYWORD, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(data)) + ); + })); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(1, 100, () -> new BytesRef(randomAlphaOfLength(10))); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.KEYWORD, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream()) + ); + })); + } + } + + /** + * Build a test case with {@code double} values. + */ + protected static void doubles( + List cases, + String name, + String evaluatorName, + BiFunction> matcher + ) { + doubles(cases, name, evaluatorName, DataTypes.DOUBLE, matcher); + } + + /** + * Build a test case with {@code double} values. + */ + protected static void doubles( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(0.0)", + () -> new TestCase( + List.of(new TypedData(List.of(0.0), DataTypes.DOUBLE, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, DoubleStream.of(0.0)) + ) + ) + ); + cases.add(new TestCaseSupplier(name + "(double)", () -> { + double mvData = randomDouble(); + return new TestCase( + List.of(new TypedData(List.of(mvData), DataTypes.DOUBLE, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, DoubleStream.of(mvData)) + ); + })); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(1, 100, ESTestCase::randomDouble); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.DOUBLE, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream().mapToDouble(Double::doubleValue)) + ); + })); + } + } + + /** + * Build a test case with {@code int} values. + */ + protected static void ints( + List cases, + String name, + String evaluatorName, + BiFunction> matcher + ) { + ints(cases, name, evaluatorName, DataTypes.INTEGER, matcher); + } + + /** + * Build a test case with {@code int} values. + */ + protected static void ints( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(0)", + () -> new TestCase( + List.of(new TypedData(List.of(0), DataTypes.INTEGER, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, IntStream.of(0)) + ) + ) + ); + cases.add(new TestCaseSupplier(name + "(int)", () -> { + int data = randomInt(); + return new TestCase( + List.of(new TypedData(List.of(data), DataTypes.INTEGER, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, IntStream.of(data)) + ); + })); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(1, 100, ESTestCase::randomInt); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.INTEGER, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream().mapToInt(Integer::intValue)) + ); + })); + } + } + + /** + * Build a test case with {@code long} values. + */ + protected static void longs( + List cases, + String name, + String evaluatorName, + BiFunction> matcher + ) { + longs(cases, name, evaluatorName, DataTypes.LONG, matcher); + } + + /** + * Build a test case with {@code long} values. + */ + protected static void longs( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(0L)", + () -> new TestCase( + List.of(new TypedData(List.of(0L), DataTypes.LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, LongStream.of(0L)) + ) + ) + ); + cases.add(new TestCaseSupplier(name + "(long)", () -> { + long data = randomLong(); + return new TestCase( + List.of(new TypedData(List.of(data), DataTypes.LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, LongStream.of(data)) + ); + })); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(1, 100, ESTestCase::randomLong); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream().mapToLong(Long::longValue)) + ); + })); + } + } + + /** + * Build a test case with unsigned {@code long} values. + */ + protected static void unsignedLongs( + List cases, + String name, + String evaluatorName, + BiFunction, Matcher> matcher + ) { + unsignedLongs(cases, name, evaluatorName, DataTypes.UNSIGNED_LONG, matcher); + } + + /** + * Build a test case with unsigned {@code long} values. + */ + protected static void unsignedLongs( + List cases, + String name, + String evaluatorName, + DataType expectedDataType, + BiFunction, Matcher> matcher + ) { + cases.add( + new TestCaseSupplier( + name + "(0UL)", + () -> new TestCase( + List.of(new TypedData(List.of(NumericUtils.asLongUnsigned(BigInteger.ZERO)), DataTypes.UNSIGNED_LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(BigInteger.ZERO)) + ) + ) + ); + cases.add(new TestCaseSupplier(name + "(unsigned long)", () -> { + long data = randomLong(); + return new TestCase( + List.of(new TypedData(List.of(data), DataTypes.UNSIGNED_LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(1, Stream.of(NumericUtils.unsignedLongAsBigInteger(data))) + ); + })); + for (Block.MvOrdering ordering : Block.MvOrdering.values()) { + cases.add(new TestCaseSupplier(name + "() " + ordering, () -> { + List mvData = randomList(1, 100, ESTestCase::randomLong); + putInOrder(mvData, ordering); + return new TestCase( + List.of(new TypedData(mvData, DataTypes.UNSIGNED_LONG, "field")), + evaluatorName + "[field=Attribute[channel=0]]", + expectedDataType, + matcher.apply(mvData.size(), mvData.stream().map(NumericUtils::unsignedLongAsBigInteger)) + ); + })); + } + } + + private static > void putInOrder(List mvData, Block.MvOrdering ordering) { + switch (ordering) { + case UNORDERED -> { + } + case ASCENDING -> Collections.sort(mvData); + default -> throw new UnsupportedOperationException("unsupported ordering [" + ordering + "]"); + } + } + + protected abstract Expression build(Source source, Expression field); + + protected abstract DataType[] supportedTypes(); + + @Override + protected final List argSpec() { + return List.of(required(supportedTypes())); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } + + @Override + protected final Expression build(Source source, List args) { + return build(source, args.get(0)); + } + + /** + * Tests a {@link Block} of values, all copied from the input pattern. + *

+ * Note that this'll sometimes be a {@link Vector} of values if the + * input pattern contained only a single value. + *

+ */ + public final void testBlockWithoutNulls() { + testBlock(false); + } + + /** + * Tests a {@link Block} of values, all copied from the input pattern with + * some null values inserted between. + */ + public final void testBlockWithNulls() { + testBlock(true); + } + + private void testBlock(boolean insertNulls) { + int positions = between(1, 1024); + TypedData data = testCase.getData().get(0); + Block oneRowBlock = BlockUtils.fromListRow(testCase.getDataValues())[0]; + ElementType elementType = LocalExecutionPlanner.toElementType(data.type()); + Block.Builder builder = elementType.newBlockBuilder(positions); + for (int p = 0; p < positions; p++) { + if (insertNulls && randomBoolean()) { + int nulls = between(1, 5); + for (int n = 0; n < nulls; n++) { + builder.appendNull(); + } + } + builder.copyFrom(oneRowBlock, 0, 1); + } + Block input = builder.build(); + Block result = evaluator(buildFieldExpression(testCase)).get().eval(new Page(input)); + + assertThat(result.getPositionCount(), equalTo(result.getPositionCount())); + for (int p = 0; p < input.getPositionCount(); p++) { + if (input.isNull(p)) { + assertThat(result.isNull(p), equalTo(true)); + continue; + } + assertThat(result.isNull(p), equalTo(false)); + assertThat(toJavaObject(result, p), testCase.getMatcher()); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgTests.java new file mode 100644 index 0000000000000..e08edf17aa47f --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAvgTests.java @@ -0,0 +1,74 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.search.aggregations.metrics.CompensatedSum; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.util.NumericUtils; +import org.hamcrest.Matcher; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.BiFunction; +import java.util.function.Supplier; +import java.util.stream.DoubleStream; + +import static org.hamcrest.Matchers.equalTo; + +public class MvAvgTests extends AbstractMultivalueFunctionTestCase { + public MvAvgTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + BiFunction> avg = (size, values) -> { + CompensatedSum sum = new CompensatedSum(); + values.forEach(sum::add); + return equalTo(sum.value() / size); + }; + List cases = new ArrayList<>(); + doubles(cases, "mv_avg", "MvAvg", DataTypes.DOUBLE, avg); + ints(cases, "mv_avg", "MvAvg", DataTypes.DOUBLE, (size, data) -> avg.apply(size, data.mapToDouble(v -> (double) v))); + longs(cases, "mv_avg", "MvAvg", DataTypes.DOUBLE, (size, data) -> avg.apply(size, data.mapToDouble(v -> (double) v))); + unsignedLongs( + cases, + "mv_avg", + "MvAvg", + DataTypes.DOUBLE, + /* + * Converting strait from BigInteger to double will round differently. + * So we have to go back to encoded `long` and then convert to double + * using the production conversion. That'll round in the same way. + */ + (size, data) -> avg.apply(size, data.mapToDouble(v -> NumericUtils.unsignedLongToDouble(NumericUtils.asLongUnsigned(v)))) + ); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvAvg(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representableNumerics(); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.DOUBLE; // Averages are always a double + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcatTests.java new file mode 100644 index 0000000000000..5fda4979e176b --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvConcatTests.java @@ -0,0 +1,83 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class MvConcatTests extends AbstractScalarFunctionTestCase { + public MvConcatTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("mv_concat basic test", () -> { + return new TestCase( + List.of( + new TypedData(List.of(new BytesRef("foo"), new BytesRef("bar"), new BytesRef("baz")), DataTypes.KEYWORD, "field"), + new TypedData(new BytesRef(", "), DataTypes.KEYWORD, "delim") + ), + "MvConcat[field=Attribute[channel=0], delim=Attribute[channel=1]]", + DataTypes.KEYWORD, + equalTo(new BytesRef("foo, bar, baz")) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new MvConcat(source, args.get(0), args.get(1)); + } + + @Override + protected List argSpec() { + return List.of(required(strings()), required(strings())); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.KEYWORD; + } + + public void testNull() { + // TODO: add these into the test parameters + BytesRef foo = new BytesRef("foo"); + BytesRef bar = new BytesRef("bar"); + BytesRef delim = new BytesRef(";"); + Expression expression = buildFieldExpression(testCase); + + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(Arrays.asList(foo, bar), null))), 0), nullValue()); + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(foo, null))), 0), nullValue()); + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(null, null))), 0), nullValue()); + + assertThat( + toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(Arrays.asList(foo, bar), Arrays.asList(delim, bar)))), 0), + nullValue() + ); + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(foo, Arrays.asList(delim, bar)))), 0), nullValue()); + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(null, Arrays.asList(delim, bar)))), 0), nullValue()); + + assertThat(toJavaObject(evaluator(expression).get().eval(row(Arrays.asList(null, delim))), 0), nullValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountTests.java new file mode 100644 index 0000000000000..baa0332bf6024 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountTests.java @@ -0,0 +1,55 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class MvCountTests extends AbstractMultivalueFunctionTestCase { + public MvCountTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + booleans(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + bytesRefs(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + doubles(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + ints(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + longs(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + unsignedLongs(cases, "mv_count", "MvCount", DataTypes.INTEGER, (size, values) -> equalTo(Math.toIntExact(values.count()))); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvCount(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representable(); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.INTEGER; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeTests.java new file mode 100644 index 0000000000000..b04a58d9fb07d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeTests.java @@ -0,0 +1,67 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; + +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class MvDedupeTests extends AbstractMultivalueFunctionTestCase { + public MvDedupeTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + booleans(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values)); + bytesRefs(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values)); + doubles(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values.mapToObj(Double::valueOf))); + ints(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values.mapToObj(Integer::valueOf))); + longs(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values.mapToObj(Long::valueOf))); + // TODO switch extraction to BigInteger so this just works. + // unsignedLongs(cases, "mv_dedupe", "MvDedupe", (size, values) -> getMatcher(values)); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvDedupe(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representable(); + } + + @SuppressWarnings("unchecked") + private static Matcher getMatcher(Stream v) { + Set values = v.collect(Collectors.toSet()); + return switch (values.size()) { + case 0 -> nullValue(); + case 1 -> equalTo(values.iterator().next()); + default -> (Matcher) (Matcher) containsInAnyOrder(values.stream().map(Matchers::equalTo).toArray(Matcher[]::new)); + }; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxTests.java new file mode 100644 index 0000000000000..30ecf8981f7e1 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxTests.java @@ -0,0 +1,57 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.util.NumericUtils; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class MvMaxTests extends AbstractMultivalueFunctionTestCase { + public MvMaxTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + booleans(cases, "mv_max", "MvMax", (size, values) -> equalTo(values.max(Comparator.naturalOrder()).get())); + bytesRefs(cases, "mv_max", "MvMax", (size, values) -> equalTo(values.max(Comparator.naturalOrder()).get())); + doubles(cases, "mv_max", "MvMax", (size, values) -> equalTo(values.max().getAsDouble())); + ints(cases, "mv_max", "MvMax", (size, values) -> equalTo(values.max().getAsInt())); + longs(cases, "mv_max", "MvMax", (size, values) -> equalTo(values.max().getAsLong())); + unsignedLongs( + cases, + "mv_max", + "MvMax", + (size, values) -> equalTo(NumericUtils.asLongUnsigned(values.reduce(BigInteger::max).get())) + ); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvMax(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representable(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianTests.java new file mode 100644 index 0000000000000..954e931eab77e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMedianTests.java @@ -0,0 +1,106 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.util.NumericUtils; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class MvMedianTests extends AbstractMultivalueFunctionTestCase { + public MvMedianTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + doubles(cases, "mv_median", "MvMedian", (size, values) -> { + int middle = size / 2; + if (size % 2 == 1) { + return equalTo(values.sorted().skip(middle).findFirst().getAsDouble()); + } + return equalTo(values.sorted().skip(middle - 1).limit(2).average().getAsDouble()); + }); + ints(cases, "mv_median", "MvMedian", (size, values) -> { + int middle = size / 2; + if (size % 2 == 1) { + return equalTo(values.sorted().skip(middle).findFirst().getAsInt()); + } + var s = values.sorted().skip(middle - 1).limit(2).iterator(); + BigInteger a = BigInteger.valueOf(s.next()); + BigInteger b = BigInteger.valueOf(s.next()); + return equalTo(a.add(b.subtract(a).divide(BigInteger.valueOf(2))).intValue()); + }); + longs(cases, "mv_median", "MvMedian", (size, values) -> { + int middle = size / 2; + if (size % 2 == 1) { + return equalTo(values.sorted().skip(middle).findFirst().getAsLong()); + } + var s = values.sorted().skip(middle - 1).limit(2).iterator(); + BigInteger a = BigInteger.valueOf(s.next()); + BigInteger b = BigInteger.valueOf(s.next()); + return equalTo(a.add(b.subtract(a).divide(BigInteger.valueOf(2))).longValue()); + }); + unsignedLongs(cases, "mv_median", "MvMedian", (size, values) -> { + int middle = size / 2; + if (size % 2 == 1) { + return equalTo(NumericUtils.asLongUnsigned(values.sorted().skip(middle).findFirst().get())); + } + var s = values.sorted().skip(middle - 1).limit(2).iterator(); + BigInteger a = s.next(); + BigInteger b = s.next(); + return equalTo(NumericUtils.asLongUnsigned(a.add(b.subtract(a).divide(BigInteger.valueOf(2))))); + }); + + cases.add( + new TestCaseSupplier( + "mv_median(<1, 2>)", + () -> new TestCase( + List.of(new TypedData(List.of(1, 2), DataTypes.INTEGER, "field")), + "MvMedian[field=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(1) + ) + ) + ); + cases.add( + new TestCaseSupplier( + "mv_median(<-1, -2>)", + () -> new TestCase( + List.of(new TypedData(List.of(-1, -2), DataTypes.INTEGER, "field")), + "MvMedian[field=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(-2) + ) + ) + ); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvMedian(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representableNumerics(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinTests.java new file mode 100644 index 0000000000000..4c4991a78a569 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinTests.java @@ -0,0 +1,57 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.util.NumericUtils; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class MvMinTests extends AbstractMultivalueFunctionTestCase { + public MvMinTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + booleans(cases, "mv_min", "MvMin", (size, values) -> equalTo(values.min(Comparator.naturalOrder()).get())); + bytesRefs(cases, "mv_min", "MvMin", (size, values) -> equalTo(values.min(Comparator.naturalOrder()).get())); + doubles(cases, "mv_min", "MvMin", (size, values) -> equalTo(values.min().getAsDouble())); + ints(cases, "mv_min", "MvMin", (size, values) -> equalTo(values.min().getAsInt())); + longs(cases, "mv_min", "MvMin", (size, values) -> equalTo(values.min().getAsLong())); + unsignedLongs( + cases, + "mv_min", + "MvMin", + (size, values) -> equalTo(NumericUtils.asLongUnsigned(values.reduce(BigInteger::min).get())) + ); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvMin(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representable(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumTests.java new file mode 100644 index 0000000000000..cfa505fd6a18b --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSumTests.java @@ -0,0 +1,48 @@ +/* + * 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.expression.function.scalar.multivalue; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class MvSumTests extends AbstractMultivalueFunctionTestCase { + public MvSumTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + List cases = new ArrayList<>(); + doubles(cases, "mv_sum", "MvSum", (size, values) -> equalTo(values.sum())); + // TODO turn these on once we are summing without overflow + // ints(cases, "mv_sum", "MvSum", (size, values) -> equalTo(values.sum())); + // longs(cases, "mv_sum", "MvSum", (size, values) -> equalTo(values.sum())); + // unsignedLongAsBigInteger(cases, "mv_sum", "MvSum", (size, values) -> equalTo(values.sum())); + return parameterSuppliersFromTypedData(cases); + } + + @Override + protected Expression build(Source source, Expression field) { + return new MvSum(source, field); + } + + @Override + protected DataType[] supportedTypes() { + return representableNumerics(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatTests.java new file mode 100644 index 0000000000000..cb40427bdc48a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ConcatTests.java @@ -0,0 +1,125 @@ +/* + * 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.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class ConcatTests extends AbstractScalarFunctionTestCase { + public ConcatTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("concat basic test", () -> { + BytesRef first = new BytesRef(randomAlphaOfLength(3)); + BytesRef second = new BytesRef(randomAlphaOfLength(3)); + return new TestCase( + List.of(new TypedData(first, DataTypes.KEYWORD, "first"), new TypedData(second, DataTypes.KEYWORD, "second")), + "ConcatEvaluator[values=[Attribute[channel=0], Attribute[channel=1]]]", + DataTypes.KEYWORD, + equalTo(new BytesRef(first.utf8ToString() + second.utf8ToString())) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.KEYWORD; + } + + private Matcher resultsMatcher(List simpleData) { + return equalTo(new BytesRef(simpleData.stream().map(o -> ((BytesRef) o.data()).utf8ToString()).collect(Collectors.joining()))); + } + + @Override + protected List argSpec() { + return List.of( + required(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()), + optional(strings()) + ); + } + + @Override + protected Expression build(Source source, List args) { + return new Concat(source, args.get(0), args.subList(1, args.size())); + } + + @Override + protected Matcher badTypeError(List specs, int badArgPosition, DataType badArgType) { + return equalTo("argument of [exp] must be [string], found value [arg" + badArgPosition + "] type [" + badArgType.typeName() + "]"); + } + + public void testMany() { + List simpleData = Stream.of("cats", " ", "and", " ", "dogs").map(s -> (Object) new BytesRef(s)).toList(); + assertThat( + toJavaObject( + evaluator( + new Concat( + Source.EMPTY, + field("a", DataTypes.KEYWORD), + IntStream.range(1, 5).mapToObj(i -> field(Integer.toString(i), DataTypes.KEYWORD)).toList() + ) + ).get().eval(row(simpleData)), + 0 + ), + equalTo(new BytesRef("cats and dogs")) + ); + } + + public void testSomeConstant() { + List simpleData = Stream.of("cats", "and", "dogs").map(s -> (Object) new BytesRef(s)).toList(); + assertThat( + toJavaObject( + evaluator( + new Concat( + Source.EMPTY, + field("a", DataTypes.KEYWORD), + List.of( + new Literal(Source.EMPTY, new BytesRef(" "), DataTypes.KEYWORD), + field("b", DataTypes.KEYWORD), + new Literal(Source.EMPTY, new BytesRef(" "), DataTypes.KEYWORD), + field("c", DataTypes.KEYWORD) + ) + ) + ).get().eval(row(simpleData)), + 0 + ), + equalTo(new BytesRef("cats and dogs")) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthTests.java new file mode 100644 index 0000000000000..eac3fbeb2149a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/LengthTests.java @@ -0,0 +1,81 @@ +/* + * 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.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.UnicodeUtil; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class LengthTests extends AbstractScalarFunctionTestCase { + public LengthTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("length basic test", () -> { + BytesRef value = new BytesRef(randomAlphaOfLength(between(0, 10000))); + return new TestCase( + List.of(new TypedData(value, DataTypes.KEYWORD, "f")), + "LengthEvaluator[val=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(UnicodeUtil.codePointCount(value)) + ); + }), + new TestCaseSupplier("empty string", () -> makeTestCase("", 0)), + new TestCaseSupplier("single ascii character", () -> makeTestCase("a", 1)), + new TestCaseSupplier("ascii string", () -> makeTestCase("clump", 5)), + new TestCaseSupplier("3 bytes, 1 code point", () -> makeTestCase("☕", 1)), + new TestCaseSupplier("6 bytes, 2 code points", () -> makeTestCase("❗️", 2)), + new TestCaseSupplier("100 random alpha", () -> makeTestCase(randomAlphaOfLength(100), 100)), + new TestCaseSupplier("100 random code points", () -> makeTestCase(randomUnicodeOfCodepointLength(100), 100)) + )); + } + + private static TestCase makeTestCase(String text, int expectedLength) { + return new TestCase( + List.of(new TypedData(new BytesRef(text), DataTypes.KEYWORD, "f")), + "LengthEvaluator[val=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(expectedLength) + ); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.INTEGER; + } + + private Matcher resultsMatcher(List typedData) { + return equalTo(UnicodeUtil.codePointCount((BytesRef) typedData.get(0).data())); + } + + @Override + protected List argSpec() { + return List.of(required(strings())); + } + + @Override + protected Expression build(Source source, List args) { + return new Length(source, args.get(0)); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitTests.java new file mode 100644 index 0000000000000..3c2f996993b11 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SplitTests.java @@ -0,0 +1,100 @@ +/* + * 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.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.joining; +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class SplitTests extends AbstractScalarFunctionTestCase { + public SplitTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("split basic test", () -> { + String delimiter = randomAlphaOfLength(1); + List strings = IntStream.range(0, between(1, 5)) + .mapToObj(i -> randomValueOtherThanMany(s -> s.contains(delimiter), () -> randomAlphaOfLength(4))) + .map(BytesRef::new) + .collect(Collectors.toList()); + String str = strings.stream().map(BytesRef::utf8ToString).collect(joining(delimiter)); + return new TestCase( + List.of( + new TypedData(new BytesRef(str), DataTypes.KEYWORD, "str"), + new TypedData(new BytesRef(delimiter), DataTypes.KEYWORD, "delim") + ), + "SplitVariableEvaluator[str=Attribute[channel=0], delim=Attribute[channel=1]]", + DataTypes.KEYWORD, + equalTo(strings.size() == 1 ? strings.get(0) : strings) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.KEYWORD; + } + + private Matcher resultsMatcher(List typedData) { + String str = ((BytesRef) typedData.get(0).data()).utf8ToString(); + String delim = ((BytesRef) typedData.get(1).data()).utf8ToString(); + List split = Arrays.stream(str.split(Pattern.quote(delim))).map(BytesRef::new).toList(); + return equalTo(split.size() == 1 ? split.get(0) : split); + } + + @Override + protected List argSpec() { + return List.of(required(strings()), required(strings())); + } + + @Override + protected Expression build(Source source, List args) { + return new Split(source, args.get(0), args.get(1)); + } + + public void testConstantDelimiter() { + EvalOperator.ExpressionEvaluator eval = evaluator( + new Split(Source.EMPTY, field("str", DataTypes.KEYWORD), new Literal(Source.EMPTY, new BytesRef(":"), DataTypes.KEYWORD)) + ).get(); + /* + * 58 is ascii for : and appears in the toString below. We don't convert the delimiter to a + * string because we aren't really sure it's printable. It could be a tab or a bell or some + * garbage. + */ + assert ':' == 58; + assertThat(eval.toString(), equalTo("SplitSingleByteEvaluator[str=Attribute[channel=0], delim=58]")); + assertThat( + toJavaObject(eval.eval(new Page(BytesRefBlock.newConstantBlockWith(new BytesRef("foo:bar"), 1))), 0), + equalTo(List.of(new BytesRef("foo"), new BytesRef("bar"))) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithTests.java new file mode 100644 index 0000000000000..20221436d0660 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithTests.java @@ -0,0 +1,71 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class StartsWithTests extends AbstractScalarFunctionTestCase { + public StartsWithTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Starts with basic test", () -> { + String str = randomAlphaOfLength(5); + String prefix = randomAlphaOfLength(5); + if (randomBoolean()) { + str = prefix + str; + } + return new TestCase( + List.of( + new TypedData(new BytesRef(str), DataTypes.KEYWORD, "str"), + new TypedData(new BytesRef(prefix), DataTypes.KEYWORD, "prefix") + ), + "StartsWithEvaluator[str=Attribute[channel=0], prefix=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(str.startsWith(prefix)) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.BOOLEAN; + } + + private Matcher resultsMatcher(List typedData) { + String str = ((BytesRef) typedData.get(0).data()).utf8ToString(); + String prefix = ((BytesRef) typedData.get(1).data()).utf8ToString(); + return equalTo(str.startsWith(prefix)); + } + + @Override + protected List argSpec() { + return List.of(required(strings()), required(strings())); + } + + @Override + protected Expression build(Source source, List args) { + return new StartsWith(source, args.get(0), args.get(1)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringTests.java new file mode 100644 index 0000000000000..83974a232e49c --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/SubstringTests.java @@ -0,0 +1,141 @@ +/* + * 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.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public class SubstringTests extends AbstractScalarFunctionTestCase { + public SubstringTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Substring basic test", () -> { + int start = between(1, 8); + int length = between(1, 10 - start); + String text = randomAlphaOfLength(10); + return new TestCase( + List.of( + new TypedData(new BytesRef(text), DataTypes.KEYWORD, "str"), + new TypedData(start, DataTypes.INTEGER, "start"), + new TypedData(length, DataTypes.INTEGER, "end") + ), + "SubstringEvaluator[str=Attribute[channel=0], start=Attribute[channel=1], length=Attribute[channel=2]]", + DataTypes.KEYWORD, + equalTo(new BytesRef(text.substring(start - 1, start + length - 1))) + ); + }))); + } + + @Override + protected DataType expectedType(List argTypes) { + return DataTypes.KEYWORD; + } + + public Matcher resultsMatcher(List typedData) { + String str = ((BytesRef) typedData.get(0).data()).utf8ToString(); + int start = (Integer) typedData.get(1).data(); + int end = (Integer) typedData.get(2).data(); + return equalTo(new BytesRef(str.substring(start - 1, start + end - 1))); + } + + public void testNoLengthToString() { + assertThat( + evaluator(new Substring(Source.EMPTY, field("str", DataTypes.KEYWORD), field("start", DataTypes.INTEGER), null)).get() + .toString(), + equalTo("SubstringNoLengthEvaluator[str=Attribute[channel=0], start=Attribute[channel=1]]") + ); + } + + @Override + protected List argSpec() { + return List.of(required(strings()), required(integers()), optional(integers())); + } + + @Override + protected Expression build(Source source, List args) { + return new Substring(source, args.get(0), args.get(1), args.size() < 3 ? null : args.get(2)); + } + + public void testWholeString() { + assertThat(process("a tiger", 0, null), equalTo("a tiger")); + assertThat(process("a tiger", 1, null), equalTo("a tiger")); + } + + public void testPositiveStartNoLength() { + assertThat(process("a tiger", 3, null), equalTo("tiger")); + } + + public void testNegativeStartNoLength() { + assertThat(process("a tiger", -3, null), equalTo("ger")); + } + + public void testPositiveStartMassiveLength() { + assertThat(process("a tiger", 3, 1000), equalTo("tiger")); + } + + public void testNegativeStartMassiveLength() { + assertThat(process("a tiger", -3, 1000), equalTo("ger")); + } + + public void testMassiveNegativeStartNoLength() { + assertThat(process("a tiger", -300, null), equalTo("a tiger")); + } + + public void testMassiveNegativeStartSmallLength() { + assertThat(process("a tiger", -300, 1), equalTo("a")); + } + + public void testPositiveStartReasonableLength() { + assertThat(process("a tiger", 1, 3), equalTo("a t")); + } + + public void testUnicode() { + final String s = "a\ud83c\udf09tiger"; + assert s.length() == 8 && s.codePointCount(0, s.length()) == 7; + assertThat(process(s, 3, 1000), equalTo("tiger")); + assertThat(process(s, -6, 1000), equalTo("\ud83c\udf09tiger")); + } + + public void testNegativeLength() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> process("a tiger", 1, -1)); + assertThat(ex.getMessage(), containsString("Length parameter cannot be negative, found [-1]")); + } + + private String process(String str, int start, Integer length) { + Block result = evaluator( + new Substring( + Source.EMPTY, + field("str", DataTypes.KEYWORD), + new Literal(Source.EMPTY, start, DataTypes.INTEGER), + length == null ? null : new Literal(Source.EMPTY, length, DataTypes.INTEGER) + ) + ).get().eval(row(List.of(new BytesRef(str)))); + return result == null ? null : ((BytesRef) toJavaObject(result, 0)).utf8ToString(); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimTests.java new file mode 100644 index 0000000000000..1a8bf4da893bd --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/TrimTests.java @@ -0,0 +1,88 @@ +/* + * 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.expression.function.scalar.string; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class TrimTests extends AbstractScalarFunctionTestCase { + public TrimTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Trim basic test", () -> { + BytesRef sampleData = addRandomLeadingOrTrailingWhitespaces(randomUnicodeOfLength(8)); + DataType dataType = randomFrom(strings()); + return new TestCase( + List.of(new TypedData(sampleData, dataType, "str")), + "TrimEvaluator[val=Attribute[channel=0]]", + dataType, + equalTo(new BytesRef(sampleData.utf8ToString().trim())) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new Trim(source, args.get(0)); + } + + @Override + protected List argSpec() { + return List.of(required(strings())); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } + + public void testTrim() { + for (int i = 0; i < 64; i++) { + String expected = randomUnicodeOfLength(8).trim(); + BytesRef result = Trim.process(addRandomLeadingOrTrailingWhitespaces(expected)); + assertThat(result.utf8ToString(), equalTo(expected)); + } + } + + static BytesRef addRandomLeadingOrTrailingWhitespaces(String expected) { + StringBuilder builder = new StringBuilder(); + if (randomBoolean()) { + builder.append(randomWhiteSpace()); + builder.append(expected); + if (randomBoolean()) { + builder.append(randomWhiteSpace()); + } + } else { + builder.append(expected); + builder.append(randomWhiteSpace()); + } + return new BytesRef(builder.toString()); + } + + private static char[] randomWhiteSpace() { + char[] randomWhitespace = new char[randomIntBetween(1, 8)]; + Arrays.fill(randomWhitespace, (char) randomIntBetween(0, 0x20)); + return randomWhitespace; + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/AbstractBinaryOperatorTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/AbstractBinaryOperatorTestCase.java new file mode 100644 index 0000000000000..1415861cb481c --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/AbstractBinaryOperatorTestCase.java @@ -0,0 +1,148 @@ +/* + * 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.expression.predicate.operator; + +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; +import org.elasticsearch.xpack.ql.tree.Location; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.Locale; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.elasticsearch.xpack.ql.type.DataTypeConverter.commonType; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; + +public abstract class AbstractBinaryOperatorTestCase extends AbstractFunctionTestCase { + + protected abstract Matcher resultsMatcher(List typedData); + + /** + * Return a {@link Matcher} to validate the results of evaluating the function + * + * @param data a list of the parameters that were passed to the evaluator + * @return a matcher to validate correctness against the given data set + */ + protected abstract Matcher resultMatcher(List data, DataType dataType); + + protected boolean rhsOk(Object o) { + return true; + } + + @Override + protected Expression build(Source source, List args) { + return build(source, args.get(0), args.get(1)); + } + + protected abstract BinaryOperator build(Source source, Expression lhs, Expression rhs); + + protected abstract boolean supportsType(DataType type); + + public final void testApplyToAllTypes() { + for (DataType lhsType : EsqlDataTypes.types()) { + if (EsqlDataTypes.isRepresentable(lhsType) == false || lhsType == DataTypes.NULL) { + continue; + } + if (supportsType(lhsType) == false) { + continue; + } + Literal lhs = randomLiteral(lhsType); + for (DataType rhsType : EsqlDataTypes.types()) { + if (EsqlDataTypes.isRepresentable(rhsType) == false || rhsType == DataTypes.NULL) { + continue; + } + if (supportsType(rhsType) == false) { + continue; + } + if (false == (lhsType == rhsType || lhsType.isNumeric() && rhsType.isNumeric())) { + continue; + } + if (lhsType != rhsType && (lhsType == DataTypes.UNSIGNED_LONG || rhsType == DataTypes.UNSIGNED_LONG)) { + continue; + } + Literal rhs = randomValueOtherThanMany(l -> rhsOk(l.value()) == false, () -> randomLiteral(rhsType)); + BinaryOperator op = build( + new Source(Location.EMPTY, lhsType.typeName() + " " + rhsType.typeName()), + field("lhs", lhsType), + field("rhs", rhsType) + ); + Object result = toJavaObject(evaluator(op).get().eval(row(List.of(lhs.value(), rhs.value()))), 0); + if (result == null) { + assertCriticalWarnings( + "Line -1:-1: evaluation of [" + op + "] failed, treating result as null. Only first 20 failures recorded.", + "java.lang.ArithmeticException: " + commonType(lhsType, rhsType).typeName() + " overflow" + ); + } else { + // The type's currently only used for distinguishing between LONG and UNSIGNED_LONG. UL requires both operands be of + // the same type, so either left or right type can be provided below. But otherwise the common type can be used + // instead. + assertThat(op.toString(), result, resultMatcher(List.of(lhs.value(), rhs.value()), lhsType)); + } + } + } + } + + public final void testResolveType() { + for (DataType lhsType : EsqlDataTypes.types()) { + if (EsqlDataTypes.isRepresentable(lhsType) == false) { + continue; + } + Literal lhs = randomLiteral(lhsType); + for (DataType rhsType : EsqlDataTypes.types()) { + if (EsqlDataTypes.isRepresentable(rhsType) == false) { + continue; + } + Literal rhs = randomLiteral(rhsType); + BinaryOperator op = build(new Source(Location.EMPTY, lhsType.typeName() + " " + rhsType.typeName()), lhs, rhs); + + if (lhsType == DataTypes.UNSIGNED_LONG || rhsType == DataTypes.UNSIGNED_LONG) { + validateUnsignedLongType(op, lhsType, rhsType); + continue; + } + validateType(op, lhsType, rhsType); + } + } + } + + private void validateUnsignedLongType(BinaryOperator op, DataType lhsType, DataType rhsType) { + Failure fail = Verifier.validateUnsignedLongOperator(op); + if (lhsType == rhsType) { + assertThat(op.toString(), fail, nullValue()); + return; + } + assertThat(op.toString(), fail, not(nullValue())); + assertThat( + op.toString(), + fail.message(), + equalTo( + String.format( + Locale.ROOT, + "first argument of [%s] is [%s] and second is [%s]. [unsigned_long] can only be operated on together " + + "with another [unsigned_long]", + op, + lhsType.typeName(), + rhsType.typeName() + ) + ) + ); + + } + + protected abstract void validateType(BinaryOperator op, DataType lhsType, DataType rhsType); +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AbstractArithmeticTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AbstractArithmeticTestCase.java new file mode 100644 index 0000000000000..732ae267b2388 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AbstractArithmeticTestCase.java @@ -0,0 +1,121 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import org.elasticsearch.xpack.esql.expression.predicate.operator.AbstractBinaryOperatorTestCase; +import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.Locale; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public abstract class AbstractArithmeticTestCase extends AbstractBinaryOperatorTestCase { + protected final Matcher resultMatcher(List data, DataType dataType) { + Number lhs = (Number) data.get(0); + Number rhs = (Number) data.get(1); + if (lhs instanceof Double || rhs instanceof Double) { + return equalTo(expectedValue(lhs.doubleValue(), rhs.doubleValue())); + } + if (lhs instanceof Long || rhs instanceof Long) { + if (dataType == DataTypes.UNSIGNED_LONG) { + return equalTo(expectedUnsignedLongValue(lhs.longValue(), rhs.longValue())); + } + return equalTo(expectedValue(lhs.longValue(), rhs.longValue())); + } + if (lhs instanceof Integer || rhs instanceof Integer) { + return equalTo(expectedValue(lhs.intValue(), rhs.intValue())); + } + throw new UnsupportedOperationException(); + } + + @Override + protected Matcher resultsMatcher(List typedData) { + Number lhs = (Number) typedData.get(0).data(); + Number rhs = (Number) typedData.get(1).data(); + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.DOUBLE))) { + return equalTo(expectedValue(lhs.doubleValue(), rhs.doubleValue())); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.UNSIGNED_LONG))) { + return equalTo(expectedUnsignedLongValue(lhs.longValue(), rhs.longValue())); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.LONG))) { + return equalTo(expectedValue(lhs.longValue(), rhs.longValue())); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.INTEGER))) { + return equalTo(expectedValue(lhs.intValue(), rhs.intValue())); + } + throw new UnsupportedOperationException(); + } + + protected abstract double expectedValue(double lhs, double rhs); + + protected abstract int expectedValue(int lhs, int rhs); + + protected abstract long expectedValue(long lhs, long rhs); + + protected abstract long expectedUnsignedLongValue(long lhs, long rhs); + + @Override + protected final boolean supportsType(DataType type) { + return type.isNumeric(); + } + + @Override + protected final void validateType(BinaryOperator op, DataType lhsType, DataType rhsType) { + if (DataTypes.isNullOrNumeric(lhsType) && DataTypes.isNullOrNumeric(rhsType)) { + assertTrue(op.toString(), op.typeResolved().resolved()); + assertThat(op.toString(), op.dataType(), equalTo(expectedType(lhsType, rhsType))); + return; + } + assertFalse(op.toString(), op.typeResolved().resolved()); + if (op instanceof Mul) { + // TODO why is mul different? + assertThat( + op.toString(), + op.typeResolved().message(), + equalTo(String.format(Locale.ROOT, "[*] has arguments with incompatible types [%s] and [%s]", lhsType, rhsType)) + ); + return; + } + assertThat( + op.toString(), + op.typeResolved().message(), + containsString( + String.format(Locale.ROOT, "argument of [%s %s] must be [numeric], found value []", lhsType.typeName(), rhsType.typeName()) + ) + ); + } + + private DataType expectedType(DataType lhsType, DataType rhsType) { + if (lhsType == DataTypes.DOUBLE || rhsType == DataTypes.DOUBLE) { + return DataTypes.DOUBLE; + } + if (lhsType == DataTypes.UNSIGNED_LONG || rhsType == DataTypes.UNSIGNED_LONG) { + assertThat(lhsType, is(DataTypes.UNSIGNED_LONG)); + assertThat(rhsType, is(DataTypes.UNSIGNED_LONG)); + return DataTypes.UNSIGNED_LONG; + } + if (lhsType == DataTypes.LONG || rhsType == DataTypes.LONG) { + return DataTypes.LONG; + } + if (lhsType == DataTypes.INTEGER || rhsType == DataTypes.INTEGER) { + return DataTypes.INTEGER; + } + if (lhsType == DataTypes.NULL || rhsType == DataTypes.NULL) { + return DataTypes.NULL; + } + throw new UnsupportedOperationException(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddTests.java new file mode 100644 index 0000000000000..00e2d310ee0e3 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/AddTests.java @@ -0,0 +1,106 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; +import static org.hamcrest.Matchers.equalTo; + +public class AddTests extends AbstractArithmeticTestCase { + public AddTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int + Int", () -> { + // Ensure we don't have an overflow + int rhs = randomIntBetween((Integer.MIN_VALUE >> 1) - 1, (Integer.MAX_VALUE >> 1) - 1); + int lhs = randomIntBetween((Integer.MIN_VALUE >> 1) - 1, (Integer.MAX_VALUE >> 1) - 1); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "AddIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.INTEGER, + equalTo(lhs + rhs) + ); + }), new TestCaseSupplier("Long + Long", () -> { + // Ensure we don't have an overflow + long rhs = randomLongBetween((Long.MIN_VALUE >> 1) - 1, (Long.MAX_VALUE >> 1) - 1); + long lhs = randomLongBetween((Long.MIN_VALUE >> 1) - 1, (Long.MAX_VALUE >> 1) - 1); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.LONG, "lhs"), new TypedData(rhs, DataTypes.LONG, "rhs")), + "AddLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.LONG, + equalTo(lhs + rhs) + ); + }), new TestCaseSupplier("Double + Double", () -> { + double rhs = randomDouble(); + double lhs = randomDouble(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.DOUBLE, "lhs"), new TypedData(rhs, DataTypes.DOUBLE, "rhs")), + "AddDoublesEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(lhs + rhs) + ); + })/*, new TestCaseSupplier("ULong + ULong", () -> { + // Ensure we don't have an overflow + // TODO: we should be able to test values over Long.MAX_VALUE too... + long rhs = randomLongBetween(0, (Long.MAX_VALUE >> 1) - 1); + long lhs = randomLongBetween(0, (Long.MAX_VALUE >> 1) - 1); + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return new TestCase( + Source.EMPTY, + List.of(new TypedData(lhs, DataTypes.UNSIGNED_LONG, "lhs"), new TypedData(rhs, DataTypes.UNSIGNED_LONG, "rhs")), + "AddUnsignedLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + equalTo(asLongUnsigned(lhsBI.add(rhsBI).longValue())) + ); + }) + */ + )); + } + + @Override + protected Add build(Source source, Expression lhs, Expression rhs) { + return new Add(source, lhs, rhs); + } + + @Override + protected double expectedValue(double lhs, double rhs) { + return lhs + rhs; + } + + @Override + protected int expectedValue(int lhs, int rhs) { + return lhs + rhs; + } + + @Override + protected long expectedValue(long lhs, long rhs) { + return lhs + rhs; + } + + @Override + protected long expectedUnsignedLongValue(long lhs, long rhs) { + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return asLongUnsigned(lhsBI.add(rhsBI).longValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivTests.java new file mode 100644 index 0000000000000..3ed0f23141dac --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/DivTests.java @@ -0,0 +1,123 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; +import static org.hamcrest.Matchers.equalTo; + +public class DivTests extends AbstractArithmeticTestCase { + public DivTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int / Int", () -> { + int lhs = randomInt(); + int rhs; + do { + rhs = randomInt(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "DivIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.INTEGER, + equalTo(lhs / rhs) + ); + }), new TestCaseSupplier("Long / Long", () -> { + long lhs = randomLong(); + long rhs; + do { + rhs = randomLong(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.LONG, "lhs"), new TypedData(rhs, DataTypes.LONG, "rhs")), + "DivLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.LONG, + equalTo(lhs / rhs) + ); + }), new TestCaseSupplier("Double / Double", () -> { + double lhs = randomDouble(); + double rhs; + do { + rhs = randomDouble(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.DOUBLE, "lhs"), new TypedData(rhs, DataTypes.DOUBLE, "rhs")), + "DivDoublesEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(lhs / rhs) + ); + })/*, new TestCaseSupplier("ULong / ULong", () -> { + // Ensure we don't have an overflow + long lhs = randomLong(); + long rhs; + do { + rhs = randomLong(); + } while (rhs == 0); + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return new TestCase( + Source.EMPTY, + List.of(new TypedData(lhs, DataTypes.UNSIGNED_LONG, "lhs"), new TypedData(rhs, DataTypes.UNSIGNED_LONG, "rhs")), + "DivUnsignedLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + equalTo(asLongUnsigned(lhsBI.divide(rhsBI).longValue())) + ); + }) + */ + )); + } + + @Override + protected boolean rhsOk(Object o) { + if (o instanceof Number n) { + return n.doubleValue() != 0; + } + return true; + } + + @Override + protected Div build(Source source, Expression lhs, Expression rhs) { + return new Div(source, lhs, rhs); + } + + @Override + protected double expectedValue(double lhs, double rhs) { + return lhs / rhs; + } + + @Override + protected int expectedValue(int lhs, int rhs) { + return lhs / rhs; + } + + @Override + protected long expectedValue(long lhs, long rhs) { + return lhs / rhs; + } + + @Override + protected long expectedUnsignedLongValue(long lhs, long rhs) { + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return asLongUnsigned(lhsBI.divide(rhsBI).longValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModTests.java new file mode 100644 index 0000000000000..0040a6c1d895a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/ModTests.java @@ -0,0 +1,123 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mod; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; +import static org.hamcrest.Matchers.equalTo; + +public class ModTests extends AbstractArithmeticTestCase { + public ModTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int % Int", () -> { + int lhs = randomInt(); + int rhs; + do { + rhs = randomInt(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "ModIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.INTEGER, + equalTo(lhs % rhs) + ); + }), new TestCaseSupplier("Long % Long", () -> { + long lhs = randomLong(); + long rhs; + do { + rhs = randomLong(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.LONG, "lhs"), new TypedData(rhs, DataTypes.LONG, "rhs")), + "ModLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.LONG, + equalTo(lhs % rhs) + ); + }), new TestCaseSupplier("Double % Double", () -> { + double lhs = randomDouble(); + double rhs; + do { + rhs = randomDouble(); + } while (rhs == 0); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.DOUBLE, "lhs"), new TypedData(rhs, DataTypes.DOUBLE, "rhs")), + "ModDoublesEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(lhs % rhs) + ); + })/*, new TestCaseSupplier("ULong % ULong", () -> { + // Ensure we don't have an overflow + long lhs = randomLong(); + long rhs; + do { + rhs = randomLong(); + } while (rhs == 0); + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return new TestCase( + Source.EMPTY, + List.of(new TypedData(lhs, DataTypes.UNSIGNED_LONG, "lhs"), new TypedData(rhs, DataTypes.UNSIGNED_LONG, "rhs")), + "ModUnsignedLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + equalTo(asLongUnsigned(lhsBI.mod(rhsBI).longValue())) + ); + }) + */ + )); + } + + @Override + protected boolean rhsOk(Object o) { + if (o instanceof Number n) { + return n.doubleValue() != 0; + } + return true; + } + + @Override + protected Mod build(Source source, Expression lhs, Expression rhs) { + return new Mod(source, lhs, rhs); + } + + @Override + protected double expectedValue(double lhs, double rhs) { + return lhs % rhs; + } + + @Override + protected int expectedValue(int lhs, int rhs) { + return lhs % rhs; + } + + @Override + protected long expectedValue(long lhs, long rhs) { + return lhs % rhs; + } + + @Override + protected long expectedUnsignedLongValue(long lhs, long rhs) { + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return asLongUnsigned(lhsBI.mod(rhsBI).longValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulTests.java new file mode 100644 index 0000000000000..73f3760c74e20 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/MulTests.java @@ -0,0 +1,105 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; +import static org.hamcrest.Matchers.equalTo; + +public class MulTests extends AbstractArithmeticTestCase { + public MulTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int * Int", () -> { + // Ensure we don't have an overflow + int rhs = randomIntBetween(-255, 255); + int lhs = randomIntBetween(-255, 255); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "MulIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.INTEGER, + equalTo(lhs * rhs) + ); + }), new TestCaseSupplier("Long * Long", () -> { + // Ensure we don't have an overflow + long rhs = randomLongBetween(-1024, 1024); + long lhs = randomLongBetween(-1024, 1024); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.LONG, "lhs"), new TypedData(rhs, DataTypes.LONG, "rhs")), + "MulLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.LONG, + equalTo(lhs * rhs) + ); + }), new TestCaseSupplier("Double * Double", () -> { + double rhs = randomDouble(); + double lhs = randomDouble(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.DOUBLE, "lhs"), new TypedData(rhs, DataTypes.DOUBLE, "rhs")), + "MulDoublesEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(lhs * rhs) + ); + })/*, new TestCaseSupplier("ULong * ULong", () -> { + // Ensure we don't have an overflow + long rhs = randomLongBetween(0, 1024); + long lhs = randomLongBetween(0, 1024); + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return new TestCase( + Source.EMPTY, + List.of(new TypedData(lhs, DataTypes.UNSIGNED_LONG, "lhs"), new TypedData(rhs, DataTypes.UNSIGNED_LONG, "rhs")), + "MulUnsignedLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + equalTo(asLongUnsigned(lhsBI.multiply(rhsBI).longValue())) + ); + }) + */ + )); + } + + @Override + protected Mul build(Source source, Expression lhs, Expression rhs) { + return new Mul(source, lhs, rhs); + } + + @Override + protected double expectedValue(double lhs, double rhs) { + return lhs * rhs; + } + + @Override + protected int expectedValue(int lhs, int rhs) { + return lhs * rhs; + } + + @Override + protected long expectedValue(long lhs, long rhs) { + return lhs * rhs; + } + + @Override + protected long expectedUnsignedLongValue(long lhs, long rhs) { + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return asLongUnsigned(lhsBI.multiply(rhsBI).longValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegTests.java new file mode 100644 index 0000000000000..fbef03ca36a71 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/NegTests.java @@ -0,0 +1,135 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.esql.expression.function.scalar.AbstractScalarFunctionTestCase; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; +import static org.hamcrest.Matchers.equalTo; + +public class NegTests extends AbstractScalarFunctionTestCase { + + public NegTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Integer", () -> { + // Ensure we don't have an overflow + int arg = randomIntBetween((Integer.MIN_VALUE + 1), Integer.MAX_VALUE); + return new TestCase( + List.of(new TypedData(arg, DataTypes.INTEGER, "arg")), + "NegIntsEvaluator[v=Attribute[channel=0]]", + DataTypes.INTEGER, + equalTo(Math.negateExact(arg)) + ); + }), new TestCaseSupplier("Long", () -> { + // Ensure we don't have an overflow + long arg = randomLongBetween((Long.MIN_VALUE + 1), Long.MAX_VALUE); + return new TestCase( + List.of(new TypedData(arg, DataTypes.LONG, "arg")), + "NegLongsEvaluator[v=Attribute[channel=0]]", + DataTypes.LONG, + equalTo(Math.negateExact(arg)) + ); + }), new TestCaseSupplier("Double", () -> { + double arg = randomDouble(); + return new TestCase( + List.of(new TypedData(arg, DataTypes.DOUBLE, "arg")), + "NegDoublesEvaluator[v=Attribute[channel=0]]", + DataTypes.DOUBLE, + equalTo(-arg) + ); + }))); + } + + @Override + protected Expression build(Source source, List args) { + return new Neg(source, args.get(0)); + } + + @Override + protected List argSpec() { + // More precisely: numerics without unsigned longs; however, `Neg::resolveType` uses `numeric`. + return List.of(required(numerics())); + } + + @Override + protected DataType expectedType(List argTypes) { + return argTypes.get(0); + } + + public void testEdgeCases() { + // Run the assertions for the current test cases type only to avoid running the same assertions multiple times. + DataType testCaseType = testCase.getData().get(0).type(); + if (testCaseType.equals(DataTypes.INTEGER)) { + assertEquals(null, process(Integer.MIN_VALUE)); + assertCriticalWarnings( + "Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", + "java.lang.ArithmeticException: integer overflow" + ); + + return; + } + if (testCaseType.equals(DataTypes.LONG)) { + assertEquals(null, process(Long.MIN_VALUE)); + assertCriticalWarnings( + "Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", + "java.lang.ArithmeticException: long overflow" + ); + + return; + } + if (testCaseType.equals(DataTypes.DOUBLE)) { + var negMaxValue = -Double.MAX_VALUE; + assertEquals(negMaxValue, process(Double.MAX_VALUE)); + assertEquals(Double.MAX_VALUE, process(negMaxValue)); + + var negMinValue = -Double.MIN_VALUE; + assertEquals(negMinValue, process(Double.MIN_VALUE)); + assertEquals(Double.MIN_VALUE, process(negMinValue)); + + assertEquals(Double.NEGATIVE_INFINITY, process(Double.POSITIVE_INFINITY)); + assertEquals(Double.POSITIVE_INFINITY, process(Double.NEGATIVE_INFINITY)); + + assertEquals(Double.NaN, process(Double.NaN)); + + return; + } + throw new AssertionError("Edge cases not tested for negation with type [" + testCaseType.typeName() + "]"); + } + + private Object process(Number val) { + return toJavaObject(evaluator(new Neg(Source.EMPTY, field("val", typeOf(val)))).get().eval(row(List.of(val))), 0); + } + + private DataType typeOf(Number val) { + if (val instanceof Integer) { + return DataTypes.INTEGER; + } + if (val instanceof Long) { + return DataTypes.LONG; + } + if (val instanceof Double) { + return DataTypes.DOUBLE; + } + throw new UnsupportedOperationException("unsupported type [" + val.getClass() + "]"); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubTests.java new file mode 100644 index 0000000000000..edf355aad5b9a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/arithmetic/SubTests.java @@ -0,0 +1,106 @@ +/* + * 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.expression.predicate.operator.arithmetic; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.elasticsearch.xpack.ql.util.NumericUtils.unsignedLongAsBigInteger; +import static org.hamcrest.Matchers.equalTo; + +public class SubTests extends AbstractArithmeticTestCase { + public SubTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int - Int", () -> { + // Ensure we don't have an overflow + int rhs = randomIntBetween((Integer.MIN_VALUE >> 1) - 1, (Integer.MAX_VALUE >> 1) - 1); + int lhs = randomIntBetween((Integer.MIN_VALUE >> 1) - 1, (Integer.MAX_VALUE >> 1) - 1); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "SubIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.INTEGER, + equalTo(lhs - rhs) + ); + }), new TestCaseSupplier("Long - Long", () -> { + // Ensure we don't have an overflow + long rhs = randomLongBetween((Long.MIN_VALUE >> 1) - 1, (Long.MAX_VALUE >> 1) - 1); + long lhs = randomLongBetween((Long.MIN_VALUE >> 1) - 1, (Long.MAX_VALUE >> 1) - 1); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.LONG, "lhs"), new TypedData(rhs, DataTypes.LONG, "rhs")), + "SubLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.LONG, + equalTo(lhs - rhs) + ); + }), new TestCaseSupplier("Double - Double", () -> { + double rhs = randomDouble(); + double lhs = randomDouble(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.DOUBLE, "lhs"), new TypedData(rhs, DataTypes.DOUBLE, "rhs")), + "SubDoublesEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.DOUBLE, + equalTo(lhs - rhs) + ); + })/*, new TestCaseSupplier("ULong - ULong", () -> { + // Ensure we don't have an overflow + // TODO: we should be able to test values over Long.MAX_VALUE too... + long rhs = randomLongBetween(0, (Long.MAX_VALUE >> 1) - 1); + long lhs = randomLongBetween(0, (Long.MAX_VALUE >> 1) - 1); + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return new TestCase( + Source.EMPTY, + List.of(new TypedData(lhs, DataTypes.UNSIGNED_LONG, "lhs"), new TypedData(rhs, DataTypes.UNSIGNED_LONG, "rhs")), + "SubUnsignedLongsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + equalTo(asLongUnsigned(lhsBI.subtract(rhsBI).longValue())) + ); + }) + */ + )); + } + + @Override + protected Sub build(Source source, Expression lhs, Expression rhs) { + return new Sub(source, lhs, rhs); + } + + @Override + protected double expectedValue(double lhs, double rhs) { + return lhs - rhs; + } + + @Override + protected int expectedValue(int lhs, int rhs) { + return lhs - rhs; + } + + @Override + protected long expectedValue(long lhs, long rhs) { + return lhs - rhs; + } + + @Override + protected long expectedUnsignedLongValue(long lhs, long rhs) { + BigInteger lhsBI = unsignedLongAsBigInteger(lhs); + BigInteger rhsBI = unsignedLongAsBigInteger(rhs); + return asLongUnsigned(lhsBI.subtract(rhsBI).longValue()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/AbstractBinaryComparisonTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/AbstractBinaryComparisonTestCase.java new file mode 100644 index 0000000000000..e21b9947271fa --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/AbstractBinaryComparisonTestCase.java @@ -0,0 +1,118 @@ +/* + * 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.expression.predicate.operator.comparison; + +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.expression.predicate.operator.AbstractBinaryOperatorTestCase; +import org.elasticsearch.xpack.ql.common.Failure; +import org.elasticsearch.xpack.ql.expression.predicate.BinaryOperator; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.Locale; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; + +public abstract class AbstractBinaryComparisonTestCase extends AbstractBinaryOperatorTestCase { + @SuppressWarnings({ "rawtypes", "unchecked" }) + protected final Matcher resultMatcher(List data, DataType dataType) { + Comparable lhs = (Comparable) data.get(0); + Comparable rhs = (Comparable) data.get(1); + if (lhs instanceof Double || rhs instanceof Double) { + return (Matcher) (Matcher) resultMatcher(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue()); + } + if (lhs instanceof Long || rhs instanceof Long) { + return (Matcher) (Matcher) resultMatcher(((Number) lhs).longValue(), ((Number) rhs).longValue()); + } + if (lhs instanceof Integer || rhs instanceof Integer) { + return (Matcher) (Matcher) resultMatcher(((Number) lhs).intValue(), ((Number) rhs).intValue()); + } + return (Matcher) (Matcher) resultMatcher(lhs, rhs); + } + + @Override + protected Matcher resultsMatcher(List typedData) { + Number lhs = (Number) typedData.get(0).data(); + Number rhs = (Number) typedData.get(1).data(); + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.DOUBLE))) { + return equalTo(resultMatcher(lhs.doubleValue(), rhs.doubleValue())); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.UNSIGNED_LONG))) { + // TODO: Is this correct behavior for unsigned long? + return resultMatcher(lhs.longValue(), rhs.longValue()); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.LONG))) { + return resultMatcher(lhs.longValue(), rhs.longValue()); + } + if (typedData.stream().anyMatch(t -> t.type().equals(DataTypes.INTEGER))) { + return resultMatcher(lhs.intValue(), rhs.intValue()); + } + throw new UnsupportedOperationException(); + } + + protected abstract > Matcher resultMatcher(T lhs, T rhs); + + protected abstract boolean isEquality(); + + @Override + protected final boolean supportsType(DataType type) { + if (type == DataTypes.BOOLEAN) { + return isEquality(); + } + return true; + } + + @Override + protected final void validateType(BinaryOperator op, DataType lhsType, DataType rhsType) { + assertTrue(op.typeResolved().resolved()); + assertThat(op.dataType(), equalTo(DataTypes.BOOLEAN)); + Failure f = Verifier.validateBinaryComparison((BinaryComparison) op); + if (isEquality() == false && lhsType == DataTypes.BOOLEAN) { + assertThat(op.toString(), f, not(nullValue())); + assertThat( + op.toString(), + f.message(), + equalTo( + String.format( + Locale.ROOT, + "first argument of [%s %s] must be [numeric, keyword, text, ip, datetime or version], found value [] type [%s]", + lhsType.typeName(), + rhsType.typeName(), + lhsType.typeName() + ) + ) + ); + return; + } + if (lhsType == rhsType || lhsType.isNumeric() && rhsType.isNumeric()) { + assertThat(op.toString(), f, nullValue()); + return; + } + assertThat(op.toString(), f, not(nullValue())); + assertThat( + op.toString(), + f.message(), + equalTo( + String.format( + Locale.ROOT, + "first argument of [%s %s] is [%s] so second argument must also be [%s] but was [%s]", + lhsType.typeName(), + rhsType.typeName(), + lhsType.isNumeric() ? "numeric" : lhsType.typeName(), + lhsType.isNumeric() ? "numeric" : lhsType.typeName(), + rhsType.typeName() + ) + ) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsTests.java new file mode 100644 index 0000000000000..60dcccc0f4a2d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/EqualsTests.java @@ -0,0 +1,58 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class EqualsTests extends AbstractBinaryComparisonTestCase { + public EqualsTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int == Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "EqualsIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs == rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(lhs.equals(rhs)); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new Equals(source, lhs, rhs); + } + + @Override + protected boolean isEquality() { + return true; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualTests.java new file mode 100644 index 0000000000000..c108f965f6e68 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanOrEqualTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class GreaterThanOrEqualTests extends AbstractBinaryComparisonTestCase { + public GreaterThanOrEqualTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int >= Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "GreaterThanOrEqualIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs >= rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(lhs.compareTo(rhs) >= 0); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new GreaterThanOrEqual(source, lhs, rhs, ZoneOffset.UTC); + } + + @Override + protected boolean isEquality() { + return false; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanTests.java new file mode 100644 index 0000000000000..561cde534e47e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/GreaterThanTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class GreaterThanTests extends AbstractBinaryComparisonTestCase { + public GreaterThanTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int > Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "GreaterThanIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs > rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(lhs.compareTo(rhs) > 0); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new GreaterThan(source, lhs, rhs, ZoneOffset.UTC); + } + + @Override + protected boolean isEquality() { + return false; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualTests.java new file mode 100644 index 0000000000000..bec73c260776d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanOrEqualTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class LessThanOrEqualTests extends AbstractBinaryComparisonTestCase { + public LessThanOrEqualTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int <= Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "LessThanOrEqualIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs <= rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(lhs.compareTo(rhs) <= 0); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new LessThanOrEqual(source, lhs, rhs, ZoneOffset.UTC); + } + + @Override + protected boolean isEquality() { + return false; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanTests.java new file mode 100644 index 0000000000000..aa80d08c56605 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/LessThanTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class LessThanTests extends AbstractBinaryComparisonTestCase { + public LessThanTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int < Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "LessThanIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs < rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(lhs.compareTo(rhs) < 0); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new LessThan(source, lhs, rhs, ZoneOffset.UTC); + } + + @Override + protected boolean isEquality() { + return false; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsTests.java new file mode 100644 index 0000000000000..cc25e4169a441 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/NotEqualsTests.java @@ -0,0 +1,59 @@ +/* + * 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.expression.predicate.operator.comparison; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.hamcrest.Matcher; + +import java.time.ZoneOffset; +import java.util.List; +import java.util.function.Supplier; + +import static org.hamcrest.Matchers.equalTo; + +public class NotEqualsTests extends AbstractBinaryComparisonTestCase { + public NotEqualsTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(List.of(new TestCaseSupplier("Int != Int", () -> { + int rhs = randomInt(); + int lhs = randomInt(); + return new TestCase( + List.of(new TypedData(lhs, DataTypes.INTEGER, "lhs"), new TypedData(rhs, DataTypes.INTEGER, "rhs")), + "NotEqualsIntsEvaluator[lhs=Attribute[channel=0], rhs=Attribute[channel=1]]", + DataTypes.BOOLEAN, + equalTo(lhs != rhs) + ); + }))); + } + + @Override + protected > Matcher resultMatcher(T lhs, T rhs) { + return equalTo(false == lhs.equals(rhs)); + } + + @Override + protected BinaryComparison build(Source source, Expression lhs, Expression rhs) { + return new NotEquals(source, lhs, rhs, ZoneOffset.UTC); + } + + @Override + protected boolean isEquality() { + return true; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java new file mode 100644 index 0000000000000..0f8b9d7846ad9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java @@ -0,0 +1,232 @@ +/* + * 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.formatter; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.IntArrayVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xpack.esql.action.ColumnInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; +import org.elasticsearch.xpack.ql.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.CSV; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.PLAIN_TEXT; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.TSV; + +public class TextFormatTests extends ESTestCase { + + public void testCsvContentType() { + assertEquals("text/csv; charset=utf-8; header=present", CSV.contentType(req())); + } + + public void testCsvContentTypeWithoutHeader() { + assertEquals("text/csv; charset=utf-8; header=absent", CSV.contentType(reqWithParam("header", "absent"))); + } + + public void testTsvContentType() { + assertEquals("text/tab-separated-values; charset=utf-8", TSV.contentType(req())); + } + + public void testCsvEscaping() { + assertEquals("string", CSV.maybeEscape("string", CSV.delimiter())); + assertEquals("", CSV.maybeEscape("", CSV.delimiter())); + assertEquals("\"\"\"\"", CSV.maybeEscape("\"", CSV.delimiter())); + assertEquals("\"\"\",\"\"\"", CSV.maybeEscape("\",\"", CSV.delimiter())); + assertEquals("\"\"\"quo\"\"ted\"\"\"", CSV.maybeEscape("\"quo\"ted\"", CSV.delimiter())); + assertEquals("\"one;two\"", CSV.maybeEscape("one;two", ';')); + } + + public void testTsvEscaping() { + assertEquals("string", TSV.maybeEscape("string", null)); + assertEquals("", TSV.maybeEscape("", null)); + assertEquals("\"", TSV.maybeEscape("\"", null)); + assertEquals("\\t", TSV.maybeEscape("\t", null)); + assertEquals("\\n\"\\t", TSV.maybeEscape("\n\"\t", null)); + } + + public void testCsvFormatWithEmptyData() { + String text = format(CSV, req(), emptyData()); + assertEquals("name\r\n", text); + } + + public void testTsvFormatWithEmptyData() { + String text = format(TSV, req(), emptyData()); + assertEquals("name\n", text); + } + + public void testCsvFormatWithRegularData() { + String text = format(CSV, req(), regularData()); + assertEquals(""" + string,number\r + Along The River Bank,708\r + Mind Train,280\r + """, text); + } + + public void testCsvFormatNoHeaderWithRegularData() { + String text = format(CSV, reqWithParam("header", "absent"), regularData()); + assertEquals(""" + Along The River Bank,708\r + Mind Train,280\r + """, text); + } + + public void testCsvFormatWithCustomDelimiterRegularData() { + Set forbidden = Set.of('"', '\r', '\n', '\t'); + Character delim = randomValueOtherThanMany(forbidden::contains, () -> randomAlphaOfLength(1).charAt(0)); + String text = format(CSV, reqWithParam("delimiter", String.valueOf(delim)), regularData()); + List terms = Arrays.asList("string", "number", "Along The River Bank", "708", "Mind Train", "280"); + List expectedTerms = terms.stream() + .map(x -> x.contains(String.valueOf(delim)) ? '"' + x + '"' : x) + .collect(Collectors.toList()); + StringBuffer sb = new StringBuffer(); + do { + sb.append(expectedTerms.remove(0)); + sb.append(delim); + sb.append(expectedTerms.remove(0)); + sb.append("\r\n"); + } while (expectedTerms.size() > 0); + assertEquals(sb.toString(), text); + } + + public void testTsvFormatWithRegularData() { + String text = format(TSV, req(), regularData()); + assertEquals(""" + string\tnumber + Along The River Bank\t708 + Mind Train\t280 + """, text); + } + + public void testCsvFormatWithEscapedData() { + String text = format(CSV, req(), escapedData()); + assertEquals(""" + first,""\"special""\"\r + normal,""\"quo""ted"", + "\r + commas,"a,b,c, + ,d,e,\t + "\r + """, text); + } + + public void testCsvFormatWithCustomDelimiterEscapedData() { + String text = format(CSV, reqWithParam("delimiter", "\\"), escapedData()); + assertEquals(""" + first\\""\"special""\"\r + normal\\""\"quo""ted"", + "\r + commas\\"a,b,c, + ,d,e,\t + "\r + """, text); + } + + public void testTsvFormatWithEscapedData() { + String text = format(TSV, req(), escapedData()); + assertEquals(""" + first\t"special" + normal\t"quo"ted",\\n + commas\ta,b,c,\\n,d,e,\\t\\n + """, text); + } + + public void testInvalidCsvDelims() { + List invalid = Arrays.asList("\"", "\r", "\n", "\t", "", "ab"); + + for (String c : invalid) { + Exception e = expectThrows(IllegalArgumentException.class, () -> format(CSV, reqWithParam("delimiter", c), emptyData())); + String msg; + if (c.length() == 1) { + msg = c.equals("\t") + ? "illegal delimiter [TAB] specified as delimiter for the [csv] format; choose the [tsv] format instead" + : "illegal reserved character specified as delimiter [" + c + "]"; + } else { + msg = "invalid " + (c.length() > 0 ? "multi-character" : "empty") + " delimiter [" + c + "]"; + } + assertEquals(msg, e.getMessage()); + } + } + + public void testPlainTextEmptyCursorWithColumns() { + assertEquals(""" + name \s + --------------- + """, format(PLAIN_TEXT, req(), emptyData())); + } + + public void testPlainTextEmptyCursorWithoutColumns() { + assertEquals(StringUtils.EMPTY, PLAIN_TEXT.format(req(), new EsqlQueryResponse(emptyList(), emptyList(), false))); + } + + private static EsqlQueryResponse emptyData() { + return new EsqlQueryResponse(singletonList(new ColumnInfo("name", "keyword")), emptyList(), false); + } + + private static EsqlQueryResponse regularData() { + // headers + List headers = asList(new ColumnInfo("string", "keyword"), new ColumnInfo("number", "integer")); + + // values + List values = List.of( + new Page( + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("Along The River Bank")) + .appendBytesRef(new BytesRef("Mind Train")) + .build(), + new IntArrayVector(new int[] { 11 * 60 + 48, 4 * 60 + 40 }, 2).asBlock() + ) + ); + + return new EsqlQueryResponse(headers, values, false); + } + + private static EsqlQueryResponse escapedData() { + // headers + List headers = asList(new ColumnInfo("first", "keyword"), new ColumnInfo("\"special\"", "keyword")); + + // values + List values = List.of( + new Page( + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("normal")).appendBytesRef(new BytesRef("commas")).build(), + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("\"quo\"ted\",\n")) + .appendBytesRef(new BytesRef("a,b,c,\n,d,e,\t\n")) + .build() + ) + ); + + return new EsqlQueryResponse(headers, values, false); + } + + private static RestRequest req() { + return new FakeRestRequest(); + } + + private static RestRequest reqWithParam(String paramName, String paramVal) { + return new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withParams(singletonMap(paramName, paramVal)).build(); + } + + private String format(TextFormat format, RestRequest request, EsqlQueryResponse response) { + return format.format(request, response); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java new file mode 100644 index 0000000000000..b113c52cfc457 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java @@ -0,0 +1,143 @@ +/* + * 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.formatter; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleArrayVector; +import org.elasticsearch.compute.data.LongArrayVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.action.ColumnInfo; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; + +import java.util.Arrays; +import java.util.List; + +import static org.elasticsearch.xpack.ql.util.DateUtils.UTC_DATE_TIME_FORMATTER; +import static org.hamcrest.Matchers.arrayWithSize; + +public class TextFormatterTests extends ESTestCase { + private final List columns = Arrays.asList( + new ColumnInfo("foo", "keyword"), + new ColumnInfo("bar", "long"), + new ColumnInfo("15charwidename!", "double"), + new ColumnInfo("null_field1", "integer"), + new ColumnInfo("superduperwidename!!!", "double"), + new ColumnInfo("baz", "keyword"), + new ColumnInfo("date", "date"), + new ColumnInfo("null_field2", "keyword") + ); + EsqlQueryResponse esqlResponse = new EsqlQueryResponse( + columns, + List.of( + new Page( + BytesRefBlock.newBlockBuilder(2) + .appendBytesRef(new BytesRef("15charwidedata!")) + .appendBytesRef(new BytesRef("dog")) + .build(), + new LongArrayVector(new long[] { 1, 2 }, 2).asBlock(), + new DoubleArrayVector(new double[] { 6.888, 123124.888 }, 2).asBlock(), + Block.constantNullBlock(2), + new DoubleArrayVector(new double[] { 12, 9912 }, 2).asBlock(), + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("rabbit")).appendBytesRef(new BytesRef("goat")).build(), + new LongArrayVector( + new long[] { + UTC_DATE_TIME_FORMATTER.parseMillis("1953-09-02T00:00:00.000Z"), + UTC_DATE_TIME_FORMATTER.parseMillis("2000-03-15T21:34:37.443Z") }, + 2 + ).asBlock(), + Block.constantNullBlock(2) + ) + ), + randomBoolean() + ); + + TextFormatter formatter = new TextFormatter(esqlResponse); + + /** + * Tests for {@link TextFormatter#format} with header, values + * of exactly the minimum column size, column names of exactly + * the minimum column size, column headers longer than the + * minimum column size, and values longer than the minimum + * column size. + */ + public void testFormatWithHeader() { + String[] result = formatter.format(true).split("\n"); + assertThat(result, arrayWithSize(4)); + assertEquals( + " foo | bar |15charwidename!| null_field1 |superduperwidename!!!| baz |" + + " date | null_field2 ", + result[0] + ); + assertEquals( + "---------------+---------------+---------------+---------------+---------------------+---------------+" + + "------------------------+---------------", + result[1] + ); + assertEquals( + "15charwidedata!|1 |6.888 |null |12.0 |rabbit |" + + "1953-09-02T00:00:00.000Z|null ", + result[2] + ); + assertEquals( + "dog |2 |123124.888 |null |9912.0 |goat |" + + "2000-03-15T21:34:37.443Z|null ", + result[3] + ); + } + + /** + * Tests for {@link TextFormatter#format} without header and + * truncation of long columns. + */ + public void testFormatWithoutHeader() { + EsqlQueryResponse response = new EsqlQueryResponse( + columns, + List.of( + new Page( + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("doggie")).appendBytesRef(new BytesRef("dog")).build(), + new LongArrayVector(new long[] { 4, 2 }, 2).asBlock(), + new DoubleArrayVector(new double[] { 1, 123124.888 }, 2).asBlock(), + Block.constantNullBlock(2), + new DoubleArrayVector(new double[] { 77.0, 9912.0 }, 2).asBlock(), + BytesRefBlock.newBlockBuilder(2).appendBytesRef(new BytesRef("wombat")).appendBytesRef(new BytesRef("goat")).build(), + new LongArrayVector( + new long[] { + UTC_DATE_TIME_FORMATTER.parseMillis("1955-01-21T01:02:03.342Z"), + UTC_DATE_TIME_FORMATTER.parseMillis("2231-12-31T23:59:59.999Z") }, + 2 + ).asBlock(), + Block.constantNullBlock(2) + ) + ), + randomBoolean() + ); + + String[] result = new TextFormatter(response).format(false).split("\n"); + assertThat(result, arrayWithSize(2)); + assertEquals( + "doggie |4 |1.0 |null |77.0 |wombat |" + + "1955-01-21T01:02:03.342Z|null ", + result[0] + ); + assertEquals( + "dog |2 |123124.888 |null |9912.0 |goat |" + + "2231-12-31T23:59:59.999Z|null ", + result[1] + ); + } + + /** + * Ensure that our estimates are perfect in at least some cases. + */ + public void testEstimateSize() { + assertEquals(formatter.format(true).length(), formatter.estimateSize(esqlResponse.values().size() + 2)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypesTests.java new file mode 100644 index 0000000000000..99114fc48311e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypesTests.java @@ -0,0 +1,592 @@ +/* + * 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.io.stream; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.ByteBufferStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.dissect.DissectParser; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.EqualsHashCodeTestUtils; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.SerializationTestUtils; +import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Avg; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.function.aggregate.CountDistinct; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Max; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Median; +import org.elasticsearch.xpack.esql.expression.function.aggregate.MedianAbsoluteDeviation; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Min; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Percentile; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.StartsWith; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.DissectExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.GrokExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.MvExpandExec; +import org.elasticsearch.xpack.esql.plan.physical.OrderExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.RowExec; +import org.elasticsearch.xpack.esql.plan.physical.ShowExec; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NameId; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunction; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.ArithmeticOperation; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mod; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NotEquals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.NullEquals; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.DateEsField; +import org.elasticsearch.xpack.ql.type.EsField; +import org.elasticsearch.xpack.ql.type.KeywordEsField; +import org.elasticsearch.xpack.ql.type.TextEsField; +import org.elasticsearch.xpack.ql.type.UnsupportedEsField; +import org.elasticsearch.xpack.ql.util.DateUtils; + +import java.io.IOException; +import java.time.ZoneId; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Stream; + +import static org.elasticsearch.xpack.esql.SerializationTestUtils.serializeDeserialize; +import static org.hamcrest.Matchers.equalTo; + +public class PlanNamedTypesTests extends ESTestCase { + + // List of known serializable plan nodes - this should be kept up to date or retrieved + // programmatically. Excludes LocalSourceExec + static final List> PHYSICAL_PLAN_NODE_CLS = List.of( + AggregateExec.class, + DissectExec.class, + EsQueryExec.class, + EsSourceExec.class, + EvalExec.class, + EnrichExec.class, + ExchangeExec.class, + ExchangeSinkExec.class, + ExchangeSourceExec.class, + FieldExtractExec.class, + FilterExec.class, + FragmentExec.class, + GrokExec.class, + LimitExec.class, + MvExpandExec.class, + OrderExec.class, + ProjectExec.class, + RowExec.class, + ShowExec.class, + TopNExec.class + ); + + // Tests that all physical plan nodes have a suitably named serialization entry. + public void testPhysicalPlanEntries() { + var expected = PHYSICAL_PLAN_NODE_CLS.stream().map(Class::getSimpleName).toList(); + var actual = PlanNamedTypes.namedTypeEntries() + .stream() + .filter(e -> e.categoryClass().isAssignableFrom(PhysicalPlan.class)) + .map(PlanNameRegistry.Entry::name) + .toList(); + assertThat(actual, equalTo(expected)); + } + + // Tests that all names are unique - there should be a good reason if this is not the case. + public void testUniqueNames() { + var actual = PlanNamedTypes.namedTypeEntries().stream().map(PlanNameRegistry.Entry::name).distinct().toList(); + assertThat(actual.size(), equalTo(PlanNamedTypes.namedTypeEntries().size())); + } + + // Tests that reader from the original(outer) stream and inner(plan) streams work together. + public void testWrappedStreamSimple() throws IOException { + // write + BytesStreamOutput bso = new BytesStreamOutput(); + bso.writeString("hello"); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + var plan = new RowExec(Source.EMPTY, List.of(field("foo", DataTypes.LONG))); + out.writePhysicalPlanNode(plan); + bso.writeVInt(11_345); + + // read + StreamInput in = ByteBufferStreamInput.wrap(BytesReference.toBytes(bso.bytes())); + assertThat(in.readString(), equalTo("hello")); + var planStreamInput = new PlanStreamInput(in, planNameRegistry, SerializationTestUtils.writableRegistry(), EsqlTestUtils.TEST_CFG); + var deser = (RowExec) planStreamInput.readPhysicalPlanNode(); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(plan, unused -> deser); + assertThat(in.readVInt(), equalTo(11_345)); + } + + public void testUnsupportedAttributeSimple() throws IOException { + var orig = new UnsupportedAttribute( + Source.EMPTY, + "foo", + new UnsupportedEsField("foo", "keyword"), + "field not supported", + new NameId(53) + ); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeUnsupportedAttr(out, orig); + var deser = PlanNamedTypes.readUnsupportedAttr(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + assertThat(deser.id(), equalTo(orig.id())); + } + + public void testUnsupportedAttribute() { + Stream.generate(PlanNamedTypesTests::randomUnsupportedAttribute) + .limit(100) + .forEach(PlanNamedTypesTests::assertNamedExpressionAndId); + } + + public void testFieldAttributeSimple() throws IOException { + var orig = new FieldAttribute( + Source.EMPTY, + null, // parent, can be null + "bar", // name + DataTypes.KEYWORD, + randomEsField(), + null, // qualifier, can be null + Nullability.TRUE, + new NameId(53), + true // synthetic + ); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeFieldAttribute(out, orig); + var deser = PlanNamedTypes.readFieldAttribute(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + assertThat(deser.id(), equalTo(orig.id())); + } + + public void testFieldAttribute() { + Stream.generate(PlanNamedTypesTests::randomFieldAttribute).limit(100).forEach(PlanNamedTypesTests::assertNamedExpressionAndId); + } + + public void testKeywordEsFieldSimple() throws IOException { + var orig = new KeywordEsField( + "BarKeyField", // name + Map.of(), // no properties + true, // hasDocValues + 5, // precision + true, // normalized + true // alias + ); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeKeywordEsField(out, orig); + var deser = PlanNamedTypes.readKeywordEsField(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testKeywordEsField() { + Stream.generate(PlanNamedTypesTests::randomKeywordEsField).limit(100).forEach(PlanNamedTypesTests::assertNamedEsField); + } + + public void testTextdEsFieldSimple() throws IOException { + var orig = new TextEsField( + "BarKeyField", // name + Map.of(), // no properties + true, // hasDocValues + true // alias + ); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeTextEsField(out, orig); + var deser = PlanNamedTypes.readTextEsField(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testTextEsField() { + Stream.generate(PlanNamedTypesTests::randomTextEsField).limit(100).forEach(PlanNamedTypesTests::assertNamedEsField); + } + + public void testEsDateFieldSimple() throws IOException { + var orig = DateEsField.dateEsField("birth_date", Map.of(), false); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeDateEsField(out, orig); + var deser = PlanNamedTypes.readDateEsField(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testBinComparisonSimple() throws IOException { + var orig = new Equals(Source.EMPTY, field("foo", DataTypes.DOUBLE), field("bar", DataTypes.DOUBLE)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + out.writeNamed(BinaryComparison.class, orig); + var deser = (Equals) planStreamInput(bso).readNamed(BinaryComparison.class); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testBinComparison() { + Stream.generate(PlanNamedTypesTests::randomBinaryComparison) + .limit(100) + .forEach(obj -> assertNamedType(BinaryComparison.class, obj)); + } + + public void testAggFunctionSimple() throws IOException { + var orig = new Avg(Source.EMPTY, field("foo_val", DataTypes.DOUBLE)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + out.writeNamed(AggregateFunction.class, orig); + var deser = (Avg) planStreamInput(bso).readNamed(AggregateFunction.class); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testAggFunction() { + Stream.generate(PlanNamedTypesTests::randomAggFunction).limit(100).forEach(obj -> assertNamedType(AggregateFunction.class, obj)); + } + + public void testArithmeticOperationSimple() throws IOException { + var orig = new Add(Source.EMPTY, field("foo", DataTypes.LONG), field("bar", DataTypes.LONG)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + out.writeNamed(ArithmeticOperation.class, orig); + var deser = (Add) planStreamInput(bso).readNamed(ArithmeticOperation.class); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testArithmeticOperation() { + Stream.generate(PlanNamedTypesTests::randomArithmeticOperation) + .limit(100) + .forEach(obj -> assertNamedType(ArithmeticOperation.class, obj)); + } + + public void testSubStringSimple() throws IOException { + var orig = new Substring(Source.EMPTY, field("foo", DataTypes.KEYWORD), new Literal(Source.EMPTY, 1, DataTypes.INTEGER), null); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeSubstring(out, orig); + var deser = PlanNamedTypes.readSubstring(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testStartsWithSimple() throws IOException { + var orig = new StartsWith(Source.EMPTY, field("foo", DataTypes.KEYWORD), new Literal(Source.EMPTY, "fo", DataTypes.KEYWORD)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeStartsWith(out, orig); + var deser = PlanNamedTypes.readStartsWith(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testRoundSimple() throws IOException { + var orig = new Round(Source.EMPTY, field("value", DataTypes.DOUBLE), new Literal(Source.EMPTY, 1, DataTypes.INTEGER)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeRound(out, orig); + var deser = PlanNamedTypes.readRound(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testPowSimple() throws IOException { + var orig = new Pow(Source.EMPTY, field("value", DataTypes.DOUBLE), new Literal(Source.EMPTY, 1, DataTypes.INTEGER)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writePow(out, orig); + var deser = PlanNamedTypes.readPow(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testAliasSimple() throws IOException { + var orig = new Alias(Source.EMPTY, "alias_name", field("a", DataTypes.LONG)); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeAlias(out, orig); + var deser = PlanNamedTypes.readAlias(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + assertThat(orig.id(), equalTo(deser.id())); + } + + public void testLiteralSimple() throws IOException { + var orig = new Literal(Source.EMPTY, 1, DataTypes.INTEGER); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeLiteral(out, orig); + var deser = PlanNamedTypes.readLiteral(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testOrderSimple() throws IOException { + var orig = new Order(Source.EMPTY, field("val", DataTypes.INTEGER), Order.OrderDirection.ASC, Order.NullsPosition.FIRST); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeOrder(out, orig); + var deser = PlanNamedTypes.readOrder(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testFieldSortSimple() throws IOException { + var orig = new EsQueryExec.FieldSort(field("val", DataTypes.LONG), Order.OrderDirection.ASC, Order.NullsPosition.FIRST); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeFieldSort(out, orig); + var deser = PlanNamedTypes.readFieldSort(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testEsIndexSimple() throws IOException { + var orig = new EsIndex("test*", Map.of("first_name", new KeywordEsField("first_name")), Set.of("test1", "test2")); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeEsIndex(out, orig); + var deser = PlanNamedTypes.readEsIndex(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + public void testDissectParserSimple() throws IOException { + String pattern = "%{b} %{c}"; + var orig = new Dissect.Parser(pattern, ",", new DissectParser(pattern, ",")); + BytesStreamOutput bso = new BytesStreamOutput(); + PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry); + PlanNamedTypes.writeDissectParser(out, orig); + var deser = PlanNamedTypes.readDissectParser(planStreamInput(bso)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser); + } + + private static void assertNamedExpressionAndId(NamedExpression origObj) { + var deserObj = serializeDeserialize(origObj, PlanStreamOutput::writeExpression, PlanStreamInput::readNamedExpression); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origObj, unused -> deserObj); + assertThat(deserObj.id(), equalTo(origObj.id())); + } + + private static void assertNamedType(Class type, T origObj) { + var deserObj = serializeDeserialize(origObj, (o, v) -> o.writeNamed(type, origObj), i -> i.readNamed(type)); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origObj, unused -> deserObj); + } + + private static void assertNamedEsField(EsField origObj) { + var deserObj = serializeDeserialize(origObj, (o, v) -> o.writeNamed(EsField.class, v), PlanStreamInput::readEsFieldNamed); + EqualsHashCodeTestUtils.checkEqualsAndHashCode(origObj, unused -> deserObj); + } + + static UnsupportedAttribute randomUnsupportedAttribute() { + return new UnsupportedAttribute( + Source.EMPTY, + randomAlphaOfLength(randomIntBetween(1, 25)), // name + randomUnsupportedEsField(), // field + randomStringOrNull(), // customMessage + nameIdOrNull() + ); + } + + static FieldAttribute randomFieldAttributeOrNull() { + return randomBoolean() ? randomFieldAttribute() : null; + } + + static FieldAttribute randomFieldAttribute() { + return new FieldAttribute( + Source.EMPTY, + randomFieldAttributeOrNull(), // parent + randomAlphaOfLength(randomIntBetween(1, 25)), // name + randomDataType(), + randomEsField(), + randomStringOrNull(), // qualifier + randomNullability(), + nameIdOrNull(), + randomBoolean() // synthetic + ); + } + + static KeywordEsField randomKeywordEsField() { + return new KeywordEsField( + randomAlphaOfLength(randomIntBetween(1, 25)), // name + randomProperties(), + randomBoolean(), // hasDocValues + randomIntBetween(1, 12), // precision + randomBoolean(), // normalized + randomBoolean() // alias + ); + } + + static TextEsField randomTextEsField() { + return new TextEsField( + randomAlphaOfLength(randomIntBetween(1, 25)), // name + randomProperties(), + randomBoolean(), // hasDocValues + randomBoolean() // alias + ); + } + + static BinaryComparison randomBinaryComparison() { + int v = randomIntBetween(0, 6); + var left = field(randomName(), randomDataType()); + var right = field(randomName(), randomDataType()); + return switch (v) { + case 0 -> new Equals(Source.EMPTY, left, right, zoneIdOrNull()); + case 1 -> new NullEquals(Source.EMPTY, left, right, zoneIdOrNull()); + case 2 -> new NotEquals(Source.EMPTY, left, right, zoneIdOrNull()); + case 3 -> new GreaterThan(Source.EMPTY, left, right, zoneIdOrNull()); + case 4 -> new GreaterThanOrEqual(Source.EMPTY, left, right, zoneIdOrNull()); + case 5 -> new LessThan(Source.EMPTY, left, right, zoneIdOrNull()); + case 6 -> new LessThanOrEqual(Source.EMPTY, left, right, zoneIdOrNull()); + default -> throw new AssertionError(v); + }; + } + + static AggregateFunction randomAggFunction() { + int v = randomIntBetween(0, 8); + var field = field(randomName(), randomDataType()); + var right = field(randomName(), randomDataType()); + return switch (v) { + case 0 -> new Avg(Source.EMPTY, field); + case 1 -> new Count(Source.EMPTY, field); + case 2 -> new Sum(Source.EMPTY, field); + case 3 -> new Min(Source.EMPTY, field); + case 4 -> new Max(Source.EMPTY, field); + case 5 -> new Median(Source.EMPTY, field); + case 6 -> new MedianAbsoluteDeviation(Source.EMPTY, field); + case 7 -> new CountDistinct(Source.EMPTY, field, right); + case 8 -> new Percentile(Source.EMPTY, field, right); + default -> throw new AssertionError(v); + }; + } + + static ArithmeticOperation randomArithmeticOperation() { + int v = randomIntBetween(0, 4); + var left = field(randomName(), randomDataType()); + var right = field(randomName(), randomDataType()); + return switch (v) { + case 0 -> new Add(Source.EMPTY, left, right); + case 1 -> new Sub(Source.EMPTY, left, right); + case 2 -> new Mul(Source.EMPTY, left, right); + case 3 -> new Div(Source.EMPTY, left, right); + case 4 -> new Mod(Source.EMPTY, left, right); + default -> throw new AssertionError(v); + }; + } + + static NameId nameIdOrNull() { + return randomBoolean() ? new NameId() : null; + } + + static ZoneId zoneIdOrNull() { + return randomBoolean() ? DateUtils.UTC : null; + } + + static Nullability randomNullability() { + int i = randomInt(2); + return switch (i) { + case 0 -> Nullability.UNKNOWN; + case 1 -> Nullability.TRUE; + case 2 -> Nullability.FALSE; + default -> throw new AssertionError(i); + }; + } + + static EsField randomEsField() { + return randomEsField(0); + } + + static EsField randomEsField(int depth) { + return new EsField( + randomAlphaOfLength(randomIntBetween(1, 25)), + randomDataType(), + randomProperties(depth), + randomBoolean(), // aggregatable + randomBoolean() // isAlias + ); + } + + static UnsupportedEsField randomUnsupportedEsField() { + return new UnsupportedEsField( + randomAlphaOfLength(randomIntBetween(1, 25)), // name + randomAlphaOfLength(randomIntBetween(1, 25)), // originalType + randomAlphaOfLength(randomIntBetween(1, 25)), // inherited + randomProperties() + ); + } + + static Map randomProperties() { + return randomProperties(0); + } + + static Map randomProperties(int depth) { + if (depth > 2) { + return Map.of(); // prevent infinite recursion (between EsField and properties) + } + int size = randomIntBetween(0, 5); + Map map = new HashMap<>(); + for (int i = 0; i < size; i++) { + map.put( + randomAlphaOfLength(randomIntBetween(1, 10)), // name + randomEsField(depth++) + ); + } + return Map.copyOf(map); + } + + static List DATA_TYPES = EsqlDataTypes.types().stream().toList(); + + static DataType randomDataType() { + return DATA_TYPES.get(randomIntBetween(0, DATA_TYPES.size() - 1)); + } + + static String randomStringOrNull() { + return randomBoolean() ? randomAlphaOfLength(randomIntBetween(1, 25)) : null; + } + + static String randomName() { + return randomAlphaOfLength(randomIntBetween(1, 25)); + } + + static FieldAttribute field(String name, DataType type) { + return new FieldAttribute(Source.EMPTY, name, new EsField(name, type, Collections.emptyMap(), false)); + } + + static PlanNameRegistry planNameRegistry = new PlanNameRegistry(); + + static PlanStreamInput planStreamInput(BytesStreamOutput out) { + StreamInput in = new NamedWriteableAwareStreamInput( + ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), + SerializationTestUtils.writableRegistry() + ); + return new PlanStreamInput(in, planNameRegistry, SerializationTestUtils.writableRegistry(), EsqlTestUtils.TEST_CFG); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/FoldNull.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/FoldNull.java new file mode 100644 index 0000000000000..17317e1a9015f --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/FoldNull.java @@ -0,0 +1,18 @@ +/* + * 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; + +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.optimizer.OptimizerRules; + +class FoldNull extends OptimizerRules.FoldNull { + @Override + public Expression rule(Expression e) { + return super.rule(e); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java new file mode 100644 index 0000000000000..9916e135623fa --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java @@ -0,0 +1,214 @@ +/* + * 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; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.junit.BeforeClass; + +import java.util.Map; + +import static org.elasticsearch.xpack.esql.EsqlTestUtils.L; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.statsForMissingField; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; + +//@TestLogging(value = "org.elasticsearch.xpack.esql:TRACE", reason = "debug") +public class LocalLogicalPlanOptimizerTests extends ESTestCase { + + private static EsqlParser parser; + private static Analyzer analyzer; + private static LogicalPlanOptimizer logicalOptimizer; + private static Map mapping; + + private static final Literal ONE = L(1); + + @BeforeClass + public static void init() { + parser = new EsqlParser(); + + mapping = loadMapping("mapping-basic.json"); + EsIndex test = new EsIndex("test", mapping); + IndexResolution getIndexResult = IndexResolution.valid(test); + logicalOptimizer = new LogicalPlanOptimizer(); + + analyzer = new Analyzer( + new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, EsqlTestUtils.emptyPolicyResolution()), + new Verifier(new Metrics()) + ); + } + + /** + * Expects + * LocalRelation[[first_name{f}#4],EMPTY] + */ + public void testMissingFieldInFilterNumeric() { + var plan = plan(""" + from test + | where emp_no > 10 + | keep first_name + """); + + var testStats = statsForMissingField("emp_no"); + var localPlan = localPlan(plan, testStats); + + var empty = asEmptyRelation(localPlan); + assertThat(Expressions.names(empty.output()), contains("first_name")); + } + + /** + * Expects + * LocalRelation[[first_name{f}#4],EMPTY] + */ + public void testMissingFieldInFilterString() { + var plan = plan(""" + from test + | where starts_with(last_name, "abc") + | keep first_name + """); + + var testStats = statsForMissingField("last_name"); + var localPlan = localPlan(plan, testStats); + + var empty = asEmptyRelation(localPlan); + assertThat(Expressions.names(empty.output()), contains("first_name")); + } + + /** + * Expects + * Project[[last_name{r}#6]] + * \_Eval[[null[KEYWORD] AS last_name]] + * \_Limit[10000[INTEGER]] + * \_EsRelation[test][_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, gen..] + */ + public void testMissingFieldInProject() { + var plan = plan(""" + from test + | keep last_name + """); + + var testStats = statsForMissingField("last_name"); + var localPlan = localPlan(plan, testStats); + + var project = as(localPlan, Project.class); + var projections = project.projections(); + assertThat(Expressions.names(projections), contains("last_name")); + as(projections.get(0), ReferenceAttribute.class); + var eval = as(project.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("last_name")); + var alias = as(eval.fields().get(0), Alias.class); + var literal = as(alias.child(), Literal.class); + assertThat(literal.fold(), is(nullValue())); + assertThat(literal.dataType(), is(DataTypes.KEYWORD)); + + var limit = as(eval.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + /** + * Expects + * EsqlProject[[first_name{f}#4]] + * \_Limit[10000[INTEGER]] + * \_EsRelation[test][_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !ge..] + */ + public void testMissingFieldInSort() { + var plan = plan(""" + from test + | sort last_name + | keep first_name + """); + + var testStats = statsForMissingField("last_name"); + var localPlan = localPlan(plan, testStats); + + var project = as(localPlan, Project.class); + var projections = project.projections(); + assertThat(Expressions.names(projections), contains("first_name")); + + var limit = as(project.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + /** + * Expects + * EsqlProject[[x{r}#3]] + * \_Eval[[null[INTEGER] AS x]] + * \_Limit[10000[INTEGER]] + * \_EsRelation[test][_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + */ + public void testMissingFieldInEval() { + var plan = plan(""" + from test + | eval x = emp_no + 1 + | keep x + """); + + var testStats = statsForMissingField("emp_no"); + var localPlan = localPlan(plan, testStats); + + var project = as(localPlan, Project.class); + assertThat(Expressions.names(project.projections()), contains("x")); + var eval = as(project.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("x")); + + var alias = as(eval.fields().get(0), Alias.class); + var literal = as(alias.child(), Literal.class); + assertThat(literal.fold(), is(nullValue())); + assertThat(literal.dataType(), is(DataTypes.INTEGER)); + + var limit = as(eval.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + private LocalRelation asEmptyRelation(Object o) { + var empty = as(o, LocalRelation.class); + assertThat(empty.supplier(), is(LocalSupplier.EMPTY)); + return empty; + } + + private LogicalPlan plan(String query) { + var analyzed = analyzer.analyze(parser.createStatement(query)); + // System.out.println(analyzed); + var optimized = logicalOptimizer.optimize(analyzed); + // System.out.println(optimized); + return optimized; + } + + private LogicalPlan localPlan(LogicalPlan plan, SearchStats searchStats) { + var localContext = new LocalLogicalOptimizerContext(EsqlTestUtils.TEST_CFG, searchStats); + // System.out.println(plan); + var localPlan = new LocalLogicalPlanOptimizer(localContext).localOptimize(plan); + // System.out.println(localPlan); + return localPlan; + } +} 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 new file mode 100644 index 0000000000000..926967231f186 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -0,0 +1,1479 @@ +/* + * 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; + +import org.elasticsearch.common.lucene.BytesRefs; +import org.elasticsearch.compute.aggregation.QuantileStates; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Count; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Percentile; +import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateFormat; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateParse; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring; +import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.TopN; +import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Expressions; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Nullability; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.predicate.logical.And; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.expression.predicate.nulls.IsNotNull; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.EsRelation; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.junit.BeforeClass; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonList; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.L; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptySource; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.localSource; +import static org.elasticsearch.xpack.ql.TestUtils.greaterThanOf; +import static org.elasticsearch.xpack.ql.TestUtils.greaterThanOrEqualOf; +import static org.elasticsearch.xpack.ql.TestUtils.lessThanOf; +import static org.elasticsearch.xpack.ql.TestUtils.relation; +import static org.elasticsearch.xpack.ql.TestUtils.rlike; +import static org.elasticsearch.xpack.ql.TestUtils.wildcardLike; +import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.hamcrest.Matchers.contains; +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.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.startsWith; + +//@TestLogging(value = "org.elasticsearch.xpack.esql:TRACE", reason = "debug") +public class LogicalPlanOptimizerTests extends ESTestCase { + + private static final Literal ONE = L(1); + private static final Literal TWO = L(2); + private static final Literal THREE = L(3); + + private static EsqlParser parser; + private static Analyzer analyzer; + private static LogicalPlanOptimizer logicalOptimizer; + private static Map mapping; + + @BeforeClass + public static void init() { + parser = new EsqlParser(); + + mapping = loadMapping("mapping-basic.json"); + EsIndex test = new EsIndex("test", mapping); + IndexResolution getIndexResult = IndexResolution.valid(test); + + logicalOptimizer = new LogicalPlanOptimizer(); + EnrichPolicyResolution policy = AnalyzerTestUtils.loadEnrichPolicyResolution( + "languages_idx", + "id", + "languages_idx", + "mapping-languages.json" + ); + analyzer = new Analyzer( + new AnalyzerContext( + EsqlTestUtils.TEST_CFG, + new EsqlFunctionRegistry(), + getIndexResult, + new EnrichResolution(Set.of(policy), Set.of("languages_idx", "something")) + ), + new Verifier(new Metrics()) + ); + } + + public void testEmptyProjections() { + var plan = plan(""" + from test + | keep salary + | drop salary + """); + + var relation = as(plan, LocalRelation.class); + assertThat(relation.output(), is(empty())); + assertThat(relation.supplier().get(), emptyArray()); + } + + public void testEmptyProjectionInStat() { + var plan = plan(""" + from test + | stats c = count(salary) + | drop c + """); + + var relation = as(plan, LocalRelation.class); + assertThat(relation.output(), is(empty())); + assertThat(relation.supplier().get(), emptyArray()); + } + + public void testCombineProjections() { + var plan = plan(""" + from test + | keep emp_no, *name, salary + | keep last_name + """); + + var keep = as(plan, Project.class); + assertThat(Expressions.names(keep.projections()), contains("last_name")); + var limit = as(keep.child(), Limit.class); + var relation = as(limit.child(), EsRelation.class); + } + + public void testCombineProjectionWithFilterInBetween() { + var plan = plan(""" + from test + | keep *name, salary + | where salary > 10 + | keep last_name + """); + + var keep = as(plan, Project.class); + assertThat(Expressions.names(keep.projections()), contains("last_name")); + } + + public void testCombineProjectionWhilePreservingAlias() { + var plan = plan(""" + from test + | rename first_name as x + | keep x, salary + | where salary > 10 + | rename x as y + | keep y + """); + + var keep = as(plan, Project.class); + assertThat(Expressions.names(keep.projections()), contains("y")); + var p = keep.projections().get(0); + var alias = as(p, Alias.class); + assertThat(Expressions.name(alias.child()), containsString("first_name")); + } + + public void testCombineProjectionWithAggregation() { + var plan = plan(""" + from test + | stats s = sum(salary) by last_name, first_name + | keep s, last_name, first_name + """); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(Expressions.names(agg.aggregates()), contains("s", "last_name", "first_name")); + assertThat(Expressions.names(agg.groupings()), contains("last_name", "first_name")); + } + + public void testQlComparisonOptimizationsApply() { + var plan = plan(""" + from test + | where (1 + 4) < salary + """); + + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + // The core QL optimizations rotate constants to the right. + var condition = as(filter.condition(), GreaterThan.class); + assertThat(Expressions.name(condition.left()), equalTo("salary")); + assertThat(Expressions.name(condition.right()), equalTo("1 + 4")); + var con = as(condition.right(), Literal.class); + assertThat(con.value(), equalTo(5)); + } + + public void testCombineProjectionWithPruning() { + var plan = plan(""" + from test + | rename first_name as x + | keep x, salary, last_name + | stats count(salary) by x + """); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(Expressions.names(agg.aggregates()), contains("count(salary)", "x")); + assertThat(Expressions.names(agg.groupings()), contains("x")); + var alias = as(agg.aggregates().get(1), Alias.class); + var field = as(alias.child(), FieldAttribute.class); + assertThat(field.name(), is("first_name")); + var group = as(agg.groupings().get(0), Attribute.class); + assertThat(group, is(alias.toAttribute())); + var from = as(agg.child(), EsRelation.class); + } + + public void testCombineLimits() { + var limitValues = new int[] { randomIntBetween(10, 99), randomIntBetween(100, 1000) }; + var firstLimit = randomBoolean() ? 0 : 1; + var secondLimit = firstLimit == 0 ? 1 : 0; + var oneLimit = new Limit(EMPTY, L(limitValues[firstLimit]), emptySource()); + var anotherLimit = new Limit(EMPTY, L(limitValues[secondLimit]), oneLimit); + assertEquals( + new Limit(EMPTY, L(Math.min(limitValues[0], limitValues[1])), emptySource()), + new LogicalPlanOptimizer.PushDownAndCombineLimits().rule(anotherLimit) + ); + } + + public void testMultipleCombineLimits() { + var numberOfLimits = randomIntBetween(3, 10); + var minimum = randomIntBetween(10, 99); + var limitWithMinimum = randomIntBetween(0, numberOfLimits - 1); + + var fa = getFieldAttribute("a", INTEGER); + var relation = localSource(singletonList(fa), singletonList(1)); + LogicalPlan plan = relation; + + for (int i = 0; i < numberOfLimits; i++) { + var value = i == limitWithMinimum ? minimum : randomIntBetween(100, 1000); + plan = new Limit(EMPTY, L(value), plan); + } + assertEquals(new Limit(EMPTY, L(minimum), relation), new LogicalPlanOptimizer().optimize(plan)); + } + + public void testCombineFilters() { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + + Filter fa = new Filter(EMPTY, relation, conditionA); + Filter fb = new Filter(EMPTY, fa, conditionB); + + assertEquals( + new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), + new LogicalPlanOptimizer.PushDownAndCombineFilters().apply(fb) + ); + } + + public void testCombineFiltersLikeRLike() { + EsRelation relation = relation(); + RLike conditionA = rlike(getFieldAttribute("a"), "foo"); + WildcardLike conditionB = wildcardLike(getFieldAttribute("b"), "bar"); + + Filter fa = new Filter(EMPTY, relation, conditionA); + Filter fb = new Filter(EMPTY, fa, conditionB); + + assertEquals( + new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), + new LogicalPlanOptimizer.PushDownAndCombineFilters().apply(fb) + ); + } + + public void testPushDownFilter() { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + + Filter fa = new Filter(EMPTY, relation, conditionA); + List projections = singletonList(getFieldAttribute("b")); + EsqlProject keep = new EsqlProject(EMPTY, fa, projections); + Filter fb = new Filter(EMPTY, keep, conditionB); + + Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)); + assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new LogicalPlanOptimizer.PushDownAndCombineFilters().apply(fb)); + } + + public void testPushDownLikeRlikeFilter() { + EsRelation relation = relation(); + RLike conditionA = rlike(getFieldAttribute("a"), "foo"); + WildcardLike conditionB = wildcardLike(getFieldAttribute("b"), "bar"); + + Filter fa = new Filter(EMPTY, relation, conditionA); + List projections = singletonList(getFieldAttribute("b")); + EsqlProject keep = new EsqlProject(EMPTY, fa, projections); + Filter fb = new Filter(EMPTY, keep, conditionB); + + Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)); + assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new LogicalPlanOptimizer.PushDownAndCombineFilters().apply(fb)); + } + + // from ... | where a > 1 | stats count(1) by b | where count(1) >= 3 and b < 2 + // => ... | where a > 1 and b < 2 | stats count(1) by b | where count(1) >= 3 + public void testSelectivelyPushDownFilterPastFunctionAgg() { + EsRelation relation = relation(); + GreaterThan conditionA = greaterThanOf(getFieldAttribute("a"), ONE); + LessThan conditionB = lessThanOf(getFieldAttribute("b"), TWO); + GreaterThanOrEqual aggregateCondition = greaterThanOrEqualOf(new Count(EMPTY, ONE), THREE); + + Filter fa = new Filter(EMPTY, relation, conditionA); + // invalid aggregate but that's fine cause its properties are not used by this rule + Aggregate aggregate = new Aggregate(EMPTY, fa, singletonList(getFieldAttribute("b")), emptyList()); + Filter fb = new Filter(EMPTY, aggregate, new And(EMPTY, aggregateCondition, conditionB)); + + // expected + Filter expected = new Filter( + EMPTY, + new Aggregate( + EMPTY, + new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), + singletonList(getFieldAttribute("b")), + emptyList() + ), + aggregateCondition + ); + assertEquals(expected, new LogicalPlanOptimizer.PushDownAndCombineFilters().apply(fb)); + } + + public void testSelectivelyPushDownFilterPastRefAgg() { + // expected plan: "from test | where emp_no > 1 and emp_no < 3 | stats x = count(1) by emp_no | where x > 7" + LogicalPlan plan = optimizedPlan(""" + from test + | where emp_no > 1 + | stats x = count(1) by emp_no + | where x + 2 > 9 + | where emp_no < 3"""); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof GreaterThan); + var gt = (GreaterThan) filter.condition(); + assertTrue(gt.left() instanceof ReferenceAttribute); + var refAttr = (ReferenceAttribute) gt.left(); + assertEquals("x", refAttr.name()); + assertEquals(L(7), gt.right()); + + var agg = as(filter.child(), Aggregate.class); + + filter = as(agg.child(), Filter.class); + assertTrue(filter.condition() instanceof And); + var and = (And) filter.condition(); + assertTrue(and.left() instanceof GreaterThan); + gt = (GreaterThan) and.left(); + assertTrue(gt.left() instanceof FieldAttribute); + assertEquals("emp_no", ((FieldAttribute) gt.left()).name()); + assertTrue(and.right() instanceof LessThan); + var lt = (LessThan) and.right(); + assertTrue(lt.left() instanceof FieldAttribute); + assertEquals("emp_no", ((FieldAttribute) lt.left()).name()); + + assertTrue(filter.child() instanceof EsRelation); + } + + public void testNoPushDownOrFilterPastAgg() { + LogicalPlan plan = optimizedPlan(""" + from test + | stats x = count(1) by emp_no + | where emp_no < 3 or x > 9"""); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof Or); + var or = (Or) filter.condition(); + assertTrue(or.left() instanceof LessThan); + assertTrue(or.right() instanceof GreaterThan); + + var stats = as(filter.child(), Aggregate.class); + assertTrue(stats.child() instanceof EsRelation); + } + + public void testSelectivePushDownComplexFilterPastAgg() { + // expected plan: from test | emp_no > 0 | stats x = count(1) by emp_no | where emp_no < 3 or x > 9 + LogicalPlan plan = optimizedPlan(""" + from test + | stats x = count(1) by emp_no + | where (emp_no < 3 or x > 9) and emp_no > 0"""); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof Or); + var or = (Or) filter.condition(); + assertTrue(or.left() instanceof LessThan); + assertTrue(or.right() instanceof GreaterThan); + + var stats = as(filter.child(), Aggregate.class); + filter = as(stats.child(), Filter.class); + assertTrue(filter.condition() instanceof GreaterThan); + var gt = (GreaterThan) filter.condition(); + assertTrue(gt.left() instanceof FieldAttribute); + assertEquals("emp_no", ((FieldAttribute) gt.left()).name()); + assertEquals(L(0), gt.right()); + + assertTrue(filter.child() instanceof EsRelation); + } + + public void testSelectivelyPushDownFilterPastEval() { + // expected plan: "from test | where emp_no > 1 and emp_no < 3 | eval x = emp_no + 1 | where x < 7" + LogicalPlan plan = optimizedPlan(""" + from test + | where emp_no > 1 + | eval x = emp_no + 1 + | where x + 2 < 9 + | where emp_no < 3"""); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof LessThan); + var lt = (LessThan) filter.condition(); + assertTrue(lt.left() instanceof ReferenceAttribute); + var refAttr = (ReferenceAttribute) lt.left(); + assertEquals("x", refAttr.name()); + assertEquals(L(7), lt.right()); + + var eval = as(filter.child(), Eval.class); + assertEquals(1, eval.fields().size()); + assertTrue(eval.fields().get(0) instanceof Alias); + assertEquals("x", (eval.fields().get(0)).name()); + + filter = as(eval.child(), Filter.class); + assertTrue(filter.condition() instanceof And); + var and = (And) filter.condition(); + assertTrue(and.left() instanceof GreaterThan); + var gt = (GreaterThan) and.left(); + assertTrue(gt.left() instanceof FieldAttribute); + assertEquals("emp_no", ((FieldAttribute) gt.left()).name()); + assertTrue(and.right() instanceof LessThan); + lt = (LessThan) and.right(); + assertTrue(lt.left() instanceof FieldAttribute); + assertEquals("emp_no", ((FieldAttribute) lt.left()).name()); + + assertTrue(filter.child() instanceof EsRelation); + } + + public void testNoPushDownOrFilterPastLimit() { + LogicalPlan plan = optimizedPlan(""" + from test + | limit 3 + | where emp_no < 3 or salary > 9"""); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof Or); + var or = (Or) filter.condition(); + assertTrue(or.left() instanceof LessThan); + assertTrue(or.right() instanceof GreaterThan); + + var limit2 = as(filter.child(), Limit.class); + assertTrue(limit2.child() instanceof EsRelation); + } + + public void testPushDownFilterPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename emp_no as x + | keep x + | where x > 10"""); + + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var filter = as(limit.child(), Filter.class); + var attr = filter.condition().collect(Attribute.class::isInstance).stream().findFirst().get(); + assertThat(as(attr, FieldAttribute.class).name(), is("emp_no")); + } + + public void testPushDownEvalPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename emp_no as x + | keep x + | eval y = x * 2"""); + + var keep = as(plan, Project.class); + var eval = as(keep.child(), Eval.class); + assertThat( + eval.fields(), + contains( + new Alias( + EMPTY, + "y", + new Mul(EMPTY, new FieldAttribute(EMPTY, "emp_no", mapping.get("emp_no")), new Literal(EMPTY, 2, INTEGER)) + ) + ) + ); + } + + public void testPushDownDissectPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename first_name as x + | keep x + | dissect x "%{y}" + """); + + var keep = as(plan, Project.class); + var dissect = as(keep.child(), Dissect.class); + assertThat(dissect.extractedFields(), contains(new ReferenceAttribute(Source.EMPTY, "y", DataTypes.KEYWORD))); + } + + public void testPushDownGrokPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename first_name as x + | keep x + | grok x "%{WORD:y}" + """); + + var keep = as(plan, Project.class); + var grok = as(keep.child(), Grok.class); + assertThat(grok.extractedFields(), contains(new ReferenceAttribute(Source.EMPTY, "y", DataTypes.KEYWORD))); + } + + public void testPushDownFilterPastProjectUsingEval() { + LogicalPlan plan = optimizedPlan(""" + from test + | eval y = emp_no + 1 + | rename y as x + | where x > 10"""); + + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var filter = as(limit.child(), Filter.class); + var attr = filter.condition().collect(Attribute.class::isInstance).stream().findFirst().get(); + assertThat(as(attr, ReferenceAttribute.class).name(), is("y")); + var eval = as(filter.child(), Eval.class); + as(eval.child(), EsRelation.class); + } + + public void testPushDownFilterPastProjectUsingDissect() { + LogicalPlan plan = optimizedPlan(""" + from test + | dissect first_name "%{y}" + | rename y as x + | keep x + | where x == "foo" + """); + + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var filter = as(limit.child(), Filter.class); + var attr = filter.condition().collect(Attribute.class::isInstance).stream().findFirst().get(); + assertThat(as(attr, ReferenceAttribute.class).name(), is("y")); + var dissect = as(filter.child(), Dissect.class); + as(dissect.child(), EsRelation.class); + } + + public void testPushDownFilterPastProjectUsingGrok() { + LogicalPlan plan = optimizedPlan(""" + from test + | grok first_name "%{WORD:y}" + | rename y as x + | keep x + | where x == "foo" + """); + + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var filter = as(limit.child(), Filter.class); + var attr = filter.condition().collect(Attribute.class::isInstance).stream().findFirst().get(); + assertThat(as(attr, ReferenceAttribute.class).name(), is("y")); + var grok = as(filter.child(), Grok.class); + as(grok.child(), EsRelation.class); + } + + public void testPushDownLimitPastEval() { + LogicalPlan plan = optimizedPlan(""" + from test + | eval x = emp_no + 100 + | limit 10"""); + + var eval = as(plan, Eval.class); + as(eval.child(), Limit.class); + } + + public void testPushDownLimitPastDissect() { + LogicalPlan plan = optimizedPlan(""" + from test + | dissect first_name "%{y}" + | limit 10"""); + + var dissect = as(plan, Dissect.class); + as(dissect.child(), Limit.class); + } + + public void testPushDownLimitPastGrok() { + LogicalPlan plan = optimizedPlan(""" + from test + | grok first_name "%{WORD:y}" + | limit 10"""); + + var grok = as(plan, Grok.class); + as(grok.child(), Limit.class); + } + + public void testPushDownLimitPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename emp_no as a + | keep a + | limit 10"""); + + var keep = as(plan, Project.class); + as(keep.child(), Limit.class); + } + + public void testDontPushDownLimitPastFilter() { + LogicalPlan plan = optimizedPlan(""" + from test + | limit 100 + | where emp_no > 10 + | limit 10"""); + + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + as(filter.child(), Limit.class); + } + + public void testEliminateHigherLimitDueToDescendantLimit() throws Exception { + LogicalPlan plan = optimizedPlan(""" + from test + | limit 10 + | sort emp_no + | where emp_no > 10 + | eval c = emp_no + 2 + | limit 100"""); + + var topN = as(plan, TopN.class); + var eval = as(topN.child(), Eval.class); + var filter = as(eval.child(), Filter.class); + as(filter.child(), Limit.class); + } + + public void testDoNotEliminateHigherLimitDueToDescendantLimit() throws Exception { + LogicalPlan plan = optimizedPlan(""" + from test + | limit 10 + | where emp_no > 10 + | stats c = count(emp_no) by emp_no + | limit 100"""); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + var filter = as(agg.child(), Filter.class); + as(filter.child(), Limit.class); + } + + public void testBasicNullFolding() { + FoldNull rule = new FoldNull(); + assertNullLiteral(rule.rule(new Add(EMPTY, L(randomInt()), Literal.NULL))); + assertNullLiteral(rule.rule(new Round(EMPTY, Literal.NULL, null))); + assertNullLiteral(rule.rule(new Pow(EMPTY, Literal.NULL, Literal.NULL))); + assertNullLiteral(rule.rule(new DateFormat(EMPTY, Literal.NULL, Literal.NULL, null))); + assertNullLiteral(rule.rule(new DateParse(EMPTY, Literal.NULL, Literal.NULL))); + assertNullLiteral(rule.rule(new DateTrunc(EMPTY, Literal.NULL, Literal.NULL))); + assertNullLiteral(rule.rule(new Substring(EMPTY, Literal.NULL, Literal.NULL, Literal.NULL))); + } + + public void testPruneSortBeforeStats() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | where emp_no > 10 + | stats x = sum(salary) by first_name"""); + + var limit = as(plan, Limit.class); + var stats = as(limit.child(), Aggregate.class); + var filter = as(stats.child(), Filter.class); + as(filter.child(), EsRelation.class); + } + + public void testDontPruneSortWithLimitBeforeStats() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | limit 100 + | stats x = sum(salary) by first_name"""); + + var limit = as(plan, Limit.class); + var stats = as(limit.child(), Aggregate.class); + var topN = as(stats.child(), TopN.class); + as(topN.child(), EsRelation.class); + } + + public void testCombineOrderBy() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | sort salary"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("salary", "emp_no")); + as(topN.child(), EsRelation.class); + } + + public void testCombineOrderByThroughEval() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | eval x = salary + 1 + | sort x"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("x", "emp_no")); + var eval = as(topN.child(), Eval.class); + as(eval.child(), EsRelation.class); + } + + public void testCombineOrderByThroughEvalWithTwoDefs() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | eval x = salary + 1, y = salary + 2 + | eval z = x * y + | sort z"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("z", "emp_no")); + var eval = as(topN.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("z")); + eval = as(eval.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("x", "y")); + as(eval.child(), EsRelation.class); + } + + public void testCombineOrderByThroughDissect() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | dissect first_name "%{x}" + | sort x"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("x", "emp_no")); + var dissect = as(topN.child(), Dissect.class); + as(dissect.child(), EsRelation.class); + } + + public void testCombineOrderByThroughGrok() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | grok first_name "%{WORD:x}" + | sort x"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("x", "emp_no")); + var grok = as(topN.child(), Grok.class); + as(grok.child(), EsRelation.class); + } + + public void testCombineOrderByThroughProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | keep salary, emp_no + | sort salary"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat(orderNames(topN), contains("salary", "emp_no")); + as(topN.child(), EsRelation.class); + } + + public void testCombineOrderByThroughProjectAndEval() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | rename emp_no as en + | keep salary, en + | eval e = en * 2 + | sort salary"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat(orderNames(topN), contains("salary", "emp_no")); + var eval = as(topN.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("e")); + as(eval.child(), EsRelation.class); + } + + public void testCombineOrderByThroughProjectWithAlias() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | rename salary as l + | keep l, emp_no + | sort l"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat(orderNames(topN), contains("salary", "emp_no")); + as(topN.child(), EsRelation.class); + } + + public void testCombineOrderByThroughFilter() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | where emp_no > 10 + | sort salary"""); + + var topN = as(plan, TopN.class); + assertThat(orderNames(topN), contains("salary", "emp_no")); + var filter = as(topN.child(), Filter.class); + as(filter.child(), EsRelation.class); + } + + private static List orderNames(TopN topN) { + return topN.order().stream().map(o -> as(o.child(), NamedExpression.class).name()).toList(); + } + + public void testCombineLimitWithOrderByThroughFilterAndEval() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort salary + | eval x = emp_no / 2 + | where x > 20 + | sort x + | limit 10"""); + + var topN = as(plan, TopN.class); + var filter = as(topN.child(), Filter.class); + var eval = as(filter.child(), Eval.class); + as(eval.child(), EsRelation.class); + } + + public void testCombineMultipleOrderByAndLimits() { + // expected plan: + // from test + // | sort salary, emp_no + // | limit 100 + // | where salary > 1 + // | sort emp_no, first_name + // | keep l = salary, emp_no, first_name + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no + | rename salary as l + | keep l, emp_no, first_name + | sort l + | limit 100 + | sort first_name + | where l > 1 + | sort emp_no"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat(orderNames(topN), contains("emp_no", "first_name")); + var filter = as(topN.child(), Filter.class); + var topN2 = as(filter.child(), TopN.class); + assertThat(orderNames(topN2), contains("salary", "emp_no")); + as(topN2.child(), EsRelation.class); + } + + public void testPruneRedundantSortClauses() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort salary nulls last, emp_no desc nulls first + | where salary > 2 + | eval e = emp_no * 2 + | keep salary, emp_no, e + | sort e, emp_no, salary desc, emp_no desc"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat( + topN.order(), + contains( + new Order( + EMPTY, + new ReferenceAttribute(EMPTY, "e", INTEGER, null, Nullability.TRUE, null, false), + Order.OrderDirection.ASC, + Order.NullsPosition.LAST + ), + new Order( + EMPTY, + new FieldAttribute(EMPTY, "emp_no", mapping.get("emp_no")), + Order.OrderDirection.ASC, + Order.NullsPosition.LAST + ), + new Order( + EMPTY, + new FieldAttribute(EMPTY, "salary", mapping.get("salary")), + Order.OrderDirection.DESC, + Order.NullsPosition.FIRST + ) + ) + ); + assertThat(topN.child().collect(OrderBy.class::isInstance), is(emptyList())); + } + + public void testPruneRedundantSortClausesUsingAlias() { + LogicalPlan plan = optimizedPlan(""" + from test + | sort emp_no desc + | rename emp_no as e + | keep e + | sort e"""); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + assertThat( + topN.order(), + contains( + new Order( + EMPTY, + new FieldAttribute(EMPTY, "emp_no", mapping.get("emp_no")), + Order.OrderDirection.ASC, + Order.NullsPosition.LAST + ) + ) + ); + } + + public void testSimplifyLikeNoWildcard() { + LogicalPlan plan = optimizedPlan(""" + from test + | where first_name like "foo" + """); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof Equals); + Equals equals = as(filter.condition(), Equals.class); + assertEquals(BytesRefs.toBytesRef("foo"), equals.right().fold()); + assertTrue(filter.child() instanceof EsRelation); + } + + public void testSimplifyLikeMatchAll() { + LogicalPlan plan = optimizedPlan(""" + from test + | where first_name like "*" + """); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + as(filter.condition(), IsNotNull.class); + assertTrue(filter.child() instanceof EsRelation); + } + + public void testSimplifyRLikeNoWildcard() { + LogicalPlan plan = optimizedPlan(""" + from test + | where first_name rlike "foo" + """); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + assertTrue(filter.condition() instanceof Equals); + Equals equals = as(filter.condition(), Equals.class); + assertEquals(BytesRefs.toBytesRef("foo"), equals.right().fold()); + assertTrue(filter.child() instanceof EsRelation); + } + + public void testSimplifyRLikeMatchAll() { + LogicalPlan plan = optimizedPlan(""" + from test + | where first_name rlike ".*" + """); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + + var isNotNull = as(filter.condition(), IsNotNull.class); + assertTrue(filter.child() instanceof EsRelation); + } + + public void testFoldNullInToLocalRelation() { + LogicalPlan plan = optimizedPlan(""" + from test + | where null in (first_name, ".*") + """); + assertThat(plan, instanceOf(LocalRelation.class)); + } + + public void testFoldNullListInToLocalRelation() { + LogicalPlan plan = optimizedPlan(""" + from test + | where first_name in (null, null) + """); + assertThat(plan, instanceOf(LocalRelation.class)); + } + + public void testFoldInKeyword() { + LogicalPlan plan = optimizedPlan(""" + from test + | where "foo" in ("bar", "baz") + """); + assertThat(plan, instanceOf(LocalRelation.class)); + + plan = optimizedPlan(""" + from test + | where "foo" in ("bar", "foo", "baz") + """); + var limit = as(plan, Limit.class); + as(limit.child(), EsRelation.class); + } + + public void testFoldInIP() { + LogicalPlan plan = optimizedPlan(""" + from test + | where to_ip("1.1.1.1") in (to_ip("1.1.1.2"), to_ip("1.1.1.2")) + """); + assertThat(plan, instanceOf(LocalRelation.class)); + + plan = optimizedPlan(""" + from test + | where to_ip("1.1.1.1") in (to_ip("1.1.1.1"), to_ip("1.1.1.2")) + """); + var limit = as(plan, Limit.class); + as(limit.child(), EsRelation.class); + } + + public void testFoldInVersion() { + LogicalPlan plan = optimizedPlan(""" + from test + | where to_version("1.2.3") in (to_version("1"), to_version("1.2.4")) + """); + assertThat(plan, instanceOf(LocalRelation.class)); + + plan = optimizedPlan(""" + from test + | where to_version("1.2.3") in (to_version("1"), to_version("1.2.3")) + """); + var limit = as(plan, Limit.class); + as(limit.child(), EsRelation.class); + } + + public void testFoldInNumerics() { + LogicalPlan plan = optimizedPlan(""" + from test + | where 3 in (4.0, 5, 2147483648) + """); + assertThat(plan, instanceOf(LocalRelation.class)); + + plan = optimizedPlan(""" + from test + | where 3 in (4.0, 3.0, to_long(3)) + """); + var limit = as(plan, Limit.class); + as(limit.child(), EsRelation.class); + } + + public void testEnrich() { + LogicalPlan plan = optimizedPlan(""" + from test + | eval x = to_string(languages) + | enrich languages_idx on x + """); + var enrich = as(plan, Enrich.class); + assertTrue(enrich.policyName().resolved()); + assertThat(enrich.policyName().fold(), is(BytesRefs.toBytesRef("languages_idx"))); + var eval = as(enrich.child(), Eval.class); + var limit = as(eval.child(), Limit.class); + as(limit.child(), EsRelation.class); + } + + public void testPushDownEnrichPastProject() { + LogicalPlan plan = optimizedPlan(""" + from test + | eval a = to_string(languages) + | rename a as x + | keep x + | enrich languages_idx on x + """); + + var keep = as(plan, Project.class); + as(keep.child(), Enrich.class); + } + + public void testTopNEnrich() { + LogicalPlan plan = optimizedPlan(""" + from test + | rename languages as x + | eval x = to_string(x) + | keep x + | enrich languages_idx on x + | sort language_name + """); + + var keep = as(plan, Project.class); + var topN = as(keep.child(), TopN.class); + as(topN.child(), Enrich.class); + } + + public void testEnrichNotNullFilter() { + LogicalPlan plan = optimizedPlan(""" + from test + | eval x = to_string(languages) + | enrich languages_idx on x + | where language_name is not null + | limit 10 + """); + var limit = as(plan, Limit.class); + var filter = as(limit.child(), Filter.class); + var enrich = as(filter.child(), Enrich.class); + assertTrue(enrich.policyName().resolved()); + assertThat(enrich.policyName().fold(), is(BytesRefs.toBytesRef("languages_idx"))); + var eval = as(enrich.child(), Eval.class); + as(eval.child(), EsRelation.class); + } + + /** + * Expects + * EsqlProject[[a{r}#3, last_name{f}#9]] + * \_Eval[[__a_SUM_123{r}#12 / __a_COUNT_150{r}#13 AS a]] + * \_Limit[10000[INTEGER]] + * \_Aggregate[[last_name{f}#9],[SUM(salary{f}#10) AS __a_SUM_123, COUNT(salary{f}#10) AS __a_COUNT_150, last_nam + * e{f}#9]] + * \_EsRelation[test][_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + */ + public void testSimpleAvgReplacement() { + var plan = plan(""" + from test + | stats a = avg(salary) by last_name + """); + + var project = as(plan, Project.class); + assertThat(Expressions.names(project.projections()), contains("a", "last_name")); + var eval = as(project.child(), Eval.class); + var f = eval.fields(); + assertThat(f, hasSize(1)); + assertThat(f.get(0).name(), is("a")); + var limit = as(eval.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + var aggs = agg.aggregates(); + var a = as(aggs.get(0), Alias.class); + assertThat(a.name(), startsWith("__a_SUM@")); + var sum = as(a.child(), Sum.class); + + a = as(aggs.get(1), Alias.class); + assertThat(a.name(), startsWith("__a_COUNT@")); + var count = as(a.child(), Count.class); + + assertThat(Expressions.names(agg.groupings()), contains("last_name")); + } + + /** + * Expects + * EsqlProject[[a{r}#3, c{r}#6, s{r}#9, last_name{f}#15]] + * \_Eval[[s{r}#9 / c{r}#6 AS a]] + * \_Limit[10000[INTEGER]] + * \_Aggregate[[last_name{f}#15],[COUNT(salary{f}#16) AS c, SUM(salary{f}#16) AS s, last_name{f}#15]] + * \_EsRelation[test][_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, ..] + */ + public void testClashingAggAvgReplacement() { + var plan = plan(""" + from test + | stats a = avg(salary), c = count(salary), s = sum(salary) by last_name + """); + + assertThat(Expressions.names(plan.output()), contains("a", "c", "s", "last_name")); + var project = as(plan, EsqlProject.class); + var eval = as(project.child(), Eval.class); + var f = eval.fields(); + assertThat(f, hasSize(1)); + assertThat(f.get(0).name(), is("a")); + var limit = as(eval.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + var aggs = agg.aggregates(); + assertThat(Expressions.names(aggs), contains("c", "s", "last_name")); + } + + /** + * Expects + * EsqlProject[[a{r}#3, c{r}#6, s{r}#9, last_name{f}#15]] + * \_Eval[[s{r}#9 / __a_COUNT@xxx{r}#18 AS a]] + * \_Limit[10000[INTEGER]] + * \_Aggregate[[last_name{f}#15],[COUNT(salary{f}#16) AS __a_COUNT@xxx, COUNT(languages{f}#14) AS c, SUM(salary{f}#16) AS + * s, last_name{f}#15]] + * \_EsRelation[test][_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, ..] + */ + public void testSemiClashingAvgReplacement() { + var plan = plan(""" + from test + | stats a = avg(salary), c = count(languages), s = sum(salary) by last_name + """); + + var project = as(plan, Project.class); + assertThat(Expressions.names(project.projections()), contains("a", "c", "s", "last_name")); + var eval = as(project.child(), Eval.class); + var f = eval.fields(); + assertThat(f, hasSize(1)); + assertThat(f.get(0).name(), is("a")); + var limit = as(eval.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + var aggs = agg.aggregates(); + var a = as(aggs.get(0), Alias.class); + assertThat(a.name(), startsWith("__a_COUNT@")); + var sum = as(a.child(), Count.class); + + a = as(aggs.get(1), Alias.class); + assertThat(a.name(), is("c")); + var count = as(a.child(), Count.class); + + a = as(aggs.get(2), Alias.class); + assertThat(a.name(), is("s")); + } + + /** + * Expected + * Limit[10000[INTEGER]] + * \_Aggregate[[last_name{f}#9],[PERCENTILE(salary{f}#10,50[INTEGER]) AS m, last_name{f}#9]] + * \_EsRelation[test][_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + */ + public void testMedianReplacement() { + var plan = plan(""" + from test + | stats m = median(salary) by last_name + """); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(Expressions.names(agg.aggregates()), contains("m", "last_name")); + var aggs = agg.aggregates(); + var a = as(aggs.get(0), Alias.class); + var per = as(a.child(), Percentile.class); + var literal = as(per.percentile(), Literal.class); + assertThat((int) QuantileStates.MEDIAN, is(literal.fold())); + + assertThat(Expressions.names(agg.groupings()), contains("last_name")); + } + + public void testSplittingInWithFoldableValue() { + FieldAttribute fa = getFieldAttribute("foo"); + In in = new In(EMPTY, ONE, List.of(TWO, THREE, fa, L(null))); + Or expected = new Or(EMPTY, new In(EMPTY, ONE, List.of(TWO, THREE)), new In(EMPTY, ONE, List.of(fa, L(null)))); + assertThat(new LogicalPlanOptimizer.SplitInWithFoldableValue().rule(in), equalTo(expected)); + } + + public void testPruneUnusedEval() { + var plan = plan(""" + from test + | eval garbage = salary + 3 + | keep salary + """); + + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + public void testPruneChainedEval() { + var plan = plan(""" + from test + | eval garbage_a = salary + 3 + | eval garbage_b = emp_no / garbage_a, garbage_c = garbage_a + | eval garbage_x = 1 - garbage_b/garbage_c + | keep salary + """); + var keep = as(plan, Project.class); + var limit = as(keep.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + public void testPruneEvalDueToStats() { + var plan = plan(""" + from test + | eval garbage_a = salary + 3, x = salary + | eval garbage_b = x + 3 + | stats c = count(x) + """); + + var limit = as(plan, Limit.class); + var aggregate = as(limit.child(), Aggregate.class); + assertThat(aggregate.aggregates(), hasSize(1)); + var alias = as(aggregate.aggregates().get(0), Alias.class); + var count = as(alias.child(), Count.class); + var eval = as(aggregate.child(), Eval.class); + assertThat(eval.fields(), hasSize(1)); + var field = as(eval.fields().get(0), Alias.class); + assertThat(field.name(), is("x")); + var source = as(eval.child(), EsRelation.class); + } + + public void testPruneUnusedAggSimple() { + var plan = plan(""" + from test + | stats c = count(salary), max = max(salary), min = min(salary) + | keep c + """); + + var limit = as(plan, Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(agg.groupings(), hasSize(0)); + assertThat(agg.aggregates(), hasSize(1)); + var aggOne = as(agg.aggregates().get(0), Alias.class); + assertThat(aggOne.name(), is("c")); + var count = as(aggOne.child(), Count.class); + var source = as(agg.child(), EsRelation.class); + } + + public void testPruneUnusedAggMixedWithEval() { + var plan = plan(""" + from test + | stats c = count(salary), max = max(salary), min = min(salary) + | eval x = c + | keep x + """); + + var project = as(plan, Project.class); + var eval = as(project.child(), Eval.class); + var limit = as(eval.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(agg.groupings(), hasSize(0)); + assertThat(agg.aggregates(), hasSize(1)); + var aggOne = as(agg.aggregates().get(0), Alias.class); + assertThat(aggOne.name(), is("c")); + var count = as(aggOne.child(), Count.class); + var source = as(agg.child(), EsRelation.class); + } + + public void testPruneUnusedAggsChainedAgg() { + var plan = plan(""" + from test + | stats c = count(salary), max = max(salary), min = min(salary) + | eval x = max + min + c + | eval y = min + | eval z = c + | keep c + """); + + var project = as(plan, Project.class); + var limit = as(project.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + assertThat(agg.groupings(), hasSize(0)); + var aggs = agg.aggregates(); + assertThat(aggs, hasSize(1)); + assertThat(Expressions.names(aggs), contains("c")); + var source = as(agg.child(), EsRelation.class); + } + + public void testPruneMixedAggInsideUnusedEval() { + var plan = plan(""" + from test + | stats c = count(salary), max = max(salary), min = min(salary) + | eval x = max + min + c + | eval y = min + | where y > 10 + | eval z = c + | keep c + """); + + var project = as(plan, Project.class); + var limit = as(project.child(), Limit.class); + var filter = as(limit.child(), Filter.class); + var eval = as(filter.child(), Eval.class); + var agg = as(eval.child(), Aggregate.class); + assertThat(agg.groupings(), hasSize(0)); + var aggs = agg.aggregates(); + assertThat(aggs, hasSize(2)); + assertThat(Expressions.names(aggs), contains("c", "min")); + var source = as(agg.child(), EsRelation.class); + } + + public void testNoPruningWhenDealingJustWithEvals() { + var plan = plan(""" + from test + | stats c = count(salary), max = max(salary), min = min(salary) + | eval x = max + min + c + | eval y = min + | eval z = c + """); + + var eval = as(plan, Eval.class); + eval = as(eval.child(), Eval.class); + eval = as(eval.child(), Eval.class); + var limit = as(eval.child(), Limit.class); + var agg = as(limit.child(), Aggregate.class); + } + + public void testNoPruningWhenChainedEvals() { + var plan = plan(""" + from test + | eval x = emp_no, y = x + 1, z = y + | keep z + """); + + var project = as(plan, Project.class); + var eval = as(project.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("x", "y", "z")); + var limit = as(eval.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + public void testPruningDuplicateEvals() { + var plan = plan(""" + from test + | eval x = emp_no, x = salary + | eval y = salary + | eval y = emp_no + | keep x, y + """); + + var project = as(plan, Project.class); + var eval = as(project.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("y")); + eval = as(eval.child(), Eval.class); + assertThat(Expressions.names(eval.fields()), contains("x")); + var limit = as(eval.child(), Limit.class); + var source = as(limit.child(), EsRelation.class); + } + + private LogicalPlan optimizedPlan(String query) { + return plan(query); + } + + private LogicalPlan plan(String query) { + var analyzed = analyzer.analyze(parser.createStatement(query)); + // System.out.println(analyzed); + var optimized = logicalOptimizer.optimize(analyzed); + // System.out.println(optimized); + return optimized; + } + + private void assertNullLiteral(Expression expression) { + assertEquals(Literal.class, expression.getClass()); + assertNull(expression.fold()); + } + + // TODO: move these from org.elasticsearch.xpack.ql.optimizer.OptimizerRulesTests to org.elasticsearch.xpack.ql.TestUtils + private static FieldAttribute getFieldAttribute(String name) { + return getFieldAttribute(name, INTEGER); + } + + private static FieldAttribute getFieldAttribute(String name, DataType dataType) { + return new FieldAttribute(EMPTY, name, new EsField(name + "f", dataType, emptyMap(), true)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java new file mode 100644 index 0000000000000..2eab81f02911d --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -0,0 +1,1722 @@ +/* + * 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; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.RegexpQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.TermsQueryBuilder; +import org.elasticsearch.index.query.WildcardQueryBuilder; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.EnrichResolution; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.DissectExec; +import org.elasticsearch.xpack.esql.plan.physical.EnrichExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec.FieldSort; +import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.EstimatesRowSize; +import org.elasticsearch.xpack.esql.plan.physical.EvalExec; +import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.plan.physical.FilterExec; +import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; +import org.elasticsearch.xpack.esql.plan.physical.GrokExec; +import org.elasticsearch.xpack.esql.plan.physical.LimitExec; +import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.plan.physical.ProjectExec; +import org.elasticsearch.xpack.esql.plan.physical.TopNExec; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.esql.planner.PhysicalVerificationException; +import org.elasticsearch.xpack.esql.planner.PlannerUtils; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.querydsl.query.SingleValueQuery; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.esql.stats.SearchStats; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Attribute; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.DateUtils; +import org.elasticsearch.xpack.ql.type.EsField; +import org.junit.Before; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Arrays.asList; +import static org.elasticsearch.core.Tuple.tuple; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.statsForMissingField; +import static org.elasticsearch.xpack.esql.SerializationTestUtils.assertSerialization; +import static org.elasticsearch.xpack.ql.expression.Expressions.name; +import static org.elasticsearch.xpack.ql.expression.Expressions.names; +import static org.elasticsearch.xpack.ql.expression.Order.OrderDirection.ASC; +import static org.elasticsearch.xpack.ql.expression.function.scalar.FunctionTestUtils.l; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; + +//@TestLogging(value = "org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizer:TRACE", reason = "debug") +public class PhysicalPlanOptimizerTests extends ESTestCase { + + private static final String PARAM_FORMATTING = "%1$s"; + + /** + * Estimated size of a keyword field in bytes. + */ + private static final int KEYWORD_EST = EstimatesRowSize.estimateSize(DataTypes.KEYWORD); + + private EsqlParser parser; + private Analyzer analyzer; + private LogicalPlanOptimizer logicalOptimizer; + private PhysicalPlanOptimizer physicalPlanOptimizer; + private Mapper mapper; + private Map mapping; + private int allFieldRowSize; + + private final EsqlConfiguration config; + + @ParametersFactory(argumentFormatting = PARAM_FORMATTING) + public static List readScriptSpec() { + return settings().stream().map(t -> { + var settings = Settings.builder().loadFromMap(t.v2()).build(); + return new Object[] { + t.v1(), + new EsqlConfiguration( + DateUtils.UTC, + Locale.US, + null, + null, + new QueryPragmas(settings), + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(settings) + ) }; + }).toList(); + } + + private static List>> settings() { + return asList(new Tuple<>("default", Map.of())); + } + + public PhysicalPlanOptimizerTests(String name, EsqlConfiguration config) { + this.config = config; + } + + @Before + public void init() { + parser = new EsqlParser(); + + mapping = loadMapping("mapping-basic.json"); + allFieldRowSize = mapping.values() + .stream() + .mapToInt(f -> EstimatesRowSize.estimateSize(EsqlDataTypes.widenSmallNumericTypes(f.getDataType()))) + .sum(); + EsIndex test = new EsIndex("test", mapping); + IndexResolution getIndexResult = IndexResolution.valid(test); + logicalOptimizer = new LogicalPlanOptimizer(); + physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(config)); + FunctionRegistry functionRegistry = new EsqlFunctionRegistry(); + mapper = new Mapper(functionRegistry); + var enrichResolution = new EnrichResolution( + Set.of( + new EnrichPolicyResolution( + "foo", + new EnrichPolicy(EnrichPolicy.MATCH_TYPE, null, List.of("idx"), "fld", List.of("a", "b")), + IndexResolution.valid( + new EsIndex( + "idx", + Map.ofEntries( + Map.entry("a", new EsField("a", DataTypes.INTEGER, Map.of(), true)), + Map.entry("b", new EsField("b", DataTypes.LONG, Map.of(), true)) + ) + ) + ) + ) + ), + Set.of("foo") + ); + + analyzer = new Analyzer( + new AnalyzerContext(config, functionRegistry, getIndexResult, enrichResolution), + new Verifier(new Metrics()) + ); + } + + public void testSingleFieldExtractor() { + // using a function (round()) here and following tests to prevent the optimizer from pushing the + // filter down to the source and thus change the shape of the expected physical tree. + var plan = physicalPlan(""" + from test + | where round(emp_no) > 10 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var restExtract = as(project.child(), FieldExtractExec.class); + var limit = as(restExtract.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var extract = as(filter.child(), FieldExtractExec.class); + + assertEquals(Sets.difference(mapping.keySet(), Set.of("emp_no")), Sets.newHashSet(names(restExtract.attributesToExtract()))); + assertEquals(Set.of("emp_no"), Sets.newHashSet(names(extract.attributesToExtract()))); + + var query = as(extract.child(), EsQueryExec.class); + assertThat(query.estimatedRowSize(), equalTo(Integer.BYTES + allFieldRowSize)); + } + + public void testExactlyOneExtractorPerFieldWithPruning() { + var plan = physicalPlan(""" + from test + | where round(emp_no) > 10 + | eval c = emp_no + """); + + var optimized = optimizedPlan(plan); + var eval = as(optimized, EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var restExtract = as(project.child(), FieldExtractExec.class); + var limit = as(restExtract.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var extract = as(filter.child(), FieldExtractExec.class); + + assertEquals(Sets.difference(mapping.keySet(), Set.of("emp_no")), Sets.newHashSet(names(restExtract.attributesToExtract()))); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + + var query = source(extract.child()); + // An int for doc id and one for c + assertThat(query.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2)); + } + + public void testDoubleExtractorPerFieldEvenWithAliasNoPruningDueToImplicitProjection() { + var plan = physicalPlan(""" + from test + | where round(emp_no) > 10 + | eval c = salary + | stats x = sum(c) + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregate = as(limit.child(), AggregateExec.class); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES)); + + var exchange = asRemoteExchange(aggregate.child()); + aggregate = as(exchange.child(), AggregateExec.class); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES)); + + var eval = as(aggregate.child(), EvalExec.class); + + var extract = as(eval.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("salary")); + + var filter = as(extract.child(), FilterExec.class); + extract = as(filter.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + + var query = source(extract.child()); + assertThat(query.estimatedRowSize(), equalTo(Integer.BYTES * 4 /* for doc id, emp_no, salary, and c */)); + } + + public void testTripleExtractorPerField() { + var plan = physicalPlan(""" + from test + | where round(emp_no) > 10 + | eval c = first_name + | stats x = sum(salary) + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregate = as(limit.child(), AggregateExec.class); + var exchange = asRemoteExchange(aggregate.child()); + aggregate = as(exchange.child(), AggregateExec.class); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("salary")); + + var filter = as(extract.child(), FilterExec.class); + extract = as(filter.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + + var query = source(extract.child()); + // for doc ids, emp_no, salary + int estimatedSize = Integer.BYTES * 3; + assertThat(query.estimatedRowSize(), equalTo(estimatedSize)); + } + + /** + * Expected + * LimitExec[10000[INTEGER]] + * \_AggregateExec[[],[AVG(salary{f}#14) AS x],FINAL] + * \_AggregateExec[[],[AVG(salary{f}#14) AS x],PARTIAL] + * \_FilterExec[ROUND(emp_no{f}#9) > 10[INTEGER]] + * \_TopNExec[[Order[last_name{f}#13,ASC,LAST]],10[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[salary{f}#14, first_name{f}#10, emp_no{f}#9, last_name{f}#13]] -- project away _doc + * \_FieldExtractExec[salary{f}#14, first_name{f}#10, emp_no{f}#9, last_n..] -- local field extraction + * \_EsQueryExec[test], query[][_doc{f}#16], limit[10], sort[[last_name]] + */ + public void testExtractorForField() { + var plan = physicalPlan(""" + from test + | sort last_name + | limit 10 + | where round(emp_no) > 10 + | eval c = first_name + | stats x = sum(salary) + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregateFinal = as(limit.child(), AggregateExec.class); + assertThat(aggregateFinal.estimatedRowSize(), equalTo(Long.BYTES)); + + var aggregatePartial = as(aggregateFinal.child(), AggregateExec.class); + var filter = as(aggregatePartial.child(), FilterExec.class); + var topN = as(filter.child(), TopNExec.class); + + var exchange = asRemoteExchange(topN.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("salary", "emp_no", "last_name")); + var source = source(extract.child()); + assertThat(source.limit(), is(topN.limit())); + assertThat(source.sorts(), is(sorts(topN.order()))); + + assertThat(source.limit(), is(l(10))); + assertThat(source.sorts().size(), is(1)); + FieldSort order = source.sorts().get(0); + assertThat(order.direction(), is(ASC)); + assertThat(name(order.field()), is("last_name")); + // last name is keyword, salary, emp_no, doc id, segment, forwards and backwards doc id maps are all ints + int estimatedSize = KEYWORD_EST + Integer.BYTES * 6; + assertThat(source.estimatedRowSize(), equalTo(estimatedSize)); + } + + /** + * Expected + * + * EvalExec[[emp_no{f}#538 + 1[INTEGER] AS emp_no]] + * \_EvalExec[[emp_no{f}#538 + 1[INTEGER] AS e]] + * \_LimitExec[10000[INTEGER]] + * \_ExchangeExec[GATHER,SINGLE_DISTRIBUTION] + * \_ProjectExec[[_meta_field{f}#537, emp_no{f}#538, first_name{f}#539, languages{f}#540, last_name{f}#541, salary{f}#542]] + * \_FieldExtractExec[_meta_field{f}#537, emp_no{f}#538, first_name{f}#53..] + * \_EsQueryExec[test], query[][_doc{f}#543], limit[10000] + */ + public void testExtractorMultiEvalWithDifferentNames() { + var plan = physicalPlan(""" + from test + | eval e = emp_no + 1 + | eval emp_no = emp_no + 1 + """); + + var optimized = optimizedPlan(plan); + var eval = as(optimized, EvalExec.class); + eval = as(eval.child(), EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + assertThat( + names(extract.attributesToExtract()), + contains("_meta_field", "emp_no", "first_name", "gender", "languages", "last_name", "salary") + ); + } + + /** + * Expected + * EvalExec[[emp_no{r}#120 + 1[INTEGER] AS emp_no]] + * \_EvalExec[[emp_no{f}#125 + 1[INTEGER] AS emp_no]] + * \_LimitExec[10000[INTEGER]] + * \_ExchangeExec[GATHER,SINGLE_DISTRIBUTION] + * \_ProjectExec[[_meta_field{f}#124, emp_no{f}#125, first_name{f}#126, languages{f}#127, last_name{f}#128, salary{f}#129]] + * \_FieldExtractExec[_meta_field{f}#124, emp_no{f}#125, first_name{f}#12..] + * \_EsQueryExec[test], query[][_doc{f}#130], limit[10000] + */ + public void testExtractorMultiEvalWithSameName() { + var plan = physicalPlan(""" + from test + | eval emp_no = emp_no + 1 + | eval emp_no = emp_no + 1 + """); + + var optimized = optimizedPlan(plan); + var eval = as(optimized, EvalExec.class); + eval = as(eval.child(), EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + assertThat( + names(extract.attributesToExtract()), + contains("_meta_field", "emp_no", "first_name", "gender", "languages", "last_name", "salary") + ); + } + + public void testExtractorsOverridingFields() { + var plan = physicalPlan(""" + from test + | stats emp_no = sum(emp_no) + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var node = as(limit.child(), AggregateExec.class); + var exchange = asRemoteExchange(node.child()); + var aggregate = as(exchange.child(), AggregateExec.class); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + } + + public void testDoNotExtractGroupingFields() { + var plan = physicalPlan(""" + from test + | stats x = sum(salary) by first_name + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregate = as(limit.child(), AggregateExec.class); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + assertThat(aggregate.groupings(), hasSize(1)); + + var exchange = asRemoteExchange(aggregate.child()); + aggregate = as(exchange.child(), AggregateExec.class); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + assertThat(aggregate.groupings(), hasSize(1)); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), equalTo(List.of("salary"))); + + var source = source(extract.child()); + // doc id and salary are ints. salary isn't extracted. + // TODO salary kind of is extracted. At least sometimes it is. should it count? + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES * 2)); + } + + public void testExtractGroupingFieldsIfAggd() { + var plan = physicalPlan(""" + from test + | stats x = count(first_name) by first_name + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregate = as(limit.child(), AggregateExec.class); + assertThat(aggregate.groupings(), hasSize(1)); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + + var exchange = asRemoteExchange(aggregate.child()); + aggregate = as(exchange.child(), AggregateExec.class); + assertThat(aggregate.groupings(), hasSize(1)); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), equalTo(List.of("first_name"))); + + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES + KEYWORD_EST)); + } + + public void testExtractGroupingFieldsIfAggdWithEval() { + var plan = physicalPlan(""" + from test + | eval g = first_name + | stats x = count(first_name) by first_name + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var aggregate = as(limit.child(), AggregateExec.class); + assertThat(aggregate.groupings(), hasSize(1)); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + + var exchange = asRemoteExchange(aggregate.child()); + aggregate = as(exchange.child(), AggregateExec.class); + assertThat(aggregate.groupings(), hasSize(1)); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES + KEYWORD_EST)); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), equalTo(List.of("first_name"))); + + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES + KEYWORD_EST)); + } + + public void testQueryWithAggregation() { + var plan = physicalPlan(""" + from test + | stats sum(emp_no) + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var node = as(limit.child(), AggregateExec.class); + var exchange = asRemoteExchange(node.child()); + var aggregate = as(exchange.child(), AggregateExec.class); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES)); + + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES)); + } + + public void testQueryWithAggAfterEval() { + var plan = physicalPlan(""" + from test + | stats agg_emp = sum(emp_no) + | eval x = agg_emp + 7 + """); + + var optimized = optimizedPlan(plan); + var eval = as(optimized, EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var agg = as(topLimit.child(), AggregateExec.class); + // sum and x are longs + assertThat(agg.estimatedRowSize(), equalTo(Long.BYTES * 2)); + var exchange = asRemoteExchange(agg.child()); + var aggregate = as(exchange.child(), AggregateExec.class); + // sum is long a long, x isn't calculated until the agg above + assertThat(aggregate.estimatedRowSize(), equalTo(Long.BYTES)); + var extract = as(aggregate.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("emp_no")); + } + + public void testQueryWithNull() { + var plan = physicalPlan(""" + from test + | eval nullsum = emp_no + null + | sort emp_no + | limit 1 + """); + + var optimized = optimizedPlan(plan); + var topN = as(optimized, TopNExec.class); + // no fields are added after the top n - so 0 here + assertThat(topN.estimatedRowSize(), equalTo(0)); + + var exchange = asRemoteExchange(topN.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var topNLocal = as(extract.child(), TopNExec.class); + // All fields except emp_no are loaded after this topn. We load an extra int for the doc and segment mapping. + assertThat(topNLocal.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var extractForEval = as(topNLocal.child(), FieldExtractExec.class); + var eval = as(extractForEval.child(), EvalExec.class); + var source = source(eval.child()); + // emp_no and nullsum are longs, doc id is an int + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES * 2 + Integer.BYTES)); + } + + public void testPushAndInequalitiesFilter() { + var plan = physicalPlan(""" + from test + | where emp_no + 1 > 0 + | where salary < 10 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var fieldExtract = as(project.child(), FieldExtractExec.class); + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var bq = as(source.query(), BoolQueryBuilder.class); + assertThat(bq.must(), hasSize(2)); + var first = as(sv(bq.must().get(0), "emp_no"), RangeQueryBuilder.class); + assertThat(first.fieldName(), equalTo("emp_no")); + assertThat(first.from(), equalTo(-1)); + assertThat(first.includeLower(), equalTo(false)); + assertThat(first.to(), nullValue()); + var second = as(sv(bq.must().get(1), "salary"), RangeQueryBuilder.class); + assertThat(second.fieldName(), equalTo("salary")); + assertThat(second.from(), nullValue()); + assertThat(second.to(), equalTo(10)); + assertThat(second.includeUpper(), equalTo(false)); + } + + public void testOnlyPushTranslatableConditionsInFilter() { + var plan = physicalPlan(""" + from test + | where round(emp_no) + 1 > 0 + | where salary < 10 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var limit = as(extractRest.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var extract = as(filter.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var gt = as(filter.condition(), GreaterThan.class); + as(gt.left(), Round.class); + + var rq = as(sv(source.query(), "salary"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("salary")); + assertThat(rq.to(), equalTo(10)); + assertThat(rq.includeLower(), equalTo(false)); + assertThat(rq.from(), nullValue()); + } + + public void testNoPushDownNonFoldableInComparisonFilter() { + var plan = physicalPlan(""" + from test + | where emp_no > salary + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var limit = as(extractRest.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var extract = as(filter.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + assertThat(names(filter.condition().collect(FieldAttribute.class::isInstance)), contains("emp_no", "salary")); + assertThat(names(extract.attributesToExtract()), contains("emp_no", "salary")); + assertNull(source.query()); + } + + public void testNoPushDownNonFieldAttributeInComparisonFilter() { + var plan = physicalPlan(""" + from test + | where round(emp_no) > 0 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var limit = as(extractRest.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var extract = as(filter.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var gt = as(filter.condition(), GreaterThan.class); + as(gt.left(), Round.class); + assertNull(source.query()); + } + + public void testPushBinaryLogicFilters() { + var plan = physicalPlan(""" + from test + | where emp_no + 1 > 0 or salary < 10 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var fieldExtract = as(project.child(), FieldExtractExec.class); + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + BoolQueryBuilder bq = as(source.query(), BoolQueryBuilder.class); + assertThat(bq.should(), hasSize(2)); + var rq = as(sv(bq.should().get(0), "emp_no"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("emp_no")); + assertThat(rq.from(), equalTo(-1)); + assertThat(rq.includeLower(), equalTo(false)); + assertThat(rq.to(), nullValue()); + rq = as(sv(bq.should().get(1), "salary"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("salary")); + assertThat(rq.from(), nullValue()); + assertThat(rq.to(), equalTo(10)); + assertThat(rq.includeUpper(), equalTo(false)); + } + + public void testPushMultipleBinaryLogicFilters() { + var plan = physicalPlan(""" + from test + | where emp_no + 1 > 0 or salary < 10 + | where salary <= 10000 or salary >= 50000 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var fieldExtract = as(project.child(), FieldExtractExec.class); + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var top = as(source.query(), BoolQueryBuilder.class); + assertThat(top.must(), hasSize(2)); + + var first = as(top.must().get(0), BoolQueryBuilder.class); + var rq = as(sv(first.should().get(0), "emp_no"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("emp_no")); + assertThat(rq.from(), equalTo(-1)); + assertThat(rq.includeLower(), equalTo(false)); + assertThat(rq.to(), nullValue()); + rq = as(sv(first.should().get(1), "salary"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("salary")); + assertThat(rq.from(), nullValue()); + assertThat(rq.to(), equalTo(10)); + assertThat(rq.includeUpper(), equalTo(false)); + + var second = as(top.must().get(1), BoolQueryBuilder.class); + rq = as(sv(second.should().get(0), "salary"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("salary")); + assertThat(rq.from(), nullValue()); + assertThat(rq.to(), equalTo(10000)); + assertThat(rq.includeUpper(), equalTo(true)); + rq = as(sv(second.should().get(1), "salary"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("salary")); + assertThat(rq.from(), equalTo(50000)); + assertThat(rq.includeLower(), equalTo(true)); + assertThat(rq.to(), nullValue()); + } + + public void testLimit() { + var optimized = optimizedPlan(physicalPlan(""" + from test + | limit 10 + """)); + + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var fieldExtract = as(project.child(), FieldExtractExec.class); + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + assertThat(source.limit().fold(), is(10)); + } + + /** + * TopNExec[[Order[nullsum{r}#3,ASC,LAST]],1[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !gender, languages{f}#8, last_name{f}#9, salary{f}#10, nulls + * um{r}#3]] + * \_FieldExtractExec[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + * \_TopNExec[[Order[nullsum{r}#3,ASC,LAST]],1[INTEGER]] + * \_EvalExec[[null[INTEGER] AS nullsum]] + * \_EsQueryExec[test], query[][_doc{f}#12], limit[], sort[] + */ + public void testExtractorForEvalWithoutProject() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | eval nullsum = emp_no + null + | sort nullsum + | limit 1 + """)); + var topN = as(optimized, TopNExec.class); + var exchange = asRemoteExchange(topN.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var topNLocal = as(extract.child(), TopNExec.class); + // two extra ints for forwards and backwards map + assertThat(topNLocal.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2)); + + var eval = as(topNLocal.child(), EvalExec.class); + var source = source(eval.child()); + // nullsum and doc id are ints. we don't actually load emp_no here because we know we don't need it. + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES * 2)); + } + + public void testProjectAfterTopN() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | sort emp_no + | keep first_name + | limit 2 + """)); + var topProject = as(optimized, ProjectExec.class); + assertEquals(1, topProject.projections().size()); + assertEquals("first_name", topProject.projections().get(0).name()); + var topN = as(topProject.child(), TopNExec.class); + var exchange = asRemoteExchange(topN.child()); + var project = as(exchange.child(), ProjectExec.class); + List projectionNames = project.projections().stream().map(NamedExpression::name).collect(Collectors.toList()); + assertTrue(projectionNames.containsAll(List.of("first_name", "emp_no"))); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.limit(), is(topN.limit())); + assertThat(source.sorts(), is(sorts(topN.order()))); + // an int for doc id, an int for segment id, two ints for doc id map, and int for emp_no. + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES * 5 + KEYWORD_EST)); + } + + /** + * Expected + * + * EvalExec[[emp_no{f}#248 * 10[INTEGER] AS emp_no_10]] + * \_LimitExec[10[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#247, emp_no{f}#248, first_name{f}#249, languages{f}#250, last_name{f}#251, salary{f}#252]] + * \_FieldExtractExec[_meta_field{f}#247, emp_no{f}#248, first_name{f}#24..] + * \_EsQueryExec[test], query[][_doc{f}#253], limit[10], sort[] + */ + public void testPushLimitToSource() { + var optimized = optimizedPlan(physicalPlan(""" + from test + | eval emp_no_10 = emp_no * 10 + | limit 10 + """)); + + var eval = as(optimized, EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var leaves = extract.collectLeaves(); + assertEquals(1, leaves.size()); + var source = as(leaves.get(0), EsQueryExec.class); + assertThat(source.limit().fold(), is(10)); + // extra ints for doc id and emp_no_10 + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2)); + } + + /** + * Expected + * EvalExec[[emp_no{f}#5 * 10[INTEGER] AS emp_no_10]] + * \_LimitExec[10[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !gender, languages{f}#8, last_name{f}#9, salary{f}#10]] + * \_FieldExtractExec[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + * \_EsQueryExec[test], query[{"range":{"emp_no":{"gt":0,"boost":1.0}}}][_doc{f}#12], limit[10], sort[] + */ + public void testPushLimitAndFilterToSource() { + var optimized = optimizedPlan(physicalPlan(""" + from test + | eval emp_no_10 = emp_no * 10 + | where emp_no > 0 + | limit 10 + """)); + + var eval = as(optimized, EvalExec.class); + var topLimit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + + assertThat( + names(extract.attributesToExtract()), + contains("_meta_field", "emp_no", "first_name", "gender", "languages", "last_name", "salary") + ); + + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2)); + assertThat(source.limit().fold(), is(10)); + var rq = as(sv(source.query(), "emp_no"), RangeQueryBuilder.class); + assertThat(rq.fieldName(), equalTo("emp_no")); + assertThat(rq.from(), equalTo(0)); + assertThat(rq.includeLower(), equalTo(false)); + assertThat(rq.to(), nullValue()); + } + + /** + * Expected + * TopNExec[[Order[emp_no{f}#2,ASC,LAST]],1[INTEGER]] + * \_LimitExec[1[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, !gender, languages{f}#5, last_name{f}#6, salary{f}#7]] + * \_FieldExtractExec[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, !ge..] + * \_EsQueryExec[test], query[][_doc{f}#9], limit[1], sort[] + */ + public void testQueryWithLimitSort() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | limit 1 + | sort emp_no + """)); + + var topN = as(optimized, TopNExec.class); + var limit = as(topN.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + } + + /** + * Expected + * + * ProjectExec[[emp_no{f}#7, x{r}#4]] + * \_TopNExec[[Order[emp_no{f}#7,ASC,LAST]],5[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[emp_no{f}#7, x{r}#4]] + * \_TopNExec[[Order[emp_no{f}#7,ASC,LAST]],5[INTEGER]] + * \_FieldExtractExec[emp_no{f}#7] + * \_EvalExec[[first_name{f}#8 AS x]] + * \_FieldExtractExec[first_name{f}#8] + * \_EsQueryExec[test], query[][_doc{f}#14], limit[] + */ + public void testLocalProjectIncludeLocalAlias() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | sort emp_no + | eval x = first_name + | keep emp_no, x + | limit 5 + """)); + + var project = as(optimized, ProjectExec.class); + var topN = as(project.child(), TopNExec.class); + var exchange = asRemoteExchange(topN.child()); + + project = as(exchange.child(), ProjectExec.class); + assertThat(names(project.projections()), contains("emp_no", "x")); + topN = as(project.child(), TopNExec.class); + var extract = as(topN.child(), FieldExtractExec.class); + var eval = as(extract.child(), EvalExec.class); + extract = as(eval.child(), FieldExtractExec.class); + } + + /** + * Expected + * ProjectExec[[languages{f}#10, salary{f}#12, x{r}#6]] + * \_EvalExec[[languages{f}#10 + 1[INTEGER] AS x]] + * \_TopNExec[[Order[salary{f}#12,ASC,LAST]],1[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[languages{f}#10, salary{f}#12]] + * \_FieldExtractExec[languages{f}#10] + * \_EsQueryExec[test], query[][_doc{f}#14], limit[1], sort[[salary]] + */ + public void testDoNotAliasesDefinedAfterTheExchange() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | sort salary + | limit 1 + | keep languages, salary + | eval x = languages + 1 + """)); + + var project = as(optimized, ProjectExec.class); + var eval = as(project.child(), EvalExec.class); + var topN = as(eval.child(), TopNExec.class); + var exchange = asRemoteExchange(topN.child()); + + project = as(exchange.child(), ProjectExec.class); + assertThat(names(project.projections()), contains("languages", "salary")); + var extract = as(project.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("languages", "salary")); + var source = source(extract.child()); + assertThat(source.limit(), is(topN.limit())); + assertThat(source.sorts(), is(sorts(topN.order()))); + + assertThat(source.limit(), is(l(1))); + assertThat(source.sorts().size(), is(1)); + FieldSort order = source.sorts().get(0); + assertThat(order.direction(), is(ASC)); + assertThat(name(order.field()), is("salary")); + // ints for doc id, segment id, forwards and backwards mapping, languages, and salary + assertThat(source.estimatedRowSize(), equalTo(Integer.BYTES * 6)); + } + + /** + * Expected + * TopNExec[[Order[emp_no{f}#3,ASC,LAST]],1[INTEGER]] + * \_FilterExec[emp_no{f}#3 > 10[INTEGER]] + * \_LimitExec[1[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !gender, languages{f}#6, last_name{f}#7, salary{f}#8]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !ge..] + * \_EsQueryExec[test], query[][_doc{f}#10], limit[1], sort[] + */ + public void testQueryWithLimitWhereSort() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | limit 1 + | where emp_no > 10 + | sort emp_no + """)); + + var topN = as(optimized, TopNExec.class); + var filter = as(topN.child(), FilterExec.class); + var limit = as(filter.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.limit(), is(topN.limit())); + assertThat(source.limit(), is(l(1))); + assertNull(source.sorts()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + } + + /** + * Expected + * TopNExec[[Order[x{r}#3,ASC,LAST]],3[INTEGER]] + * \_EvalExec[[emp_no{f}#5 AS x]] + * \_LimitExec[3[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !gender, languages{f}#8, last_name{f}#9, salary{f}#10]] + * \_FieldExtractExec[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, !g..] + * \_EsQueryExec[test], query[][_doc{f}#12], limit[3], sort[] + */ + public void testQueryWithLimitWhereEvalSort() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | limit 3 + | eval x = emp_no + | sort x + """)); + + var topN = as(optimized, TopNExec.class); + var eval = as(topN.child(), EvalExec.class); + var limit = as(eval.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + // an int for doc id and one for x + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2)); + } + + public void testQueryJustWithLimit() throws Exception { + var optimized = optimizedPlan(physicalPlan(""" + from test + | limit 3 + """)); + + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + } + + public void testPushDownDisjunction() { + var plan = physicalPlan(""" + from test + | where emp_no == 10010 or emp_no == 10011 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var tqb = as(sv(source.query(), "emp_no"), TermsQueryBuilder.class); + assertThat(tqb.fieldName(), is("emp_no")); + assertThat(tqb.values(), is(List.of(10010, 10011))); + } + + public void testPushDownDisjunctionAndConjunction() { + var plan = physicalPlan(""" + from test + | where first_name == "Bezalel" or first_name == "Suzette" + | where salary > 50000 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + BoolQueryBuilder query = as(source.query(), BoolQueryBuilder.class); + assertThat(query.must(), hasSize(2)); + var tq = as(sv(query.must().get(0), "first_name"), TermsQueryBuilder.class); + assertThat(tq.fieldName(), is("first_name")); + assertThat(tq.values(), is(List.of("Bezalel", "Suzette"))); + var rqb = as(sv(query.must().get(1), "salary"), RangeQueryBuilder.class); + assertThat(rqb.fieldName(), is("salary")); + assertThat(rqb.from(), is(50_000)); + assertThat(rqb.includeLower(), is(false)); + assertThat(rqb.to(), nullValue()); + } + + public void testPushDownIn() { + var plan = physicalPlan(""" + from test + | where emp_no in (10020, 10030 + 10) + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var tqb = as(sv(source.query(), "emp_no"), TermsQueryBuilder.class); + assertThat(tqb.fieldName(), is("emp_no")); + assertThat(tqb.values(), is(List.of(10020, 10040))); + } + + public void testPushDownInAndConjunction() { + var plan = physicalPlan(""" + from test + | where last_name in (concat("Sim", "mel"), "Pettey") + | where salary > 60000 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + BoolQueryBuilder bq = as(source.query(), BoolQueryBuilder.class); + assertThat(bq.must(), hasSize(2)); + var tqb = as(sv(bq.must().get(0), "last_name"), TermsQueryBuilder.class); + assertThat(tqb.fieldName(), is("last_name")); + assertThat(tqb.values(), is(List.of("Simmel", "Pettey"))); + var rqb = as(sv(bq.must().get(1), "salary"), RangeQueryBuilder.class); + assertThat(rqb.fieldName(), is("salary")); + assertThat(rqb.from(), is(60_000)); + } + + // `where "Pettey" in (last_name, "Simmel") or last_name == "Parto"` --> `where last_name in ("Pettey", "Parto")` + // LimitExec[10000[INTEGER]] + // \_ExchangeExec[] + // \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gender{f}#5, languages{f}#6, last_name{f}#7, salary{f}#8]] + // \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gen..] + // \_EsQueryExec[test], + // query[{"esql_single_value":{"field":"last_name","next":{"terms":{"last_name":["Pettey","Parto"],"boost":1.0}}}}][_doc{f}#10], + // limit[10000], sort[] + public void testPushDownRecombinedIn() { + var plan = physicalPlan(""" + from test + | where "Pettey" in (last_name, "Simmel") or last_name == "Parto" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + + var tqb = as(sv(source.query(), "last_name"), TermsQueryBuilder.class); + assertThat(tqb.fieldName(), is("last_name")); + assertThat(tqb.values(), is(List.of("Pettey", "Parto"))); + } + + /** + * Expected: + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[REMOTE_SOURCE] + * \_ExchangeExec[REMOTE_SINK] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !gender, languages{f}#6, last_name{f}#7, salary{f}#8]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !ge..] + * \_EsQueryExec[test], query[sv(not(emp_no IN (10010, 10011)))][_doc{f}#10], + * limit[10000], sort[] + */ + public void testPushDownNegatedDisjunction() { + var plan = physicalPlan(""" + from test + | where not (emp_no == 10010 or emp_no == 10011) + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var boolQuery = as(sv(source.query(), "emp_no"), BoolQueryBuilder.class); + assertThat(boolQuery.mustNot(), hasSize(1)); + var termsQuery = as(boolQuery.mustNot().get(0), TermsQueryBuilder.class); + assertThat(termsQuery.fieldName(), is("emp_no")); + assertThat(termsQuery.values(), is(List.of(10010, 10011))); + } + + /** + * Expected: + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[REMOTE_SOURCE] + * \_ExchangeExec[REMOTE_SINK] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !gender, languages{f}#6, last_name{f}#7, salary{f}#8]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !ge..] + * \_EsQueryExec[test], query[sv(emp_no, not(emp_no == 10010)) OR sv(not(first_name == "Parto"))], limit[10000], sort[] + */ + public void testPushDownNegatedConjunction() { + var plan = physicalPlan(""" + from test + | where not (emp_no == 10010 and first_name == "Parto") + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var bq = as(source.query(), BoolQueryBuilder.class); + assertThat(bq.should(), hasSize(2)); + var empNo = as(sv(bq.should().get(0), "emp_no"), BoolQueryBuilder.class); + assertThat(empNo.mustNot(), hasSize(1)); + var tq = as(empNo.mustNot().get(0), TermQueryBuilder.class); + assertThat(tq.fieldName(), equalTo("emp_no")); + assertThat(tq.value(), equalTo(10010)); + var firstName = as(sv(bq.should().get(1), "first_name"), BoolQueryBuilder.class); + assertThat(firstName.mustNot(), hasSize(1)); + tq = as(firstName.mustNot().get(0), TermQueryBuilder.class); + assertThat(tq.fieldName(), equalTo("first_name")); + assertThat(tq.value(), equalTo("Parto")); + } + + /** + * Expected: + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[REMOTE_SOURCE] + * \_ExchangeExec[REMOTE_SINK] + * \_ProjectExec[[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, !gender, languages{f}#5, last_name{f}#6, salary{f}#7]] + * \_FieldExtractExec[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, !ge..] + * \_EsQueryExec[test], query[{"bool":{"must_not":[{"term":{"emp_no":{"value":10010}}}],"boost":1.0}}][_doc{f}#9], + * limit[10000], sort[] + */ + public void testPushDownNegatedEquality() { + var plan = physicalPlan(""" + from test + | where not emp_no == 10010 + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var boolQuery = as(sv(source.query(), "emp_no"), BoolQueryBuilder.class); + assertThat(boolQuery.mustNot(), hasSize(1)); + var termQuery = as(boolQuery.mustNot().get(0), TermQueryBuilder.class); + assertThat(termQuery.fieldName(), is("emp_no")); + assertThat(termQuery.value(), is(10010)); // TODO this will match multivalued fields and we don't want that + } + + /** + * Expected: + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[REMOTE_SOURCE] + * \_ExchangeExec[REMOTE_SINK] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, !gender, languages{f}#6, last_name{f}#7, salary{f}#8]] + * \_FieldExtractExec[_meta_field{f}#9, first_name{f}#4, !gender, last_na..] + * \_LimitExec[10000[INTEGER]] + * \_FilterExec[NOT(emp_no{f}#3 == languages{f}#6)] + * \_FieldExtractExec[emp_no{f}#3, languages{f}#6] + * \_EsQueryExec[test], query[][_doc{f}#10], limit[], sort[] + */ + public void testDontPushDownNegatedEqualityBetweenAttributes() { + var plan = physicalPlan(""" + from test + | where not emp_no == languages + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var localLimit = as(extractRest.child(), LimitExec.class); + var filterExec = as(localLimit.child(), FilterExec.class); + assertThat(filterExec.condition(), instanceOf(Not.class)); + var extractForFilter = as(filterExec.child(), FieldExtractExec.class); + var source = source(extractForFilter.child()); + assertNull(source.query()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + } + + public void testEvalLike() { + var plan = physicalPlan(""" + from test + | eval x = concat(first_name, "--") + | where x like "%foo%" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var limit = as(extractRest.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var eval = as(filter.child(), EvalExec.class); + var fieldExtract = as(eval.child(), FieldExtractExec.class); + assertEquals(EsQueryExec.class, fieldExtract.child().getClass()); + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES + KEYWORD_EST)); + } + + public void testPushDownLike() { + var plan = physicalPlan(""" + from test + | where first_name like "*foo*" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + QueryBuilder query = source.query(); + assertNotNull(query); + assertEquals(WildcardQueryBuilder.class, query.getClass()); + WildcardQueryBuilder wildcard = ((WildcardQueryBuilder) query); + assertEquals("first_name", wildcard.fieldName()); + assertEquals("*foo*", wildcard.value()); + } + + public void testPushDownNotLike() { + var plan = physicalPlan(""" + from test + | where not first_name like "%foo%" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + var boolQuery = as(sv(source.query(), "first_name"), BoolQueryBuilder.class); + assertThat(boolQuery.mustNot(), hasSize(1)); + var tq = as(boolQuery.mustNot().get(0), TermQueryBuilder.class); + assertThat(tq.fieldName(), is("first_name")); + assertThat(tq.value(), is("%foo%")); + } + + public void testEvalRLike() { + var plan = physicalPlan(""" + from test + | eval x = concat(first_name, "--") + | where x rlike ".*foo.*" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var limit = as(extractRest.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + var eval = as(filter.child(), EvalExec.class); + var fieldExtract = as(eval.child(), FieldExtractExec.class); + assertEquals(EsQueryExec.class, fieldExtract.child().getClass()); + + var source = source(fieldExtract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES + KEYWORD_EST)); + } + + public void testPushDownRLike() { + var plan = physicalPlan(""" + from test + | where first_name rlike ".*foo.*" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + QueryBuilder query = source.query(); + assertNotNull(query); + assertEquals(RegexpQueryBuilder.class, query.getClass()); + RegexpQueryBuilder wildcard = ((RegexpQueryBuilder) query); + assertEquals("first_name", wildcard.fieldName()); + assertEquals(".*foo.*", wildcard.value()); + } + + public void testPushDownNotRLike() { + var plan = physicalPlan(""" + from test + | where not first_name rlike ".*foo.*" + """); + + var optimized = optimizedPlan(plan); + var topLimit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(topLimit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extractRest = as(project.child(), FieldExtractExec.class); + var source = source(extractRest.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES)); + + QueryBuilder query = source.query(); + assertNotNull(query); + assertThat(query, instanceOf(BoolQueryBuilder.class)); + var boolQuery = (BoolQueryBuilder) query; + List mustNot = boolQuery.mustNot(); + assertThat(mustNot.size(), is(1)); + assertThat(mustNot.get(0), instanceOf(RegexpQueryBuilder.class)); + var regexpQuery = (RegexpQueryBuilder) mustNot.get(0); + assertThat(regexpQuery.fieldName(), is("first_name")); + assertThat(regexpQuery.value(), is(".*foo.*")); + } + + /** + * EnrichExec[first_name{f}#3,foo,fld,idx,[a{r}#11, b{r}#12]] + * \_LimitExec[10000[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, gender{f}#4, languages{f}#5, last_name{f}#6, salary{f}#7]] + * \_FieldExtractExec[_meta_field{f}#8, emp_no{f}#2, first_name{f}#3, gen..] + * \_EsQueryExec[test], query[][_doc{f}#13], limit[10000], sort[] estimatedRowSize[216] + */ + public void testEnrich() { + var plan = physicalPlan(""" + from test + | enrich foo on first_name + """); + + var optimized = optimizedPlan(plan); + var enrich = as(optimized, EnrichExec.class); + var limit = as(enrich.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + // an int for doc id, and int for the "a" enriched field, and a long for the "b" enriched field + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES * 2 + Long.BYTES)); + } + + /** + * Expects the filter to transform the source into a local relationship + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[[],false] + * \_LocalSourceExec[[_meta_field{f}#8, emp_no{r}#2, first_name{f}#3, gender{f}#4, languages{f}#5, last_name{f}#6, salary{f}#7],EMPT + * Y] + */ + public void testLocallyMissingField() { + var testStats = statsForMissingField("emp_no"); + + var optimized = optimizedPlan(physicalPlan(""" + from test + | where emp_no > 10 + """), testStats); + + var limit = as(optimized, LimitExec.class); + var exchange = as(limit.child(), ExchangeExec.class); + var source = as(exchange.child(), LocalSourceExec.class); + assertEquals(LocalSupplier.EMPTY, source.supplier()); + } + + /** + * GrokExec[first_name{f}#4,Parser[pattern=%{WORD:b}.*, grok=org.elasticsearch.grok.Grok@60a20ab6],[b{r}#2]] + * \_LimitExec[10000[INTEGER]] + * \_ExchangeExec[] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gender{f}#5, languages{f}#6, last_name{f}#7, salary{f}#8]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gen..] + * \_EsQueryExec[test], query[][_doc{f}#10], limit[10000], sort[] estimatedRowSize[216] + */ + public void testGrok() { + var plan = physicalPlan(""" + from test + | grok first_name "%{WORD:b}.*" + """); + + var optimized = optimizedPlan(plan); + var grok = as(optimized, GrokExec.class); + var limit = as(grok.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES + KEYWORD_EST)); + } + + public void testDissect() { + var plan = physicalPlan(""" + from test + | dissect first_name "%{b} " + """); + + var optimized = optimizedPlan(plan); + var dissect = as(optimized, DissectExec.class); + var limit = as(dissect.child(), LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES + KEYWORD_EST)); + } + + public void testPushDownMetadataIndexInWildcard() { + var plan = physicalPlan(""" + from test [metadata _index] + | where _index like "test*" + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + + var tq = as(source.query(), WildcardQueryBuilder.class); + assertThat(tq.fieldName(), is("_index")); + assertThat(tq.value(), is("test*")); + } + + /* + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[[],false] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gender{f}#5, languages{f}#6, last_name{f}#7, salary{f}#8, + * _index{m}#1]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gen..] + * \_EsQueryExec[test], query[{"esql_single_value":{"field":"_index","next":{"term":{"_index":{"value":"test"}}}}}] + * [_doc{f}#10], limit[10000], sort[] estimatedRowSize[266] + */ + public void testPushDownMetadataIndexInEquality() { + var plan = physicalPlan(""" + from test [metadata _index] + | where _index == "test" + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + + var tq = as(source.query(), TermQueryBuilder.class); + assertThat(tq.fieldName(), is("_index")); + assertThat(tq.value(), is("test")); + } + + /* + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[[],false] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gender{f}#5, languages{f}#6, last_name{f}#7, salary{f}#8, + * _index{m}#1]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gen..] + * \_EsQueryExec[test], query[{"bool":{"must_not":[{"term":{"_index":{"value":"test"}}}],"boost":1.0}}] + * [_doc{f}#10], limit[10000], sort[] estimatedRowSize[266] + */ + public void testPushDownMetadataIndexInNotEquality() { + var plan = physicalPlan(""" + from test [metadata _index] + | where _index != "test" + """); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + var source = source(extract.child()); + + var bq = as(source.query(), BoolQueryBuilder.class); + assertThat(bq.mustNot().size(), is(1)); + var tq = as(bq.mustNot().get(0), TermQueryBuilder.class); + assertThat(tq.fieldName(), is("_index")); + assertThat(tq.value(), is("test")); + } + + /* + * LimitExec[10000[INTEGER]] + * \_ExchangeExec[[],false] + * \_ProjectExec[[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gender{f}#5, languages{f}#6, last_name{f}#7, salary{f}#8, _in + * dex{m}#1]] + * \_FieldExtractExec[_meta_field{f}#9, emp_no{f}#3, first_name{f}#4, gen..] + * \_LimitExec[10000[INTEGER]] + * \_FilterExec[_index{m}#1 > [74 65 73 74][KEYWORD]] + * \_FieldExtractExec[_index{m}#1] + * \_EsQueryExec[test], query[][_doc{f}#10], limit[], sort[] estimatedRowSize[266] + */ + public void testDontPushDownMetadataIndexInInequality() { + for (var t : List.of( + tuple(">", GreaterThan.class), + tuple(">=", GreaterThanOrEqual.class), + tuple("<", LessThan.class), + tuple("<=", LessThanOrEqual.class) + // no NullEquals use + )) { + var plan = physicalPlan("from test [metadata _index] | where _index " + t.v1() + " \"test\""); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + limit = as(extract.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + + var comp = as(filter.condition(), t.v2()); + var metadataAttribute = as(comp.left(), MetadataAttribute.class); + assertThat(metadataAttribute.name(), is("_index")); + + extract = as(filter.child(), FieldExtractExec.class); + var source = source(extract.child()); + } + } + + public void testDontPushDownMetadataVersionAndId() { + for (var t : List.of(tuple("_version", "2"), tuple("_id", "\"2\""))) { + var plan = physicalPlan("from test [metadata " + t.v1() + "] | where " + t.v1() + " == " + t.v2()); + + var optimized = optimizedPlan(plan); + var limit = as(optimized, LimitExec.class); + var exchange = asRemoteExchange(limit.child()); + var project = as(exchange.child(), ProjectExec.class); + var extract = as(project.child(), FieldExtractExec.class); + limit = as(extract.child(), LimitExec.class); + var filter = as(limit.child(), FilterExec.class); + + assertThat(filter.condition(), instanceOf(Equals.class)); + assertThat(((Equals) filter.condition()).left(), instanceOf(MetadataAttribute.class)); + var metadataAttribute = (MetadataAttribute) ((Equals) filter.condition()).left(); + assertThat(metadataAttribute.name(), is(t.v1())); + + extract = as(filter.child(), FieldExtractExec.class); + var source = source(extract.child()); + } + } + + private static EsQueryExec source(PhysicalPlan plan) { + if (plan instanceof ExchangeExec exchange) { + plan = exchange.child(); + } + return as(plan, EsQueryExec.class); + } + + private PhysicalPlan optimizedPlan(PhysicalPlan plan) { + return optimizedPlan(plan, EsqlTestUtils.TEST_SEARCH_STATS); + } + + private PhysicalPlan optimizedPlan(PhysicalPlan plan, SearchStats searchStats) { + // System.out.println("* Physical Before\n" + plan); + var p = EstimatesRowSize.estimateRowSize(0, physicalPlanOptimizer.optimize(plan)); + // System.out.println("* Physical After\n" + p); + // the real execution breaks the plan at the exchange and then decouples the plan + // this is of no use in the unit tests, which checks the plan as a whole instead of each + // individually hence why here the plan is kept as is + + var l = p.transformUp(FragmentExec.class, fragment -> { + var localPlan = PlannerUtils.localPlan(config, fragment, searchStats); + return EstimatesRowSize.estimateRowSize(fragment.estimatedRowSize(), localPlan); + }); + + // System.out.println("* Localized DataNode Plan\n" + l); + return l; + } + + private PhysicalPlan physicalPlan(String query) { + var logical = logicalOptimizer.optimize(analyzer.analyze(parser.createStatement(query))); + // System.out.println("Logical\n" + logical); + var physical = mapper.map(logical); + assertSerialization(physical); + return physical; + } + + private List sorts(List orders) { + return orders.stream().map(o -> new FieldSort((FieldAttribute) o.child(), o.direction(), o.nullsPosition())).toList(); + } + + private ExchangeExec asRemoteExchange(PhysicalPlan plan) { + return as(plan, ExchangeExec.class); + } + + public void testFieldExtractWithoutSourceAttributes() { + PhysicalPlan verifiedPlan = optimizedPlan(physicalPlan(""" + from test + | where round(emp_no) > 10 + """)); + // Transform the verified plan so that it is invalid (i.e. no source attributes) + List emptyAttrList = List.of(); + var badPlan = verifiedPlan.transformDown( + EsQueryExec.class, + node -> new EsSourceExec(node.source(), node.index(), emptyAttrList, node.query()) + ); + + var e = expectThrows(PhysicalVerificationException.class, () -> physicalPlanOptimizer.verify(badPlan)); + assertThat( + e.getMessage(), + containsString( + "Need to add field extractor for [[emp_no]] but cannot detect source attributes from node [EsSourceExec[test][]]" + ) + ); + } + + /** + * Asserts that a {@link QueryBuilder} is a {@link SingleValueQuery} that + * acting on the provided field name and returns the {@link QueryBuilder} + * that it wraps. + */ + private QueryBuilder sv(QueryBuilder builder, String fieldName) { + SingleValueQuery.Builder sv = as(builder, SingleValueQuery.Builder.class); + assertThat(sv.field(), equalTo(fieldName)); + return sv.next(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java new file mode 100644 index 0000000000000..a4e94254b94ae --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestLocalPhysicalPlanOptimizer.java @@ -0,0 +1,24 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; + +import java.util.List; + +public class TestLocalPhysicalPlanOptimizer extends LocalPhysicalPlanOptimizer { + + public TestLocalPhysicalPlanOptimizer(LocalPhysicalOptimizerContext context) { + super(context); + } + + @Override + protected List> batches() { + return rules(false); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java new file mode 100644 index 0000000000000..1e994a0d5721b --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/TestPhysicalPlanOptimizer.java @@ -0,0 +1,25 @@ +/* + * 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; + +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.ql.rule.RuleExecutor; + +public class TestPhysicalPlanOptimizer extends PhysicalPlanOptimizer { + + private static final Iterable> rules = initializeRules(false); + + public TestPhysicalPlanOptimizer(PhysicalOptimizerContext context) { + super(context); + } + + @Override + protected Iterable> batches() { + return rules; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/ExpressionTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/ExpressionTests.java new file mode 100644 index 0000000000000..3469cc66f21aa --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/ExpressionTests.java @@ -0,0 +1,653 @@ +/* + * 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.parser; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.plan.logical.Drop; +import org.elasticsearch.xpack.esql.plan.logical.Rename; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedStar; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.expression.predicate.logical.And; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.Project; +import org.elasticsearch.xpack.ql.type.DataType; + +import java.time.Duration; +import java.time.Period; +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.DATE_PERIOD; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypes.TIME_DURATION; +import static org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy.DEFAULT; +import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; +import static org.elasticsearch.xpack.ql.type.DataTypes.DOUBLE; +import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.type.DataTypes.LONG; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class ExpressionTests extends ESTestCase { + private final EsqlParser parser = new EsqlParser(); + + public void testBooleanLiterals() { + assertEquals(Literal.TRUE, whereExpression("true")); + assertEquals(Literal.FALSE, whereExpression("false")); + assertEquals(Literal.NULL, whereExpression("null")); + } + + public void testNumberLiterals() { + assertEquals(l(123, INTEGER), whereExpression("123")); + assertEquals(l(123, INTEGER), whereExpression("+123")); + assertEquals(l(-123, INTEGER), whereExpression("-123")); + assertEquals(l(123.123, DOUBLE), whereExpression("123.123")); + assertEquals(l(123.123, DOUBLE), whereExpression("+123.123")); + assertEquals(l(-123.123, DOUBLE), whereExpression("-123.123")); + assertEquals(l(0.123, DOUBLE), whereExpression(".123")); + assertEquals(l(0.123, DOUBLE), whereExpression("0.123")); + assertEquals(l(0.123, DOUBLE), whereExpression("+0.123")); + assertEquals(l(-0.123, DOUBLE), whereExpression("-0.123")); + assertEquals(l(12345678901L, LONG), whereExpression("12345678901")); + assertEquals(l(12345678901L, LONG), whereExpression("+12345678901")); + assertEquals(l(-12345678901L, LONG), whereExpression("-12345678901")); + assertEquals(l(123e12, DOUBLE), whereExpression("123e12")); + assertEquals(l(123e-12, DOUBLE), whereExpression("123e-12")); + assertEquals(l(123E12, DOUBLE), whereExpression("123E12")); + assertEquals(l(123E-12, DOUBLE), whereExpression("123E-12")); + } + + public void testMinusSign() { + assertEquals(l(-123, INTEGER), whereExpression("+(-123)")); + assertEquals(l(-123, INTEGER), whereExpression("+(+(-123))")); + // we could do better here. ES SQL is smarter and accounts for the number of minuses + assertEquals(new Neg(null, l(-123, INTEGER)), whereExpression("-(-123)")); + } + + public void testStringLiterals() { + assertEquals(l("abc", KEYWORD), whereExpression("\"abc\"")); + assertEquals(l("123.123", KEYWORD), whereExpression("\"123.123\"")); + + assertEquals(l("hello\"world", KEYWORD), whereExpression("\"hello\\\"world\"")); + assertEquals(l("hello'world", KEYWORD), whereExpression("\"hello'world\"")); + assertEquals(l("\"hello\"world\"", KEYWORD), whereExpression("\"\\\"hello\\\"world\\\"\"")); + assertEquals(l("\"hello\nworld\"", KEYWORD), whereExpression("\"\\\"hello\\nworld\\\"\"")); + assertEquals(l("hello\nworld", KEYWORD), whereExpression("\"hello\\nworld\"")); + assertEquals(l("hello\\world", KEYWORD), whereExpression("\"hello\\\\world\"")); + assertEquals(l("hello\rworld", KEYWORD), whereExpression("\"hello\\rworld\"")); + assertEquals(l("hello\tworld", KEYWORD), whereExpression("\"hello\\tworld\"")); + assertEquals(l("C:\\Program Files\\Elastic", KEYWORD), whereExpression("\"C:\\\\Program Files\\\\Elastic\"")); + + assertEquals(l("C:\\Program Files\\Elastic", KEYWORD), whereExpression("\"\"\"C:\\Program Files\\Elastic\"\"\"")); + assertEquals(l("\"\"hello world\"\"", KEYWORD), whereExpression("\"\"\"\"\"hello world\"\"\"\"\"")); + assertEquals(l("hello \"\"\" world", KEYWORD), whereExpression("\"hello \\\"\\\"\\\" world\"")); + assertEquals(l("hello\\nworld", KEYWORD), whereExpression("\"\"\"hello\\nworld\"\"\"")); + assertEquals(l("hello\\tworld", KEYWORD), whereExpression("\"\"\"hello\\tworld\"\"\"")); + assertEquals(l("hello world\\", KEYWORD), whereExpression("\"\"\"hello world\\\"\"\"")); + assertEquals(l("hello world\\", KEYWORD), whereExpression("\"\"\"hello world\\\"\"\"")); + assertEquals(l("\t \n \r \" \\ ", KEYWORD), whereExpression("\"\\t \\n \\r \\\" \\\\ \"")); + } + + public void testStringLiteralsExceptions() { + assertParsingException(() -> whereExpression("\"\"\"\"\"\"foo\"\""), "line 1:22: mismatched input 'foo' expecting {,"); + assertParsingException( + () -> whereExpression("\"foo\" == \"\"\"\"\"\"bar\"\"\""), + "line 1:31: mismatched input 'bar' expecting {," + ); + assertParsingException( + () -> whereExpression("\"\"\"\"\"\\\"foo\"\"\"\"\"\" != \"\"\"bar\"\"\""), + "line 1:31: mismatched input '\" != \"' expecting {," + ); + assertParsingException( + () -> whereExpression("\"\"\"\"\"\\\"foo\"\"\\\"\"\"\" == \"\"\"\"\"\\\"bar\\\"\\\"\"\"\"\"\""), + "line 1:55: token recognition error at: '\"'" + ); + assertParsingException( + () -> whereExpression("\"\"\"\"\"\" foo \"\"\"\" == abc"), + "line 1:23: mismatched input 'foo' expecting {," + ); + } + + public void testBooleanLiteralsCondition() { + Expression expression = whereExpression("true and false"); + assertThat(expression, instanceOf(And.class)); + And and = (And) expression; + assertThat(and.left(), equalTo(Literal.TRUE)); + assertThat(and.right(), equalTo(Literal.FALSE)); + } + + public void testArithmeticOperationCondition() { + Expression expression = whereExpression("-a-b*c == 123"); + assertThat(expression, instanceOf(Equals.class)); + Equals eq = (Equals) expression; + assertThat(eq.right(), instanceOf(Literal.class)); + assertThat(((Literal) eq.right()).value(), equalTo(123)); + assertThat(eq.left(), instanceOf(Sub.class)); + Sub sub = (Sub) eq.left(); + assertThat(sub.left(), instanceOf(Neg.class)); + Neg subLeftNeg = (Neg) sub.left(); + assertThat(subLeftNeg.field(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) subLeftNeg.field()).name(), equalTo("a")); + Mul mul = (Mul) sub.right(); + assertThat(mul.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(mul.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) mul.left()).name(), equalTo("b")); + assertThat(((UnresolvedAttribute) mul.right()).name(), equalTo("c")); + } + + public void testConjunctionDisjunctionCondition() { + Expression expression = whereExpression("not aaa and b or c"); + assertThat(expression, instanceOf(Or.class)); + Or or = (Or) expression; + assertThat(or.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) or.right()).name(), equalTo("c")); + assertThat(or.left(), instanceOf(And.class)); + And and = (And) or.left(); + assertThat(and.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) and.right()).name(), equalTo("b")); + assertThat(and.left(), instanceOf(Not.class)); + Not not = (Not) and.left(); + assertThat(not.children().size(), equalTo(1)); + assertThat(not.children().get(0), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) not.children().get(0)).name(), equalTo("aaa")); + } + + public void testParenthesizedExpression() { + Expression expression = whereExpression("((a and ((b and c))) or (((x or y))))"); + assertThat(expression, instanceOf(Or.class)); + Or or = (Or) expression; + + assertThat(or.right(), instanceOf(Or.class)); + Or orRight = (Or) or.right(); + assertThat(orRight.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) orRight.right()).name(), equalTo("y")); + assertThat(orRight.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(orRight.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) orRight.left()).name(), equalTo("x")); + + assertThat(or.left(), instanceOf(And.class)); + And and = (And) or.left(); + assertThat(and.right(), instanceOf(And.class)); + And andRight = (And) and.right(); + assertThat(andRight.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) andRight.right()).name(), equalTo("c")); + assertThat(andRight.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) andRight.left()).name(), equalTo("b")); + + assertThat(and.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) and.left()).name(), equalTo("a")); + } + + public void testCommandNamesAsIdentifiers() { + Expression expr = whereExpression("from and where"); + assertThat(expr, instanceOf(And.class)); + And and = (And) expr; + + assertThat(and.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) and.left()).name(), equalTo("from")); + + assertThat(and.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) and.right()).name(), equalTo("where")); + } + + public void testIdentifiersCaseSensitive() { + Expression expr = whereExpression("hElLo"); + + assertThat(expr, instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) expr).name(), equalTo("hElLo")); + } + + /* + * a > 1 and b > 1 + 2 => (a > 1) and (b > (1 + 2)) + */ + public void testOperatorsPrecedenceWithConjunction() { + Expression expression = whereExpression("a > 1 and b > 1 + 2"); + assertThat(expression, instanceOf(And.class)); + And and = (And) expression; + + assertThat(and.left(), instanceOf(GreaterThan.class)); + GreaterThan gt = (GreaterThan) and.left(); + assertThat(gt.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) gt.left()).name(), equalTo("a")); + assertThat(gt.right(), instanceOf(Literal.class)); + assertThat(((Literal) gt.right()).value(), equalTo(1)); + + assertThat(and.right(), instanceOf(GreaterThan.class)); + gt = (GreaterThan) and.right(); + assertThat(gt.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) gt.left()).name(), equalTo("b")); + assertThat(gt.right(), instanceOf(Add.class)); + Add add = (Add) gt.right(); + assertThat(((Literal) add.right()).value(), equalTo(2)); + assertThat(((Literal) add.left()).value(), equalTo(1)); + } + + /* + * a <= 1 or b >= 5 / 2 and c != 5 => (a <= 1) or (b >= (5 / 2) and not(c == 5)) + */ + public void testOperatorsPrecedenceWithDisjunction() { + Expression expression = whereExpression("a <= 1 or b >= 5 / 2 and c != 5"); + assertThat(expression, instanceOf(Or.class)); + Or or = (Or) expression; + + assertThat(or.left(), instanceOf(LessThanOrEqual.class)); + LessThanOrEqual lte = (LessThanOrEqual) or.left(); + assertThat(lte.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) lte.left()).name(), equalTo("a")); + assertThat(lte.right(), instanceOf(Literal.class)); + assertThat(((Literal) lte.right()).value(), equalTo(1)); + + assertThat(or.right(), instanceOf(And.class)); + And and = (And) or.right(); + assertThat(and.left(), instanceOf(GreaterThanOrEqual.class)); + GreaterThanOrEqual gte = (GreaterThanOrEqual) and.left(); + assertThat(gte.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) gte.left()).name(), equalTo("b")); + assertThat(gte.right(), instanceOf(Div.class)); + Div div = (Div) gte.right(); + assertThat(div.right(), instanceOf(Literal.class)); + assertThat(((Literal) div.right()).value(), equalTo(2)); + assertThat(div.left(), instanceOf(Literal.class)); + assertThat(((Literal) div.left()).value(), equalTo(5)); + + assertThat(and.right(), instanceOf(Not.class)); + assertThat(((Not) and.right()).field(), instanceOf(Equals.class)); + Equals e = (Equals) ((Not) and.right()).field(); + assertThat(e.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) e.left()).name(), equalTo("c")); + assertThat(e.right(), instanceOf(Literal.class)); + assertThat(((Literal) e.right()).value(), equalTo(5)); + } + + /* + * not a == 1 or not b >= 5 and c == 5 => (not (a == 1)) or ((not (b >= 5)) and c == 5) + */ + public void testOperatorsPrecedenceWithNegation() { + Expression expression = whereExpression("not a == 1 or not b >= 5 and c == 5"); + assertThat(expression, instanceOf(Or.class)); + Or or = (Or) expression; + + assertThat(or.left(), instanceOf(Not.class)); + assertThat(((Not) or.left()).field(), instanceOf(Equals.class)); + Equals e = (Equals) ((Not) or.left()).field(); + assertThat(e.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) e.left()).name(), equalTo("a")); + assertThat(e.right(), instanceOf(Literal.class)); + assertThat(((Literal) e.right()).value(), equalTo(1)); + + assertThat(or.right(), instanceOf(And.class)); + And and = (And) or.right(); + assertThat(and.left(), instanceOf(Not.class)); + assertThat(((Not) and.left()).field(), instanceOf(GreaterThanOrEqual.class)); + GreaterThanOrEqual gte = (GreaterThanOrEqual) ((Not) and.left()).field(); + assertThat(gte.right(), instanceOf(Literal.class)); + assertThat(((Literal) gte.right()).value(), equalTo(5)); + + assertThat(and.right(), instanceOf(Equals.class)); + e = (Equals) and.right(); + assertThat(e.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) e.left()).name(), equalTo("c")); + assertThat(e.right(), instanceOf(Literal.class)); + assertThat(((Literal) e.right()).value(), equalTo(5)); + } + + public void testOperatorsPrecedenceExpressionsEquality() { + assertThat(whereExpression("a-1>2 or b>=5 and c-1>=5"), equalTo(whereExpression("((a-1)>2 or (b>=5 and (c-1)>=5))"))); + assertThat( + whereExpression("a*5==25 and b>5 and c%4>=1 or true or false"), + equalTo(whereExpression("(((((a*5)==25) and (b>5) and ((c%4)>=1)) or true) or false)")) + ); + assertThat( + whereExpression("a*4-b*5<100 and b/2+c*6>=50 or c%5+x>=5"), + equalTo(whereExpression("((((a*4)-(b*5))<100) and (((b/2)+(c*6))>=50)) or (((c%5)+x)>=5)")) + ); + assertThat( + whereExpression("true and false or true and c/12+x*5-y%2>=50"), + equalTo(whereExpression("((true and false) or (true and (((c/12)+(x*5)-(y%2))>=50)))")) + ); + assertThat( + whereExpression("10 days > 5 hours and 1/5 minutes > 8 seconds * 3 and -1 minutes > foo"), + equalTo(whereExpression("((10 days) > (5 hours)) and ((1/(5 minutes) > ((8 seconds) * 3))) and (-1 minute > foo)")) + ); + assertThat( + whereExpression("10 DAYS > 5 HOURS and 1/5 MINUTES > 8 SECONDS * 3 and -1 MINUTES > foo"), + equalTo(whereExpression("((10 days) > (5 hours)) and ((1/(5 minutes) > ((8 seconds) * 3))) and (-1 minute > foo)")) + ); + } + + public void testFunctionExpressions() { + assertEquals(new UnresolvedFunction(EMPTY, "fn", DEFAULT, new ArrayList<>()), whereExpression("fn()")); + assertEquals( + new UnresolvedFunction( + EMPTY, + "invoke", + DEFAULT, + new ArrayList<>( + List.of( + new UnresolvedAttribute(EMPTY, "a"), + new Add(EMPTY, new UnresolvedAttribute(EMPTY, "b"), new UnresolvedAttribute(EMPTY, "c")) + ) + ) + ), + whereExpression("invoke(a, b + c)") + ); + assertEquals(whereExpression("(invoke((a + b)))"), whereExpression("invoke(a+b)")); + assertEquals(whereExpression("((fn()) + fn(fn()))"), whereExpression("fn() + fn(fn())")); + } + + public void testUnquotedIdentifiers() { + for (String identifier : List.of("a", "_a", "a_b", "a9", "abc123", "a_____9", "__a_b", "@a", "_1", "@2")) { + assertEquals(new UnresolvedAttribute(EMPTY, identifier), whereExpression(identifier)); + } + } + + public void testDurationLiterals() { + int value = randomInt(Integer.MAX_VALUE); + + assertEquals(l(Duration.ZERO, TIME_DURATION), whereExpression("0 millisecond")); + assertEquals(l(Duration.ofMillis(value), TIME_DURATION), whereExpression(value + "millisecond")); + assertEquals(l(Duration.ofMillis(value), TIME_DURATION), whereExpression(value + " milliseconds")); + + assertEquals(l(Duration.ZERO, TIME_DURATION), whereExpression("0 second")); + assertEquals(l(Duration.ofSeconds(value), TIME_DURATION), whereExpression(value + "second")); + assertEquals(l(Duration.ofSeconds(value), TIME_DURATION), whereExpression(value + " seconds")); + + assertEquals(l(Duration.ZERO, TIME_DURATION), whereExpression("0 minute")); + assertEquals(l(Duration.ofMinutes(value), TIME_DURATION), whereExpression(value + "minute")); + assertEquals(l(Duration.ofMinutes(value), TIME_DURATION), whereExpression(value + " minutes")); + + assertEquals(l(Duration.ZERO, TIME_DURATION), whereExpression("0 hour")); + assertEquals(l(Duration.ofHours(value), TIME_DURATION), whereExpression(value + "hour")); + assertEquals(l(Duration.ofHours(value), TIME_DURATION), whereExpression(value + " hours")); + + assertEquals(l(Duration.ofHours(-value), TIME_DURATION), whereExpression("-" + value + " hours")); + } + + public void testDatePeriodLiterals() { + int value = randomInt(Integer.MAX_VALUE); + int weeksValue = randomInt(Integer.MAX_VALUE / 7); + + assertEquals(l(Period.ZERO, DATE_PERIOD), whereExpression("0 day")); + assertEquals(l(Period.ofDays(value), DATE_PERIOD), whereExpression(value + "day")); + assertEquals(l(Period.ofDays(value), DATE_PERIOD), whereExpression(value + " days")); + + assertEquals(l(Period.ZERO, DATE_PERIOD), whereExpression("0week")); + assertEquals(l(Period.ofDays(weeksValue * 7), DATE_PERIOD), whereExpression(weeksValue + "week")); + assertEquals(l(Period.ofDays(weeksValue * 7), DATE_PERIOD), whereExpression(weeksValue + " weeks")); + + assertEquals(l(Period.ZERO, DATE_PERIOD), whereExpression("0 month")); + assertEquals(l(Period.ofMonths(value), DATE_PERIOD), whereExpression(value + "month")); + assertEquals(l(Period.ofMonths(value), DATE_PERIOD), whereExpression(value + " months")); + + assertEquals(l(Period.ZERO, DATE_PERIOD), whereExpression("0year")); + assertEquals(l(Period.ofYears(value), DATE_PERIOD), whereExpression(value + "year")); + assertEquals(l(Period.ofYears(value), DATE_PERIOD), whereExpression(value + " years")); + + assertEquals(l(Period.ofYears(-value), DATE_PERIOD), whereExpression("-" + value + " years")); + } + + public void testUnknownNumericQualifier() { + assertParsingException(() -> whereExpression("1 decade"), "Unexpected numeric qualifier 'decade'"); + } + + public void testQualifiedDecimalLiteral() { + assertParsingException(() -> whereExpression("1.1 hours"), "extraneous input 'hours' expecting "); + } + + public void testOverflowingValueForDuration() { + for (String unit : List.of("milliseconds", "seconds", "minutes", "hours")) { + assertParsingException( + () -> parse("row x = 9223372036854775808 " + unit), // unsigned_long (Long.MAX_VALUE + 1) + "line 1:10: Number [9223372036854775808] outside of [" + unit + "] range" + ); + assertParsingException( + () -> parse("row x = 18446744073709551616 " + unit), // double (UNSIGNED_LONG_MAX + 1) + "line 1:10: Number [18446744073709551616] outside of [" + unit + "] range" + ); + } + assertParsingException( + () -> parse("row x = 153722867280912931 minutes"), // Long.MAX_VALUE / 60 + 1 + "line 1:10: Number [153722867280912931] outside of [minutes] range" + ); + assertParsingException( + () -> parse("row x = 2562047788015216 hours"), // Long.MAX_VALUE / 3600 + 1 + "line 1:10: Number [2562047788015216] outside of [hours] range" + ); + } + + public void testOverflowingValueForPeriod() { + for (String unit : List.of("days", "weeks", "months", "years")) { + assertParsingException( + () -> parse("row x = 2147483648 " + unit), // long (Integer.MAX_VALUE + 1) + "line 1:10: Number [2147483648] outside of [" + unit + "] range" + ); + } + assertParsingException( + () -> parse("row x = 306783379 weeks"), // Integer.MAX_VALUE / 7 + 1 + "line 1:10: Number [306783379] outside of [weeks] range" + ); + } + + public void testWildcardProjectKeepPatterns() { + String[] exp = new String[] { + "a*", + "*a", + "a.*", + "a.a.*.*.a", + "*.a.a.a.*", + "*abc.*", + "a*b*c", + "*a*", + "*a*b", + "a*b*", + "*a*b*c*", + "a*b*c*", + "*a*b*c", + "a*b*c*a.b*", + "a*b*c*a.b.*", + "*a.b.c*b*c*a.b.*" }; + List projections; + Project p; + for (String e : exp) { + p = projectExpression(e); + projections = p.projections(); + assertThat(projections.size(), equalTo(1)); + assertThat("Projection [" + e + "] has an unexpected type", projections.get(0), instanceOf(UnresolvedAttribute.class)); + UnresolvedAttribute ua = (UnresolvedAttribute) projections.get(0); + assertThat(ua.name(), equalTo(e)); + assertThat(ua.unresolvedMessage(), equalTo("Unknown column [" + e + "]")); + } + } + + public void testWildcardProjectKeep() { + Project p = projectExpression("*"); + List projections = p.projections(); + assertThat(projections.size(), equalTo(1)); + assertThat(projections.get(0), instanceOf(UnresolvedStar.class)); + UnresolvedStar us = (UnresolvedStar) projections.get(0); + assertThat(us.qualifier(), equalTo(null)); + assertThat(us.unresolvedMessage(), equalTo("Cannot determine columns for [*]")); + } + + public void testWildcardProjectAwayPatterns() { + String[] exp = new String[] { + "a*", + "*a", + "a.*", + "a.a.*.*.a", + "*.a.a.a.*", + "*abc.*", + "a*b*c", + "*a*", + "*a*b", + "a*b*", + "*a*b*c*", + "a*b*c*", + "*a*b*c", + "a*b*c*a.b*", + "a*b*c*a.b.*", + "*a.b.c*b*c*a.b.*" }; + List removals; + for (String e : exp) { + Drop d = dropExpression(e); + removals = d.removals(); + assertThat(removals.size(), equalTo(1)); + assertThat("Projection [" + e + "] has an unexpected type", removals.get(0), instanceOf(UnresolvedAttribute.class)); + UnresolvedAttribute ursa = (UnresolvedAttribute) removals.get(0); + assertThat(ursa.name(), equalTo(e)); + assertThat(ursa.unresolvedMessage(), equalTo("Unknown column [" + e + "]")); + } + } + + public void testForbidWildcardProjectAway() { + assertParsingException(() -> dropExpression("foo, *"), "line 1:21: Removing all fields is not allowed [*]"); + } + + public void testForbidMultipleIncludeStar() { + var errorMsg = "Cannot specify [*] more than once"; + assertParsingException(() -> projectExpression("a, *, *, b"), errorMsg); + assertParsingException(() -> projectExpression("a, *, b, *, c"), errorMsg); + assertParsingException(() -> projectExpression("a, b, *, c, d, *"), errorMsg); + } + + public void testProjectKeepPatterns() { + String[] exp = new String[] { "abc", "abc.xyz", "a.b.c.d.e" }; + List projections; + for (String e : exp) { + Project p = projectExpression(e); + projections = p.projections(); + assertThat(projections.size(), equalTo(1)); + assertThat(projections.get(0), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) projections.get(0)).name(), equalTo(e)); + } + } + + public void testProjectAwayPatterns() { + String[] exp = new String[] { "abc", "abc.xyz", "a.b.c.d.e" }; + for (String e : exp) { + Drop d = dropExpression(e); + List removals = d.removals(); + assertThat(removals.size(), equalTo(1)); + assertThat(removals.get(0), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) removals.get(0)).name(), equalTo(e)); + } + } + + public void testProjectRename() { + String[] newName = new String[] { "a", "a.b", "a", "x.y" }; + String[] oldName = new String[] { "b", "a.c", "x.y", "a" }; + List renamings; + for (int i = 0; i < newName.length; i++) { + Rename r = renameExpression(oldName[i] + " AS " + newName[i]); + renamings = r.renamings(); + assertThat(renamings.size(), equalTo(1)); + assertThat(renamings.get(0), instanceOf(Alias.class)); + Alias a = (Alias) renamings.get(0); + assertThat(a.child(), instanceOf(UnresolvedAttribute.class)); + UnresolvedAttribute ua = (UnresolvedAttribute) a.child(); + assertThat(a.name(), equalTo(newName[i])); + assertThat(ua.name(), equalTo(oldName[i])); + } + } + + public void testMultipleProjectPatterns() { + LogicalPlan plan = parse("from a | rename y as x | keep abc, xyz*, x, *"); + Project p = as(plan, Project.class); + List projections = p.projections(); + assertThat(projections.size(), equalTo(4)); + assertThat(projections.get(0), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) projections.get(0)).name(), equalTo("abc")); + assertThat(projections.get(1), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) projections.get(1)).name(), equalTo("xyz*")); + assertThat(projections.get(2), instanceOf(UnresolvedAttribute.class)); + assertThat(projections.get(3), instanceOf(UnresolvedStar.class)); + } + + public void testForbidWildcardProjectRename() { + assertParsingException( + () -> renameExpression("b* AS a*"), + "line 1:18: Using wildcards (*) in renaming projections is not allowed [b* AS a*]" + ); + } + + public void testSimplifyInWithSingleElementList() { + Expression e = whereExpression("a IN (1)"); + assertThat(e, instanceOf(Equals.class)); + Equals eq = (Equals) e; + assertThat(eq.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) eq.left()).name(), equalTo("a")); + assertThat(eq.right(), instanceOf(Literal.class)); + assertThat(eq.right().fold(), equalTo(1)); + + e = whereExpression("1 IN (a)"); + assertThat(e, instanceOf(Equals.class)); + eq = (Equals) e; + assertThat(eq.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) eq.right()).name(), equalTo("a")); + assertThat(eq.left(), instanceOf(Literal.class)); + assertThat(eq.left().fold(), equalTo(1)); + + e = whereExpression("1 NOT IN (a)"); + assertThat(e, instanceOf(Not.class)); + e = e.children().get(0); + assertThat(e, instanceOf(Equals.class)); + eq = (Equals) e; + assertThat(eq.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) eq.right()).name(), equalTo("a")); + assertThat(eq.left(), instanceOf(Literal.class)); + assertThat(eq.left().fold(), equalTo(1)); + + } + + private Expression whereExpression(String e) { + return ((Filter) parse("from a | where " + e)).condition(); + } + + private Drop dropExpression(String e) { + return (Drop) parse("from a | drop " + e); + } + + private Rename renameExpression(String e) { + return (Rename) parse("from a | rename " + e); + } + + private Project projectExpression(String e) { + return (Project) parse("from a | keep " + e); + } + + private LogicalPlan parse(String s) { + return parser.createStatement(s); + } + + private Literal l(Object value, DataType type) { + return new Literal(null, value, type); + } + + private void assertParsingException(ThrowingRunnable expression, String expectedError) { + ParsingException e = expectThrows(ParsingException.class, "Expected syntax error", expression); + assertThat(e.getMessage(), containsString(expectedError)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java new file mode 100644 index 0000000000000..9536a8160f150 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java @@ -0,0 +1,813 @@ +/* + * 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.parser; + +import org.elasticsearch.core.Tuple; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Enrich; +import org.elasticsearch.xpack.esql.plan.logical.EsqlUnresolvedRelation; +import org.elasticsearch.xpack.esql.plan.logical.Eval; +import org.elasticsearch.xpack.esql.plan.logical.Explain; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.logical.InlineStats; +import org.elasticsearch.xpack.esql.plan.logical.MvExpand; +import org.elasticsearch.xpack.esql.plan.logical.Row; +import org.elasticsearch.xpack.ql.expression.Alias; +import org.elasticsearch.xpack.ql.expression.EmptyAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.NamedExpression; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.ReferenceAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.regex.RLike; +import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike; +import org.elasticsearch.xpack.ql.plan.logical.Aggregate; +import org.elasticsearch.xpack.ql.plan.logical.Filter; +import org.elasticsearch.xpack.ql.plan.logical.Limit; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.plan.logical.OrderBy; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.elasticsearch.xpack.ql.expression.Literal.FALSE; +import static org.elasticsearch.xpack.ql.expression.Literal.TRUE; +import static org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy.DEFAULT; +import static org.elasticsearch.xpack.ql.tree.Source.EMPTY; +import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD; +import static org.elasticsearch.xpack.ql.util.NumericUtils.asLongUnsigned; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class StatementParserTests extends ESTestCase { + + private static String FROM = "from test"; + EsqlParser parser = new EsqlParser(); + + public void testRowCommand() { + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "a", integer(1)), new Alias(EMPTY, "b", integer(2)))), + statement("row a = 1, b = 2") + ); + } + + public void testRowCommandImplicitFieldName() { + assertEquals( + new Row( + EMPTY, + List.of(new Alias(EMPTY, "1", integer(1)), new Alias(EMPTY, "2", integer(2)), new Alias(EMPTY, "c", integer(3))) + ), + statement("row 1, 2, c = 3") + ); + } + + public void testRowCommandLong() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalLong(2147483648L)))), statement("row c = 2147483648")); + } + + public void testRowCommandHugeInt() { + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalUnsignedLong("9223372036854775808")))), + statement("row c = 9223372036854775808") + ); + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalDouble(18446744073709551616.)))), + statement("row c = 18446744073709551616") + ); + } + + public void testRowCommandDouble() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalDouble(1.0)))), statement("row c = 1.0")); + } + + public void testRowCommandMultivalueInt() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", integers(1, 2, -5)))), statement("row c = [1, 2, -5]")); + } + + public void testRowCommandMultivalueLong() { + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalLongs(2147483648L, 2147483649L, -434366649L)))), + statement("row c = [2147483648, 2147483649, -434366649]") + ); + } + + public void testRowCommandMultivalueLongAndInt() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalLongs(2147483648L, 1L)))), statement("row c = [2147483648, 1]")); + } + + public void testRowCommandMultivalueHugeInts() { + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalDoubles(18446744073709551616., 18446744073709551617.)))), + statement("row c = [18446744073709551616, 18446744073709551617]") + ); + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalUnsignedLongs("9223372036854775808", "9223372036854775809")))), + statement("row c = [9223372036854775808, 9223372036854775809]") + ); + } + + public void testRowCommandMultivalueHugeIntAndNormalInt() { + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalDoubles(18446744073709551616., 1.0)))), + statement("row c = [18446744073709551616, 1]") + ); + assertEquals( + new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalUnsignedLongs("9223372036854775808", "1")))), + statement("row c = [9223372036854775808, 1]") + ); + } + + public void testRowCommandMultivalueDouble() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalDoubles(1.0, 2.0, -3.4)))), statement("row c = [1.0, 2.0, -3.4]")); + } + + public void testRowCommandBoolean() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalBoolean(false)))), statement("row c = false")); + } + + public void testRowCommandMultivalueBoolean() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalBooleans(false, true)))), statement("row c = [false, true]")); + } + + public void testRowCommandString() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalString("chicken")))), statement("row c = \"chicken\"")); + } + + public void testRowCommandMultivalueString() { + assertEquals(new Row(EMPTY, List.of(new Alias(EMPTY, "c", literalStrings("cat", "dog")))), statement("row c = [\"cat\", \"dog\"]")); + } + + public void testRowCommandWithEscapedFieldName() { + assertEquals( + new Row( + EMPTY, + List.of( + new Alias(EMPTY, "a.b.c", integer(1)), + new Alias(EMPTY, "b", integer(2)), + new Alias(EMPTY, "@timestamp", new Literal(EMPTY, "2022-26-08T00:00:00", KEYWORD)) + ) + ), + statement("row a.b.c = 1, `b` = 2, `@timestamp`=\"2022-26-08T00:00:00\"") + ); + } + + public void testCompositeCommand() { + assertEquals( + new Filter(EMPTY, new Row(EMPTY, List.of(new Alias(EMPTY, "a", integer(1)))), TRUE), + statement("row a = 1 | where true") + ); + } + + public void testMultipleCompositeCommands() { + assertEquals( + new Filter( + EMPTY, + new Filter(EMPTY, new Filter(EMPTY, new Row(EMPTY, List.of(new Alias(EMPTY, "a", integer(1)))), TRUE), FALSE), + TRUE + ), + statement("row a = 1 | where true | where false | where true") + ); + } + + public void testEval() { + assertEquals( + new Eval(EMPTY, PROCESSING_CMD_INPUT, List.of(new Alias(EMPTY, "b", attribute("a")))), + processingCommand("eval b = a") + ); + + assertEquals( + new Eval( + EMPTY, + PROCESSING_CMD_INPUT, + List.of(new Alias(EMPTY, "b", attribute("a")), new Alias(EMPTY, "c", new Add(EMPTY, attribute("a"), integer(1)))) + ), + processingCommand("eval b = a, c = a + 1") + ); + } + + public void testEvalImplicitNames() { + assertEquals(new Eval(EMPTY, PROCESSING_CMD_INPUT, List.of(new Alias(EMPTY, "a", attribute("a")))), processingCommand("eval a")); + + assertEquals( + new Eval( + EMPTY, + PROCESSING_CMD_INPUT, + List.of( + new Alias( + EMPTY, + "fn(a+1)", + new UnresolvedFunction(EMPTY, "fn", DEFAULT, List.of(new Add(EMPTY, attribute("a"), integer(1)))) + ) + ) + ), + processingCommand("eval fn(a + 1)") + ); + } + + public void testStatsWithGroups() { + assertEquals( + new Aggregate( + EMPTY, + PROCESSING_CMD_INPUT, + List.of(attribute("c"), attribute("d.e")), + List.of( + new Alias(EMPTY, "b", new UnresolvedFunction(EMPTY, "min", DEFAULT, List.of(attribute("a")))), + attribute("c"), + attribute("d.e") + ) + ), + processingCommand("stats b = min(a) by c, d.e") + ); + } + + public void testStatsWithoutGroups() { + assertEquals( + new Aggregate( + EMPTY, + PROCESSING_CMD_INPUT, + List.of(), + List.of( + new Alias(EMPTY, "min(a)", new UnresolvedFunction(EMPTY, "min", DEFAULT, List.of(attribute("a")))), + new Alias(EMPTY, "c", integer(1)) + ) + ), + processingCommand("stats min(a), c = 1") + ); + } + + public void testStatsWithoutAggs() throws Exception { + assertEquals( + new Aggregate(EMPTY, PROCESSING_CMD_INPUT, List.of(attribute("a")), List.of(attribute("a"))), + processingCommand("stats by a") + ); + } + + public void testStatsWithoutAggsOrGroup() throws Exception { + expectError("from text | stats", "At least one aggregation or grouping expression required in [stats]"); + } + + public void testAggsWithGroupKeyAsAgg() throws Exception { + var queries = new String[] { """ + row a = 1, b = 2 + | stats a by a + """, """ + row a = 1, b = 2 + | stats a by a + | sort a + """, """ + row a = 1, b = 2 + | stats a = a by a + """, """ + row a = 1, b = 2 + | stats x = a by a + """ }; + + for (String query : queries) { + expectError(query, "Cannot specify grouping expression [a] as an aggregate"); + } + } + + public void testInlineStatsWithGroups() { + assertEquals( + new InlineStats( + EMPTY, + PROCESSING_CMD_INPUT, + List.of(attribute("c"), attribute("d.e")), + List.of( + new Alias(EMPTY, "b", new UnresolvedFunction(EMPTY, "min", DEFAULT, List.of(attribute("a")))), + attribute("c"), + attribute("d.e") + ) + ), + processingCommand("inlinestats b = min(a) by c, d.e") + ); + } + + public void testInlineStatsWithoutGroups() { + assertEquals( + new InlineStats( + EMPTY, + PROCESSING_CMD_INPUT, + List.of(), + List.of( + new Alias(EMPTY, "min(a)", new UnresolvedFunction(EMPTY, "min", DEFAULT, List.of(attribute("a")))), + new Alias(EMPTY, "c", integer(1)) + ) + ), + processingCommand("inlinestats min(a), c = 1") + ); + } + + public void testIdentifiersAsIndexPattern() { + assertIdentifierAsIndexPattern("foo", "from `foo`"); + assertIdentifierAsIndexPattern("foo,test-*", "from `foo`,`test-*`"); + assertIdentifierAsIndexPattern("foo,test-*", "from foo,test-*"); + assertIdentifierAsIndexPattern("123-test@foo_bar+baz1", "from 123-test@foo_bar+baz1"); + assertIdentifierAsIndexPattern("foo,test-*,abc", "from `foo`,`test-*`,abc"); + assertIdentifierAsIndexPattern("foo, test-*, abc, xyz", "from `foo, test-*, abc, xyz`"); + assertIdentifierAsIndexPattern("foo, test-*, abc, xyz,test123", "from `foo, test-*, abc, xyz`, test123"); + assertIdentifierAsIndexPattern("foo,test,xyz", "from foo, test,xyz"); + assertIdentifierAsIndexPattern( + ",", + "from , ``" + ); + } + + public void testIdentifierAsFieldName() { + String[] operators = new String[] { "==", "!=", ">", "<", ">=", "<=" }; + Class[] expectedOperators = new Class[] { + Equals.class, + Not.class, + GreaterThan.class, + LessThan.class, + GreaterThanOrEqual.class, + LessThanOrEqual.class }; + String[] identifiers = new String[] { "abc", "`abc`", "ab_c", "a.b.c", "@a", "a.@b", "`a@b.c`" }; + String[] expectedIdentifiers = new String[] { "abc", "abc", "ab_c", "a.b.c", "@a", "a.@b", "a@b.c" }; + LogicalPlan where; + for (int i = 0; i < operators.length; i++) { + for (int j = 0; j < identifiers.length; j++) { + where = processingCommand("where " + identifiers[j] + operators[i] + "123"); + assertThat(where, instanceOf(Filter.class)); + Filter filter = (Filter) where; + assertThat(filter.children().size(), equalTo(1)); + assertThat(filter.condition(), instanceOf(expectedOperators[i])); + BinaryComparison comparison; + if (filter.condition() instanceof Not not) { + assertThat(not.children().get(0), instanceOf(Equals.class)); + comparison = (BinaryComparison) (not.children().get(0)); + } else { + comparison = (BinaryComparison) filter.condition(); + } + assertThat(comparison.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) comparison.left()).name(), equalTo(expectedIdentifiers[j])); + assertThat(comparison.right(), instanceOf(Literal.class)); + assertThat(((Literal) comparison.right()).value(), equalTo(123)); + assertThat(filter.child(), equalTo(PROCESSING_CMD_INPUT)); + } + } + } + + public void testBooleanLiteralCondition() { + LogicalPlan where = processingCommand("where true"); + assertThat(where, instanceOf(Filter.class)); + Filter w = (Filter) where; + assertThat(w.child(), equalTo(PROCESSING_CMD_INPUT)); + assertThat(w.condition(), equalTo(TRUE)); + } + + public void testBasicLimitCommand() { + LogicalPlan plan = statement("from text | where true | limit 5"); + assertThat(plan, instanceOf(Limit.class)); + Limit limit = (Limit) plan; + assertThat(limit.limit(), instanceOf(Literal.class)); + assertThat(((Literal) limit.limit()).value(), equalTo(5)); + assertThat(limit.children().size(), equalTo(1)); + assertThat(limit.children().get(0), instanceOf(Filter.class)); + assertThat(limit.children().get(0).children().size(), equalTo(1)); + assertThat(limit.children().get(0).children().get(0), instanceOf(EsqlUnresolvedRelation.class)); + } + + public void testLimitConstraints() { + expectError("from text | limit -1", "extraneous input '-' expecting INTEGER_LITERAL"); + } + + public void testBasicSortCommand() { + LogicalPlan plan = statement("from text | where true | sort a+b asc nulls first, x desc nulls last | sort y asc | sort z desc"); + assertThat(plan, instanceOf(OrderBy.class)); + OrderBy orderBy = (OrderBy) plan; + assertThat(orderBy.order().size(), equalTo(1)); + Order order = orderBy.order().get(0); + assertThat(order.direction(), equalTo(Order.OrderDirection.DESC)); + assertThat(order.nullsPosition(), equalTo(Order.NullsPosition.FIRST)); + assertThat(order.child(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) order.child()).name(), equalTo("z")); + + assertThat(orderBy.children().size(), equalTo(1)); + assertThat(orderBy.children().get(0), instanceOf(OrderBy.class)); + orderBy = (OrderBy) orderBy.children().get(0); + assertThat(orderBy.order().size(), equalTo(1)); + order = orderBy.order().get(0); + assertThat(order.direction(), equalTo(Order.OrderDirection.ASC)); + assertThat(order.nullsPosition(), equalTo(Order.NullsPosition.LAST)); + assertThat(order.child(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) order.child()).name(), equalTo("y")); + + assertThat(orderBy.children().size(), equalTo(1)); + assertThat(orderBy.children().get(0), instanceOf(OrderBy.class)); + orderBy = (OrderBy) orderBy.children().get(0); + assertThat(orderBy.order().size(), equalTo(2)); + order = orderBy.order().get(0); + assertThat(order.direction(), equalTo(Order.OrderDirection.ASC)); + assertThat(order.nullsPosition(), equalTo(Order.NullsPosition.FIRST)); + assertThat(order.child(), instanceOf(Add.class)); + Add add = (Add) order.child(); + assertThat(add.left(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) add.left()).name(), equalTo("a")); + assertThat(add.right(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) add.right()).name(), equalTo("b")); + order = orderBy.order().get(1); + assertThat(order.direction(), equalTo(Order.OrderDirection.DESC)); + assertThat(order.nullsPosition(), equalTo(Order.NullsPosition.LAST)); + assertThat(order.child(), instanceOf(UnresolvedAttribute.class)); + assertThat(((UnresolvedAttribute) order.child()).name(), equalTo("x")); + + assertThat(orderBy.children().size(), equalTo(1)); + assertThat(orderBy.children().get(0), instanceOf(Filter.class)); + assertThat(orderBy.children().get(0).children().size(), equalTo(1)); + assertThat(orderBy.children().get(0).children().get(0), instanceOf(EsqlUnresolvedRelation.class)); + } + + public void testSubquery() { + assertEquals(new Explain(EMPTY, PROCESSING_CMD_INPUT), statement("explain [ row a = 1 ]")); + } + + public void testSubqueryWithPipe() { + assertEquals( + new Limit(EMPTY, integer(10), new Explain(EMPTY, PROCESSING_CMD_INPUT)), + statement("explain [ row a = 1 ] | limit 10") + ); + } + + public void testNestedSubqueries() { + assertEquals( + new Limit( + EMPTY, + integer(10), + new Explain(EMPTY, new Limit(EMPTY, integer(5), new Explain(EMPTY, new Limit(EMPTY, integer(1), PROCESSING_CMD_INPUT)))) + ), + statement("explain [ explain [ row a = 1 | limit 1 ] | limit 5 ] | limit 10") + ); + } + + public void testSubquerySpacing() { + assertEquals(statement("explain [ explain [ from a ] | where b == 1 ]"), statement("explain[explain[from a]|where b==1]")); + } + + public void testBlockComments() { + String query = " explain [ from foo ] | limit 10 "; + LogicalPlan expected = statement(query); + + int wsIndex = query.indexOf(' '); + + do { + String queryWithComment = query.substring(0, wsIndex) + + "/*explain [ \nfrom bar ] | where a > b*/" + + query.substring(wsIndex + 1); + + assertEquals(expected, statement(queryWithComment)); + + wsIndex = query.indexOf(' ', wsIndex + 1); + } while (wsIndex >= 0); + } + + public void testSingleLineComments() { + String query = " explain [ from foo ] | limit 10 "; + LogicalPlan expected = statement(query); + + int wsIndex = query.indexOf(' '); + + do { + String queryWithComment = query.substring(0, wsIndex) + + "//explain [ from bar ] | where a > b \n" + + query.substring(wsIndex + 1); + + assertEquals(expected, statement(queryWithComment)); + + wsIndex = query.indexOf(' ', wsIndex + 1); + } while (wsIndex >= 0); + } + + public void testSuggestAvailableSourceCommandsOnParsingError() { + for (Tuple queryWithUnexpectedCmd : List.of( + Tuple.tuple("frm foo", "frm"), + Tuple.tuple("expln[from bar]", "expln"), + Tuple.tuple("not-a-thing logs", "not-a-thing"), + Tuple.tuple("high5 a", "high5"), + Tuple.tuple("a+b = c", "a+b"), + Tuple.tuple("a//hi", "a"), + Tuple.tuple("a/*hi*/", "a"), + Tuple.tuple("explain [ frm a ]", "frm") + )) { + ParsingException pe = expectThrows(ParsingException.class, () -> statement(queryWithUnexpectedCmd.v1())); + assertThat( + pe.getMessage(), + allOf( + containsString("mismatched input '" + queryWithUnexpectedCmd.v2() + "'"), + containsString("'explain'"), + containsString("'from'"), + containsString("'row'") + ) + ); + } + } + + public void testSuggestAvailableProcessingCommandsOnParsingError() { + for (Tuple queryWithUnexpectedCmd : List.of( + Tuple.tuple("from a | filter b > 1", "filter"), + Tuple.tuple("from a | explain [ row 1 ]", "explain"), + Tuple.tuple("from a | not-a-thing", "not-a-thing"), + Tuple.tuple("from a | high5 a", "high5"), + Tuple.tuple("from a | a+b = c", "a+b"), + Tuple.tuple("from a | a//hi", "a"), + Tuple.tuple("from a | a/*hi*/", "a"), + Tuple.tuple("explain [ from a | evl b = c ]", "evl") + )) { + ParsingException pe = expectThrows(ParsingException.class, () -> statement(queryWithUnexpectedCmd.v1())); + assertThat( + pe.getMessage(), + allOf( + containsString("mismatched input '" + queryWithUnexpectedCmd.v2() + "'"), + containsString("'eval'"), + containsString("'stats'"), + containsString("'where'") + ) + ); + } + } + + public void testDeprecatedIsNullFunction() { + expectError( + "from test | eval x = is_null(f)", + "line 1:23: is_null function is not supported anymore, please use 'is null'/'is not null' predicates instead" + ); + expectError( + "row x = is_null(f)", + "line 1:10: is_null function is not supported anymore, please use 'is null'/'is not null' predicates instead" + ); + } + + public void testMetadataFieldOnOtherSources() { + expectError( + "row a = 1 [metadata _index]", + "1:11: mismatched input '[' expecting {, PIPE, 'and', COMMA, 'or', '+', '-', '*', '/', '%'}" + ); + expectError("show functions [metadata _index]", "line 1:16: mismatched input '[' expecting {, PIPE}"); + expectError( + "explain [from foo] [metadata _index]", + "line 1:20: mismatched input '[' expecting {PIPE, COMMA, OPENING_BRACKET, ']'}" + ); + } + + public void testMetadataFieldMultipleDeclarations() { + expectError("from test [metadata _index, _version, _index]", "1:40: metadata field [_index] already declared [@1:21]"); + } + + public void testMetadataFieldUnsupportedPrimitiveType() { + expectError("from test [metadata _tier]", "line 1:22: unsupported metadata field [_tier]"); + } + + public void testMetadataFieldUnsupportedCustomType() { + expectError("from test [metadata _feature]", "line 1:22: unsupported metadata field [_feature]"); + } + + public void testMetadataFieldNotFoundNonExistent() { + expectError("from test [metadata _doesnot_compute]", "line 1:22: unsupported metadata field [_doesnot_compute]"); + } + + public void testMetadataFieldNotFoundNormalField() { + expectError("from test [metadata emp_no]", "line 1:22: unsupported metadata field [emp_no]"); + } + + public void testDissectPattern() { + LogicalPlan cmd = processingCommand("dissect a \"%{foo}\""); + assertEquals(Dissect.class, cmd.getClass()); + Dissect dissect = (Dissect) cmd; + assertEquals("%{foo}", dissect.parser().pattern()); + assertEquals("", dissect.parser().appendSeparator()); + assertEquals(List.of(referenceAttribute("foo", KEYWORD)), dissect.extractedFields()); + + cmd = processingCommand("dissect a \"%{foo}\" append_separator=\",\""); + assertEquals(Dissect.class, cmd.getClass()); + dissect = (Dissect) cmd; + assertEquals("%{foo}", dissect.parser().pattern()); + assertEquals(",", dissect.parser().appendSeparator()); + assertEquals(List.of(referenceAttribute("foo", KEYWORD)), dissect.extractedFields()); + + for (Tuple queryWithUnexpectedCmd : List.of( + Tuple.tuple("from a | dissect foo \"\"", "[]"), + Tuple.tuple("from a | dissect foo \" \"", "[ ]"), + Tuple.tuple("from a | dissect foo \"no fields\"", "[no fields]") + )) { + expectError(queryWithUnexpectedCmd.v1(), "Invalid pattern for dissect: " + queryWithUnexpectedCmd.v2()); + } + + expectError("from a | dissect foo \"%{*a}:%{&a}\"", "Reference keys not supported in dissect patterns: [%{*a}]"); + expectError("from a | dissect foo \"%{bar}\" invalid_option=3", "Invalid option for dissect: [invalid_option]"); + expectError( + "from a | dissect foo \"%{bar}\" append_separator=3", + "Invalid value for dissect append_separator: expected a string, but was [3]" + ); + } + + public void testGrokPattern() { + LogicalPlan cmd = processingCommand("grok a \"%{WORD:foo}\""); + assertEquals(Grok.class, cmd.getClass()); + Grok dissect = (Grok) cmd; + assertEquals("%{WORD:foo}", dissect.parser().pattern()); + assertEquals(List.of(referenceAttribute("foo", KEYWORD)), dissect.extractedFields()); + + ParsingException pe = expectThrows(ParsingException.class, () -> statement("row a = \"foo bar\" | grok a \"%{_invalid_:x}\"")); + assertThat( + pe.getMessage(), + containsString("Invalid pattern [%{_invalid_:x}] for grok: Unable to find pattern [_invalid_] in Grok's pattern dictionary") + ); + } + + public void testLikeRLike() { + LogicalPlan cmd = processingCommand("where foo like \"*bar*\""); + assertEquals(Filter.class, cmd.getClass()); + Filter filter = (Filter) cmd; + assertEquals(WildcardLike.class, filter.condition().getClass()); + WildcardLike like = (WildcardLike) filter.condition(); + assertEquals("*bar*", like.pattern().pattern()); + + cmd = processingCommand("where foo rlike \".*bar.*\""); + assertEquals(Filter.class, cmd.getClass()); + filter = (Filter) cmd; + assertEquals(RLike.class, filter.condition().getClass()); + RLike rlike = (RLike) filter.condition(); + assertEquals(".*bar.*", rlike.pattern().asJavaRegex()); + + expectError("from a | where foo like 12", "mismatched input '12'"); + expectError("from a | where foo rlike 12", "mismatched input '12'"); + } + + public void testEnrich() { + assertEquals( + new Enrich(EMPTY, PROCESSING_CMD_INPUT, new Literal(EMPTY, "countries", KEYWORD), new EmptyAttribute(EMPTY), null, List.of()), + processingCommand("enrich countries") + ); + + assertEquals( + new Enrich( + EMPTY, + PROCESSING_CMD_INPUT, + new Literal(EMPTY, "countries", KEYWORD), + new UnresolvedAttribute(EMPTY, "country_code"), + null, + List.of() + ), + processingCommand("enrich countries ON country_code") + ); + + expectError("from a | enrich countries on foo* ", "Using wildcards (*) in ENRICH WITH projections is not allowed [foo*]"); + expectError("from a | enrich countries on foo with bar*", "Using wildcards (*) in ENRICH WITH projections is not allowed [bar*]"); + expectError( + "from a | enrich countries on foo with x = bar* ", + "Using wildcards (*) in ENRICH WITH projections is not allowed [bar*]" + ); + expectError( + "from a | enrich countries on foo with x* = bar ", + "Using wildcards (*) in ENRICH WITH projections is not allowed [x*]" + ); + } + + public void testMvExpand() { + LogicalPlan cmd = processingCommand("mv_expand a"); + assertEquals(MvExpand.class, cmd.getClass()); + MvExpand expand = (MvExpand) cmd; + assertThat(expand.target(), equalTo(attribute("a"))); + } + + public void testUsageOfProject() { + processingCommand("project a"); + assertWarnings("PROJECT command is no longer supported, please use KEEP instead"); + } + + public void testInputParams() { + LogicalPlan stm = statement("row x = ?, y = ?", List.of(new TypedParamValue("integer", 1), new TypedParamValue("keyword", "2"))); + assertThat(stm, instanceOf(Row.class)); + Row row = (Row) stm; + assertThat(row.fields().size(), is(2)); + + NamedExpression field = row.fields().get(0); + assertThat(field.name(), is("x")); + assertThat(field, instanceOf(Alias.class)); + Alias alias = (Alias) field; + assertThat(alias.child().fold(), is(1)); + + field = row.fields().get(1); + assertThat(field.name(), is("y")); + assertThat(field, instanceOf(Alias.class)); + alias = (Alias) field; + assertThat(alias.child().fold(), is("2")); + } + + public void testMissingInputParams() { + expectError("row x = ?, y = ?", List.of(new TypedParamValue("integer", 1)), "Not enough actual parameters 1"); + } + + private void assertIdentifierAsIndexPattern(String identifier, String statement) { + LogicalPlan from = statement(statement); + assertThat(from, instanceOf(EsqlUnresolvedRelation.class)); + EsqlUnresolvedRelation table = (EsqlUnresolvedRelation) from; + assertThat(table.table().index(), is(identifier)); + } + + private LogicalPlan statement(String e) { + return statement(e, List.of()); + } + + private LogicalPlan statement(String e, List params) { + return parser.createStatement(e, params); + } + + private LogicalPlan processingCommand(String e) { + return parser.createStatement("row a = 1 | " + e); + } + + private static final LogicalPlan PROCESSING_CMD_INPUT = new Row(EMPTY, List.of(new Alias(EMPTY, "a", integer(1)))); + + private static UnresolvedAttribute attribute(String name) { + return new UnresolvedAttribute(EMPTY, name); + } + + private static ReferenceAttribute referenceAttribute(String name, DataType type) { + return new ReferenceAttribute(EMPTY, name, type); + } + + private static Literal integer(int i) { + return new Literal(EMPTY, i, DataTypes.INTEGER); + } + + private static Literal integers(int... ints) { + return new Literal(EMPTY, Arrays.stream(ints).boxed().toList(), DataTypes.INTEGER); + } + + private static Literal literalLong(long i) { + return new Literal(EMPTY, i, DataTypes.LONG); + } + + private static Literal literalLongs(long... longs) { + return new Literal(EMPTY, Arrays.stream(longs).boxed().toList(), DataTypes.LONG); + } + + private static Literal literalDouble(double d) { + return new Literal(EMPTY, d, DataTypes.DOUBLE); + } + + private static Literal literalDoubles(double... doubles) { + return new Literal(EMPTY, Arrays.stream(doubles).boxed().toList(), DataTypes.DOUBLE); + } + + private static Literal literalUnsignedLong(String ulong) { + return new Literal(EMPTY, asLongUnsigned(new BigInteger(ulong)), DataTypes.UNSIGNED_LONG); + } + + private static Literal literalUnsignedLongs(String... ulongs) { + return new Literal(EMPTY, Arrays.stream(ulongs).map(s -> asLongUnsigned(new BigInteger(s))).toList(), DataTypes.UNSIGNED_LONG); + } + + private static Literal literalBoolean(boolean b) { + return new Literal(EMPTY, b, DataTypes.BOOLEAN); + } + + private static Literal literalBooleans(boolean... booleans) { + List v = new ArrayList<>(booleans.length); + for (boolean b : booleans) { + v.add(b); + } + return new Literal(EMPTY, v, DataTypes.BOOLEAN); + } + + private static Literal literalString(String s) { + return new Literal(EMPTY, s, DataTypes.KEYWORD); + } + + private static Literal literalStrings(String... strings) { + return new Literal(EMPTY, Arrays.asList(strings), DataTypes.KEYWORD); + } + + private void expectError(String query, String errorMessage) { + ParsingException e = expectThrows(ParsingException.class, "Expected syntax error for " + query, () -> statement(query)); + assertThat(e.getMessage(), containsString(errorMessage)); + } + + private void expectError(String query, List params, String errorMessage) { + ParsingException e = expectThrows(ParsingException.class, "Expected syntax error for " + query, () -> statement(query, params)); + assertThat(e.getMessage(), containsString(errorMessage)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/EvalMapperTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/EvalMapperTests.java new file mode 100644 index 0000000000000..2a01c51ac6e6e --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/EvalMapperTests.java @@ -0,0 +1,143 @@ +/* + * 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.planner; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.SerializationTestUtils; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateFormat; +import org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Abs; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Concat; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Length; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.StartsWith; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.predicate.logical.And; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Not; +import org.elasticsearch.xpack.ql.expression.predicate.logical.Or; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Add; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Div; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Mul; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Neg; +import org.elasticsearch.xpack.ql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThanOrEqual; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThan; +import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.LessThanOrEqual; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.time.Duration; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.function.Supplier; + +public class EvalMapperTests extends ESTestCase { + private static final FieldAttribute DOUBLE1 = field("foo", DataTypes.DOUBLE); + private static final FieldAttribute DOUBLE2 = field("bar", DataTypes.DOUBLE); + private static final FieldAttribute LONG = field("long", DataTypes.LONG); + private static final FieldAttribute DATE = field("date", DataTypes.DATETIME); + + private static final EsqlConfiguration TEST_CONFIG = new EsqlConfiguration(ZoneOffset.UTC, Locale.US, "test", null, null, 10000000); + + @ParametersFactory(argumentFormatting = "%1$s") + public static List params() { + Literal literal = new Literal(Source.EMPTY, new BytesRef("something"), DataTypes.KEYWORD); + Literal datePattern = new Literal(Source.EMPTY, new BytesRef("yyyy"), DataTypes.KEYWORD); + Literal dateInterval = new Literal(Source.EMPTY, Duration.ofHours(1), EsqlDataTypes.TIME_DURATION); + + List params = new ArrayList<>(); + for (Expression e : new Expression[] { + new Add(Source.EMPTY, DOUBLE1, DOUBLE2), + new Sub(Source.EMPTY, DOUBLE1, DOUBLE2), + new Mul(Source.EMPTY, DOUBLE1, DOUBLE2), + new Div(Source.EMPTY, DOUBLE1, DOUBLE2), + new Neg(Source.EMPTY, DOUBLE1), + new Abs(Source.EMPTY, DOUBLE1), + new Equals(Source.EMPTY, DOUBLE1, DOUBLE2), + new GreaterThan(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new GreaterThanOrEqual(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new LessThan(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new LessThanOrEqual(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new And( + Source.EMPTY, + new LessThan(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new LessThanOrEqual(Source.EMPTY, DOUBLE1, DOUBLE2, null) + ), + new Or( + Source.EMPTY, + new LessThan(Source.EMPTY, DOUBLE1, DOUBLE2, null), + new LessThanOrEqual(Source.EMPTY, DOUBLE1, DOUBLE2, null) + ), + new Not(Source.EMPTY, new LessThan(Source.EMPTY, DOUBLE1, DOUBLE2, null)), + new Concat(Source.EMPTY, literal, Collections.emptyList()), + new Round(Source.EMPTY, DOUBLE1, LONG), + new Pow(Source.EMPTY, DOUBLE1, DOUBLE2), + DOUBLE1, + literal, + new Length(Source.EMPTY, literal), + new DateFormat(Source.EMPTY, DATE, datePattern, TEST_CONFIG), + new DateFormat(Source.EMPTY, literal, datePattern, TEST_CONFIG), + new StartsWith(Source.EMPTY, literal, literal), + new Substring(Source.EMPTY, literal, LONG, LONG), + new DateTrunc(Source.EMPTY, DATE, dateInterval) }) { + params.add(new Object[] { e.nodeString(), e }); + } + + return params; + } + + private final String nodeString; + private final Expression expression; + + public EvalMapperTests(String nodeString, Expression expression) { + this.nodeString = nodeString; + this.expression = expression; + } + + public void testEvaluatorSuppliers() { + Layout.Builder lb = new Layout.Builder(); + lb.appendChannel(DOUBLE1.id()); + lb.appendChannel(DOUBLE2.id()); + lb.appendChannel(DATE.id()); + lb.appendChannel(LONG.id()); + Layout layout = lb.build(); + + Supplier supplier = EvalMapper.toEvaluator(expression, layout); + EvalOperator.ExpressionEvaluator evaluator1 = supplier.get(); + EvalOperator.ExpressionEvaluator evaluator2 = supplier.get(); + assertNotNull(evaluator1); + assertNotNull(evaluator2); + assertTrue(evaluator1 != evaluator2); + } + + // Test serialization of expressions, since we have convenient access to some expressions. + public void testExpressionSerialization() { + SerializationTestUtils.assertSerialization(expression); + } + + private static FieldAttribute field(String name, DataType type) { + return new FieldAttribute(Source.EMPTY, name, new EsField(name, type, Collections.emptyMap(), false)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracterTests.java new file mode 100644 index 0000000000000..16bf25e4809f0 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/GrokEvaluatorExtracterTests.java @@ -0,0 +1,239 @@ +/* + * 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.planner; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BooleanBlock; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.grok.Grok; +import org.elasticsearch.grok.GrokBuiltinPatterns; +import org.elasticsearch.test.ESTestCase; + +import java.util.Map; + +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class GrokEvaluatorExtracterTests extends ESTestCase { + final Map KEY_TO_BLOCK = Map.of("a", 0, "b", 1, "c", 2, "d", 3, "e", 4, "f", 5); + final Map TYPES = Map.of( + "a", + ElementType.BYTES_REF, + "b", + ElementType.INT, + "c", + ElementType.LONG, + "d", + ElementType.DOUBLE, + "e", + ElementType.DOUBLE, + "f", + ElementType.BOOLEAN + ); + + public void testSingleValue() { + String pattern = "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean}"; + + GrokEvaluatorExtracter extracter = buildExtracter(pattern, KEY_TO_BLOCK, TYPES); + String[] input = { "foo 10 100 12.3 15.5 false", "wrong", "bar 20 200 14.3 16.5 true" }; + BytesRefBlock inputBlock = buildInputBlock(new int[] { 1, 1, 1 }, input); + Block.Builder[] targetBlocks = buidDefaultTargetBlocks(3); + for (int i = 0; i < input.length; i++) { + extracter.computeRow(inputBlock, i, targetBlocks, new BytesRef()); + } + + checkStringBlock(targetBlocks[0], new int[] { 1, 0, 1 }, "foo", "bar"); + checkIntBlock(targetBlocks[1], new int[] { 1, 0, 1 }, 10, 20); + checkLongBlock(targetBlocks[2], new int[] { 1, 0, 1 }, 100, 200); + checkDoubleBlock(targetBlocks[3], new int[] { 1, 0, 1 }, 12.3F, 14.3F); + checkDoubleBlock(targetBlocks[4], new int[] { 1, 0, 1 }, 15.5D, 16.5D); + checkBooleanBlock(targetBlocks[5], new int[] { 1, 0, 1 }, false, true); + } + + public void testMvPattern() { + String pattern = "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean} " + + "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean}"; + + GrokEvaluatorExtracter extracter = buildExtracter(pattern, KEY_TO_BLOCK, TYPES); + String[] input = { "foo 10 100 12.3 15.5 false bar 20 200 14.3 16.5 true" }; + BytesRefBlock inputBlock = buildInputBlock(new int[] { 1 }, input); + Block.Builder[] targetBlocks = buidDefaultTargetBlocks(1); + for (int i = 0; i < input.length; i++) { + extracter.computeRow(inputBlock, i, targetBlocks, new BytesRef()); + } + + checkStringBlock(targetBlocks[0], new int[] { 2 }, "foo", "bar"); + checkIntBlock(targetBlocks[1], new int[] { 2 }, 10, 20); + checkLongBlock(targetBlocks[2], new int[] { 2 }, 100, 200); + checkDoubleBlock(targetBlocks[3], new int[] { 2 }, 12.3F, 14.3F); + checkDoubleBlock(targetBlocks[4], new int[] { 2 }, 15.5D, 16.5D); + checkBooleanBlock(targetBlocks[5], new int[] { 2 }, false, true); + } + + public void testMvInput() { + String pattern = "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean}"; + + GrokEvaluatorExtracter extracter = buildExtracter(pattern, KEY_TO_BLOCK, TYPES); + String[] input = { "foo 10 100 12.3 15.5 false", "wrong", "bar 20 200 14.3 16.5 true", "baz 30 300 34.3 36.5 true" }; + BytesRefBlock inputBlock = buildInputBlock(new int[] { 3, 1 }, input); + Block.Builder[] targetBlocks = buidDefaultTargetBlocks(4); + for (int i = 0; i < input.length; i++) { + extracter.computeRow(inputBlock, i, targetBlocks, new BytesRef()); + } + + checkStringBlock(targetBlocks[0], new int[] { 2, 1 }, "foo", "bar", "baz"); + checkIntBlock(targetBlocks[1], new int[] { 2, 1 }, 10, 20, 30); + checkLongBlock(targetBlocks[2], new int[] { 2, 1 }, 100, 200, 300); + checkDoubleBlock(targetBlocks[3], new int[] { 2, 1 }, 12.3F, 14.3F, 34.3F); + checkDoubleBlock(targetBlocks[4], new int[] { 2, 1 }, 15.5D, 16.5D, 36.5D); + checkBooleanBlock(targetBlocks[5], new int[] { 2, 1 }, false, true, true); + } + + public void testMvInputAndPattern() { + String pattern = "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean} " + + "%{WORD:a} %{NUMBER:b:int} %{NUMBER:c:long} %{NUMBER:d:float} %{NUMBER:e:double} %{WORD:f:boolean}"; + + GrokEvaluatorExtracter extracter = buildExtracter(pattern, KEY_TO_BLOCK, TYPES); + String[] input = { + "foo 10 100 12.3 15.5 false bar 20 200 14.3 16.5 true", + "wrong", + "baz 30 300 34.3 36.5 true bax 80 800 84.3 86.5 false" }; + BytesRefBlock inputBlock = buildInputBlock(new int[] { 3 }, input); + Block.Builder[] targetBlocks = buidDefaultTargetBlocks(3); + for (int i = 0; i < input.length; i++) { + extracter.computeRow(inputBlock, i, targetBlocks, new BytesRef()); + } + + checkStringBlock(targetBlocks[0], new int[] { 4 }, "foo", "bar", "baz", "bax"); + checkIntBlock(targetBlocks[1], new int[] { 4 }, 10, 20, 30, 80); + checkLongBlock(targetBlocks[2], new int[] { 4 }, 100, 200, 300, 800); + checkDoubleBlock(targetBlocks[3], new int[] { 4 }, 12.3F, 14.3F, 34.3F, 84.3F); + checkDoubleBlock(targetBlocks[4], new int[] { 4 }, 15.5D, 16.5D, 36.5D, 86.5D); + checkBooleanBlock(targetBlocks[5], new int[] { 4 }, false, true, true, false); + } + + private void checkStringBlock(Block.Builder builder, int[] itemsPerRow, String... expectedValues) { + int nextString = 0; + assertThat(builder, instanceOf(BytesRefBlock.Builder.class)); + BytesRefBlock block = (BytesRefBlock) builder.build(); + BytesRef spare = new BytesRef(); + for (int i = 0; i < itemsPerRow.length; i++) { + int valueCount = block.getValueCount(i); + assertThat(valueCount, is(itemsPerRow[i])); + int firstPosition = block.getFirstValueIndex(i); + for (int j = 0; j < itemsPerRow[i]; j++) { + assertThat(block.getBytesRef(firstPosition + j, spare).utf8ToString(), is(expectedValues[nextString++])); + } + } + } + + private void checkIntBlock(Block.Builder builder, int[] itemsPerRow, int... expectedValues) { + int nextString = 0; + assertThat(builder, instanceOf(IntBlock.Builder.class)); + IntBlock block = (IntBlock) builder.build(); + for (int i = 0; i < itemsPerRow.length; i++) { + int valueCount = block.getValueCount(i); + assertThat(valueCount, is(itemsPerRow[i])); + int firstPosition = block.getFirstValueIndex(i); + for (int j = 0; j < itemsPerRow[i]; j++) { + assertThat(block.getInt(firstPosition + j), is(expectedValues[nextString++])); + } + } + } + + private void checkLongBlock(Block.Builder builder, int[] itemsPerRow, long... expectedValues) { + int nextString = 0; + assertThat(builder, instanceOf(LongBlock.Builder.class)); + LongBlock block = (LongBlock) builder.build(); + for (int i = 0; i < itemsPerRow.length; i++) { + int valueCount = block.getValueCount(i); + assertThat(valueCount, is(itemsPerRow[i])); + int firstPosition = block.getFirstValueIndex(i); + for (int j = 0; j < itemsPerRow[i]; j++) { + assertThat(block.getLong(firstPosition + j), is(expectedValues[nextString++])); + } + } + } + + private void checkDoubleBlock(Block.Builder builder, int[] itemsPerRow, double... expectedValues) { + int nextString = 0; + assertThat(builder, instanceOf(DoubleBlock.Builder.class)); + DoubleBlock block = (DoubleBlock) builder.build(); + for (int i = 0; i < itemsPerRow.length; i++) { + int valueCount = block.getValueCount(i); + assertThat(valueCount, is(itemsPerRow[i])); + int firstPosition = block.getFirstValueIndex(i); + for (int j = 0; j < itemsPerRow[i]; j++) { + assertThat(block.getDouble(firstPosition + j), is(expectedValues[nextString++])); + } + } + } + + private void checkBooleanBlock(Block.Builder builder, int[] itemsPerRow, boolean... expectedValues) { + int nextString = 0; + assertThat(builder, instanceOf(BooleanBlock.Builder.class)); + BooleanBlock block = (BooleanBlock) builder.build(); + for (int i = 0; i < itemsPerRow.length; i++) { + int valueCount = block.getValueCount(i); + assertThat(valueCount, is(itemsPerRow[i])); + int firstPosition = block.getFirstValueIndex(i); + for (int j = 0; j < itemsPerRow[i]; j++) { + assertThat(block.getBoolean(firstPosition + j), is(expectedValues[nextString++])); + } + } + } + + private BytesRefBlock buildInputBlock(int[] mvSize, String... input) { + int nextString = 0; + BytesRefBlock.Builder inputBuilder = BytesRefBlock.newBlockBuilder(input.length); + for (int i = 0; i < mvSize.length; i++) { + if (mvSize[i] == 0) { + inputBuilder.appendNull(); + } else if (mvSize[i] == 1) { + inputBuilder.appendBytesRef(new BytesRef(input[nextString++])); + } else { + inputBuilder.beginPositionEntry(); + for (int j = 0; j < mvSize[i]; j++) { + inputBuilder.appendBytesRef(new BytesRef(input[nextString++])); + } + inputBuilder.endPositionEntry(); + } + } + for (String s : input) { + if (s == null) { + inputBuilder.appendNull(); + } else { + inputBuilder.appendBytesRef(new BytesRef(s)); + } + } + return inputBuilder.build(); + } + + private Block.Builder[] buidDefaultTargetBlocks(int estimatedSize) { + return new Block.Builder[] { + BytesRefBlock.newBlockBuilder(estimatedSize), + IntBlock.newBlockBuilder(estimatedSize), + LongBlock.newBlockBuilder(estimatedSize), + DoubleBlock.newBlockBuilder(estimatedSize), + DoubleBlock.newBlockBuilder(estimatedSize), + BooleanBlock.newBlockBuilder(estimatedSize) }; + } + + private GrokEvaluatorExtracter buildExtracter(String pattern, Map keyToBlock, Map types) { + var builtinPatterns = GrokBuiltinPatterns.get(true); + Grok grok = new Grok(builtinPatterns, pattern, logger::warn); + GrokEvaluatorExtracter extracter = new GrokEvaluatorExtracter(grok, pattern, keyToBlock, types); + return extracter; + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java new file mode 100644 index 0000000000000..dab7b3ee41922 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java @@ -0,0 +1,169 @@ +/* + * 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.planner; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.mapper.MapperServiceTestCase; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.test.TestSearchContext; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataTypes; +import org.elasticsearch.xpack.ql.type.EsField; +import org.hamcrest.Matcher; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class LocalExecutionPlannerTests extends MapperServiceTestCase { + @ParametersFactory + public static Iterable parameters() throws Exception { + List params = new ArrayList<>(); + params.add(new Object[] { false }); + params.add(new Object[] { true }); + return params; + } + + private final QueryPragmas pragmas = new QueryPragmas(Settings.EMPTY); + private final boolean estimatedRowSizeIsHuge; + + private Directory directory = newDirectory(); + private IndexReader reader; + + public LocalExecutionPlannerTests(@Name("estimatedRowSizeIsHuge") boolean estimatedRowSizeIsHuge) { + this.estimatedRowSizeIsHuge = estimatedRowSizeIsHuge; + } + + @After + public void closeIndex() throws IOException { + IOUtils.close(reader, directory); + } + + public void testLuceneSourceOperatorHugeRowSize() throws IOException { + int estimatedRowSize = randomEstimatedRowSize(estimatedRowSizeIsHuge); + LocalExecutionPlanner.LocalExecutionPlan plan = planner().plan( + new EsQueryExec(Source.EMPTY, index(), List.of(), null, null, null, estimatedRowSize) + ); + LocalExecutionPlanner.DriverSupplier supplier = plan.driverFactories.get(0).driverSupplier(); + var factory = (LuceneSourceOperator.LuceneSourceOperatorFactory) supplier.physicalOperation().sourceOperatorFactory; + assertThat(factory.maxPageSize(), maxPageSizeMatcher(estimatedRowSizeIsHuge, estimatedRowSize)); + assertThat(factory.limit(), equalTo(Integer.MAX_VALUE)); + } + + public void testLuceneTopNSourceOperator() throws IOException { + int estimatedRowSize = randomEstimatedRowSize(estimatedRowSizeIsHuge); + FieldAttribute sortField = new FieldAttribute(Source.EMPTY, "field", new EsField("field", DataTypes.INTEGER, Map.of(), true)); + EsQueryExec.FieldSort sort = new EsQueryExec.FieldSort(sortField, Order.OrderDirection.ASC, Order.NullsPosition.LAST); + Literal limit = new Literal(Source.EMPTY, 10, DataTypes.INTEGER); + LocalExecutionPlanner.LocalExecutionPlan plan = planner().plan( + new EsQueryExec(Source.EMPTY, index(), List.of(), null, limit, List.of(sort), estimatedRowSize) + ); + LocalExecutionPlanner.DriverSupplier supplier = plan.driverFactories.get(0).driverSupplier(); + var factory = (LuceneTopNSourceOperator.LuceneTopNSourceOperatorFactory) supplier.physicalOperation().sourceOperatorFactory; + assertThat(factory.maxPageSize(), maxPageSizeMatcher(estimatedRowSizeIsHuge, estimatedRowSize)); + assertThat(factory.limit(), equalTo(10)); + } + + private int randomEstimatedRowSize(boolean huge) { + int hugeBoundary = SourceOperator.MIN_TARGET_PAGE_SIZE * 10; + return huge ? between(hugeBoundary, Integer.MAX_VALUE) : between(1, hugeBoundary); + } + + private Matcher maxPageSizeMatcher(boolean estimatedRowSizeIsHuge, int estimatedRowSize) { + if (estimatedRowSizeIsHuge) { + return equalTo(SourceOperator.MIN_TARGET_PAGE_SIZE); + } + return equalTo(SourceOperator.TARGET_PAGE_SIZE / estimatedRowSize); + } + + private LocalExecutionPlanner planner() throws IOException { + return new LocalExecutionPlanner( + "test", + null, + BigArrays.NON_RECYCLING_INSTANCE, + config(), + null, + null, + null, + esPhysicalOperationProviders() + ); + } + + private EsqlConfiguration config() { + return new EsqlConfiguration( + randomZone(), + randomLocale(random()), + "test_user", + "test_cluser", + pragmas, + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(null) + ); + } + + private EsPhysicalOperationProviders esPhysicalOperationProviders() throws IOException { + return new EsPhysicalOperationProviders(List.of(searchContext())); + } + + private SearchContext searchContext() throws IOException { + return new TestSearchContext(createSearchExecutionContext(createMapperService(mapping(b -> {})), new IndexSearcher(reader()))); + } + + private IndexReader reader() { + if (reader != null) { + return reader; + } + try ( + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + for (int d = 0; d < 10; d++) { + List doc = new ArrayList<>(); + doc.add(new SortedNumericDocValuesField("s", d)); + writer.addDocument(doc); + } + reader = writer.getReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return reader; + } + + private EsIndex index() { + return new EsIndex("test", Map.of()); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java new file mode 100644 index 0000000000000..c088cae6f20c9 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java @@ -0,0 +1,299 @@ +/* + * 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.planner; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.Describable; +import org.elasticsearch.compute.aggregation.GroupingAggregator; +import org.elasticsearch.compute.aggregation.blockhash.BlockHash; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.HashAggregationOperator; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; +import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.operator.SourceOperator.SourceOperatorFactory; +import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; +import org.elasticsearch.xpack.esql.plan.physical.AggregateExec; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlannerContext; +import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.PhysicalOperation; +import org.elasticsearch.xpack.ql.expression.Attribute; + +import java.util.List; +import java.util.Random; +import java.util.function.Supplier; + +import static com.carrotsearch.randomizedtesting.generators.RandomNumbers.randomIntBetween; +import static java.util.stream.Collectors.joining; + +public class TestPhysicalOperationProviders extends AbstractPhysicalOperationProviders { + + private final Page testData; + private final List columnNames; + + public TestPhysicalOperationProviders(Page testData, List columnNames) { + this.testData = testData; + this.columnNames = columnNames; + } + + @Override + public PhysicalOperation fieldExtractPhysicalOperation(FieldExtractExec fieldExtractExec, PhysicalOperation source) { + Layout.Builder layout = source.layout.builder(); + PhysicalOperation op = source; + for (Attribute attr : fieldExtractExec.attributesToExtract()) { + layout.appendChannel(attr.id()); + op = op.with(new TestFieldExtractOperatorFactory(attr.name()), layout.build()); + } + return op; + } + + @Override + public PhysicalOperation sourcePhysicalOperation(EsQueryExec esQueryExec, LocalExecutionPlannerContext context) { + Layout.Builder layout = new Layout.Builder(); + for (int i = 0; i < esQueryExec.output().size(); i++) { + layout.appendChannel(esQueryExec.output().get(i).id()); + } + return PhysicalOperation.fromSource(new TestSourceOperatorFactory(), layout.build()); + } + + @Override + public Operator.OperatorFactory ordinalGroupingOperatorFactory( + PhysicalOperation source, + AggregateExec aggregateExec, + List aggregatorFactories, + Attribute attrSource, + ElementType groupElementType, + LocalExecutionPlannerContext context + ) { + int channelIndex = source.layout.numberOfChannels(); + return new TestOrdinalsGroupingAggregationOperatorFactory( + channelIndex, + aggregatorFactories, + groupElementType, + context.bigArrays(), + attrSource.name() + ); + } + + private class TestSourceOperator extends SourceOperator { + + boolean finished = false; + + @Override + public Page getOutput() { + if (finished == false) { + finish(); + } + + Block[] fakeSourceAttributesBlocks = new Block[1]; + // a block that contains the position of each document as int + // will be used to "filter" and extract the block's values later on. Basically, a replacement for _doc, _shard and _segment ids + IntBlock.Builder docIndexBlockBuilder = IntBlock.newBlockBuilder(testData.getPositionCount()); + for (int i = 0; i < testData.getPositionCount(); i++) { + docIndexBlockBuilder.appendInt(i); + } + fakeSourceAttributesBlocks[0] = docIndexBlockBuilder.build(); // instead of _doc + Page newPageWithSourceAttributes = new Page(fakeSourceAttributesBlocks); + return newPageWithSourceAttributes; + } + + @Override + public boolean isFinished() { + return finished; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public void close() { + + } + } + + private class TestSourceOperatorFactory implements SourceOperatorFactory { + + SourceOperator op = new TestSourceOperator(); + + @Override + public SourceOperator get(DriverContext driverContext) { + return op; + } + + @Override + public String describe() { + return "TestSourceOperator"; + } + } + + private class TestFieldExtractOperator implements Operator { + + private Page lastPage; + boolean finished; + String columnName; + + TestFieldExtractOperator(String columnName) { + this.columnName = columnName; + } + + @Override + public void addInput(Page page) { + Block block = extractBlockForColumn(page, columnName); + lastPage = page.appendBlock(block); + } + + @Override + public Page getOutput() { + Page l = lastPage; + lastPage = null; + return l; + } + + @Override + public boolean isFinished() { + return finished && lastPage == null; + } + + @Override + public void finish() { + finished = true; + } + + @Override + public boolean needsInput() { + return lastPage == null; + } + + @Override + public void close() { + + } + } + + private class TestFieldExtractOperatorFactory implements Operator.OperatorFactory { + + final String columnName; + final Operator op; + + TestFieldExtractOperatorFactory(String columnName) { + this.columnName = columnName; + this.op = new TestFieldExtractOperator(columnName); + } + + @Override + public Operator get(DriverContext driverContext) { + return op; + } + + @Override + public String describe() { + return "TestFieldExtractOperator"; + } + } + + private class TestHashAggregationOperator extends HashAggregationOperator { + + private final String columnName; + + TestHashAggregationOperator( + List aggregators, + Supplier blockHash, + String columnName, + DriverContext driverContext + ) { + super(aggregators, blockHash, driverContext); + this.columnName = columnName; + } + + @Override + protected Page wrapPage(Page page) { + return page.appendBlock(extractBlockForColumn(page, columnName)); + } + } + + /** + * Pretends to be the {@link OrdinalsGroupingOperator} but always delegates to the + * {@link HashAggregationOperator}. + */ + private class TestOrdinalsGroupingAggregationOperatorFactory implements Operator.OperatorFactory { + private int groupByChannel; + private List aggregators; + private ElementType groupElementType; + private BigArrays bigArrays; + private String columnName; + + TestOrdinalsGroupingAggregationOperatorFactory( + int channelIndex, + List aggregatorFactories, + ElementType groupElementType, + BigArrays bigArrays, + String name + ) { + this.groupByChannel = channelIndex; + this.aggregators = aggregatorFactories; + this.groupElementType = groupElementType; + this.bigArrays = bigArrays; + this.columnName = name; + } + + @Override + public Operator get(DriverContext driverContext) { + Random random = Randomness.get(); + int pageSize = random.nextBoolean() ? randomIntBetween(random, 1, 16) : randomIntBetween(random, 1, 10 * 1024); + return new TestHashAggregationOperator( + aggregators, + () -> BlockHash.build( + List.of(new HashAggregationOperator.GroupSpec(groupByChannel, groupElementType)), + bigArrays, + pageSize + ), + columnName, + driverContext + ); + } + + @Override + public String describe() { + return "TestHashAggregationOperator(mode = " + + "" + + ", aggs = " + + aggregators.stream().map(Describable::describe).collect(joining(", ")) + + ")"; + } + } + + private Block extractBlockForColumn(Page page, String columnName) { + var columnIndex = -1; + // locate the block index corresponding to "columnName" + for (int i = 0, size = columnNames.size(); i < size && columnIndex < 0; i++) { + if (columnNames.get(i).equals(columnName)) { + columnIndex = i; + } + } + if (columnIndex < 0) { + throw new EsqlIllegalArgumentException("Cannot find column named [{}] in {}", columnName, columnNames); + } + // this is the first block added by TestSourceOperator + IntBlock docIndexBlock = page.getBlock(0); + // use its filtered position to extract the data needed for "columnName" block + Block loadedBlock = testData.getBlock(columnIndex); + int[] filteredPositions = new int[docIndexBlock.getPositionCount()]; + for (int c = 0; c < docIndexBlock.getPositionCount(); c++) { + filteredPositions[c] = (Integer) docIndexBlock.getInt(c); + } + return loadedBlock.filter(filteredPositions); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestTests.java new file mode 100644 index 0000000000000..fae2a1caeab4c --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestTests.java @@ -0,0 +1,196 @@ +/* + * 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.plugin; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.analysis.Analyzer; +import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; +import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerContext; +import org.elasticsearch.xpack.esql.optimizer.PhysicalPlanOptimizer; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.planner.Mapper; +import org.elasticsearch.xpack.esql.session.EsqlConfiguration; +import org.elasticsearch.xpack.esql.session.EsqlConfigurationSerializationTests; +import org.elasticsearch.xpack.esql.stats.Metrics; +import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry; +import org.elasticsearch.xpack.ql.index.EsIndex; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.io.IOException; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.xpack.esql.EsqlTestUtils.emptyPolicyResolution; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.loadMapping; + +public class DataNodeRequestTests extends AbstractWireSerializingTestCase { + + @Override + protected Writeable.Reader instanceReader() { + return DataNodeRequest::new; + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + SearchModule searchModule = new SearchModule(Settings.EMPTY, List.of()); + return new NamedWriteableRegistry(searchModule.getNamedWriteables()); + } + + @Override + protected DataNodeRequest createTestInstance() { + var sessionId = randomAlphaOfLength(10); + String query = randomFrom(""" + from test + | where round(emp_no) > 10 + | eval c = salary + | stats x = avg(c) + """, """ + from test + | sort last_name + | limit 10 + | where round(emp_no) > 10 + | eval c = first_name + | stats x = avg(salary) + """); + List shardIds = randomList(1, 10, () -> new ShardId("index-" + between(1, 10), "n/a", between(1, 10))); + PhysicalPlan physicalPlan = mapAndMaybeOptimize(parse(query)); + Map aliasFilters = Map.of( + new Index("concrete-index", "n/a"), + AliasFilter.of(new TermQueryBuilder("id", "1"), "alias-1") + ); + DataNodeRequest request = new DataNodeRequest( + sessionId, + EsqlConfigurationSerializationTests.randomConfiguration(), + shardIds, + aliasFilters, + physicalPlan + ); + request.setParentTask(randomAlphaOfLength(10), randomNonNegativeLong()); + return request; + } + + @Override + protected DataNodeRequest mutateInstance(DataNodeRequest in) throws IOException { + return switch (between(0, 5)) { + case 0 -> { + var request = new DataNodeRequest(randomAlphaOfLength(20), in.configuration(), in.shardIds(), in.aliasFilters(), in.plan()); + request.setParentTask(in.getParentTask()); + yield request; + } + case 1 -> { + var request = new DataNodeRequest( + in.sessionId(), + EsqlConfigurationSerializationTests.randomConfiguration(), + in.shardIds(), + in.aliasFilters(), + in.plan() + ); + request.setParentTask(in.getParentTask()); + yield request; + } + case 2 -> { + List shardIds = randomList(1, 10, () -> new ShardId("new-index-" + between(1, 10), "n/a", between(1, 10))); + var request = new DataNodeRequest(in.sessionId(), in.configuration(), shardIds, in.aliasFilters(), in.plan()); + request.setParentTask(in.getParentTask()); + yield request; + } + case 3 -> { + String newQuery = randomFrom(""" + from test + | where round(emp_no) > 100 + | eval c = salary + | stats x = avg(c) + """, """ + from test + | sort last_name + | limit 10 + | where round(emp_no) > 100 + | eval c = first_name + | stats x = avg(salary) + """); + var request = new DataNodeRequest( + in.sessionId(), + in.configuration(), + in.shardIds(), + in.aliasFilters(), + mapAndMaybeOptimize(parse(newQuery)) + ); + request.setParentTask(in.getParentTask()); + yield request; + } + case 4 -> { + final Map aliasFilters; + if (randomBoolean()) { + aliasFilters = Map.of(); + } else { + aliasFilters = Map.of(new Index("concrete-index", "n/a"), AliasFilter.of(new TermQueryBuilder("id", "2"), "alias-2")); + } + var request = new DataNodeRequest(in.sessionId(), in.configuration(), in.shardIds(), aliasFilters, in.plan()); + request.setParentTask(request.getParentTask()); + yield request; + } + case 5 -> { + var request = new DataNodeRequest(in.sessionId(), in.configuration(), in.shardIds(), in.aliasFilters(), in.plan()); + request.setParentTask( + randomValueOtherThan(request.getParentTask().getNodeId(), () -> randomAlphaOfLength(10)), + randomNonNegativeLong() + ); + yield request; + } + default -> throw new AssertionError("invalid value"); + }; + } + + static LogicalPlan parse(String query) { + Map mapping = loadMapping("mapping-basic.json"); + EsIndex test = new EsIndex("test", mapping); + IndexResolution getIndexResult = IndexResolution.valid(test); + var logicalOptimizer = new LogicalPlanOptimizer(); + var analyzer = new Analyzer( + new AnalyzerContext(EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), getIndexResult, emptyPolicyResolution()), + new Verifier(new Metrics()) + ); + return logicalOptimizer.optimize(analyzer.analyze(new EsqlParser().createStatement(query))); + } + + static PhysicalPlan mapAndMaybeOptimize(LogicalPlan logicalPlan) { + var configuration = new EsqlConfiguration( + ZoneOffset.UTC, + Locale.US, + null, + null, + new QueryPragmas(Settings.EMPTY), + EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(Settings.EMPTY) + ); + var physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); + FunctionRegistry functionRegistry = new EsqlFunctionRegistry(); + var mapper = new Mapper(functionRegistry); + var physical = mapper.map(logicalPlan); + if (randomBoolean()) { + physical = physicalPlanOptimizer.optimize(physical); + } + return physical; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParserTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParserTests.java new file mode 100644 index 0000000000000..789d6e5adbfc7 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/EsqlMediaTypeParserTests.java @@ -0,0 +1,116 @@ +/* + * 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.plugin; + +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.xcontent.MediaType; +import org.elasticsearch.xcontent.NamedXContentRegistry; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; + +import java.util.Collections; +import java.util.Map; + +import static org.elasticsearch.xpack.esql.formatter.TextFormat.CSV; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.PLAIN_TEXT; +import static org.elasticsearch.xpack.esql.formatter.TextFormat.TSV; +import static org.elasticsearch.xpack.esql.plugin.EsqlMediaTypeParser.getResponseMediaType; +import static org.hamcrest.CoreMatchers.is; + +public class EsqlMediaTypeParserTests extends ESTestCase { + + public void testPlainTextDetection() { + MediaType text = getResponseMediaType(reqWithAccept("text/plain"), createTestInstance(false)); + assertThat(text, is(PLAIN_TEXT)); + } + + public void testCsvDetection() { + MediaType text = getResponseMediaType(reqWithAccept("text/csv"), createTestInstance(false)); + assertThat(text, is(CSV)); + + text = getResponseMediaType(reqWithAccept("text/csv; delimiter=x"), createTestInstance(false)); + assertThat(text, is(CSV)); + } + + public void testTsvDetection() { + MediaType text = getResponseMediaType(reqWithAccept("text/tab-separated-values"), createTestInstance(false)); + assertThat(text, is(TSV)); + } + + public void testMediaTypeDetectionWithParameters() { + assertThat(getResponseMediaType(reqWithAccept("text/plain; charset=utf-8"), createTestInstance(false)), is(PLAIN_TEXT)); + assertThat(getResponseMediaType(reqWithAccept("text/plain; header=present"), createTestInstance(false)), is(PLAIN_TEXT)); + assertThat( + getResponseMediaType(reqWithAccept("text/plain; charset=utf-8; header=present"), createTestInstance(false)), + is(PLAIN_TEXT) + ); + + assertThat(getResponseMediaType(reqWithAccept("text/csv; charset=utf-8"), createTestInstance(false)), is(CSV)); + assertThat(getResponseMediaType(reqWithAccept("text/csv; header=present"), createTestInstance(false)), is(CSV)); + assertThat(getResponseMediaType(reqWithAccept("text/csv; charset=utf-8; header=present"), createTestInstance(false)), is(CSV)); + + assertThat(getResponseMediaType(reqWithAccept("text/tab-separated-values; charset=utf-8"), createTestInstance(false)), is(TSV)); + assertThat(getResponseMediaType(reqWithAccept("text/tab-separated-values; header=present"), createTestInstance(false)), is(TSV)); + assertThat( + getResponseMediaType(reqWithAccept("text/tab-separated-values; charset=utf-8; header=present"), createTestInstance(false)), + is(TSV) + ); + } + + public void testInvalidFormat() { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> getResponseMediaType(reqWithAccept("text/garbage"), createTestInstance(false)) + ); + assertEquals(e.getMessage(), "Invalid request content type: Accept=[text/garbage], Content-Type=[application/json], format=[null]"); + } + + public void testColumnarWithAcceptText() { + var accept = randomFrom("text/plain", "text/csv", "text/tab-separated-values"); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> getResponseMediaType(reqWithAccept(accept), createTestInstance(true)) + ); + assertEquals(e.getMessage(), "Invalid use of [columnar] argument: cannot be used in combination with [txt, csv, tsv] formats"); + } + + public void testColumnarWithParamText() { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> getResponseMediaType(reqWithParams(Map.of("format", randomFrom("txt", "csv", "tsv"))), createTestInstance(true)) + ); + assertEquals(e.getMessage(), "Invalid use of [columnar] argument: cannot be used in combination with [txt, csv, tsv] formats"); + } + + public void testNoFormat() { + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> getResponseMediaType(new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).build(), createTestInstance(false)) + ); + assertEquals(e.getMessage(), "Invalid request content type: Accept=[null], Content-Type=[null], format=[null]"); + } + + private static RestRequest reqWithAccept(String acceptHeader) { + return new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withHeaders( + Map.of("Content-Type", Collections.singletonList("application/json"), "Accept", Collections.singletonList(acceptHeader)) + ).build(); + } + + private static RestRequest reqWithParams(Map params) { + return new FakeRestRequest.Builder(NamedXContentRegistry.EMPTY).withHeaders( + Map.of("Content-Type", Collections.singletonList("application/json")) + ).withParams(params).build(); + } + + protected EsqlQueryRequest createTestInstance(boolean columnar) { + var request = new EsqlQueryRequest(); + request.columnar(columnar); + return request; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryNegateTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryNegateTests.java new file mode 100644 index 0000000000000..8a3baebb3da35 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryNegateTests.java @@ -0,0 +1,32 @@ +/* + * 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.querydsl.query; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.ql.querydsl.query.MatchAll; +import org.elasticsearch.xpack.ql.querydsl.query.NotQuery; +import org.elasticsearch.xpack.ql.tree.Source; + +import static org.hamcrest.Matchers.equalTo; + +/** + * Assertions that negating {@link SingleValueQuery} spits out the classes + * we expect. See {@link SingleValueQueryTests} for tests that it matches + * the docs we expect. + */ +public class SingleValueQueryNegateTests extends ESTestCase { + public void testNot() { + var sv = new SingleValueQuery(new MatchAll(Source.EMPTY), "foo"); + assertThat(sv.negate(Source.EMPTY), equalTo(new SingleValueQuery(new NotQuery(Source.EMPTY, new MatchAll(Source.EMPTY)), "foo"))); + } + + public void testNotNot() { + var sv = new SingleValueQuery(new MatchAll(Source.EMPTY), "foo"); + assertThat(sv.negate(Source.EMPTY).negate(Source.EMPTY), equalTo(sv)); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuerySerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuerySerializationTests.java new file mode 100644 index 0000000000000..06c9febec324a --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQuerySerializationTests.java @@ -0,0 +1,60 @@ +/* + * 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.querydsl.query; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + +import java.util.List; + +public class SingleValueQuerySerializationTests extends AbstractWireSerializingTestCase { + @Override + protected SingleValueQuery.Builder createTestInstance() { + return new SingleValueQuery.Builder(randomQuery(), randomFieldName(), new SingleValueQuery.Stats()); + } + + private QueryBuilder randomQuery() { + return new TermQueryBuilder(randomAlphaOfLength(1), randomAlphaOfLength(2)); + } + + private String randomFieldName() { + return randomAlphaOfLength(3); + } + + @Override + protected SingleValueQuery.Builder mutateInstance(SingleValueQuery.Builder instance) { + return switch (between(0, 1)) { + case 0 -> new SingleValueQuery.Builder( + randomValueOtherThan(instance.next(), this::randomQuery), + instance.field(), + new SingleValueQuery.Stats() + ); + case 1 -> new SingleValueQuery.Builder( + instance.next(), + randomValueOtherThan(instance.field(), this::randomFieldName), + new SingleValueQuery.Stats() + ); + default -> throw new IllegalArgumentException(); + }; + } + + @Override + protected Writeable.Reader instanceReader() { + return SingleValueQuery.Builder::new; + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry( + List.of(new NamedWriteableRegistry.Entry(QueryBuilder.class, TermQueryBuilder.NAME, TermQueryBuilder::new)) + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryTests.java new file mode 100644 index 0000000000000..cc5b05537c4c6 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/SingleValueQueryTests.java @@ -0,0 +1,355 @@ +/* + * 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.querydsl.query; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.apache.lucene.document.DoubleField; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.KeywordField; +import org.apache.lucene.document.LongField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MapperServiceTestCase; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; +import org.elasticsearch.index.query.MatchPhraseQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.ql.querydsl.query.MatchAll; +import org.elasticsearch.xpack.ql.querydsl.query.RangeQuery; +import org.elasticsearch.xpack.ql.tree.Source; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; + +public class SingleValueQueryTests extends MapperServiceTestCase { + interface Setup { + XContentBuilder mapping(XContentBuilder builder) throws IOException; + + List> build(RandomIndexWriter iw) throws IOException; + + void assertStats(SingleValueQuery.Builder builder, boolean subHasTwoPhase); + } + + @ParametersFactory + public static List params() { + List params = new ArrayList<>(); + for (String fieldType : new String[] { "long", "integer", "short", "byte", "double", "float", "keyword" }) { + params.add(new Object[] { new StandardSetup(fieldType, false) }); + params.add(new Object[] { new StandardSetup(fieldType, true) }); + } + params.add(new Object[] { new FieldMissingSetup() }); + return params; + } + + private final Setup setup; + + public SingleValueQueryTests(Setup setup) { + this.setup = setup; + } + + public void testMatchAll() throws IOException { + testCase( + new SingleValueQuery(new MatchAll(Source.EMPTY), "foo").asBuilder(), + false, + false, + (fieldValues, count) -> assertThat(count, equalTo((int) fieldValues.stream().filter(l -> l.size() == 1).count())) + ); + } + + public void testMatchSome() throws IOException { + int max = between(1, 100); + testCase( + new SingleValueQuery.Builder(new RangeQueryBuilder("i").lt(max), "foo", new SingleValueQuery.Stats()), + false, + false, + (fieldValues, count) -> { + int expected = 0; + for (int i = 0; i < max; i++) { + if (fieldValues.get(i).size() == 1) { + expected++; + } + } + assertThat(count, equalTo(expected)); + } + ); + } + + public void testSubPhrase() throws IOException { + testCase( + new SingleValueQuery.Builder(new MatchPhraseQueryBuilder("str", "fox jumped"), "foo", new SingleValueQuery.Stats()), + false, + true, + (fieldValues, count) -> assertThat(count, equalTo((int) fieldValues.stream().filter(l -> l.size() == 1).count())) + ); + } + + public void testMatchNone() throws IOException { + testCase( + new SingleValueQuery.Builder(new MatchNoneQueryBuilder(), "foo", new SingleValueQuery.Stats()), + true, + false, + (fieldValues, count) -> assertThat(count, equalTo(0)) + ); + } + + public void testRewritesToMatchNone() throws IOException { + testCase( + new SingleValueQuery.Builder(new TermQueryBuilder("missing", 0), "foo", new SingleValueQuery.Stats()), + true, + false, + (fieldValues, count) -> assertThat(count, equalTo(0)) + ); + } + + public void testNotMatchAll() throws IOException { + testCase( + new SingleValueQuery(new MatchAll(Source.EMPTY), "foo").negate(Source.EMPTY).asBuilder(), + true, + false, + (fieldValues, count) -> assertThat(count, equalTo(0)) + ); + } + + public void testNotMatchNone() throws IOException { + testCase( + new SingleValueQuery(new MatchAll(Source.EMPTY).negate(Source.EMPTY), "foo").negate(Source.EMPTY).asBuilder(), + false, + false, + (fieldValues, count) -> assertThat(count, equalTo((int) fieldValues.stream().filter(l -> l.size() == 1).count())) + ); + } + + public void testNotMatchSome() throws IOException { + int max = between(1, 100); + testCase( + new SingleValueQuery(new RangeQuery(Source.EMPTY, "i", null, false, max, false, null), "foo").negate(Source.EMPTY).asBuilder(), + false, + true, + (fieldValues, count) -> { + int expected = 0; + for (int i = max; i < 100; i++) { + if (fieldValues.get(i).size() == 1) { + expected++; + } + } + assertThat(count, equalTo(expected)); + } + ); + } + + @FunctionalInterface + interface TestCase { + void run(List> fieldValues, int count) throws IOException; + } + + private void testCase(SingleValueQuery.Builder builder, boolean rewritesToMatchNone, boolean subHasTwoPhase, TestCase testCase) + throws IOException { + MapperService mapper = createMapperService(mapping(setup::mapping)); + try (Directory d = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), d)) { + List> fieldValues = setup.build(iw); + try (IndexReader reader = iw.getReader()) { + SearchExecutionContext ctx = createSearchExecutionContext(mapper, new IndexSearcher(reader)); + QueryBuilder rewritten = builder.rewrite(ctx); + Query query = rewritten.toQuery(ctx); + testCase.run(fieldValues, ctx.searcher().count(query)); + if (rewritesToMatchNone) { + assertThat(rewritten, instanceOf(MatchNoneQueryBuilder.class)); + assertThat(builder.stats().missingField(), equalTo(0)); + assertThat(builder.stats().rewrittenToMatchNone(), equalTo(1)); + assertThat(builder.stats().numericSingle(), equalTo(0)); + assertThat(builder.stats().numericMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().numericMultiApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsSingle(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiApprox(), equalTo(0)); + assertThat(builder.stats().bytesApprox(), equalTo(0)); + assertThat(builder.stats().bytesNoApprox(), equalTo(0)); + } else { + assertThat(builder.stats().rewrittenToMatchNone(), equalTo(0)); + setup.assertStats(builder, subHasTwoPhase); + } + assertThat(builder.stats().noNextScorer(), equalTo(0)); + } + } + } + + private record StandardSetup(String fieldType, boolean multivaluedField) implements Setup { + @Override + public XContentBuilder mapping(XContentBuilder builder) throws IOException { + builder.startObject("i").field("type", "long").endObject(); + builder.startObject("str").field("type", "text").endObject(); + return builder.startObject("foo").field("type", fieldType).endObject(); + } + + @Override + public List> build(RandomIndexWriter iw) throws IOException { + List> fieldValues = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + // i == 10 forces at least one multivalued field when we're configured for multivalued fields + boolean makeMultivalued = multivaluedField && (i == 10 || randomBoolean()); + List values; + if (makeMultivalued) { + int count = between(2, 10); + Set set = new HashSet<>(count); + while (set.size() < count) { + set.add(randomValue()); + } + values = List.copyOf(set); + } else { + values = List.of(randomValue()); + } + fieldValues.add(values); + iw.addDocument(docFor(i, values)); + } + + return fieldValues; + } + + private Object randomValue() { + return switch (fieldType) { + case "long" -> randomLong(); + case "integer" -> randomInt(); + case "short" -> randomShort(); + case "byte" -> randomByte(); + case "double" -> randomDouble(); + case "float" -> randomFloat(); + case "keyword" -> randomAlphaOfLength(5); + default -> throw new UnsupportedOperationException(); + }; + } + + private List docFor(int i, Iterable values) { + List fields = new ArrayList<>(); + fields.add(new LongField("i", i)); + fields.add(new TextField("str", "the quick brown fox jumped over the lazy dog", Field.Store.NO)); + switch (fieldType) { + case "long", "integer", "short", "byte" -> { + for (Object v : values) { + long l = ((Number) v).longValue(); + fields.add(new LongField("foo", l, Field.Store.NO)); + } + } + case "double", "float" -> { + for (Object v : values) { + double d = ((Number) v).doubleValue(); + fields.add(new DoubleField("foo", d, Field.Store.NO)); + } + } + case "keyword" -> { + for (Object v : values) { + fields.add(new KeywordField("foo", v.toString(), Field.Store.NO)); + } + } + default -> throw new UnsupportedOperationException(); + } + return fields; + } + + @Override + public void assertStats(SingleValueQuery.Builder builder, boolean subHasTwoPhase) { + assertThat(builder.stats().missingField(), equalTo(0)); + switch (fieldType) { + case "long", "integer", "short", "byte", "double", "float" -> { + assertThat(builder.stats().ordinalsSingle(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiApprox(), equalTo(0)); + assertThat(builder.stats().bytesApprox(), equalTo(0)); + assertThat(builder.stats().bytesNoApprox(), equalTo(0)); + + if (multivaluedField) { + assertThat(builder.stats().numericSingle(), greaterThanOrEqualTo(0)); + if (subHasTwoPhase) { + assertThat(builder.stats().numericMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().numericMultiApprox(), greaterThan(0)); + } else { + assertThat(builder.stats().numericMultiNoApprox(), greaterThan(0)); + assertThat(builder.stats().numericMultiApprox(), equalTo(0)); + } + } else { + assertThat(builder.stats().numericSingle(), greaterThan(0)); + assertThat(builder.stats().numericMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().numericMultiApprox(), equalTo(0)); + } + } + case "keyword" -> { + assertThat(builder.stats().numericSingle(), equalTo(0)); + assertThat(builder.stats().numericMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().numericMultiApprox(), equalTo(0)); + assertThat(builder.stats().bytesApprox(), equalTo(0)); + assertThat(builder.stats().bytesNoApprox(), equalTo(0)); + if (multivaluedField) { + assertThat(builder.stats().ordinalsSingle(), greaterThanOrEqualTo(0)); + if (subHasTwoPhase) { + assertThat(builder.stats().ordinalsMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiApprox(), greaterThan(0)); + } else { + assertThat(builder.stats().ordinalsMultiNoApprox(), greaterThan(0)); + assertThat(builder.stats().ordinalsMultiApprox(), equalTo(0)); + } + } else { + assertThat(builder.stats().ordinalsSingle(), greaterThan(0)); + assertThat(builder.stats().ordinalsMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiApprox(), equalTo(0)); + } + } + default -> throw new UnsupportedOperationException(); + } + } + } + + private record FieldMissingSetup() implements Setup { + @Override + public XContentBuilder mapping(XContentBuilder builder) throws IOException { + builder.startObject("str").field("type", "text").endObject(); + return builder.startObject("i").field("type", "long").endObject(); + } + + @Override + public List> build(RandomIndexWriter iw) throws IOException { + List> fieldValues = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + iw.addDocument( + List.of(new LongField("i", i), new TextField("str", "the quick brown fox jumped over the lazy dog", Field.Store.NO)) + ); + fieldValues.add(List.of()); + } + return fieldValues; + } + + @Override + public void assertStats(SingleValueQuery.Builder builder, boolean subHasTwoPhase) { + assertThat(builder.stats().missingField(), equalTo(1)); + assertThat(builder.stats().numericSingle(), equalTo(0)); + assertThat(builder.stats().numericMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().numericMultiApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsSingle(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiNoApprox(), equalTo(0)); + assertThat(builder.stats().ordinalsMultiApprox(), equalTo(0)); + assertThat(builder.stats().bytesApprox(), equalTo(0)); + assertThat(builder.stats().bytesNoApprox(), equalTo(0)); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlConfigurationSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlConfigurationSerializationTests.java new file mode 100644 index 0000000000000..bc1146c492a73 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/EsqlConfigurationSerializationTests.java @@ -0,0 +1,60 @@ +/* + * 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.session; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.compute.lucene.DataPartitioning; +import org.elasticsearch.test.AbstractWireSerializingTestCase; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; + +import java.io.IOException; + +public class EsqlConfigurationSerializationTests extends AbstractWireSerializingTestCase { + + @Override + protected Writeable.Reader instanceReader() { + return EsqlConfiguration::new; + } + + private static QueryPragmas randomQueryPragmas() { + return new QueryPragmas( + Settings.builder().put(QueryPragmas.DATA_PARTITIONING.getKey(), randomFrom(DataPartitioning.values())).build() + ); + } + + public static EsqlConfiguration randomConfiguration() { + var zoneId = randomZone(); + var locale = randomLocale(random()); + var username = randomAlphaOfLengthBetween(1, 10); + var clusterName = randomAlphaOfLengthBetween(3, 10); + var truncation = randomNonNegativeInt(); + + return new EsqlConfiguration(zoneId, locale, username, clusterName, randomQueryPragmas(), truncation); + } + + @Override + protected EsqlConfiguration createTestInstance() { + return randomConfiguration(); + } + + @Override + protected EsqlConfiguration mutateInstance(EsqlConfiguration in) throws IOException { + int ordinal = between(0, 5); + return new EsqlConfiguration( + ordinal == 0 ? randomValueOtherThan(in.zoneId(), () -> randomZone().normalized()) : in.zoneId(), + ordinal == 1 ? randomValueOtherThan(in.locale(), () -> randomLocale(random())) : in.locale(), + ordinal == 2 ? randomAlphaOfLength(15) : in.username(), + ordinal == 3 ? randomAlphaOfLength(15) : in.clusterName(), + ordinal == 4 + ? new QueryPragmas(Settings.builder().put(QueryPragmas.EXCHANGE_BUFFER_SIZE.getKey(), between(1, 10)).build()) + : in.pragmas(), + ordinal == 5 ? in.resultTruncationMaxSize() + randomIntBetween(3, 10) : in.resultTruncationMaxSize() + ); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/DisabledSearchStats.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/DisabledSearchStats.java new file mode 100644 index 0000000000000..72c255d5e5388 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/DisabledSearchStats.java @@ -0,0 +1,50 @@ +/* + * 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.stats; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.xpack.ql.type.DataType; + +import static java.util.Collections.emptyList; + +public class DisabledSearchStats extends SearchStats { + + public DisabledSearchStats() { + super(emptyList()); + } + + @Override + public long count() { + return -1; + } + + @Override + public long count(String field) { + return -1; + } + + @Override + public long count(String field, BytesRef value) { + return -1; + } + + @Override + public boolean exists(String field) { + return true; + } + + @Override + public byte[] min(String field, DataType dataType) { + return null; + } + + @Override + public byte[] max(String field, DataType dataType) { + return null; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java new file mode 100644 index 0000000000000..87aea66d1eeb7 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java @@ -0,0 +1,118 @@ +/* + * 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.stats; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.fieldcaps.FieldCapabilities; +import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.esql.EsqlTestUtils; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.analysis.VerificationException; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolver; +import org.elasticsearch.xpack.esql.execution.PlanExecutor; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeRegistry; +import org.elasticsearch.xpack.ql.index.IndexResolver; +import org.junit.After; +import org.junit.Before; +import org.mockito.stubbing.Answer; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonMap; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PlanExecutorMetricsTests extends ESTestCase { + + private ThreadPool threadPool; + + @Before + public void setUpThreadPool() throws Exception { + threadPool = new TestThreadPool(PlanExecutorMetricsTests.class.getSimpleName()); + } + + @After + public void shutdownThreadPool() throws Exception { + terminate(threadPool); + } + + public void testFailedMetric() { + Client client = mock(Client.class); + IndexResolver idxResolver = new IndexResolver(client, randomAlphaOfLength(10), EsqlDataTypeRegistry.INSTANCE, Set::of); + var planExecutor = new PlanExecutor(idxResolver, new EnrichPolicyResolver(null, idxResolver, threadPool)); + String[] indices = new String[] { "test" }; + + // simulate a valid field_caps response so we can parse and correctly analyze de query + FieldCapabilitiesResponse fieldCapabilitiesResponse = mock(FieldCapabilitiesResponse.class); + when(fieldCapabilitiesResponse.getIndices()).thenReturn(indices); + when(fieldCapabilitiesResponse.get()).thenReturn(fields(indices)); + doAnswer((Answer) invocation -> { + @SuppressWarnings("unchecked") + ActionListener listener = (ActionListener) invocation.getArguments()[1]; + listener.onResponse(fieldCapabilitiesResponse); + return null; + }).when(client).fieldCaps(any(), any()); + + var request = new EsqlQueryRequest(); + // test a failed query: xyz field doesn't exist + request.query("from test | stats m = max(xyz)"); + planExecutor.esql(request, randomAlphaOfLength(10), EsqlTestUtils.TEST_CFG, new ActionListener() { + @Override + public void onResponse(PhysicalPlan physicalPlan) { + fail("this shouldn't happen"); + } + + @Override + public void onFailure(Exception e) { + assertThat(e, instanceOf(VerificationException.class)); + } + }); + + // check we recorded the failure and that the query actually came + assertEquals(1, planExecutor.metrics().stats().get("queries._all.failed")); + assertEquals(1, planExecutor.metrics().stats().get("queries._all.total")); + assertEquals(0, planExecutor.metrics().stats().get("features.stats")); + + // fix the failing query: foo field does exist + request.query("from test | stats m = max(foo)"); + planExecutor.esql(request, randomAlphaOfLength(10), EsqlTestUtils.TEST_CFG, new ActionListener() { + @Override + public void onResponse(PhysicalPlan physicalPlan) {} + + @Override + public void onFailure(Exception e) { + fail("this shouldn't happen"); + } + }); + + // check the new metrics + assertEquals(1, planExecutor.metrics().stats().get("queries._all.failed")); + assertEquals(2, planExecutor.metrics().stats().get("queries._all.total")); + assertEquals(1, planExecutor.metrics().stats().get("features.stats")); + } + + private Map> fields(String[] indices) { + FieldCapabilities fooField = new FieldCapabilities("foo", "integer", false, true, true, indices, null, null, emptyMap()); + FieldCapabilities barField = new FieldCapabilities("bar", "long", false, true, true, indices, null, null, emptyMap()); + Map> fields = new HashMap<>(); + fields.put(fooField.getName(), singletonMap(fooField.getName(), fooField)); + fields.put(barField.getName(), singletonMap(barField.getName(), barField)); + return fields; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/VerifierMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/VerifierMetricsTests.java new file mode 100644 index 0000000000000..08dd7174c8eaa --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/VerifierMetricsTests.java @@ -0,0 +1,199 @@ +/* + * 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.stats; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.watcher.common.stats.Counters; +import org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils; +import org.elasticsearch.xpack.esql.analysis.Verifier; +import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.ql.index.IndexResolution; + +import static org.elasticsearch.xpack.esql.analysis.AnalyzerTestUtils.analyzer; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.DISSECT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.EVAL; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.GROK; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.LIMIT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.SORT; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.STATS; +import static org.elasticsearch.xpack.esql.stats.FeatureMetric.WHERE; +import static org.elasticsearch.xpack.esql.stats.Metrics.FPREFIX; + +public class VerifierMetricsTests extends ESTestCase { + + private EsqlParser parser = new EsqlParser(); + + public void testDissectQuery() { + Counters c = esql("from employees | dissect concat(first_name, \" \", last_name) \"%{a} %{b}\""); + assertEquals(1L, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(0, stats(c)); + assertEquals(0, where(c)); + } + + public void testEvalQuery() { + Counters c = esql("from employees | eval name_len = length(first_name)"); + assertEquals(0, dissect(c)); + assertEquals(1L, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(0, stats(c)); + assertEquals(0, where(c)); + } + + public void testGrokQuery() { + Counters c = esql("from employees | grok concat(first_name, \" \", last_name) \"%{WORD:a} %{WORD:b}\""); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(1L, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(0, stats(c)); + assertEquals(0, where(c)); + } + + public void testLimitQuery() { + Counters c = esql("from employees | limit 2"); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(0, stats(c)); + assertEquals(0, where(c)); + } + + public void testSortQuery() { + Counters c = esql("from employees | sort first_name desc nulls first"); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(1L, sort(c)); + assertEquals(0, stats(c)); + assertEquals(0, where(c)); + } + + public void testStatsQuery() { + Counters c = esql("from employees | stats l = max(languages)"); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(1L, stats(c)); + assertEquals(0, where(c)); + } + + public void testWhereQuery() { + Counters c = esql("from employees | where languages > 2"); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(0, sort(c)); + assertEquals(0, stats(c)); + assertEquals(1L, where(c)); + } + + public void testTwoWhereQuery() { + Counters c = esql("from employees | where languages > 2 | limit 5 | sort first_name | where first_name == \"George\""); + assertEquals(0, dissect(c)); + assertEquals(0, eval(c)); + assertEquals(0, grok(c)); + assertEquals(1L, limit(c)); + assertEquals(1L, sort(c)); + assertEquals(0, stats(c)); + assertEquals(1L, where(c)); + } + + public void testTwoQueriesExecuted() { + Metrics metrics = new Metrics(); + Verifier verifier = new Verifier(metrics); + esqlWithVerifier(""" + from employees + | where languages > 2 + | limit 5 + | eval name_len = length(first_name) + | sort first_name + | limit 3 + """, verifier); + esqlWithVerifier(""" + from employees + | where languages > 2 + | sort first_name desc nulls first + | dissect concat(first_name, " ", last_name) "%{a} %{b}" + | grok concat(first_name, " ", last_name) "%{WORD:a} %{WORD:b}" + | stats x = max(languages) + | sort x + | stats y = min(x) by x + """, verifier); + Counters c = metrics.stats(); + assertEquals(1L, dissect(c)); + assertEquals(1L, eval(c)); + assertEquals(1L, grok(c)); + assertEquals(2L, limit(c)); + assertEquals(2L, sort(c)); + assertEquals(1L, stats(c)); + assertEquals(2L, where(c)); + } + + private long dissect(Counters c) { + return c.get(FPREFIX + DISSECT); + } + + private long eval(Counters c) { + return c.get(FPREFIX + EVAL); + } + + private long grok(Counters c) { + return c.get(FPREFIX + GROK); + } + + private long limit(Counters c) { + return c.get(FPREFIX + LIMIT); + } + + private long sort(Counters c) { + return c.get(FPREFIX + SORT); + } + + private long stats(Counters c) { + return c.get(FPREFIX + STATS); + } + + private long where(Counters c) { + return c.get(FPREFIX + WHERE); + } + + private Counters esql(String sql) { + return esql(sql, null); + } + + private void esqlWithVerifier(String esql, Verifier verifier) { + esql(esql, verifier); + } + + private Counters esql(String esql, Verifier v) { + IndexResolution mapping = AnalyzerTestUtils.analyzerDefaultMapping(); + + Verifier verifier = v; + Metrics metrics = null; + if (v == null) { + metrics = new Metrics(); + verifier = new Verifier(metrics); + } + analyzer(mapping, verifier).analyze(parser.createStatement(esql)); + + return metrics == null ? null : metrics.stats(); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java new file mode 100644 index 0000000000000..937488d2ed546 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java @@ -0,0 +1,189 @@ +/* + * 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.tree; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.dissect.DissectParser; +import org.elasticsearch.xpack.core.enrich.EnrichPolicy; +import org.elasticsearch.xpack.esql.enrich.EnrichPolicyResolution; +import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch; +import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.Concat; +import org.elasticsearch.xpack.esql.plan.logical.Dissect; +import org.elasticsearch.xpack.esql.plan.logical.Grok; +import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.esql.plan.physical.OutputExec; +import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import org.elasticsearch.xpack.esql.type.EsqlDataTypes; +import org.elasticsearch.xpack.ql.capabilities.UnresolvedException; +import org.elasticsearch.xpack.ql.expression.Expression; +import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.Literal; +import org.elasticsearch.xpack.ql.expression.Order; +import org.elasticsearch.xpack.ql.expression.UnresolvedAlias; +import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute; +import org.elasticsearch.xpack.ql.expression.UnresolvedNamedExpression; +import org.elasticsearch.xpack.ql.expression.UnresolvedStar; +import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.ql.index.IndexResolution; +import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.ql.tree.Node; +import org.elasticsearch.xpack.ql.tree.NodeSubclassTests; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.EsField; + +import java.io.IOException; +import java.lang.reflect.Modifier; +import java.lang.reflect.ParameterizedType; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Predicate; + +public class EsqlNodeSubclassTests> extends NodeSubclassTests { + + private static final List> CLASSES_WITH_MIN_TWO_CHILDREN = List.of(Concat.class, CIDRMatch.class); + + // List of classes that are "unresolved" NamedExpression subclasses, therefore not suitable for use with logical/physical plan nodes. + private static final List> UNRESOLVED_CLASSES = List.of( + UnresolvedAttribute.class, + UnresolvedAlias.class, + UnresolvedException.class, + UnresolvedFunction.class, + UnresolvedNamedExpression.class, + UnresolvedStar.class + ); + + public EsqlNodeSubclassTests(Class subclass) { + super(subclass); + } + + @Override + protected Object pluggableMakeArg(Class> toBuildClass, Class argClass) throws Exception { + if (argClass == Dissect.Parser.class) { + // Dissect.Parser is a record / final, cannot be mocked + String pattern = randomDissectPattern(); + String appendSeparator = randomAlphaOfLength(16); + return new Dissect.Parser(pattern, appendSeparator, new DissectParser(pattern, appendSeparator)); + } else if (argClass == Grok.Parser.class) { + // Grok.Parser is a record / final, cannot be mocked + return Grok.pattern(Source.EMPTY, randomGrokPattern()); + } else if (argClass == EsQueryExec.FieldSort.class) { + return randomFieldSort(); + } else if (toBuildClass == Pow.class && Expression.class.isAssignableFrom(argClass)) { + return randomResolvedExpression(randomBoolean() ? FieldAttribute.class : Literal.class); + } else if (isPlanNodeClass(toBuildClass) && Expression.class.isAssignableFrom(argClass)) { + return randomResolvedExpression(argClass); + } else if (argClass == EnrichPolicyResolution.class) { + // EnrichPolicyResolution is a record + return new EnrichPolicyResolution( + randomAlphaOfLength(5), + new EnrichPolicy( + randomAlphaOfLength(10), + null, + List.of(randomAlphaOfLength(5)), + randomAlphaOfLength(5), + List.of(randomAlphaOfLength(5), randomAlphaOfLength(5)) + ), + IndexResolution.invalid(randomAlphaOfLength(5)) + ); + } else if (argClass == Integer.class) { + return randomInt(); + } + + return null; + } + + @Override + protected Object pluggableMakeParameterizedArg(Class> toBuildClass, ParameterizedType pt) { + if (toBuildClass == OutputExec.class && pt.getRawType() == Consumer.class) { + // pageConsumer just needs a BiConsumer. But the consumer has to have reasonable + // `equals` for randomValueOtherThan, so we just ensure that a new instance is + // created each time which uses Object::equals identity. + return new Consumer() { + @Override + public void accept(Page page) { + // do nothing + } + }; + } + return null; + } + + @Override + protected boolean hasAtLeastTwoChildren(Class> toBuildClass) { + return CLASSES_WITH_MIN_TWO_CHILDREN.stream().anyMatch(toBuildClass::equals); + } + + static final Predicate CLASSNAME_FILTER = className -> (className.startsWith("org.elasticsearch.xpack.ql") != false + || className.startsWith("org.elasticsearch.xpack.esql") != false); + + @Override + protected Predicate pluggableClassNameFilter() { + return CLASSNAME_FILTER; + } + + /** Scans the {@code .class} files to identify all classes and checks if they are subclasses of {@link Node}. */ + @ParametersFactory + @SuppressWarnings("rawtypes") + public static List nodeSubclasses() throws IOException { + return subclassesOf(Node.class, CLASSNAME_FILTER).stream() + .filter(c -> testClassFor(c) == null) + .map(c -> new Object[] { c }) + .toList(); + } + + static boolean isPlanNodeClass(Class> toBuildClass) { + return PhysicalPlan.class.isAssignableFrom(toBuildClass) || LogicalPlan.class.isAssignableFrom(toBuildClass); + } + + Expression randomResolvedExpression(Class argClass) throws Exception { + assert Expression.class.isAssignableFrom(argClass); + @SuppressWarnings("unchecked") + Class asNodeSubclass = (Class) argClass; + if (Modifier.isAbstract(argClass.getModifiers())) { + while (true) { + var candidate = randomFrom(subclassesOf(asNodeSubclass)); + if (UNRESOLVED_CLASSES.contains(candidate) == false) { + asNodeSubclass = candidate; + break; + } + } + } + return makeNode(asNodeSubclass); + } + + static String randomDissectPattern() { + return randomFrom(Set.of("%{a} %{b}", "%{b} %{c}", "%{a} %{b} %{c}", "%{b} %{c} %{d}", "%{x}")); + } + + static String randomGrokPattern() { + return randomFrom( + Set.of("%{NUMBER:b:int} %{NUMBER:c:float} %{NUMBER:d:double} %{WORD:e:boolean}", "[a-zA-Z0-9._-]+", "%{LOGLEVEL}") + ); + } + + static List DATA_TYPES = EsqlDataTypes.types().stream().toList(); + + static EsQueryExec.FieldSort randomFieldSort() { + return new EsQueryExec.FieldSort( + field(randomAlphaOfLength(16), randomFrom(DATA_TYPES)), + randomFrom(EnumSet.allOf(Order.OrderDirection.class)), + randomFrom(EnumSet.allOf(Order.NullsPosition.class)) + ); + } + + static FieldAttribute field(String name, DataType type) { + return new FieldAttribute(Source.EMPTY, name, new EsField(name, type, Collections.emptyMap(), false)); + } +} diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/expression/MetadataAttribute.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/expression/MetadataAttribute.java new file mode 100644 index 0000000000000..c61fe9f240ea7 --- /dev/null +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/expression/MetadataAttribute.java @@ -0,0 +1,95 @@ +/* + * 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.ql.expression; + +import org.elasticsearch.core.Tuple; +import org.elasticsearch.xpack.ql.tree.NodeInfo; +import org.elasticsearch.xpack.ql.tree.Source; +import org.elasticsearch.xpack.ql.type.DataType; +import org.elasticsearch.xpack.ql.type.DataTypes; + +import java.util.Map; + +import static org.elasticsearch.core.Tuple.tuple; + +public class MetadataAttribute extends TypedAttribute { + + private static final Map> ATTRIBUTES_MAP = Map.of( + "_version", + tuple(DataTypes.LONG, false), // _version field is not searchable + "_index", + tuple(DataTypes.KEYWORD, true), + "_id", + tuple(DataTypes.KEYWORD, false) // actually searchable, but fielddata access on the _id field is disallowed by default + ); + + private final boolean searchable; + + public MetadataAttribute( + Source source, + String name, + DataType dataType, + String qualifier, + Nullability nullability, + NameId id, + boolean synthetic, + boolean searchable + ) { + super(source, name, dataType, qualifier, nullability, id, synthetic); + this.searchable = searchable; + } + + public MetadataAttribute(Source source, String name, DataType dataType, boolean searchable) { + this(source, name, dataType, null, Nullability.TRUE, null, false, searchable); + } + + @Override + protected MetadataAttribute clone( + Source source, + String name, + DataType type, + String qualifier, + Nullability nullability, + NameId id, + boolean synthetic + ) { + return new MetadataAttribute(source, name, type, qualifier, nullability, id, synthetic, searchable); + } + + @Override + protected String label() { + return "m"; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MetadataAttribute::new, name(), dataType(), qualifier(), nullable(), id(), synthetic(), searchable); + } + + public boolean searchable() { + return searchable; + } + + private MetadataAttribute withSource(Source source) { + return new MetadataAttribute(source, name(), dataType(), qualifier(), nullable(), id(), synthetic(), searchable()); + } + + public static MetadataAttribute create(Source source, String name) { + var t = ATTRIBUTES_MAP.get(name); + return t != null ? new MetadataAttribute(source, name, t.v1(), t.v2()) : null; + } + + public static DataType dataType(String name) { + var t = ATTRIBUTES_MAP.get(name); + return t != null ? t.v1() : null; + } + + public static boolean isSupported(String name) { + return ATTRIBUTES_MAP.containsKey(name); + } +} diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java index 99a5a8a966a71..74a16259635d4 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java @@ -1776,7 +1776,7 @@ private static boolean replace( // default implementation nullifies all nullable expressions protected Expression nullify(Expression exp, Expression nullExp) { - return exp.nullable() == Nullability.TRUE ? new Literal(exp.source(), null, DataTypes.NULL) : exp; + return exp.nullable() == Nullability.TRUE ? Literal.of(exp, null) : exp; } // placeholder for non-null diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/plan/QueryPlan.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/plan/QueryPlan.java index b6668b69f9df0..419c3acba314d 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/plan/QueryPlan.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/plan/QueryPlan.java @@ -55,7 +55,7 @@ public AttributeSet inputSet() { // public PlanType transformExpressionsOnly(Function rule) { - return transformPropertiesOnly(Expression.class, rule); + return transformPropertiesOnly(Object.class, e -> doTransformExpression(e, exp -> exp.transformDown(rule))); } public PlanType transformExpressionsOnly(Class typeToken, Function rule) { diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslator.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslator.java index c308034a37b0b..9152ed872d5b6 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslator.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslator.java @@ -9,6 +9,8 @@ import org.elasticsearch.xpack.ql.expression.Expression; import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.TypedAttribute; import org.elasticsearch.xpack.ql.querydsl.query.NestedQuery; import org.elasticsearch.xpack.ql.querydsl.query.Query; import org.elasticsearch.xpack.ql.util.Check; @@ -38,4 +40,19 @@ public static FieldAttribute checkIsFieldAttribute(Expression e) { Check.isTrue(e instanceof FieldAttribute, "Expected a FieldAttribute but received [{}]", e); return (FieldAttribute) e; } + + public static TypedAttribute checkIsPushableAttribute(Expression e) { + Check.isTrue( + e instanceof FieldAttribute || e instanceof MetadataAttribute, + "Expected a FieldAttribute or MetadataAttribute but received [{}]", + e + ); + return (TypedAttribute) e; + } + + public static String pushableAttributeName(TypedAttribute attribute) { + return attribute instanceof FieldAttribute fa + ? fa.exactAttribute().name() // equality should always be against an exact match (which is important for strings) + : attribute.name(); + } } diff --git a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslators.java b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslators.java index bbbebff5a93cf..c2c51863dbb77 100644 --- a/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslators.java +++ b/x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/planner/ExpressionTranslators.java @@ -14,6 +14,8 @@ import org.elasticsearch.xpack.ql.expression.Expression; import org.elasticsearch.xpack.ql.expression.Expressions; import org.elasticsearch.xpack.ql.expression.FieldAttribute; +import org.elasticsearch.xpack.ql.expression.MetadataAttribute; +import org.elasticsearch.xpack.ql.expression.TypedAttribute; import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction; import org.elasticsearch.xpack.ql.expression.function.scalar.string.StartsWith; import org.elasticsearch.xpack.ql.expression.predicate.Range; @@ -127,25 +129,31 @@ protected Query asQuery(RegexMatch e, TranslatorHandler handler) { } public static Query doTranslate(RegexMatch e, TranslatorHandler handler) { - Query q = null; - String targetFieldName = null; + Query q; + Expression field = e.field(); - if (e.field() instanceof FieldAttribute) { - targetFieldName = handler.nameOf(((FieldAttribute) e.field()).exactAttribute()); - if (e instanceof Like l) { - q = new WildcardQuery(e.source(), targetFieldName, l.pattern().asLuceneWildcard(), l.caseInsensitive()); - } - if (e instanceof WildcardLike l) { - q = new WildcardQuery(e.source(), targetFieldName, l.pattern().asLuceneWildcard(), l.caseInsensitive()); - } - if (e instanceof RLike rl) { - q = new RegexQuery(e.source(), targetFieldName, rl.pattern().asJavaRegex(), rl.caseInsensitive()); - } + if (field instanceof FieldAttribute fa) { + q = translateField(e, handler.nameOf(fa.exactAttribute())); + } else if (field instanceof MetadataAttribute ma) { + q = translateField(e, handler.nameOf(ma)); } else { q = new ScriptQuery(e.source(), e.asScript()); } - return wrapIfNested(q, e.field()); + return wrapIfNested(q, field); + } + + private static Query translateField(RegexMatch e, String targetFieldName) { + if (e instanceof Like l) { + return new WildcardQuery(e.source(), targetFieldName, l.pattern().asLuceneWildcard(), l.caseInsensitive()); + } + if (e instanceof WildcardLike l) { + return new WildcardQuery(e.source(), targetFieldName, l.pattern().asLuceneWildcard(), l.caseInsensitive()); + } + if (e instanceof RLike rl) { + return new RegexQuery(e.source(), targetFieldName, rl.pattern().asJavaRegex(), rl.caseInsensitive()); + } + return null; } } @@ -275,9 +283,9 @@ public static Query doTranslate(BinaryComparison bc, TranslatorHandler handler) } static Query translate(BinaryComparison bc, TranslatorHandler handler) { - FieldAttribute field = checkIsFieldAttribute(bc.left()); + TypedAttribute attribute = checkIsPushableAttribute(bc.left()); Source source = bc.source(); - String name = handler.nameOf(field); + String name = handler.nameOf(attribute); Object value = valueOf(bc.right()); String format = null; boolean isDateLiteralComparison = false; @@ -298,9 +306,9 @@ static Query translate(BinaryComparison bc, TranslatorHandler handler) { } format = formatter.pattern(); isDateLiteralComparison = true; - } else if (field.dataType() == IP && value instanceof BytesRef bytesRef) { + } else if (attribute.dataType() == IP && value instanceof BytesRef bytesRef) { value = DocValueFormat.IP.format(bytesRef); - } else if (field.dataType() == VERSION) { + } else if (attribute.dataType() == VERSION) { // VersionStringFieldMapper#indexedValueForSearch() only accepts as input String or BytesRef with the String (i.e. not // encoded) representation of the version as it'll do the encoding itself. if (value instanceof BytesRef bytesRef) { @@ -308,12 +316,12 @@ static Query translate(BinaryComparison bc, TranslatorHandler handler) { } else if (value instanceof Version version) { value = version.toString(); } - } else if (field.dataType() == UNSIGNED_LONG && value instanceof Long ul) { + } else if (attribute.dataType() == UNSIGNED_LONG && value instanceof Long ul) { value = unsignedLongAsNumber(ul); } ZoneId zoneId = null; - if (DataTypes.isDateTime(field.dataType())) { + if (DataTypes.isDateTime(attribute.dataType())) { zoneId = bc.zoneId(); } if (bc instanceof GreaterThan) { @@ -329,9 +337,7 @@ static Query translate(BinaryComparison bc, TranslatorHandler handler) { return new RangeQuery(source, name, null, false, value, true, format, zoneId); } if (bc instanceof Equals || bc instanceof NullEquals || bc instanceof NotEquals) { - // equality should always be against an exact match - // (which is important for strings) - name = field.exactAttribute().name(); + name = pushableAttributeName(attribute); Query query; if (isDateLiteralComparison) { @@ -415,14 +421,14 @@ private static boolean needsTypeSpecificValueHandling(DataType fieldType) { } private static Query translate(In in, TranslatorHandler handler) { - FieldAttribute field = checkIsFieldAttribute(in.value()); + TypedAttribute attribute = checkIsPushableAttribute(in.value()); Set terms = new LinkedHashSet<>(); List queries = new ArrayList<>(); for (Expression rhs : in.list()) { if (DataTypes.isNull(rhs.dataType()) == false) { - if (needsTypeSpecificValueHandling(field.dataType())) { + if (needsTypeSpecificValueHandling(attribute.dataType())) { // delegates to BinaryComparisons translator to ensure consistent handling of date and time values Query query = BinaryComparisons.translate(new Equals(in.source(), in.value(), rhs, in.zoneId()), handler); @@ -438,7 +444,7 @@ private static Query translate(In in, TranslatorHandler handler) { } if (terms.isEmpty() == false) { - String fieldName = field.exactAttribute().name(); + String fieldName = pushableAttributeName(attribute); queries.add(new TermsQuery(in.source(), fieldName, terms)); } diff --git a/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java b/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java index 22065d458bb95..b14e46a96a9e6 100644 --- a/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java +++ b/x-pack/plugin/ql/src/test/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRulesTests.java @@ -1661,7 +1661,7 @@ public void testIsNullAndComparison() throws Exception { IsNull isNull = new IsNull(EMPTY, fa); And and = new And(EMPTY, isNull, greaterThanOf(fa, ONE)); - assertEquals(new And(EMPTY, isNull, NULL), new PropagateNullable().rule(and)); + assertEquals(new And(EMPTY, isNull, nullOf(BOOLEAN)), new PropagateNullable().rule(and)); } // a IS NULL AND b < 1 AND c < 1 AND a < 1 => a IS NULL AND b < 1 AND c < 1 => a IS NULL AND b < 1 AND c < 1 @@ -1674,7 +1674,7 @@ public void testIsNullAndMultipleComparison() throws Exception { And top = new And(EMPTY, and, lessThanOf(fa, ONE)); Expression optimized = new PropagateNullable().rule(top); - Expression expected = new And(EMPTY, and, NULL); + Expression expected = new And(EMPTY, and, nullOf(BOOLEAN)); assertEquals(Predicates.splitAnd(expected), Predicates.splitAnd(optimized)); } @@ -1683,12 +1683,16 @@ public void testIsNullAndDeeplyNestedExpression() throws Exception { FieldAttribute fa = getFieldAttribute(); IsNull isNull = new IsNull(EMPTY, fa); - Expression nullified = new And(EMPTY, greaterThanOf(new Div(EMPTY, new Add(EMPTY, fa, ONE), TWO), ONE), new Add(EMPTY, fa, TWO)); + Expression nullified = new And( + EMPTY, + greaterThanOf(new Div(EMPTY, new Add(EMPTY, fa, ONE), TWO), ONE), + greaterThanOf(new Add(EMPTY, fa, TWO), ONE) + ); Expression kept = new And(EMPTY, isNull, lessThanOf(getFieldAttribute("b"), THREE)); And and = new And(EMPTY, nullified, kept); Expression optimized = new PropagateNullable().rule(and); - Expression expected = new And(EMPTY, new And(EMPTY, NULL, NULL), kept); + Expression expected = new And(EMPTY, new And(EMPTY, nullOf(BOOLEAN), nullOf(BOOLEAN)), kept); assertEquals(Predicates.splitAnd(expected), Predicates.splitAnd(optimized)); } @@ -1767,6 +1771,10 @@ public void testPushDownFilterThroughAgg() throws Exception { } + private Literal nullOf(DataType dataType) { + return new Literal(Source.EMPTY, null, dataType); + } + private void assertNullLiteral(Expression expression) { assertEquals(Literal.class, expression.getClass()); assertNull(expression.fold()); diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index 45440ec7d8462..5f49e70cbc990 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -319,6 +319,7 @@ public class Constants { "cluster:monitor/xpack/enrich/coordinator_stats", "cluster:monitor/xpack/enrich/stats", "cluster:monitor/xpack/eql/stats/dist", + "cluster:monitor/xpack/esql/stats/dist", "cluster:monitor/xpack/info", "cluster:monitor/xpack/info/aggregate_metric", "cluster:monitor/xpack/info/analytics", @@ -329,6 +330,7 @@ public class Constants { "cluster:monitor/xpack/info/enrich", "cluster:monitor/xpack/info/enterprise_search", "cluster:monitor/xpack/info/eql", + "cluster:monitor/xpack/info/esql", "cluster:monitor/xpack/info/frozen_indices", "cluster:monitor/xpack/info/graph", "cluster:monitor/xpack/info/ilm", @@ -389,6 +391,7 @@ public class Constants { "cluster:monitor/xpack/usage/enrich", "cluster:monitor/xpack/usage/enterprise_search", "cluster:monitor/xpack/usage/eql", + "cluster:monitor/xpack/usage/esql", "cluster:monitor/xpack/usage/frozen_indices", "cluster:monitor/xpack/usage/graph", "cluster:monitor/xpack/usage/health_api", @@ -481,6 +484,7 @@ public class Constants { "indices:data/read/close_point_in_time", "indices:data/read/eql", "indices:data/read/eql/async/get", + "indices:data/read/esql", "indices:data/read/explain", "indices:data/read/field_caps", "indices:data/read/get", diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java index 782c57c4ae8c1..c7a6c1fbd498d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java @@ -266,6 +266,7 @@ private static boolean shouldAuthorizeIndexActionNameOnly(String action, Transpo case "indices:data/write/reindex": case "indices:data/read/sql": case "indices:data/read/sql/translate": + case "indices:data/read/esql": if (request instanceof BulkShardRequest) { return false; }