Browse Source

ESQL: Regex improvements (#106429)

This makes a couple of changes to regex processing in the compute
engine:
1. Process utf-8 strings directly. This should save a ton of time.
2. Snip the `toString` output if it is too big - I chose 64kb of
   strings.
3. I changed the formatting of the automaton to a slightly customized
   `dot` output. Because automata are graphs. Everyone knows it. And
   they are a lot easier to read as graphs. `dot` is easy to convert
   into a graph.
4. I implement `EvaluatorMapper` for regex operations which is pretty
   standard for the rest of our operations.
Nik Everett 1 year ago
parent
commit
bffd2a964c
20 changed files with 614 additions and 453 deletions
  1. 32 2
      benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/EvalBenchmark.java
  2. 5 0
      docs/changelog/106429.yaml
  3. 0 146
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java
  4. 0 146
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java
  5. 21 15
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/AutomataMatchEvaluator.java
  6. 0 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/EvalMapper.java
  7. 0 31
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMapper.java
  8. 0 23
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMatch.java
  9. 137 0
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/AutomataMatch.java
  10. 13 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java
  11. 19 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java
  12. 2 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java
  13. 2 2
      x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
  14. 28 15
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java
  15. 135 27
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java
  16. 9 34
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSortTests.java
  17. 157 0
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLikeTests.java
  18. 50 0
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLikeTests.java
  19. 2 2
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java
  20. 2 2
      x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java

+ 32 - 2
benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/EvalBenchmark.java

@@ -8,6 +8,7 @@
 
 package org.elasticsearch.benchmark.compute.operator;
 
+import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.breaker.NoopCircuitBreaker;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.compute.data.Block;
@@ -26,11 +27,13 @@ import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Equa
 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.expression.function.scalar.string.RLike;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 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.regex.RLikePattern;
 import org.elasticsearch.xpack.ql.tree.Source;
 import org.elasticsearch.xpack.ql.type.DataTypes;
 import org.elasticsearch.xpack.ql.type.EsField;
@@ -58,7 +61,6 @@ import java.util.concurrent.TimeUnit;
 @State(Scope.Thread)
 @Fork(1)
 public class EvalBenchmark {
-    private static final BigArrays BIG_ARRAYS = BigArrays.NON_RECYCLING_INSTANCE;  // TODO real big arrays?
     private static final BlockFactory blockFactory = BlockFactory.getInstance(
         new NoopCircuitBreaker("noop"),
         BigArrays.NON_RECYCLING_INSTANCE
@@ -82,7 +84,9 @@ public class EvalBenchmark {
         }
     }
 
-    @Param({ "abs", "add", "date_trunc", "equal_to_const", "long_equal_to_long", "long_equal_to_int", "mv_min", "mv_min_ascending" })
+    @Param(
+        { "abs", "add", "date_trunc", "equal_to_const", "long_equal_to_long", "long_equal_to_int", "mv_min", "mv_min_ascending", "rlike" }
+    )
     public String operation;
 
     private static Operator operator(String operation) {
@@ -134,6 +138,11 @@ public class EvalBenchmark {
                 FieldAttribute longField = longField();
                 yield EvalMapper.toEvaluator(new MvMin(Source.EMPTY, longField), layout(longField)).get(driverContext);
             }
+            case "rlike" -> {
+                FieldAttribute keywordField = keywordField();
+                RLike rlike = new RLike(Source.EMPTY, keywordField, new RLikePattern(".ar"));
+                yield EvalMapper.toEvaluator(rlike, layout(keywordField)).get(driverContext);
+            }
             default -> throw new UnsupportedOperationException();
         };
     }
@@ -146,6 +155,10 @@ public class EvalBenchmark {
         return new FieldAttribute(Source.EMPTY, "int", new EsField("int", DataTypes.INTEGER, Map.of(), true));
     }
 
+    private static FieldAttribute keywordField() {
+        return new FieldAttribute(Source.EMPTY, "keyword", new EsField("keyword", DataTypes.KEYWORD, Map.of(), true));
+    }
+
     private static Layout layout(FieldAttribute... fields) {
         Layout.Builder layout = new Layout.Builder();
         layout.append(Arrays.asList(fields));
@@ -205,6 +218,15 @@ public class EvalBenchmark {
                     }
                 }
             }
+            case "rlike" -> {
+                BooleanVector v = actual.<BooleanBlock>getBlock(1).asVector();
+                for (int i = 0; i < BLOCK_LENGTH; i++) {
+                    boolean expected = i % 2 == 1;
+                    if (v.getBoolean(i) != expected) {
+                        throw new AssertionError("[" + operation + "] expected [" + expected + "] but was [" + v.getBoolean(i) + "]");
+                    }
+                }
+            }
             default -> throw new UnsupportedOperationException();
         }
     }
@@ -250,6 +272,14 @@ public class EvalBenchmark {
                 }
                 yield new Page(builder.build());
             }
+            case "rlike" -> {
+                var builder = blockFactory.newBytesRefVectorBuilder(BLOCK_LENGTH);
+                BytesRef[] values = new BytesRef[] { new BytesRef("foo"), new BytesRef("bar") };
+                for (int i = 0; i < BLOCK_LENGTH; i++) {
+                    builder.appendBytesRef(values[i % 2]);
+                }
+                yield new Page(builder.build().asBlock());
+            }
             default -> throw new UnsupportedOperationException();
         };
     }

+ 5 - 0
docs/changelog/106429.yaml

@@ -0,0 +1,5 @@
+pr: 106429
+summary: "ESQL: Regex improvements"
+area: ES|QL
+type: enhancement
+issues: []

+ 0 - 146
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java

@@ -1,146 +0,0 @@
-// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
-// or more contributor license agreements. Licensed under the Elastic License
-// 2.0; 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.IllegalArgumentException;
-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.DriverContext;
-import org.elasticsearch.compute.operator.EvalOperator;
-import org.elasticsearch.core.Releasables;
-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;
-
-  private final DriverContext driverContext;
-
-  public PowIntEvaluator(Source source, EvalOperator.ExpressionEvaluator base,
-      EvalOperator.ExpressionEvaluator exponent, DriverContext driverContext) {
-    this.warnings = new Warnings(source);
-    this.base = base;
-    this.exponent = exponent;
-    this.driverContext = driverContext;
-  }
-
-  @Override
-  public Block.Ref eval(Page page) {
-    try (Block.Ref baseRef = base.eval(page)) {
-      DoubleBlock baseBlock = (DoubleBlock) baseRef.block();
-      try (Block.Ref exponentRef = exponent.eval(page)) {
-        DoubleBlock exponentBlock = (DoubleBlock) exponentRef.block();
-        DoubleVector baseVector = baseBlock.asVector();
-        if (baseVector == null) {
-          return Block.Ref.floating(eval(page.getPositionCount(), baseBlock, exponentBlock));
-        }
-        DoubleVector exponentVector = exponentBlock.asVector();
-        if (exponentVector == null) {
-          return Block.Ref.floating(eval(page.getPositionCount(), baseBlock, exponentBlock));
-        }
-        return Block.Ref.floating(eval(page.getPositionCount(), baseVector, exponentVector));
-      }
-    }
-  }
-
-  public IntBlock eval(int positionCount, DoubleBlock baseBlock, DoubleBlock exponentBlock) {
-    try(IntBlock.Builder result = driverContext.blockFactory().newIntBlockBuilder(positionCount)) {
-      position: for (int p = 0; p < positionCount; p++) {
-        if (baseBlock.isNull(p)) {
-          result.appendNull();
-          continue position;
-        }
-        if (baseBlock.getValueCount(p) != 1) {
-          if (baseBlock.getValueCount(p) > 1) {
-            warnings.registerException(new IllegalArgumentException("single-value function encountered multi-value"));
-          }
-          result.appendNull();
-          continue position;
-        }
-        if (exponentBlock.isNull(p)) {
-          result.appendNull();
-          continue position;
-        }
-        if (exponentBlock.getValueCount(p) != 1) {
-          if (exponentBlock.getValueCount(p) > 1) {
-            warnings.registerException(new IllegalArgumentException("single-value function encountered multi-value"));
-          }
-          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) {
-    try(IntBlock.Builder result = driverContext.blockFactory().newIntBlockBuilder(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 + "]";
-  }
-
-  @Override
-  public void close() {
-    Releasables.closeExpectNoException(base, exponent);
-  }
-
-  static class Factory implements EvalOperator.ExpressionEvaluator.Factory {
-    private final Source source;
-
-    private final EvalOperator.ExpressionEvaluator.Factory base;
-
-    private final EvalOperator.ExpressionEvaluator.Factory exponent;
-
-    public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory base,
-        EvalOperator.ExpressionEvaluator.Factory exponent) {
-      this.source = source;
-      this.base = base;
-      this.exponent = exponent;
-    }
-
-    @Override
-    public PowIntEvaluator get(DriverContext context) {
-      return new PowIntEvaluator(source, base.get(context), exponent.get(context), context);
-    }
-
-    @Override
-    public String toString() {
-      return "PowIntEvaluator[" + "base=" + base + ", exponent=" + exponent + "]";
-    }
-  }
-}

+ 0 - 146
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java

@@ -1,146 +0,0 @@
-// Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
-// or more contributor license agreements. Licensed under the Elastic License
-// 2.0; 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.IllegalArgumentException;
-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.DriverContext;
-import org.elasticsearch.compute.operator.EvalOperator;
-import org.elasticsearch.core.Releasables;
-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;
-
-  private final DriverContext driverContext;
-
-  public PowLongEvaluator(Source source, EvalOperator.ExpressionEvaluator base,
-      EvalOperator.ExpressionEvaluator exponent, DriverContext driverContext) {
-    this.warnings = new Warnings(source);
-    this.base = base;
-    this.exponent = exponent;
-    this.driverContext = driverContext;
-  }
-
-  @Override
-  public Block.Ref eval(Page page) {
-    try (Block.Ref baseRef = base.eval(page)) {
-      DoubleBlock baseBlock = (DoubleBlock) baseRef.block();
-      try (Block.Ref exponentRef = exponent.eval(page)) {
-        DoubleBlock exponentBlock = (DoubleBlock) exponentRef.block();
-        DoubleVector baseVector = baseBlock.asVector();
-        if (baseVector == null) {
-          return Block.Ref.floating(eval(page.getPositionCount(), baseBlock, exponentBlock));
-        }
-        DoubleVector exponentVector = exponentBlock.asVector();
-        if (exponentVector == null) {
-          return Block.Ref.floating(eval(page.getPositionCount(), baseBlock, exponentBlock));
-        }
-        return Block.Ref.floating(eval(page.getPositionCount(), baseVector, exponentVector));
-      }
-    }
-  }
-
-  public LongBlock eval(int positionCount, DoubleBlock baseBlock, DoubleBlock exponentBlock) {
-    try(LongBlock.Builder result = driverContext.blockFactory().newLongBlockBuilder(positionCount)) {
-      position: for (int p = 0; p < positionCount; p++) {
-        if (baseBlock.isNull(p)) {
-          result.appendNull();
-          continue position;
-        }
-        if (baseBlock.getValueCount(p) != 1) {
-          if (baseBlock.getValueCount(p) > 1) {
-            warnings.registerException(new IllegalArgumentException("single-value function encountered multi-value"));
-          }
-          result.appendNull();
-          continue position;
-        }
-        if (exponentBlock.isNull(p)) {
-          result.appendNull();
-          continue position;
-        }
-        if (exponentBlock.getValueCount(p) != 1) {
-          if (exponentBlock.getValueCount(p) > 1) {
-            warnings.registerException(new IllegalArgumentException("single-value function encountered multi-value"));
-          }
-          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) {
-    try(LongBlock.Builder result = driverContext.blockFactory().newLongBlockBuilder(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 + "]";
-  }
-
-  @Override
-  public void close() {
-    Releasables.closeExpectNoException(base, exponent);
-  }
-
-  static class Factory implements EvalOperator.ExpressionEvaluator.Factory {
-    private final Source source;
-
-    private final EvalOperator.ExpressionEvaluator.Factory base;
-
-    private final EvalOperator.ExpressionEvaluator.Factory exponent;
-
-    public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory base,
-        EvalOperator.ExpressionEvaluator.Factory exponent) {
-      this.source = source;
-      this.base = base;
-      this.exponent = exponent;
-    }
-
-    @Override
-    public PowLongEvaluator get(DriverContext context) {
-      return new PowLongEvaluator(source, base.get(context), exponent.get(context), context);
-    }
-
-    @Override
-    public String toString() {
-      return "PowLongEvaluator[" + "base=" + base + ", exponent=" + exponent + "]";
-    }
-  }
-}

+ 21 - 15
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMatchEvaluator.java → x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/AutomataMatchEvaluator.java

@@ -2,13 +2,13 @@
 // or more contributor license agreements. Licensed under the Elastic License
 // 2.0; you may not use this file except in compliance with the Elastic License
 // 2.0.
-package org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex;
+package org.elasticsearch.xpack.esql.expression.function.scalar.string;
 
 import java.lang.IllegalArgumentException;
 import java.lang.Override;
 import java.lang.String;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.BooleanBlock;
 import org.elasticsearch.compute.data.BooleanVector;
@@ -22,22 +22,25 @@ import org.elasticsearch.xpack.esql.expression.function.Warnings;
 import org.elasticsearch.xpack.ql.tree.Source;
 
 /**
- * {@link EvalOperator.ExpressionEvaluator} implementation for {@link RegexMatch}.
+ * {@link EvalOperator.ExpressionEvaluator} implementation for {@link AutomataMatch}.
  * This class is generated. Do not edit it.
  */
-public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluator {
+public final class AutomataMatchEvaluator implements EvalOperator.ExpressionEvaluator {
   private final Warnings warnings;
 
   private final EvalOperator.ExpressionEvaluator input;
 
-  private final CharacterRunAutomaton pattern;
+  private final ByteRunAutomaton automaton;
+
+  private final String pattern;
 
   private final DriverContext driverContext;
 
-  public RegexMatchEvaluator(Source source, EvalOperator.ExpressionEvaluator input,
-      CharacterRunAutomaton pattern, DriverContext driverContext) {
+  public AutomataMatchEvaluator(Source source, EvalOperator.ExpressionEvaluator input,
+      ByteRunAutomaton automaton, String pattern, DriverContext driverContext) {
     this.warnings = new Warnings(source);
     this.input = input;
+    this.automaton = automaton;
     this.pattern = pattern;
     this.driverContext = driverContext;
   }
@@ -68,7 +71,7 @@ public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluat
           result.appendNull();
           continue position;
         }
-        result.appendBoolean(RegexMatch.process(inputBlock.getBytesRef(inputBlock.getFirstValueIndex(p), inputScratch), pattern));
+        result.appendBoolean(AutomataMatch.process(inputBlock.getBytesRef(inputBlock.getFirstValueIndex(p), inputScratch), automaton, pattern));
       }
       return result.build();
     }
@@ -78,7 +81,7 @@ public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluat
     try(BooleanVector.Builder result = driverContext.blockFactory().newBooleanVectorBuilder(positionCount)) {
       BytesRef inputScratch = new BytesRef();
       position: for (int p = 0; p < positionCount; p++) {
-        result.appendBoolean(RegexMatch.process(inputVector.getBytesRef(p, inputScratch), pattern));
+        result.appendBoolean(AutomataMatch.process(inputVector.getBytesRef(p, inputScratch), automaton, pattern));
       }
       return result.build();
     }
@@ -86,7 +89,7 @@ public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluat
 
   @Override
   public String toString() {
-    return "RegexMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]";
+    return "AutomataMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]";
   }
 
   @Override
@@ -99,23 +102,26 @@ public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluat
 
     private final EvalOperator.ExpressionEvaluator.Factory input;
 
-    private final CharacterRunAutomaton pattern;
+    private final ByteRunAutomaton automaton;
+
+    private final String pattern;
 
     public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory input,
-        CharacterRunAutomaton pattern) {
+        ByteRunAutomaton automaton, String pattern) {
       this.source = source;
       this.input = input;
+      this.automaton = automaton;
       this.pattern = pattern;
     }
 
     @Override
-    public RegexMatchEvaluator get(DriverContext context) {
-      return new RegexMatchEvaluator(source, input.get(context), pattern, context);
+    public AutomataMatchEvaluator get(DriverContext context) {
+      return new AutomataMatchEvaluator(source, input.get(context), automaton, pattern, context);
     }
 
     @Override
     public String toString() {
-      return "RegexMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]";
+      return "AutomataMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]";
     }
   }
 }

+ 0 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/EvalMapper.java

@@ -25,7 +25,6 @@ import org.elasticsearch.xpack.esql.evaluator.mapper.ExpressionMapper;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.ComparisonMapper;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.InMapper;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.InsensitiveEqualsMapper;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RegexMapper;
 import org.elasticsearch.xpack.esql.planner.Layout;
 import org.elasticsearch.xpack.ql.QlIllegalArgumentException;
 import org.elasticsearch.xpack.ql.expression.Attribute;
@@ -48,7 +47,6 @@ public final class EvalMapper {
         ComparisonMapper.LESS_THAN,
         ComparisonMapper.LESS_THAN_OR_EQUAL,
         InMapper.IN_MAPPER,
-        RegexMapper.REGEX_MATCH,
         new InsensitiveEqualsMapper(),
         new BooleanLogic(),
         new Nots(),

+ 0 - 31
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMapper.java

@@ -1,31 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex;
-
-import org.apache.lucene.util.automaton.CharacterRunAutomaton;
-import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
-import org.elasticsearch.xpack.esql.evaluator.EvalMapper;
-import org.elasticsearch.xpack.esql.evaluator.mapper.ExpressionMapper;
-import org.elasticsearch.xpack.esql.planner.Layout;
-import org.elasticsearch.xpack.ql.expression.predicate.regex.AbstractStringPattern;
-import org.elasticsearch.xpack.ql.expression.predicate.regex.RegexMatch;
-
-public abstract class RegexMapper extends ExpressionMapper<RegexMatch<?>> {
-
-    public static final ExpressionMapper<?> REGEX_MATCH = new RegexMapper() {
-        @Override
-        public ExpressionEvaluator.Factory map(RegexMatch<?> expression, Layout layout) {
-            return dvrCtx -> new RegexMatchEvaluator(
-                expression.source(),
-                EvalMapper.toEvaluator(expression.field(), layout).get(dvrCtx),
-                new CharacterRunAutomaton(((AbstractStringPattern) expression.pattern()).createAutomaton()),
-                dvrCtx
-            );
-        }
-    };
-}

+ 0 - 23
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMatch.java

@@ -1,23 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.esql.evaluator.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());
-    }
-}

+ 137 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/AutomataMatch.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.xpack.esql.expression.function.scalar.string;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.Automaton;
+import org.apache.lucene.util.automaton.ByteRunAutomaton;
+import org.apache.lucene.util.automaton.Operations;
+import org.apache.lucene.util.automaton.Transition;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
+import org.elasticsearch.compute.ann.Evaluator;
+import org.elasticsearch.compute.ann.Fixed;
+import org.elasticsearch.compute.data.BooleanBlock;
+import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.xpack.ql.tree.Source;
+
+/**
+ * Matches {@link BytesRef}s against {@link Automaton automata}.
+ */
+public class AutomataMatch {
+    /**
+     * Build an {@link EvalOperator.ExpressionEvaluator.Factory} that will match
+     * {@link BytesRef}s against {@link Automaton automata} and return a {@link BooleanBlock}.
+     */
+    public static EvalOperator.ExpressionEvaluator.Factory toEvaluator(
+        Source source,
+        EvalOperator.ExpressionEvaluator.Factory field,
+        Automaton utf32Automaton
+    ) {
+        /*
+         * ByteRunAutomaton has a way to convert utf32 to utf8, but if we used it
+         * we couldn't get a nice toDot - so we call UTF32ToUTF8 ourselves.
+         */
+        Automaton automaton = Operations.determinize(new UTF32ToUTF8().convert(utf32Automaton), Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
+        ByteRunAutomaton run = new ByteRunAutomaton(automaton, true, Operations.DEFAULT_DETERMINIZE_WORK_LIMIT);
+        return new AutomataMatchEvaluator.Factory(source, field, run, toDot(automaton));
+    }
+
+    @Evaluator
+    static boolean process(BytesRef input, @Fixed(includeInToString = false) ByteRunAutomaton automaton, @Fixed String pattern) {
+        if (input == null) {
+            return false;
+        }
+        return automaton.run(input.bytes, input.offset, input.length);
+    }
+
+    private static final int MAX_LENGTH = 1024 * 64;
+
+    /**
+     * Convert an {@link Automaton} to <a href="https://graphviz.org/doc/info/lang.html">dot</a>.
+     * <p>
+     *  This was borrowed from {@link Automaton#toDot} but has been modified to snip if the length
+     *  grows too much and to format the bytes differently.
+     * </p>
+     */
+    public static String toDot(Automaton automaton) {
+        StringBuilder b = new StringBuilder();
+        b.append("digraph Automaton {\n");
+        b.append("  rankdir = LR\n");
+        b.append("  node [width=0.2, height=0.2, fontsize=8]\n");
+        int numStates = automaton.getNumStates();
+        if (numStates > 0) {
+            b.append("  initial [shape=plaintext,label=\"\"]\n");
+            b.append("  initial -> 0\n");
+        }
+
+        Transition t = new Transition();
+
+        too_big: for (int state = 0; state < numStates; ++state) {
+            b.append("  ");
+            b.append(state);
+            if (automaton.isAccept(state)) {
+                b.append(" [shape=doublecircle,label=\"").append(state).append("\"]\n");
+            } else {
+                b.append(" [shape=circle,label=\"").append(state).append("\"]\n");
+            }
+
+            int numTransitions = automaton.initTransition(state, t);
+
+            for (int i = 0; i < numTransitions; ++i) {
+                automaton.getNextTransition(t);
+
+                assert t.max >= t.min;
+
+                b.append("  ");
+                b.append(state);
+                b.append(" -> ");
+                b.append(t.dest);
+                b.append(" [label=\"");
+                appendByte(t.min, b);
+                if (t.max != t.min) {
+                    b.append('-');
+                    appendByte(t.max, b);
+                }
+
+                b.append("\"]\n");
+                if (b.length() >= MAX_LENGTH) {
+                    b.append("...snip...");
+                    break too_big;
+                }
+            }
+        }
+
+        b.append('}');
+        return b.toString();
+    }
+
+    static void appendByte(int c, StringBuilder b) {
+        if (c > 255) {
+            throw new UnsupportedOperationException("can only format bytes but got [" + c + "]");
+        }
+        if (c == 34) {
+            b.append("\\\"");
+            return;
+        }
+        if (c == 92) {
+            b.append("\\\\");
+            return;
+        }
+        if (c >= 33 && c <= 126) {
+            b.appendCodePoint(c);
+            return;
+        }
+        b.append("0x");
+        String hex = Integer.toHexString(c);
+        switch (hex.length()) {
+            case 1 -> b.append('0').append(hex);
+            case 2 -> b.append(hex);
+            default -> throw new UnsupportedOperationException("can only format bytes");
+        }
+    }
+}

+ 13 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RLike.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java

@@ -5,17 +5,21 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex;
+package org.elasticsearch.xpack.esql.expression.function.scalar.string;
 
+import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.xpack.esql.evaluator.mapper.EvaluatorMapper;
 import org.elasticsearch.xpack.ql.expression.Expression;
 import org.elasticsearch.xpack.ql.expression.predicate.regex.RLikePattern;
 import org.elasticsearch.xpack.ql.tree.NodeInfo;
 import org.elasticsearch.xpack.ql.tree.Source;
 
+import java.util.function.Function;
+
 import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT;
 import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString;
 
-public class RLike extends org.elasticsearch.xpack.ql.expression.predicate.regex.RLike {
+public class RLike extends org.elasticsearch.xpack.ql.expression.predicate.regex.RLike implements EvaluatorMapper {
     public RLike(Source source, Expression value, RLikePattern pattern) {
         super(source, value, pattern);
     }
@@ -38,4 +42,11 @@ public class RLike extends org.elasticsearch.xpack.ql.expression.predicate.regex
     protected TypeResolution resolveType() {
         return isString(field(), sourceText(), DEFAULT);
     }
+
+    @Override
+    public EvalOperator.ExpressionEvaluator.Factory toEvaluator(
+        Function<Expression, EvalOperator.ExpressionEvaluator.Factory> toEvaluator
+    ) {
+        return AutomataMatch.toEvaluator(source(), toEvaluator.apply(field()), pattern().createAutomaton());
+    }
 }

+ 19 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/WildcardLike.java → x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java

@@ -5,17 +5,22 @@
  * 2.0.
  */
 
-package org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex;
+package org.elasticsearch.xpack.esql.expression.function.scalar.string;
 
+import org.apache.lucene.util.automaton.Automata;
+import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.xpack.esql.evaluator.mapper.EvaluatorMapper;
 import org.elasticsearch.xpack.ql.expression.Expression;
 import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardPattern;
 import org.elasticsearch.xpack.ql.tree.NodeInfo;
 import org.elasticsearch.xpack.ql.tree.Source;
 
+import java.util.function.Function;
+
 import static org.elasticsearch.xpack.ql.expression.TypeResolutions.ParamOrdinal.DEFAULT;
 import static org.elasticsearch.xpack.ql.expression.TypeResolutions.isString;
 
-public class WildcardLike extends org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike {
+public class WildcardLike extends org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardLike implements EvaluatorMapper {
     public WildcardLike(Source source, Expression left, WildcardPattern pattern) {
         super(source, left, pattern, false);
     }
@@ -34,4 +39,16 @@ public class WildcardLike extends org.elasticsearch.xpack.ql.expression.predicat
     protected TypeResolution resolveType() {
         return isString(field(), sourceText(), DEFAULT);
     }
+
+    @Override
+    public EvalOperator.ExpressionEvaluator.Factory toEvaluator(
+        Function<Expression, EvalOperator.ExpressionEvaluator.Factory> toEvaluator
+    ) {
+        return AutomataMatch.toEvaluator(
+            source(),
+            toEvaluator.apply(field()),
+            // The empty pattern will accept the empty string
+            pattern().pattern().length() == 0 ? Automata.makeEmptyString() : pattern().createAutomaton()
+        );
+    }
 }

+ 2 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java

@@ -26,8 +26,6 @@ import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Inse
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThan;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThanOrEqual;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.NotEquals;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RLike;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.WildcardLike;
 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;
@@ -110,6 +108,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.string.EndsWith;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.LTrim;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Left;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Length;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.RTrim;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Replace;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Right;
@@ -119,6 +118,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.ToLower;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.ToUpper;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Trim;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Div;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Mod;

+ 2 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java

@@ -23,11 +23,11 @@ import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Grea
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.InsensitiveEquals;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThan;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThanOrEqual;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RLike;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.Order;
 import org.elasticsearch.xpack.esql.expression.UnresolvedNamePattern;
 import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Div;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Mod;

+ 28 - 15
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java

@@ -181,11 +181,20 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
      */
     protected abstract Expression build(Source source, List<Expression> args);
 
-    protected Expression buildFieldExpression(TestCaseSupplier.TestCase testCase) {
+    /**
+     * Build an {@link Expression} where all inputs are field references,
+     * <strong>except</strong> those that have been marked with {@link TestCaseSupplier.TypedData#forceLiteral()}.
+     */
+    protected final Expression buildFieldExpression(TestCaseSupplier.TestCase testCase) {
         return build(testCase.getSource(), testCase.getDataAsFields());
     }
 
-    protected Expression buildDeepCopyOfFieldExpression(TestCaseSupplier.TestCase testCase) {
+    /**
+     * Build an {@link Expression} where all inputs are anonymous functions
+     * that make a copy of the values from a field <strong>except</strong>
+     * those that have been marked with {@link TestCaseSupplier.TypedData#forceLiteral()}.
+     */
+    protected final Expression buildDeepCopyOfFieldExpression(TestCaseSupplier.TestCase testCase) {
         return build(testCase.getSource(), testCase.getDataAsDeepCopiedFields());
     }
 
@@ -255,7 +264,7 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
         }
         assertFalse("expected resolved", expression.typeResolved().unresolved());
         expression = new FoldNull().rule(expression);
-        assertThat(expression.dataType(), equalTo(testCase.expectedType));
+        assertThat(expression.dataType(), equalTo(testCase.expectedType()));
         logger.info("Result type: " + expression.dataType());
 
         Object result;
@@ -278,7 +287,7 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
     private Object toJavaObjectUnsignedLongAware(Block block, int position) {
         Object result;
         result = toJavaObject(block, position);
-        if (result != null && testCase.expectedType == DataTypes.UNSIGNED_LONG) {
+        if (result != null && testCase.expectedType() == DataTypes.UNSIGNED_LONG) {
             assertThat(result, instanceOf(Long.class));
             result = NumericUtils.unsignedLongAsBigInteger((Long) result);
         }
@@ -524,7 +533,7 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
         assumeTrue("All test data types must be representable in order to build fields", testCase.allTypesAreRepresentable());
         var factory = evaluator(buildFieldExpression(testCase));
         try (ExpressionEvaluator ev = factory.get(driverContext())) {
-            assertThat(ev.toString(), equalTo(testCase.evaluatorToString));
+            assertThat(ev.toString(), testCase.evaluatorToString());
         }
     }
 
@@ -532,7 +541,7 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
         assumeTrue("nothing to do if a type error", testCase.getExpectedTypeError() == null);
         assumeTrue("All test data types must be representable in order to build fields", testCase.allTypesAreRepresentable());
         var factory = evaluator(buildFieldExpression(testCase));
-        assertThat(factory.toString(), equalTo(testCase.evaluatorToString));
+        assertThat(factory.toString(), testCase.evaluatorToString());
     }
 
     public final void testFold() {
@@ -544,12 +553,12 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
         }
         assertFalse(expression.typeResolved().unresolved());
         Expression nullOptimized = new FoldNull().rule(expression);
-        assertThat(nullOptimized.dataType(), equalTo(testCase.expectedType));
+        assertThat(nullOptimized.dataType(), equalTo(testCase.expectedType()));
         assertTrue(nullOptimized.foldable());
         if (testCase.foldingExceptionClass() == null) {
             Object result = nullOptimized.fold();
             // Decode unsigned longs into BigIntegers
-            if (testCase.expectedType == DataTypes.UNSIGNED_LONG && result != null) {
+            if (testCase.expectedType() == DataTypes.UNSIGNED_LONG && result != null) {
                 result = NumericUtils.unsignedLongAsBigInteger((Long) result);
             }
             assertThat(result, testCase.getMatcher());
@@ -670,11 +679,13 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
                     }).toList();
                     return new TestCaseSupplier.TestCase(
                         data,
-                        oc.evaluatorToString,
-                        oc.expectedType,
+                        oc.evaluatorToString(),
+                        oc.expectedType(),
                         nullValue(),
                         null,
-                        oc.getExpectedTypeError()
+                        oc.getExpectedTypeError(),
+                        null,
+                        null
                     );
                 }));
 
@@ -691,11 +702,13 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
                                 .toList();
                             return new TestCaseSupplier.TestCase(
                                 data,
-                                "LiteralsEvaluator[lit=null]",
-                                entirelyNullPreservesType == false && oc.getData().size() == 1 ? DataTypes.NULL : oc.expectedType,
+                                equalTo("LiteralsEvaluator[lit=null]"),
+                                entirelyNullPreservesType == false && oc.getData().size() == 1 ? DataTypes.NULL : oc.expectedType(),
                                 nullValue(),
                                 null,
-                                oc.getExpectedTypeError()
+                                oc.getExpectedTypeError(),
+                                null,
+                                null
                             );
                         }));
                     }
@@ -1067,7 +1080,7 @@ public abstract class AbstractFunctionTestCase extends ESTestCase {
         if (testCase.getData().stream().anyMatch(t -> t.type() == DataTypes.NULL)) {
             return;
         }
-        signatures.putIfAbsent(testCase.getData().stream().map(TestCaseSupplier.TypedData::type).toList(), testCase.expectedType);
+        signatures.putIfAbsent(testCase.getData().stream().map(TestCaseSupplier.TypedData::type).toList(), testCase.expectedType());
     }
 
     @AfterClass

+ 135 - 27
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java

@@ -54,7 +54,7 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
     implements
         Supplier<TestCaseSupplier.TestCase> {
 
-    private static Logger logger = LogManager.getLogger(TestCaseSupplier.class);
+    private static final Logger logger = LogManager.getLogger(TestCaseSupplier.class);
     /**
      * Build a test case without types.
      *
@@ -1130,51 +1130,57 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
         /**
          * The {@link Source} this test case should be run with
          */
-        private Source source;
+        private final Source source;
         /**
          * The parameter values and types to pass into the function for this test run
          */
-        private List<TypedData> data;
+        private final List<TypedData> data;
 
         /**
          * The expected toString output for the evaluator this function invocation should generate
          */
-        String evaluatorToString;
+        private final Matcher<String> evaluatorToString;
         /**
          * The expected output type for the case being tested
          */
-        DataType expectedType;
+        private final DataType expectedType;
         /**
          * A matcher to validate the output of the function run on the given input data
          */
-        private Matcher<Object> matcher;
+        private final Matcher<Object> matcher;
 
         /**
          * Warnings this test is expected to produce
          */
-        private String[] expectedWarnings;
-
-        private Class<? extends Throwable> foldingExceptionClass;
-        private String foldingExceptionMessage;
+        private final String[] expectedWarnings;
 
         private final String expectedTypeError;
         private final boolean allTypesAreRepresentable;
 
+        private final Class<? extends Throwable> foldingExceptionClass;
+        private final String foldingExceptionMessage;
+
         public TestCase(List<TypedData> data, String evaluatorToString, DataType expectedType, Matcher<Object> matcher) {
-            this(data, evaluatorToString, expectedType, matcher, null, null);
+            this(data, equalTo(evaluatorToString), expectedType, matcher);
+        }
+
+        public TestCase(List<TypedData> data, Matcher<String> evaluatorToString, DataType expectedType, Matcher<Object> matcher) {
+            this(data, evaluatorToString, expectedType, matcher, null, null, null, null);
         }
 
         public static TestCase typeError(List<TypedData> data, String expectedTypeError) {
-            return new TestCase(data, null, null, null, null, expectedTypeError);
+            return new TestCase(data, null, null, null, null, expectedTypeError, null, null);
         }
 
         TestCase(
             List<TypedData> data,
-            String evaluatorToString,
+            Matcher<String> evaluatorToString,
             DataType expectedType,
             Matcher<Object> matcher,
             String[] expectedWarnings,
-            String expectedTypeError
+            String expectedTypeError,
+            Class<? extends Throwable> foldingExceptionClass,
+            String foldingExceptionMessage
         ) {
             this.source = Source.EMPTY;
             this.data = data;
@@ -1184,6 +1190,8 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
             this.expectedWarnings = expectedWarnings;
             this.expectedTypeError = expectedTypeError;
             this.allTypesAreRepresentable = data.stream().allMatch(d -> EsqlDataTypes.isRepresentable(d.type));
+            this.foldingExceptionClass = foldingExceptionClass;
+            this.foldingExceptionMessage = foldingExceptionMessage;
         }
 
         public Source getSource() {
@@ -1195,15 +1203,15 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
         }
 
         public List<Expression> getDataAsFields() {
-            return data.stream().map(t -> AbstractFunctionTestCase.field(t.name(), t.type())).collect(Collectors.toList());
+            return data.stream().map(TypedData::asField).collect(Collectors.toList());
         }
 
         public List<Expression> getDataAsDeepCopiedFields() {
-            return data.stream().map(t -> AbstractFunctionTestCase.deepCopyOfField(t.name(), t.type())).collect(Collectors.toList());
+            return data.stream().map(TypedData::asDeepCopyOfField).collect(Collectors.toList());
         }
 
         public List<Expression> getDataAsLiterals() {
-            return data.stream().map(t -> new Literal(Source.synthetic(t.name()), t.data(), t.type())).collect(Collectors.toList());
+            return data.stream().map(TypedData::asLiteral).collect(Collectors.toList());
         }
 
         public List<Object> getDataValues() {
@@ -1242,13 +1250,28 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
             } else {
                 newWarnings = new String[] { warning };
             }
-            return new TestCase(data, evaluatorToString, expectedType, matcher, newWarnings, expectedTypeError);
+            return new TestCase(
+                data,
+                evaluatorToString,
+                expectedType,
+                matcher,
+                newWarnings,
+                expectedTypeError,
+                foldingExceptionClass,
+                foldingExceptionMessage
+            );
+        }
+
+        public TestCase withFoldingException(Class<? extends Throwable> clazz, String message) {
+            return new TestCase(data, evaluatorToString, expectedType, matcher, expectedWarnings, expectedTypeError, clazz, message);
+        }
+
+        public DataType expectedType() {
+            return expectedType;
         }
 
-        public <T extends Throwable> TestCase withFoldingException(Class<T> clazz, String message) {
-            foldingExceptionClass = clazz;
-            foldingExceptionMessage = message;
-            return this;
+        public Matcher<String> evaluatorToString() {
+            return evaluatorToString;
         }
     }
 
@@ -1265,18 +1288,55 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
 
     /**
      * 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
-     * @param name - a name for the value, used for generating test case names
      */
-    public record TypedData(Object data, DataType type, String name) {
-
+    public static class TypedData {
         public static final TypedData NULL = new TypedData(null, DataTypes.NULL, "<null>");
 
+        private final Object data;
+        private final DataType type;
+        private final String name;
+        private final boolean forceLiteral;
+
+        /**
+         * @param data value to test against
+         * @param type type of the value, for building expressions
+         * @param name a name for the value, used for generating test case names
+         * @param forceLiteral should this data always be converted to a literal and <strong>never</strong> to a field reference?
+         */
+        private TypedData(Object data, DataType type, String name, boolean forceLiteral) {
+            this.data = data;
+            this.type = type;
+            this.name = name;
+            this.forceLiteral = forceLiteral;
+        }
+
+        /**
+         * @param data value to test against
+         * @param type type of the value, for building expressions
+         * @param name a name for the value, used for generating test case names
+         */
+        public TypedData(Object data, DataType type, String name) {
+            this(data, type, name, false);
+        }
+
+        /**
+         * Build a value, guessing the type via reflection.
+         * @param data value to test against
+         * @param name a name for the value, used for generating test case names
+         */
         public TypedData(Object data, String name) {
             this(data, EsqlDataTypes.fromJava(data), name);
         }
 
+        /**
+         * Return a {@link TypedData} that always returns a {@link Literal} from
+         * {@link #asField} and {@link #asDeepCopyOfField}. Use this for things that
+         * must be constants.
+         */
+        public TypedData forceLiteral() {
+            return new TypedData(data, type, name, true);
+        }
+
         @Override
         public String toString() {
             if (type == DataTypes.UNSIGNED_LONG && data instanceof Long longData) {
@@ -1284,5 +1344,53 @@ public record TestCaseSupplier(String name, List<DataType> types, Supplier<TestC
             }
             return type.toString() + "(" + (data == null ? "null" : data.toString()) + ")";
         }
+
+        /**
+         * Convert this into reference to a field.
+         */
+        public Expression asField() {
+            if (forceLiteral) {
+                return asLiteral();
+            }
+            return AbstractFunctionTestCase.field(name, type);
+        }
+
+        /**
+         * Convert this into an anonymous function that performs a copy of the values loaded from a field.
+         */
+        public Expression asDeepCopyOfField() {
+            if (forceLiteral) {
+                return asLiteral();
+            }
+            return AbstractFunctionTestCase.deepCopyOfField(name, type);
+        }
+
+        /**
+         * Convert this into a {@link Literal}.
+         */
+        public Literal asLiteral() {
+            return new Literal(Source.synthetic(name), data, type);
+        }
+
+        /**
+         * Value to test against.
+         */
+        public Object data() {
+            return data;
+        }
+
+        /**
+         * Type of the value. For building {@link Expression}s.
+         */
+        public DataType type() {
+            return type;
+        }
+
+        /**
+         * A name for the value. Used to generate test names.
+         */
+        public String name() {
+            return name;
+        }
     }
 }

+ 9 - 34
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSortTests.java

@@ -15,7 +15,6 @@ import org.elasticsearch.compute.data.ElementType;
 import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase;
 import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier;
 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.DataTypes;
 
@@ -47,30 +46,6 @@ public class MvSortTests extends AbstractFunctionTestCase {
         return new MvSort(source, args.get(0), args.size() > 1 ? args.get(1) : null);
     }
 
-    /**
-     * Override to create the second argument as a Literal instead of a FieldAttribute.
-     */
-    @Override
-    protected Expression buildFieldExpression(TestCaseSupplier.TestCase testCase) {
-        List<Expression> args = new ArrayList<>(2);
-        List<TestCaseSupplier.TypedData> data = testCase.getData();
-        args.add(AbstractFunctionTestCase.field(data.get(0).name(), data.get(0).type()));
-        args.add(new Literal(Source.synthetic(data.get(1).name()), data.get(1).data(), data.get(1).type()));
-        return build(testCase.getSource(), args);
-    }
-
-    /**
-     * Override to create the second argument as a Literal instead of a FieldAttribute.
-     */
-    @Override
-    protected Expression buildDeepCopyOfFieldExpression(TestCaseSupplier.TestCase testCase) {
-        List<Expression> args = new ArrayList<>(2);
-        List<TestCaseSupplier.TypedData> data = testCase.getData();
-        args.add(AbstractFunctionTestCase.deepCopyOfField(data.get(0).name(), data.get(0).type()));
-        args.add(new Literal(Source.synthetic(data.get(1).name()), data.get(1).data(), data.get(1).type()));
-        return build(testCase.getSource(), args);
-    }
-
     private static void booleans(List<TestCaseSupplier> suppliers) {
         suppliers.add(new TestCaseSupplier(List.of(DataTypes.BOOLEAN, DataTypes.KEYWORD), () -> {
             List<Boolean> field = randomList(1, 10, () -> randomBoolean());
@@ -78,7 +53,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.BOOLEAN, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.BOOLEAN + "[field=Attribute[channel=0], order=true]",
                 DataTypes.BOOLEAN,
@@ -95,7 +70,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.INTEGER, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.INT + "[field=Attribute[channel=0], order=false]",
                 DataTypes.INTEGER,
@@ -111,7 +86,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.LONG, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.LONG + "[field=Attribute[channel=0], order=true]",
                 DataTypes.LONG,
@@ -125,7 +100,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.DATETIME, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.LONG + "[field=Attribute[channel=0], order=false]",
                 DataTypes.DATETIME,
@@ -141,7 +116,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.DOUBLE, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.DOUBLE + "[field=Attribute[channel=0], order=true]",
                 DataTypes.DOUBLE,
@@ -157,7 +132,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.KEYWORD, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.BYTES_REF + "[field=Attribute[channel=0], order=false]",
                 DataTypes.KEYWORD,
@@ -171,7 +146,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.TEXT, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.BYTES_REF + "[field=Attribute[channel=0], order=true]",
                 DataTypes.TEXT,
@@ -185,7 +160,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.IP, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.BYTES_REF + "[field=Attribute[channel=0], order=false]",
                 DataTypes.IP,
@@ -199,7 +174,7 @@ public class MvSortTests extends AbstractFunctionTestCase {
             return new TestCaseSupplier.TestCase(
                 List.of(
                     new TestCaseSupplier.TypedData(field, DataTypes.VERSION, "field"),
-                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order")
+                    new TestCaseSupplier.TypedData(order, DataTypes.KEYWORD, "order").forceLiteral()
                 ),
                 "MvSort" + ElementType.BYTES_REF + "[field=Attribute[channel=0], order=true]",
                 DataTypes.VERSION,

+ 157 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLikeTests.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.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.AbstractFunctionTestCase;
+import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier;
+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.regex.RLikePattern;
+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;
+import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.startsWith;
+
+public class RLikeTests extends AbstractFunctionTestCase {
+    public RLikeTests(@Name("TestCase") Supplier<TestCaseSupplier.TestCase> testCaseSupplier) {
+        this.testCase = testCaseSupplier.get();
+    }
+
+    @ParametersFactory
+    public static Iterable<Object[]> parameters() {
+        return parameters(() -> randomAlphaOfLength(1) + "?");
+    }
+
+    static Iterable<Object[]> parameters(Supplier<String> optionalPattern) {
+        List<TestCaseSupplier> cases = new ArrayList<>();
+        cases.add(
+            new TestCaseSupplier(
+                "null",
+                List.of(DataTypes.NULL, DataTypes.KEYWORD, DataTypes.BOOLEAN),
+                () -> new TestCaseSupplier.TestCase(
+                    List.of(
+                        new TestCaseSupplier.TypedData(null, DataTypes.NULL, "e"),
+                        new TestCaseSupplier.TypedData(new BytesRef(randomAlphaOfLength(10)), DataTypes.KEYWORD, "pattern").forceLiteral(),
+                        new TestCaseSupplier.TypedData(false, DataTypes.BOOLEAN, "caseInsensitive").forceLiteral()
+                    ),
+                    "LiteralsEvaluator[lit=null]",
+                    DataTypes.BOOLEAN,
+                    nullValue()
+                )
+            )
+        );
+        casesForString(cases, "empty string", () -> "", false, optionalPattern);
+        casesForString(cases, "single ascii character", () -> randomAlphaOfLength(1), true, optionalPattern);
+        casesForString(cases, "ascii string", () -> randomAlphaOfLengthBetween(2, 100), true, optionalPattern);
+        casesForString(cases, "3 bytes, 1 code point", () -> "☕", false, optionalPattern);
+        casesForString(cases, "6 bytes, 2 code points", () -> "❗️", false, optionalPattern);
+        casesForString(cases, "100 random code points", () -> randomUnicodeOfCodepointLength(100), true, optionalPattern);
+        for (DataType type : EsqlDataTypes.types()) {
+            if (type == DataTypes.KEYWORD || type == DataTypes.TEXT || type == DataTypes.NULL) {
+                continue;
+            }
+            if (EsqlDataTypes.isRepresentable(type) == false) {
+                continue;
+            }
+            cases.add(
+                new TestCaseSupplier(
+                    List.of(type, DataTypes.KEYWORD, DataTypes.BOOLEAN),
+                    () -> TestCaseSupplier.TestCase.typeError(
+                        List.of(
+                            new TestCaseSupplier.TypedData(randomLiteral(type).value(), type, "e"),
+                            new TestCaseSupplier.TypedData(new BytesRef(randomAlphaOfLength(10)), DataTypes.KEYWORD, "pattern")
+                                .forceLiteral(),
+                            new TestCaseSupplier.TypedData(false, DataTypes.BOOLEAN, "caseInsensitive").forceLiteral()
+                        ),
+                        "argument of [] must be [string], found value [e] type [" + type.typeName() + "]"
+                    )
+                )
+            );
+        }
+        return parameterSuppliersFromTypedData(cases);
+    }
+
+    record TextAndPattern(String text, String pattern) {}
+
+    private static void casesForString(
+        List<TestCaseSupplier> cases,
+        String title,
+        Supplier<String> textSupplier,
+        boolean canGenerateDifferent,
+        Supplier<String> optionalPattern
+    ) {
+        cases(cases, title + " matches self", () -> {
+            String text = textSupplier.get();
+            return new TextAndPattern(text, text);
+        }, true);
+        cases(cases, title + " doesn't match self with trailing", () -> {
+            String text = textSupplier.get();
+            return new TextAndPattern(text, text + randomAlphaOfLength(1));
+        }, false);
+        cases(cases, title + " matches self with optional trailing", () -> {
+            String text = randomAlphaOfLength(1);
+            return new TextAndPattern(text, text + optionalPattern.get());
+        }, true);
+        if (canGenerateDifferent) {
+            cases(cases, title + " doesn't match different", () -> {
+                String text = textSupplier.get();
+                String different = randomValueOtherThan(text, textSupplier);
+                return new TextAndPattern(text, different);
+            }, false);
+        }
+    }
+
+    private static void cases(List<TestCaseSupplier> cases, String title, Supplier<TextAndPattern> textAndPattern, boolean expected) {
+        for (DataType type : new DataType[] { DataTypes.KEYWORD, DataTypes.TEXT }) {
+            cases.add(new TestCaseSupplier(title + " with " + type.esType(), List.of(type, type, DataTypes.BOOLEAN), () -> {
+                TextAndPattern v = textAndPattern.get();
+                return new TestCaseSupplier.TestCase(
+                    List.of(
+                        new TestCaseSupplier.TypedData(new BytesRef(v.text), type, "e"),
+                        new TestCaseSupplier.TypedData(new BytesRef(v.pattern), type, "pattern").forceLiteral(),
+                        new TestCaseSupplier.TypedData(false, DataTypes.BOOLEAN, "caseInsensitive").forceLiteral()
+                    ),
+                    startsWith("AutomataMatchEvaluator[input=Attribute[channel=0], pattern=digraph Automaton {\n"),
+                    DataTypes.BOOLEAN,
+                    equalTo(expected)
+                );
+            }));
+        }
+    }
+
+    @Override
+    protected void assertSimpleWithNulls(List<Object> data, Block value, int nullBlock) {
+        assumeFalse("generated test cases containing nulls by hand", true);
+    }
+
+    @Override
+    protected Expression build(Source source, List<Expression> args) {
+        Expression expression = args.get(0);
+        Literal pattern = (Literal) args.get(1);
+        Literal caseInsensitive = (Literal) args.get(2);
+        return new RLike(
+            source,
+            expression,
+            new RLikePattern(((BytesRef) pattern.fold()).utf8ToString()),
+            (Boolean) caseInsensitive.fold()
+        );
+    }
+}

+ 50 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLikeTests.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.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.AbstractFunctionTestCase;
+import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier;
+import org.elasticsearch.xpack.ql.expression.Expression;
+import org.elasticsearch.xpack.ql.expression.Literal;
+import org.elasticsearch.xpack.ql.expression.predicate.regex.WildcardPattern;
+import org.elasticsearch.xpack.ql.tree.Source;
+
+import java.util.List;
+import java.util.function.Supplier;
+
+import static org.hamcrest.Matchers.equalTo;
+
+public class WildcardLikeTests extends AbstractFunctionTestCase {
+    public WildcardLikeTests(@Name("TestCase") Supplier<TestCaseSupplier.TestCase> testCaseSupplier) {
+        this.testCase = testCaseSupplier.get();
+    }
+
+    @ParametersFactory
+    public static Iterable<Object[]> parameters() {
+        return RLikeTests.parameters(() -> "*");
+    }
+
+    @Override
+    protected void assertSimpleWithNulls(List<Object> data, Block value, int nullBlock) {
+        assumeFalse("generated test cases containing nulls by hand", true);
+    }
+
+    @Override
+    protected Expression build(Source source, List<Expression> args) {
+        Expression expression = args.get(0);
+        Literal pattern = (Literal) args.get(1);
+        Literal caseInsensitive = (Literal) args.get(2);
+        assertThat(caseInsensitive.fold(), equalTo(false));
+        return new WildcardLike(source, expression, new WildcardPattern(((BytesRef) pattern.fold()).utf8ToString()));
+    }
+}

+ 2 - 2
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java

@@ -24,8 +24,6 @@ import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Grea
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThan;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThanOrEqual;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.NotEquals;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RLike;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.Order;
 import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.Avg;
@@ -59,7 +57,9 @@ 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.LTrim;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike;
 import org.elasticsearch.xpack.esql.expression.function.scalar.string.Substring;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Div;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Mod;

+ 2 - 2
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java

@@ -14,8 +14,8 @@ import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.Grea
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.GreaterThanOrEqual;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThan;
 import org.elasticsearch.xpack.esql.evaluator.predicate.operator.comparison.LessThanOrEqual;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.RLike;
-import org.elasticsearch.xpack.esql.evaluator.predicate.operator.regex.WildcardLike;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike;
+import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add;
 import org.elasticsearch.xpack.esql.plan.logical.Dissect;
 import org.elasticsearch.xpack.esql.plan.logical.Enrich;