Browse Source

ESQL: Prevent `CONCAT` from using a ton of memory (#99716)

This prevents `CONCAT` from using an unbounded amount of memory by
hooking it's temporary value into the circuit breaker. To do so, it
makes *all* `ExpressionEvaluator`s `Releasable`. Most of the changes in
this PR just plumb that through to every evaluator. The rest of the
changes correctly release evaluators after their use.

I considered another tactic but didn't like it as much, even though the
number of changes would be smaller - I could have created a fresh,
`Releasable` temporary value for every `Page`. It would be pretty
contained keep the releasable there. But I wanted to share the temporary
state across runs to avoid a bunch of allocations.

Here's a script that used to crash before this PR but is fine after:
```
curl -uelastic:password -XDELETE localhost:9200/test
curl -HContent-Type:application/json -uelastic:password -XPUT localhost:9200/test -d'{
   "mappings": {
      "properties": {
         "short": {
            "type": "keyword"
         }
      }
   }
}'
curl -HContent-Type:application/json -uelastic:password -XPUT localhost:9200/test/_doc/1?refresh -d'{"short": "short"}'

echo -n '{"query": "FROM test ' > /tmp/evil
for i in {0..9}; do
   echo -n '| EVAL short = CONCAT(short' >> /tmp/evil
   for j in {1..9}; do
      echo -n ', short' >> /tmp/evil
   done
   echo -n ')' >> /tmp/evil
done
echo '| EVAL len = LENGTH(short) | KEEP len"}'>> /tmp/evil
curl -HContent-Type:application/json -uelastic:password -XPOST localhost:9200/_query?pretty --data-binary @/tmp/evil
```
Nik Everett 2 years ago
parent
commit
5e3ab06151
100 changed files with 671 additions and 24 deletions
  1. 1 0
      docs/reference/esql/functions/signature/to_ip.svg
  2. 1 0
      docs/reference/esql/functions/types/concat.asciidoc
  3. 6 0
      docs/reference/esql/functions/types/to_ip.asciidoc
  4. 68 5
      x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java
  5. 1 0
      x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorProcessor.java
  6. 18 0
      x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Types.java
  7. 1 1
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AbstractPageMappingOperator.java
  8. 5 0
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ColumnExtractOperator.java
  9. 8 0
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverContext.java
  10. 17 1
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EvalOperator.java
  11. 6 0
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FilterOperator.java
  12. 3 0
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupe.java
  13. 4 2
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/StringExtractOperator.java
  14. 1 8
      x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java
  15. 13 1
      x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ColumnExtractOperatorTests.java
  16. 3 0
      x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java
  17. 3 0
      x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java
  18. 22 6
      x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/StringExtractOperatorTests.java
  19. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsBoolsEvaluator.java
  20. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsDoublesEvaluator.java
  21. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsIntsEvaluator.java
  22. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsKeywordsEvaluator.java
  23. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsLongsEvaluator.java
  24. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanDoublesEvaluator.java
  25. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanIntsEvaluator.java
  26. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanKeywordsEvaluator.java
  27. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanLongsEvaluator.java
  28. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualDoublesEvaluator.java
  29. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualIntsEvaluator.java
  30. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualKeywordsEvaluator.java
  31. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualLongsEvaluator.java
  32. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanDoublesEvaluator.java
  33. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanIntsEvaluator.java
  34. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanKeywordsEvaluator.java
  35. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanLongsEvaluator.java
  36. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualDoublesEvaluator.java
  37. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualIntsEvaluator.java
  38. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualKeywordsEvaluator.java
  39. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualLongsEvaluator.java
  40. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsBoolsEvaluator.java
  41. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsDoublesEvaluator.java
  42. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsIntsEvaluator.java
  43. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsKeywordsEvaluator.java
  44. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsLongsEvaluator.java
  45. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/logical/NotEvaluator.java
  46. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMatchEvaluator.java
  47. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestBooleanEvaluator.java
  48. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestBytesRefEvaluator.java
  49. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestDoubleEvaluator.java
  50. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestIntEvaluator.java
  51. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestLongEvaluator.java
  52. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastBooleanEvaluator.java
  53. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastBytesRefEvaluator.java
  54. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastDoubleEvaluator.java
  55. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastIntEvaluator.java
  56. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastLongEvaluator.java
  57. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractConstantEvaluator.java
  58. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractEvaluator.java
  59. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatConstantEvaluator.java
  60. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatEvaluator.java
  61. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseConstantEvaluator.java
  62. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseEvaluator.java
  63. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java
  64. 4 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/NowEvaluator.java
  65. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatchEvaluator.java
  66. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsDoubleEvaluator.java
  67. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsIntEvaluator.java
  68. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AbsLongEvaluator.java
  69. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AcosEvaluator.java
  70. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AsinEvaluator.java
  71. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Atan2Evaluator.java
  72. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/AtanEvaluator.java
  73. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToDoubleEvaluator.java
  74. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToLongEvaluator.java
  75. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastIntToUnsignedLongEvaluator.java
  76. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToDoubleEvaluator.java
  77. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastLongToUnsignedLongEvaluator.java
  78. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CastUnsignedLongToDoubleEvaluator.java
  79. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CeilDoubleEvaluator.java
  80. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CosEvaluator.java
  81. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/CoshEvaluator.java
  82. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/FloorDoubleEvaluator.java
  83. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsFiniteEvaluator.java
  84. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsInfiniteEvaluator.java
  85. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/IsNaNEvaluator.java
  86. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10DoubleEvaluator.java
  87. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10IntEvaluator.java
  88. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10LongEvaluator.java
  89. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/Log10UnsignedLongEvaluator.java
  90. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowDoubleEvaluator.java
  91. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowIntEvaluator.java
  92. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/PowLongEvaluator.java
  93. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleEvaluator.java
  94. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundDoubleNoDecimalsEvaluator.java
  95. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundIntEvaluator.java
  96. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundLongEvaluator.java
  97. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/RoundUnsignedLongEvaluator.java
  98. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinEvaluator.java
  99. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SinhEvaluator.java
  100. 6 0
      x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/math/SqrtDoubleEvaluator.java

+ 1 - 0
docs/reference/esql/functions/signature/to_ip.svg

@@ -0,0 +1 @@
+<svg version="1.1" xmlns:xlink="http://www.w3.org/1999/xlink" xmlns="http://www.w3.org/2000/svg" width="252" height="46" viewbox="0 0 252 46"><defs><style type="text/css">#guide .c{fill:none;stroke:#222222;}#guide .k{fill:#000000;font-family:Roboto Mono,Sans-serif;font-size:20px;}#guide .s{fill:#e4f4ff;stroke:#222222;}#guide .syn{fill:#8D8D8D;font-family:Roboto Mono,Sans-serif;font-size:20px;}</style></defs><path class="c" d="M0 31h5m80 0h10m32 0h10m68 0h10m32 0h5"/><rect class="s" x="5" y="5" width="80" height="36"/><text class="k" x="15" y="31">TO_IP</text><rect class="s" x="95" y="5" width="32" height="36" rx="7"/><text class="syn" x="105" y="31">(</text><rect class="s" x="137" y="5" width="68" height="36" rx="7"/><text class="k" x="147" y="31">arg1</text><rect class="s" x="215" y="5" width="32" height="36" rx="7"/><text class="syn" x="225" y="31">)</text></svg>

+ 1 - 0
docs/reference/esql/functions/types/concat.asciidoc

@@ -2,4 +2,5 @@
 |===
 arg1 | arg2... | result
 keyword | keyword | keyword
+text | text | keyword
 |===

+ 6 - 0
docs/reference/esql/functions/types/to_ip.asciidoc

@@ -0,0 +1,6 @@
+[%header.monospaced.styled,format=dsv,separator=|]
+|===
+arg1 | result
+ip | ip
+keyword | ip
+|===

+ 68 - 5
x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java

@@ -47,9 +47,15 @@ public class EvaluatorImplementer {
     private final ProcessFunction processFunction;
     private final ClassName implementation;
 
-    public EvaluatorImplementer(Elements elements, ExecutableElement processFunction, String extraName, List<TypeMirror> warnExceptions) {
+    public EvaluatorImplementer(
+        Elements elements,
+        javax.lang.model.util.Types types,
+        ExecutableElement processFunction,
+        String extraName,
+        List<TypeMirror> warnExceptions
+    ) {
         this.declarationType = (TypeElement) processFunction.getEnclosingElement();
-        this.processFunction = new ProcessFunction(processFunction, warnExceptions);
+        this.processFunction = new ProcessFunction(elements, types, processFunction, warnExceptions);
 
         this.implementation = ClassName.get(
             elements.getPackageOf(declarationType).toString(),
@@ -87,6 +93,7 @@ public class EvaluatorImplementer {
         }
         builder.addMethod(realEval(false));
         builder.addMethod(toStringMethod());
+        builder.addMethod(close());
         return builder.build();
     }
 
@@ -219,6 +226,20 @@ public class EvaluatorImplementer {
         return builder.build();
     }
 
+    private MethodSpec close() {
+        MethodSpec.Builder builder = MethodSpec.methodBuilder("close").addAnnotation(Override.class);
+        builder.addModifiers(Modifier.PUBLIC);
+
+        List<String> invocations = processFunction.args.stream().map(ProcessFunctionArg::closeInvocation).filter(s -> s != null).toList();
+        if (invocations.isEmpty() == false) {
+            builder.addStatement(
+                "$T.closeExpectNoException(" + invocations.stream().collect(Collectors.joining(", ")) + ")",
+                Types.RELEASABLES
+            );
+        }
+        return builder.build();
+    }
+
     private interface ProcessFunctionArg {
         /**
          * Type containing the actual data for a page of values for this field. Usually a
@@ -276,7 +297,15 @@ public class EvaluatorImplementer {
          */
         void buildInvocation(StringBuilder pattern, List<Object> args, boolean blockStyle);
 
+        /**
+         * Accumulate invocation pattern and arguments to implement {@link Object#toString()}.
+         */
         void buildToStringInvocation(StringBuilder pattern, List<Object> args, String prefix);
+
+        /**
+         * The string to close this argument or {@code null}.
+         */
+        String closeInvocation();
     }
 
     private record StandardProcessFunctionArg(TypeName type, String name) implements ProcessFunctionArg {
@@ -368,6 +397,11 @@ public class EvaluatorImplementer {
             args.add(prefix + name + "=");
             args.add(name);
         }
+
+        @Override
+        public String closeInvocation() {
+            return name;
+        }
     }
 
     private record ArrayProcessFunctionArg(TypeName componentType, String name) implements ProcessFunctionArg {
@@ -470,9 +504,16 @@ public class EvaluatorImplementer {
             args.add(Arrays.class);
             args.add(name);
         }
+
+        @Override
+        public String closeInvocation() {
+            return "() -> Releasables.close(" + name + ")";
+        }
     }
 
-    private record FixedProcessFunctionArg(TypeName type, String name, boolean includeInToString) implements ProcessFunctionArg {
+    private record FixedProcessFunctionArg(TypeName type, String name, boolean includeInToString, boolean releasable)
+        implements
+            ProcessFunctionArg {
         @Override
         public TypeName dataType(boolean blockStyle) {
             return type;
@@ -534,6 +575,11 @@ public class EvaluatorImplementer {
                 args.add(name);
             }
         }
+
+        @Override
+        public String closeInvocation() {
+            return releasable ? name : null;
+        }
     }
 
     private record BuilderProcessFunctionArg(ClassName type, String name) implements ProcessFunctionArg {
@@ -593,6 +639,11 @@ public class EvaluatorImplementer {
         public void buildToStringInvocation(StringBuilder pattern, List<Object> args, String prefix) {
             // Don't want to include
         }
+
+        @Override
+        public String closeInvocation() {
+            return null;
+        }
     }
 
     private static class ProcessFunction {
@@ -601,7 +652,12 @@ public class EvaluatorImplementer {
         private final BuilderProcessFunctionArg builderArg;
         private final List<TypeMirror> warnExceptions;
 
-        private ProcessFunction(ExecutableElement function, List<TypeMirror> warnExceptions) {
+        private ProcessFunction(
+            Elements elements,
+            javax.lang.model.util.Types types,
+            ExecutableElement function,
+            List<TypeMirror> warnExceptions
+        ) {
             this.function = function;
             args = new ArrayList<>();
             BuilderProcessFunctionArg builderArg = null;
@@ -610,7 +666,14 @@ public class EvaluatorImplementer {
                 String name = v.getSimpleName().toString();
                 Fixed fixed = v.getAnnotation(Fixed.class);
                 if (fixed != null) {
-                    args.add(new FixedProcessFunctionArg(type, name, fixed.includeInToString()));
+                    args.add(
+                        new FixedProcessFunctionArg(
+                            type,
+                            name,
+                            fixed.includeInToString(),
+                            Types.extendsSuper(types, v.asType(), "org.elasticsearch.core.Releasable")
+                        )
+                    );
                     continue;
                 }
                 if (type instanceof ClassName c

+ 1 - 0
x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorProcessor.java

@@ -75,6 +75,7 @@ public class EvaluatorProcessor implements Processor {
                         "evaluator",
                         new EvaluatorImplementer(
                             env.getElementUtils(),
+                            env.getTypeUtils(),
                             (ExecutableElement) evaluatorMethod,
                             evaluatorAnn.extraName(),
                             warnExceptions(evaluatorMethod)

+ 18 - 0
x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/Types.java

@@ -12,8 +12,12 @@ import com.squareup.javapoet.ClassName;
 import com.squareup.javapoet.ParameterizedTypeName;
 import com.squareup.javapoet.TypeName;
 
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.List;
 
+import javax.lang.model.type.TypeMirror;
+
 /**
  * Types used by the code generator.
  */
@@ -113,6 +117,8 @@ public class Types {
 
     static final ClassName BYTES_REF = ClassName.get("org.apache.lucene.util", "BytesRef");
 
+    static final ClassName RELEASABLES = ClassName.get("org.elasticsearch.core", "Releasables");
+
     static ClassName blockType(TypeName elementType) {
         if (elementType.equals(TypeName.BOOLEAN)) {
             return BOOLEAN_BLOCK;
@@ -265,4 +271,16 @@ public class Types {
         throw new IllegalArgumentException("unknown element type for [" + t + "]");
     }
 
+    static boolean extendsSuper(javax.lang.model.util.Types types, TypeMirror c, String superName) {
+        Deque<TypeMirror> mirrors = new ArrayDeque<>();
+        mirrors.add(c);
+        while (mirrors.isEmpty() == false) {
+            TypeMirror m = mirrors.pop();
+            if (m.toString().equals(superName)) {
+                return true;
+            }
+            mirrors.addAll(types.directSupertypes(m));
+        }
+        return false;
+    }
 }

+ 1 - 1
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/AbstractPageMappingOperator.java

@@ -79,7 +79,7 @@ public abstract class AbstractPageMappingOperator implements Operator {
     }
 
     @Override
-    public final void close() {}
+    public void close() {}
 
     public static class Status implements Operator.Status {
         public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(

+ 5 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/ColumnExtractOperator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.BytesRefBlock;
 import org.elasticsearch.compute.data.ElementType;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
+import org.elasticsearch.core.Releasables;
 
 import java.util.function.Supplier;
 
@@ -91,4 +92,8 @@ public class ColumnExtractOperator extends AbstractPageMappingOperator {
         void computeRow(BytesRefBlock input, int row, Block.Builder[] target, BytesRef spare);
     }
 
+    @Override
+    public void close() {
+        Releasables.closeExpectNoException(inputEvaluator);
+    }
 }

+ 8 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverContext.java

@@ -7,6 +7,7 @@
 
 package org.elasticsearch.compute.operator;
 
+import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.compute.data.BlockFactory;
 import org.elasticsearch.core.Releasable;
@@ -56,6 +57,13 @@ public class DriverContext {
         return bigArrays;
     }
 
+    /**
+     * The {@link CircuitBreaker} to use to track memory.
+     */
+    public CircuitBreaker breaker() {
+        return bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST);
+    }
+
     public BlockFactory blockFactory() {
         return blockFactory;
     }

+ 17 - 1
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/EvalOperator.java

@@ -9,6 +9,8 @@ package org.elasticsearch.compute.operator;
 
 import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.core.Releasable;
+import org.elasticsearch.core.Releasables;
 
 /**
  * Evaluates a tree of functions for every position in the block, resulting in a
@@ -25,6 +27,7 @@ public class EvalOperator extends AbstractPageMappingOperator {
 
         @Override
         public String describe() {
+            // TODO ThrowingDriverContext blows up when combined with Concat
             return "EvalOperator[evaluator=" + evaluator.get(new ThrowingDriverContext()) + "]";
         }
     }
@@ -45,13 +48,23 @@ public class EvalOperator extends AbstractPageMappingOperator {
         return getClass().getSimpleName() + "[evaluator=" + evaluator + "]";
     }
 
-    public interface ExpressionEvaluator {
+    @Override
+    public void close() {
+        Releasables.closeExpectNoException(evaluator);
+    }
 
+    /**
+     * Evaluates an expression {@code a + b} or {@code log(c)} one {@link Page} at a time.
+     */
+    public interface ExpressionEvaluator extends Releasable {
         /** A Factory for creating ExpressionEvaluators. */
         interface Factory {
             ExpressionEvaluator get(DriverContext driverContext);
         }
 
+        /**
+         * Evaluate the expression.
+         */
         Block eval(Page page);
     }
 
@@ -65,5 +78,8 @@ public class EvalOperator extends AbstractPageMappingOperator {
         public String toString() {
             return "ConstantNull";
         }
+
+        @Override
+        public void close() {}
     };
 }

+ 6 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FilterOperator.java

@@ -11,6 +11,7 @@ import org.elasticsearch.compute.data.Block;
 import org.elasticsearch.compute.data.BooleanBlock;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
+import org.elasticsearch.core.Releasables;
 
 import java.util.Arrays;
 
@@ -79,4 +80,9 @@ public class FilterOperator extends AbstractPageMappingOperator {
     public String toString() {
         return "FilterOperator[" + "evaluator=" + evaluator + ']';
     }
+
+    @Override
+    public void close() {
+        Releasables.closeExpectNoException(evaluator);
+    }
 }

+ 3 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/MultivalueDedupe.java

@@ -152,6 +152,9 @@ public final class MultivalueDedupe {
         public String toString() {
             return "MvDedupe[field=" + field + "]";
         }
+
+        @Override
+        public void close() {}
     }
 
     private MultivalueDedupe() {}

+ 4 - 2
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/StringExtractOperator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.BytesRefBlock;
 import org.elasticsearch.compute.data.ElementType;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
+import org.elasticsearch.core.Releasables;
 
 import java.util.Arrays;
 import java.util.Map;
@@ -137,7 +138,8 @@ public class StringExtractOperator extends AbstractPageMappingOperator {
         return "StringExtractOperator[fields=[" + Arrays.stream(fieldNames).collect(Collectors.joining(", ")) + "]]";
     }
 
-    public interface ExtractEvaluator {
-        Map<String, Object> computeRow(Page page, int position);
+    @Override
+    public void close() {
+        Releasables.closeExpectNoException(inputEvaluator);
     }
 }

+ 1 - 8
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java

@@ -205,14 +205,7 @@ public class TopNOperator implements Operator, Accountable {
 
         @Override
         public TopNOperator get(DriverContext driverContext) {
-            return new TopNOperator(
-                driverContext.bigArrays().breakerService().getBreaker(CircuitBreaker.REQUEST),
-                topCount,
-                elementTypes,
-                encoders,
-                sortOrders,
-                maxPageSize
-            );
+            return new TopNOperator(driverContext.breaker(), topCount, elementTypes, encoders, sortOrders, maxPageSize);
         }
 
         @Override

+ 13 - 1
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ColumnExtractOperatorTests.java

@@ -47,7 +47,19 @@ public class ColumnExtractOperatorTests extends OperatorTestCase {
     @Override
     protected Operator.OperatorFactory simple(BigArrays bigArrays) {
         Supplier<ColumnExtractOperator.Evaluator> expEval = () -> new FirstWord(0);
-        return new ColumnExtractOperator.Factory(new ElementType[] { ElementType.BYTES_REF }, dvrCtx -> page -> page.getBlock(0), expEval);
+        return new ColumnExtractOperator.Factory(
+            new ElementType[] { ElementType.BYTES_REF },
+            dvrCtx -> new EvalOperator.ExpressionEvaluator() {
+                @Override
+                public Block eval(Page page) {
+                    return page.getBlock(0);
+                }
+
+                @Override
+                public void close() {}
+            },
+            expEval
+        );
     }
 
     @Override

+ 3 - 0
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java

@@ -36,6 +36,9 @@ public class EvalOperatorTests extends OperatorTestCase {
             }
             return result.build().asBlock();
         }
+
+        @Override
+        public void close() {}
     }
 
     @Override

+ 3 - 0
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java

@@ -38,6 +38,9 @@ public class FilterOperatorTests extends OperatorTestCase {
             }
             return result.build().asBlock();
         }
+
+        @Override
+        public void close() {}
     }
 
     @Override

+ 22 - 6
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/StringExtractOperatorTests.java

@@ -42,7 +42,19 @@ public class StringExtractOperatorTests extends OperatorTestCase {
     @Override
     protected Operator.OperatorFactory simple(BigArrays bigArrays) {
         Supplier<Function<String, Map<String, String>>> expEval = () -> new FirstWord("test");
-        return new StringExtractOperator.StringExtractOperatorFactory(new String[] { "test" }, dvrCtx -> page -> page.getBlock(0), expEval);
+        return new StringExtractOperator.StringExtractOperatorFactory(
+            new String[] { "test" },
+            dvrCtx -> new EvalOperator.ExpressionEvaluator() {
+                @Override
+                public Block eval(Page page) {
+                    return page.getBlock(0);
+                }
+
+                @Override
+                public void close() {}
+            },
+            expEval
+        );
     }
 
     @Override
@@ -77,11 +89,15 @@ public class StringExtractOperatorTests extends OperatorTestCase {
 
     public void testMultivalueDissectInput() {
 
-        StringExtractOperator operator = new StringExtractOperator(
-            new String[] { "test" },
-            (page) -> page.getBlock(0),
-            new FirstWord("test")
-        );
+        StringExtractOperator operator = new StringExtractOperator(new String[] { "test" }, new EvalOperator.ExpressionEvaluator() {
+            @Override
+            public Block eval(Page page) {
+                return page.getBlock(0);
+            }
+
+            @Override
+            public void close() {}
+        }, new FirstWord("test"));
 
         BytesRefBlock.Builder builder = BytesRefBlock.newBlockBuilder(1);
         builder.beginPositionEntry();

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsBoolsEvaluator.java

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.BooleanVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}.
@@ -82,4 +83,9 @@ public final class EqualsBoolsEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "EqualsBoolsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}.
@@ -84,4 +85,9 @@ public final class EqualsDoublesEvaluator implements EvalOperator.ExpressionEval
   public String toString() {
     return "EqualsDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}.
@@ -84,4 +85,9 @@ public final class EqualsIntsEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "EqualsIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}.
@@ -89,4 +90,9 @@ public final class EqualsKeywordsEvaluator implements EvalOperator.ExpressionEva
   public String toString() {
     return "EqualsKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/EqualsLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Equals}.
@@ -84,4 +85,9 @@ public final class EqualsLongsEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "EqualsLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}.
@@ -84,4 +85,9 @@ public final class GreaterThanDoublesEvaluator implements EvalOperator.Expressio
   public String toString() {
     return "GreaterThanDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}.
@@ -84,4 +85,9 @@ public final class GreaterThanIntsEvaluator implements EvalOperator.ExpressionEv
   public String toString() {
     return "GreaterThanIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}.
@@ -89,4 +90,9 @@ public final class GreaterThanKeywordsEvaluator implements EvalOperator.Expressi
   public String toString() {
     return "GreaterThanKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThan}.
@@ -84,4 +85,9 @@ public final class GreaterThanLongsEvaluator implements EvalOperator.ExpressionE
   public String toString() {
     return "GreaterThanLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}.
@@ -84,4 +85,9 @@ public final class GreaterThanOrEqualDoublesEvaluator implements EvalOperator.Ex
   public String toString() {
     return "GreaterThanOrEqualDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}.
@@ -84,4 +85,9 @@ public final class GreaterThanOrEqualIntsEvaluator implements EvalOperator.Expre
   public String toString() {
     return "GreaterThanOrEqualIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}.
@@ -89,4 +90,9 @@ public final class GreaterThanOrEqualKeywordsEvaluator implements EvalOperator.E
   public String toString() {
     return "GreaterThanOrEqualKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/GreaterThanOrEqualLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link GreaterThanOrEqual}.
@@ -84,4 +85,9 @@ public final class GreaterThanOrEqualLongsEvaluator implements EvalOperator.Expr
   public String toString() {
     return "GreaterThanOrEqualLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}.
@@ -84,4 +85,9 @@ public final class LessThanDoublesEvaluator implements EvalOperator.ExpressionEv
   public String toString() {
     return "LessThanDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}.
@@ -84,4 +85,9 @@ public final class LessThanIntsEvaluator implements EvalOperator.ExpressionEvalu
   public String toString() {
     return "LessThanIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}.
@@ -89,4 +90,9 @@ public final class LessThanKeywordsEvaluator implements EvalOperator.ExpressionE
   public String toString() {
     return "LessThanKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThan}.
@@ -84,4 +85,9 @@ public final class LessThanLongsEvaluator implements EvalOperator.ExpressionEval
   public String toString() {
     return "LessThanLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}.
@@ -84,4 +85,9 @@ public final class LessThanOrEqualDoublesEvaluator implements EvalOperator.Expre
   public String toString() {
     return "LessThanOrEqualDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}.
@@ -84,4 +85,9 @@ public final class LessThanOrEqualIntsEvaluator implements EvalOperator.Expressi
   public String toString() {
     return "LessThanOrEqualIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}.
@@ -89,4 +90,9 @@ public final class LessThanOrEqualKeywordsEvaluator implements EvalOperator.Expr
   public String toString() {
     return "LessThanOrEqualKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/LessThanOrEqualLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link LessThanOrEqual}.
@@ -84,4 +85,9 @@ public final class LessThanOrEqualLongsEvaluator implements EvalOperator.Express
   public String toString() {
     return "LessThanOrEqualLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsBoolsEvaluator.java

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.BooleanVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}.
@@ -82,4 +83,9 @@ public final class NotEqualsBoolsEvaluator implements EvalOperator.ExpressionEva
   public String toString() {
     return "NotEqualsBoolsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsDoublesEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}.
@@ -84,4 +85,9 @@ public final class NotEqualsDoublesEvaluator implements EvalOperator.ExpressionE
   public String toString() {
     return "NotEqualsDoublesEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsIntsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}.
@@ -84,4 +85,9 @@ public final class NotEqualsIntsEvaluator implements EvalOperator.ExpressionEval
   public String toString() {
     return "NotEqualsIntsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsKeywordsEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}.
@@ -89,4 +90,9 @@ public final class NotEqualsKeywordsEvaluator implements EvalOperator.Expression
   public String toString() {
     return "NotEqualsKeywordsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/comparison/NotEqualsLongsEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link NotEquals}.
@@ -84,4 +85,9 @@ public final class NotEqualsLongsEvaluator implements EvalOperator.ExpressionEva
   public String toString() {
     return "NotEqualsLongsEvaluator[" + "lhs=" + lhs + ", rhs=" + rhs + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(lhs, rhs);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/logical/NotEvaluator.java

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.BooleanVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Not}.
@@ -65,4 +66,9 @@ public final class NotEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "NotEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/evaluator/predicate/operator/regex/RegexMatchEvaluator.java

@@ -16,6 +16,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link RegexMatch}.
@@ -75,4 +76,9 @@ public final class RegexMatchEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "RegexMatchEvaluator[" + "input=" + input + ", pattern=" + pattern + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(input);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestBooleanEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.BooleanVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Greatest}.
@@ -86,4 +87,9 @@ public final class GreatestBooleanEvaluator implements EvalOperator.ExpressionEv
   public String toString() {
     return "GreatestBooleanEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestBytesRefEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Greatest}.
@@ -95,4 +96,9 @@ public final class GreatestBytesRefEvaluator implements EvalOperator.ExpressionE
   public String toString() {
     return "GreatestBytesRefEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestDoubleEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Greatest}.
@@ -86,4 +87,9 @@ public final class GreatestDoubleEvaluator implements EvalOperator.ExpressionEva
   public String toString() {
     return "GreatestDoubleEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestIntEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Greatest}.
@@ -86,4 +87,9 @@ public final class GreatestIntEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "GreatestIntEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/GreatestLongEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Greatest}.
@@ -86,4 +87,9 @@ public final class GreatestLongEvaluator implements EvalOperator.ExpressionEvalu
   public String toString() {
     return "GreatestLongEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastBooleanEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.BooleanVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Least}.
@@ -86,4 +87,9 @@ public final class LeastBooleanEvaluator implements EvalOperator.ExpressionEvalu
   public String toString() {
     return "LeastBooleanEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastBytesRefEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Least}.
@@ -95,4 +96,9 @@ public final class LeastBytesRefEvaluator implements EvalOperator.ExpressionEval
   public String toString() {
     return "LeastBytesRefEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastDoubleEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Least}.
@@ -86,4 +87,9 @@ public final class LeastDoubleEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "LeastDoubleEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastIntEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Least}.
@@ -85,4 +86,9 @@ public final class LeastIntEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "LeastIntEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/conditional/LeastLongEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Least}.
@@ -86,4 +87,9 @@ public final class LeastLongEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "LeastLongEvaluator[" + "values=" + Arrays.toString(values) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(() -> Releasables.close(values));
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractConstantEvaluator.java

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateExtract}.
@@ -74,4 +75,9 @@ public final class DateExtractConstantEvaluator implements EvalOperator.Expressi
   public String toString() {
     return "DateExtractConstantEvaluator[" + "value=" + value + ", chronoField=" + chronoField + ", zone=" + zone + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(value);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateExtractEvaluator.java

@@ -17,6 +17,7 @@ import org.elasticsearch.compute.data.LongVector;
 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;
 
@@ -108,4 +109,9 @@ public final class DateExtractEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "DateExtractEvaluator[" + "value=" + value + ", chronoField=" + chronoField + ", zone=" + zone + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(value, chronoField);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatConstantEvaluator.java

@@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateFormat}.
@@ -72,4 +73,9 @@ public final class DateFormatConstantEvaluator implements EvalOperator.Expressio
   public String toString() {
     return "DateFormatConstantEvaluator[" + "val=" + val + ", formatter=" + formatter + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateFormatEvaluator.java

@@ -16,6 +16,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateFormat}.
@@ -92,4 +93,9 @@ public final class DateFormatEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "DateFormatEvaluator[" + "val=" + val + ", formatter=" + formatter + ", locale=" + locale + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, formatter);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseConstantEvaluator.java

@@ -16,6 +16,7 @@ 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;
 
@@ -90,4 +91,9 @@ public final class DateParseConstantEvaluator implements EvalOperator.Expression
   public String toString() {
     return "DateParseConstantEvaluator[" + "val=" + val + ", formatter=" + formatter + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateParseEvaluator.java

@@ -16,6 +16,7 @@ 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;
 
@@ -109,4 +110,9 @@ public final class DateParseEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "DateParseEvaluator[" + "val=" + val + ", formatter=" + formatter + ", zoneId=" + zoneId + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, formatter);
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateTrunc}.
@@ -70,4 +71,9 @@ public final class DateTruncEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "DateTruncEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(fieldVal);
+  }
 }

+ 4 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/NowEvaluator.java

@@ -43,4 +43,8 @@ public final class NowEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "NowEvaluator[" + "now=" + now + "]";
   }
+
+  @Override
+  public void close() {
+  }
 }

+ 6 - 0
x-pack/plugin/esql/src/main/java/generated/org/elasticsearch/xpack/esql/expression/function/scalar/ip/CIDRMatchEvaluator.java

@@ -16,6 +16,7 @@ import org.elasticsearch.compute.data.BytesRefVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link CIDRMatch}.
@@ -116,4 +117,9 @@ public final class CIDRMatchEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "CIDRMatchEvaluator[" + "ip=" + ip + ", cidrs=" + Arrays.toString(cidrs) + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(ip, () -> Releasables.close(cidrs));
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}.
@@ -66,4 +67,9 @@ public final class AbsDoubleEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "AbsDoubleEvaluator[" + "fieldVal=" + fieldVal + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(fieldVal);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}.
@@ -65,4 +66,9 @@ public final class AbsIntEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "AbsIntEvaluator[" + "fieldVal=" + fieldVal + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(fieldVal);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Abs}.
@@ -65,4 +66,9 @@ public final class AbsLongEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "AbsLongEvaluator[" + "fieldVal=" + fieldVal + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(fieldVal);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class AcosEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "AcosEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class AsinEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "AsinEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Atan2}.
@@ -82,4 +83,9 @@ public final class Atan2Evaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "Atan2Evaluator[" + "y=" + y + ", x=" + x + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(y, x);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Atan}.
@@ -65,4 +66,9 @@ public final class AtanEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "AtanEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -67,4 +68,9 @@ public final class CastIntToDoubleEvaluator implements EvalOperator.ExpressionEv
   public String toString() {
     return "CastIntToDoubleEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -67,4 +68,9 @@ public final class CastIntToLongEvaluator implements EvalOperator.ExpressionEval
   public String toString() {
     return "CastIntToLongEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -68,4 +69,9 @@ public final class CastIntToUnsignedLongEvaluator implements EvalOperator.Expres
   public String toString() {
     return "CastIntToUnsignedLongEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -68,4 +69,9 @@ public final class CastLongToDoubleEvaluator implements EvalOperator.ExpressionE
   public String toString() {
     return "CastLongToDoubleEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -66,4 +67,9 @@ public final class CastLongToUnsignedLongEvaluator implements EvalOperator.Expre
   public String toString() {
     return "CastLongToUnsignedLongEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cast}.
@@ -68,4 +69,9 @@ public final class CastUnsignedLongToDoubleEvaluator implements EvalOperator.Exp
   public String toString() {
     return "CastUnsignedLongToDoubleEvaluator[" + "v=" + v + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(v);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Ceil}.
@@ -65,4 +66,9 @@ public final class CeilDoubleEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "CeilDoubleEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Cos}.
@@ -65,4 +66,9 @@ public final class CosEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "CosEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class CoshEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "CoshEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Floor}.
@@ -65,4 +66,9 @@ public final class FloorDoubleEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "FloorDoubleEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsFinite}.
@@ -67,4 +68,9 @@ public final class IsFiniteEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "IsFiniteEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsInfinite}.
@@ -67,4 +68,9 @@ public final class IsInfiniteEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "IsInfiniteEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link IsNaN}.
@@ -67,4 +68,9 @@ public final class IsNaNEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "IsNaNEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class Log10DoubleEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "Log10DoubleEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.IntVector;
 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;
 
@@ -83,4 +84,9 @@ public final class Log10IntEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "Log10IntEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 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;
 
@@ -83,4 +84,9 @@ public final class Log10LongEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "Log10LongEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 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;
 
@@ -83,4 +84,9 @@ public final class Log10UnsignedLongEvaluator implements EvalOperator.Expression
   public String toString() {
     return "Log10UnsignedLongEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -98,4 +99,9 @@ public final class PowDoubleEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "PowDoubleEvaluator[" + "base=" + base + ", exponent=" + exponent + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(base, exponent);
+  }
 }

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

@@ -14,6 +14,7 @@ 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;
 
@@ -99,4 +100,9 @@ public final class PowIntEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "PowIntEvaluator[" + "base=" + base + ", exponent=" + exponent + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(base, exponent);
+  }
 }

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

@@ -14,6 +14,7 @@ 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;
 
@@ -99,4 +100,9 @@ public final class PowLongEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "PowLongEvaluator[" + "base=" + base + ", exponent=" + exponent + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(base, exponent);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}.
@@ -84,4 +85,9 @@ public final class RoundDoubleEvaluator implements EvalOperator.ExpressionEvalua
   public String toString() {
     return "RoundDoubleEvaluator[" + "val=" + val + ", decimals=" + decimals + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, decimals);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}.
@@ -66,4 +67,9 @@ public final class RoundDoubleNoDecimalsEvaluator implements EvalOperator.Expres
   public String toString() {
     return "RoundDoubleNoDecimalsEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}.
@@ -84,4 +85,9 @@ public final class RoundIntEvaluator implements EvalOperator.ExpressionEvaluator
   public String toString() {
     return "RoundIntEvaluator[" + "val=" + val + ", decimals=" + decimals + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, decimals);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}.
@@ -82,4 +83,9 @@ public final class RoundLongEvaluator implements EvalOperator.ExpressionEvaluato
   public String toString() {
     return "RoundLongEvaluator[" + "val=" + val + ", decimals=" + decimals + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, decimals);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.LongVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Round}.
@@ -82,4 +83,9 @@ public final class RoundUnsignedLongEvaluator implements EvalOperator.Expression
   public String toString() {
     return "RoundUnsignedLongEvaluator[" + "val=" + val + ", decimals=" + decimals + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val, decimals);
+  }
 }

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

@@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 import org.elasticsearch.compute.data.Page;
 import org.elasticsearch.compute.operator.DriverContext;
 import org.elasticsearch.compute.operator.EvalOperator;
+import org.elasticsearch.core.Releasables;
 
 /**
  * {@link EvalOperator.ExpressionEvaluator} implementation for {@link Sin}.
@@ -65,4 +66,9 @@ public final class SinEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "SinEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class SinhEvaluator implements EvalOperator.ExpressionEvaluator {
   public String toString() {
     return "SinhEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

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

@@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.DoubleVector;
 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;
 
@@ -82,4 +83,9 @@ public final class SqrtDoubleEvaluator implements EvalOperator.ExpressionEvaluat
   public String toString() {
     return "SqrtDoubleEvaluator[" + "val=" + val + "]";
   }
+
+  @Override
+  public void close() {
+    Releasables.closeExpectNoException(val);
+  }
 }

Some files were not shown because too many files changed in this diff