Procházet zdrojové kódy

QL: Introduce parameterized rule and executor (#92428)

Extend the QL rule (and executor) to allow passing a Context object to
 pass runtime information to certain rules. This avoids having to
 encapsulate the state into the rule by passing them as argument
 allowing the same rule instance to be reused across multiple queries at
 the same time. A common example of this are Analyzers which rely on the
 session Configuration or IndexResolution to perform analysis.

To minimize the amount of change the existing Rule and RuleExecutor are
 kept in place with minimal changes and two subclasses are introduced:
 ParameterizedRule and ParameterizedRuleExecutor - the former accepts
 passing of the Context object to each rule while the latter keeps the
 state and passes it when encountering a ParameterizedRule.

The majority of the PR is propagating this changes in the tests, mainly
 replacing the creation of the Analyzer; to prevent future refactoring
 noise, the creation has been centralized into an utility class that
 handles the various variants.
Costin Leau před 2 roky
rodič
revize
d919beab5b
45 změnil soubory, kde provedl 402 přidání a 265 odebrání
  1. 5 0
      docs/changelog/92428.yaml
  2. 22 18
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java
  3. 13 0
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/AnalyzerContext.java
  4. 0 28
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/AnalyzerRule.java
  5. 7 7
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java
  6. 2 3
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/planner/Mapper.java
  7. 3 4
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/planner/QueryFolder.java
  8. 2 1
      x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlSession.java
  9. 40 0
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/AnalyzerTestUtils.java
  10. 2 4
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/AnalyzerTests.java
  11. 3 5
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/VerifierTests.java
  12. 2 4
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java
  13. 2 6
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/TomlFoldTests.java
  14. 2 4
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/planner/AbstractQueryTranslatorTestCase.java
  15. 4 4
      x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/stats/VerifierMetricsTests.java
  16. 19 1
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java
  17. 1 6
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java
  18. 19 0
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/ParameterizedRule.java
  19. 31 0
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/ParameterizedRuleExecutor.java
  20. 4 6
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/Rule.java
  21. 23 18
      x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/RuleExecutor.java
  22. 39 42
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java
  23. 27 0
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerContext.java
  24. 7 9
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java
  25. 2 2
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/Debug.java
  26. 2 3
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/Mapper.java
  27. 4 6
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java
  28. 6 4
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/SqlSession.java
  29. 46 0
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerTestUtils.java
  30. 4 3
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerTests.java
  31. 28 15
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/FieldAttributeTests.java
  32. 3 5
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java
  33. 3 4
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/DatabaseFunctionTests.java
  34. 3 4
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/UserFunctionTests.java
  35. 2 4
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java
  36. 2 4
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java
  37. 2 7
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerRunTests.java
  38. 2 4
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumnsTests.java
  39. 2 9
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java
  40. 2 2
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTypesTests.java
  41. 2 3
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java
  42. 2 5
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorSpecTests.java
  43. 2 3
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java
  44. 2 5
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/VerifierTests.java
  45. 2 3
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/stats/VerifierMetricsTests.java

+ 5 - 0
docs/changelog/92428.yaml

@@ -0,0 +1,5 @@
+pr: 92428
+summary: Introduce parameterized rule and executor
+area: Query Languages
+type: enhancement
+issues: []

+ 22 - 18
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/Analyzer.java

@@ -9,6 +9,8 @@ package org.elasticsearch.xpack.eql.analysis;
 
 import org.elasticsearch.xpack.eql.expression.OptionalMissingAttribute;
 import org.elasticsearch.xpack.eql.expression.OptionalUnresolvedAttribute;
+import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules;
+import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AnalyzerRule;
 import org.elasticsearch.xpack.ql.common.Failure;
 import org.elasticsearch.xpack.ql.expression.Attribute;
 import org.elasticsearch.xpack.ql.expression.Expression;
@@ -17,12 +19,11 @@ import org.elasticsearch.xpack.ql.expression.NamedExpression;
 import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute;
 import org.elasticsearch.xpack.ql.expression.function.Function;
 import org.elasticsearch.xpack.ql.expression.function.FunctionDefinition;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction;
 import org.elasticsearch.xpack.ql.plan.logical.Filter;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor;
 import org.elasticsearch.xpack.ql.rule.RuleExecutor;
-import org.elasticsearch.xpack.ql.session.Configuration;
 import org.elasticsearch.xpack.ql.type.DataTypes;
 
 import java.util.Collection;
@@ -32,27 +33,28 @@ import static java.util.Arrays.asList;
 import static org.elasticsearch.xpack.eql.analysis.AnalysisUtils.resolveAgainstList;
 import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AddMissingEqualsToBoolField;
 
-public class Analyzer extends RuleExecutor<LogicalPlan> {
+public class Analyzer extends ParameterizedRuleExecutor<LogicalPlan, AnalyzerContext> {
+
+    private static final Iterable<RuleExecutor.Batch<LogicalPlan>> rules;
+
+    static {
+        var optional = new Batch<>("Optional", Limiter.ONCE, new ResolveOrReplaceOptionalRefs());
+        var resolution = new Batch<>("Resolution", new ResolveRefs(), new ResolveFunctions());
+        var cleanup = new Batch<>("Finish Analysis", Limiter.ONCE, new AddMissingEqualsToBoolField());
+
+        rules = asList(optional, resolution, cleanup);
+    }
 
-    private final Configuration configuration;
-    private final FunctionRegistry functionRegistry;
     private final Verifier verifier;
 
-    public Analyzer(Configuration configuration, FunctionRegistry functionRegistry, Verifier verifier) {
-        this.configuration = configuration;
-        this.functionRegistry = functionRegistry;
+    public Analyzer(AnalyzerContext context, Verifier verifier) {
+        super(context);
         this.verifier = verifier;
     }
 
     @Override
-    protected Iterable<RuleExecutor<LogicalPlan>.Batch> batches() {
-        Batch optional = new Batch("Optional", Limiter.ONCE, new ResolveOrReplaceOptionalRefs());
-
-        Batch resolution = new Batch("Resolution", new ResolveRefs(), new ResolveFunctions());
-
-        Batch cleanup = new Batch("Finish Analysis", Limiter.ONCE, new AddMissingEqualsToBoolField());
-
-        return asList(optional, resolution, cleanup);
+    protected Iterable<RuleExecutor.Batch<LogicalPlan>> batches() {
+        return rules;
     }
 
     public LogicalPlan analyze(LogicalPlan plan) {
@@ -99,10 +101,12 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
         }
     }
 
-    private class ResolveFunctions extends AnalyzerRule<LogicalPlan> {
+    private static class ResolveFunctions extends AnalyzerRules.ParameterizedAnalyzerRule<LogicalPlan, AnalyzerContext> {
 
         @Override
-        protected LogicalPlan rule(LogicalPlan plan) {
+        protected LogicalPlan rule(LogicalPlan plan, AnalyzerContext context) {
+            var configuration = context.configuration();
+            var functionRegistry = context.functionRegistry();
             return plan.transformExpressionsUp(UnresolvedFunction.class, uf -> {
                 if (uf.analyzed()) {
                     return uf;

+ 13 - 0
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/AnalyzerContext.java

@@ -0,0 +1,13 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.eql.analysis;
+
+import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
+import org.elasticsearch.xpack.ql.session.Configuration;
+
+public record AnalyzerContext(Configuration configuration, FunctionRegistry functionRegistry) {}

+ 0 - 28
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/analysis/AnalyzerRule.java

@@ -1,28 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0; you may not use this file except in compliance with the Elastic License
- * 2.0.
- */
-
-package org.elasticsearch.xpack.eql.analysis;
-
-import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
-import org.elasticsearch.xpack.ql.rule.Rule;
-
-public abstract class AnalyzerRule<SubPlan extends LogicalPlan> extends Rule<SubPlan, LogicalPlan> {
-
-    // transformUp (post-order) - that is first children and then the node
-    // but with a twist; only if the tree is not resolved or analyzed
-    @Override
-    public final LogicalPlan apply(LogicalPlan plan) {
-        return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t));
-    }
-
-    @Override
-    protected abstract LogicalPlan rule(SubPlan plan);
-
-    protected boolean skipResolved() {
-        return true;
-    }
-}

+ 7 - 7
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/optimizer/Optimizer.java

@@ -78,8 +78,8 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<LogicalPlan>.Batch> batches() {
-        Batch substitutions = new Batch(
+    protected Iterable<RuleExecutor.Batch<LogicalPlan>> batches() {
+        var substitutions = new Batch<>(
             "Substitution",
             Limiter.ONCE,
             new ReplaceWildcards(),
@@ -89,7 +89,7 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new AddMandatoryJoinKeyFilter()
         );
 
-        Batch operators = new Batch(
+        var operators = new Batch<>(
             "Operator Optimization",
             new ConstantFolding(),
             // boolean
@@ -107,13 +107,13 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new PushDownAndCombineFilters()
         );
 
-        Batch constraints = new Batch("Infer constraints", Limiter.ONCE, new PropagateJoinKeyConstraints());
+        var constraints = new Batch<>("Infer constraints", Limiter.ONCE, new PropagateJoinKeyConstraints());
 
-        Batch ordering = new Batch("Implicit Order", new SortByLimit(), new PushDownOrderBy());
+        var ordering = new Batch<>("Implicit Order", new SortByLimit(), new PushDownOrderBy());
 
-        Batch local = new Batch("Skip Elasticsearch", new SkipEmptyFilter(), new SkipEmptyJoin(), new SkipQueryOnLimitZero());
+        var local = new Batch<>("Skip Elasticsearch", new SkipEmptyFilter(), new SkipEmptyJoin(), new SkipQueryOnLimitZero());
 
-        Batch label = new Batch("Set as Optimized", Limiter.ONCE, new SetAsOptimized());
+        var label = new Batch<>("Set as Optimized", Limiter.ONCE, new SetAsOptimized());
 
         return asList(substitutions, operators, constraints, operators, ordering, local, label);
     }

+ 2 - 3
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/planner/Mapper.java

@@ -50,8 +50,8 @@ class Mapper extends RuleExecutor<PhysicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<PhysicalPlan>.Batch> batches() {
-        Batch conversion = new Batch("Mapping", new SimpleExecMapper());
+    protected Iterable<RuleExecutor.Batch<PhysicalPlan>> batches() {
+        var conversion = new Batch<>("Mapping", new SimpleExecMapper());
 
         return Arrays.asList(conversion);
     }
@@ -135,7 +135,6 @@ class Mapper extends RuleExecutor<PhysicalPlan> {
         }
 
         @SuppressWarnings("unchecked")
-        @Override
         protected final PhysicalPlan rule(UnplannedExec plan) {
             LogicalPlan subPlan = plan.plan();
             if (subPlanToken.isInstance(subPlan)) {

+ 3 - 4
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/planner/QueryFolder.java

@@ -40,9 +40,9 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<PhysicalPlan>.Batch> batches() {
-        Batch fold = new Batch("Fold queries", new FoldProject(), new FoldFilter(), new FoldOrderBy(), new FoldLimit());
-        Batch finish = new Batch("Finish query", Limiter.ONCE, new PlanOutputToQueryRef());
+    protected Iterable<RuleExecutor.Batch<PhysicalPlan>> batches() {
+        var fold = new Batch<>("Fold queries", new FoldProject(), new FoldFilter(), new FoldOrderBy(), new FoldLimit());
+        var finish = new Batch<>("Finish query", Limiter.ONCE, new PlanOutputToQueryRef());
 
         return Arrays.asList(fold, finish);
     }
@@ -139,7 +139,6 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
             return plan.transformUp(typeToken(), this::rule);
         }
 
-        @Override
         protected abstract PhysicalPlan rule(SubPlan plan);
     }
 

+ 2 - 1
x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlSession.java

@@ -13,6 +13,7 @@ import org.elasticsearch.client.internal.ParentTaskAssigningClient;
 import org.elasticsearch.common.breaker.CircuitBreaker;
 import org.elasticsearch.tasks.TaskCancelledException;
 import org.elasticsearch.xpack.eql.analysis.Analyzer;
+import org.elasticsearch.xpack.eql.analysis.AnalyzerContext;
 import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
 import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
 import org.elasticsearch.xpack.eql.analysis.Verifier;
@@ -59,7 +60,7 @@ public class EqlSession {
         this.indexResolver = indexResolver;
         this.preAnalyzer = preAnalyzer;
         this.postAnalyzer = postAnalyzer;
-        this.analyzer = new Analyzer(cfg, functionRegistry, verifier);
+        this.analyzer = new Analyzer(new AnalyzerContext(cfg, functionRegistry), verifier);
         this.optimizer = optimizer;
         this.planner = planner;
         this.circuitBreaker = circuitBreaker;

+ 40 - 0
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/AnalyzerTestUtils.java

@@ -0,0 +1,40 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.eql.analysis;
+
+import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
+import org.elasticsearch.xpack.eql.session.EqlConfiguration;
+import org.elasticsearch.xpack.eql.stats.Metrics;
+import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
+
+import static org.elasticsearch.xpack.eql.EqlTestUtils.TEST_CFG;
+
+public final class AnalyzerTestUtils {
+
+    private AnalyzerTestUtils() {}
+
+    public static Analyzer analyzer() {
+        return new Analyzer(new AnalyzerContext(TEST_CFG, new EqlFunctionRegistry()), new Verifier(new Metrics()));
+    }
+
+    public static Analyzer analyzer(Verifier verifier) {
+        return analyzer(TEST_CFG, new EqlFunctionRegistry(), verifier);
+    }
+
+    public static Analyzer analyzer(EqlConfiguration configuration) {
+        return analyzer(configuration, new EqlFunctionRegistry());
+    }
+
+    public static Analyzer analyzer(EqlConfiguration configuration, FunctionRegistry registry) {
+        return analyzer(configuration, registry, new Verifier(new Metrics()));
+    }
+
+    public static Analyzer analyzer(EqlConfiguration configuration, FunctionRegistry registry, Verifier verifier) {
+        return new Analyzer(new AnalyzerContext(configuration, registry), verifier);
+    }
+}

+ 2 - 4
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/AnalyzerTests.java

@@ -10,7 +10,6 @@ package org.elasticsearch.xpack.eql.analysis;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.eql.expression.OptionalMissingAttribute;
 import org.elasticsearch.xpack.eql.expression.OptionalResolvedAttribute;
-import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
 import org.elasticsearch.xpack.eql.expression.function.scalar.string.Concat;
 import org.elasticsearch.xpack.eql.expression.function.scalar.string.ToString;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
@@ -19,7 +18,6 @@ import org.elasticsearch.xpack.eql.plan.logical.KeyedFilter;
 import org.elasticsearch.xpack.eql.plan.logical.LimitWithOffset;
 import org.elasticsearch.xpack.eql.plan.logical.Sample;
 import org.elasticsearch.xpack.eql.plan.logical.Sequence;
-import org.elasticsearch.xpack.eql.stats.Metrics;
 import org.elasticsearch.xpack.ql.expression.Expression;
 import org.elasticsearch.xpack.ql.expression.FieldAttribute;
 import org.elasticsearch.xpack.ql.expression.Literal;
@@ -41,7 +39,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.elasticsearch.xpack.eql.EqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 
 public class AnalyzerTests extends ESTestCase {
 
@@ -265,7 +263,7 @@ public class AnalyzerTests extends ESTestCase {
 
     private LogicalPlan accept(IndexResolution resolution, String eql) {
         PreAnalyzer preAnalyzer = new PreAnalyzer();
-        Analyzer analyzer = new Analyzer(TEST_CFG, new EqlFunctionRegistry(), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer();
         EqlParser parser = new EqlParser();
         LogicalPlan plan = parser.createStatement(eql);
         return analyzer.analyze(preAnalyzer.preAnalyze(plan, resolution));

+ 3 - 5
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/VerifierTests.java

@@ -9,14 +9,11 @@ package org.elasticsearch.xpack.eql.analysis;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.xpack.eql.EqlTestUtils;
-import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
 import org.elasticsearch.xpack.eql.parser.ParsingException;
 import org.elasticsearch.xpack.eql.plan.logical.KeyedFilter;
 import org.elasticsearch.xpack.eql.plan.logical.Sample;
 import org.elasticsearch.xpack.eql.session.EqlConfiguration;
-import org.elasticsearch.xpack.eql.stats.Metrics;
 import org.elasticsearch.xpack.ql.expression.EmptyAttribute;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
@@ -29,6 +26,7 @@ import java.util.Map;
 import java.util.function.Function;
 
 import static java.util.Collections.emptyMap;
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 import static org.hamcrest.Matchers.startsWith;
 
 public class VerifierTests extends ESTestCase {
@@ -48,7 +46,7 @@ public class VerifierTests extends ESTestCase {
     private LogicalPlan accept(IndexResolution resolution, String eql) {
         EqlParser parser = new EqlParser();
         PreAnalyzer preAnalyzer = new PreAnalyzer();
-        Analyzer analyzer = new Analyzer(EqlTestUtils.TEST_CFG, new EqlFunctionRegistry(), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer();
 
         LogicalPlan plan = parser.createStatement(eql);
         return analyzer.analyze(preAnalyzer.preAnalyze(plan, resolution));
@@ -471,7 +469,7 @@ public class VerifierTests extends ESTestCase {
             new TaskId("test", 123),
             null
         );
-        Analyzer analyzer = new Analyzer(eqlConfiguration, new EqlFunctionRegistry(), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(eqlConfiguration);
         IndexResolution resolution = IndexResolution.valid(new EsIndex("irrelevant", loadEqlMapping("mapping-default.json")));
         return analyzer.analyze(preAnalyzer.preAnalyze(new EqlParser().createStatement("any where true"), resolution));
     }

+ 2 - 4
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/OptimizerTests.java

@@ -12,8 +12,6 @@ import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.eql.analysis.Analyzer;
 import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
 import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
-import org.elasticsearch.xpack.eql.analysis.Verifier;
-import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
 import org.elasticsearch.xpack.eql.expression.function.scalar.string.ToString;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
 import org.elasticsearch.xpack.eql.plan.logical.AbstractJoin;
@@ -23,7 +21,6 @@ import org.elasticsearch.xpack.eql.plan.logical.Sample;
 import org.elasticsearch.xpack.eql.plan.logical.Sequence;
 import org.elasticsearch.xpack.eql.plan.logical.Tail;
 import org.elasticsearch.xpack.eql.plan.physical.LocalRelation;
-import org.elasticsearch.xpack.eql.stats.Metrics;
 import org.elasticsearch.xpack.ql.TestUtils;
 import org.elasticsearch.xpack.ql.expression.Attribute;
 import org.elasticsearch.xpack.ql.expression.EmptyAttribute;
@@ -69,6 +66,7 @@ import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonList;
 import static java.util.stream.Collectors.toList;
 import static org.elasticsearch.xpack.eql.EqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.ql.TestUtils.UTC;
 import static org.elasticsearch.xpack.ql.expression.Literal.TRUE;
 import static org.elasticsearch.xpack.ql.tree.Source.EMPTY;
@@ -92,7 +90,7 @@ public class OptimizerTests extends ESTestCase {
     private LogicalPlan accept(IndexResolution resolution, String eql) {
         PreAnalyzer preAnalyzer = new PreAnalyzer();
         PostAnalyzer postAnalyzer = new PostAnalyzer();
-        Analyzer analyzer = new Analyzer(TEST_CFG, new EqlFunctionRegistry(), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer();
         return optimizer.optimize(
             postAnalyzer.postAnalyze(analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(eql), resolution)), TEST_CFG)
         );

+ 2 - 6
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/optimizer/TomlFoldTests.java

@@ -11,11 +11,8 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.eql.analysis.Analyzer;
-import org.elasticsearch.xpack.eql.analysis.Verifier;
-import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
 import org.elasticsearch.xpack.eql.plan.physical.LocalRelation;
-import org.elasticsearch.xpack.eql.stats.Metrics;
 import org.elasticsearch.xpack.ql.expression.Alias;
 import org.elasticsearch.xpack.ql.expression.Expression;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
@@ -30,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singletonList;
 import static java.util.stream.Collectors.toList;
-import static org.elasticsearch.xpack.eql.EqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.ql.tree.Source.EMPTY;
 
 public class TomlFoldTests extends ESTestCase {
@@ -38,8 +35,7 @@ public class TomlFoldTests extends ESTestCase {
     protected static final String PARAM_FORMATTING = "%1$s.test -> %2$s";
 
     private static final EqlParser PARSER = new EqlParser();
-    private static final EqlFunctionRegistry FUNCTION_REGISTRY = new EqlFunctionRegistry();
-    private static final Analyzer ANALYZER = new Analyzer(TEST_CFG, FUNCTION_REGISTRY, new Verifier(new Metrics()));
+    private static final Analyzer ANALYZER = analyzer();
 
     private final int num;
     private final EqlFoldSpec spec;

+ 2 - 4
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/planner/AbstractQueryTranslatorTestCase.java

@@ -12,16 +12,14 @@ import org.elasticsearch.xpack.eql.EqlTestUtils;
 import org.elasticsearch.xpack.eql.analysis.Analyzer;
 import org.elasticsearch.xpack.eql.analysis.PostAnalyzer;
 import org.elasticsearch.xpack.eql.analysis.PreAnalyzer;
-import org.elasticsearch.xpack.eql.analysis.Verifier;
-import org.elasticsearch.xpack.eql.expression.function.EqlFunctionRegistry;
 import org.elasticsearch.xpack.eql.optimizer.Optimizer;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
 import org.elasticsearch.xpack.eql.plan.physical.PhysicalPlan;
 import org.elasticsearch.xpack.eql.session.EqlConfiguration;
-import org.elasticsearch.xpack.eql.stats.Metrics;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.ql.type.TypesTests.loadMapping;
 
 public abstract class AbstractQueryTranslatorTestCase extends ESTestCase {
@@ -29,7 +27,7 @@ public abstract class AbstractQueryTranslatorTestCase extends ESTestCase {
     protected PreAnalyzer preAnalyzer = new PreAnalyzer();
     protected PostAnalyzer postAnalyzer = new PostAnalyzer();
     protected EqlConfiguration configuration = EqlTestUtils.randomConfiguration();
-    protected Analyzer analyzer = new Analyzer(configuration, new EqlFunctionRegistry(), new Verifier(new Metrics()));
+    protected Analyzer analyzer = analyzer(configuration);
     protected Optimizer optimizer = new Optimizer();
     protected Planner planner = new Planner();
 

+ 4 - 4
x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/stats/VerifierMetricsTests.java

@@ -18,8 +18,10 @@ import org.elasticsearch.xpack.eql.optimizer.OptimizerTests;
 import org.elasticsearch.xpack.eql.parser.EqlParser;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 
+import java.util.Arrays;
 import java.util.Set;
 
+import static org.elasticsearch.xpack.eql.analysis.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.eql.stats.FeatureMetric.EVENT;
 import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN;
 import static org.elasticsearch.xpack.eql.stats.FeatureMetric.JOIN_KEYS_FIVE_OR_MORE;
@@ -179,7 +181,7 @@ public class VerifierMetricsTests extends ESTestCase {
     private Counters eql(String query) {
         Metrics metrics = new Metrics();
         Verifier verifier = new Verifier(metrics);
-        Analyzer analyzer = new Analyzer(EqlTestUtils.randomConfiguration(), eqlFunctionRegistry, verifier);
+        Analyzer analyzer = analyzer(EqlTestUtils.randomConfiguration(), eqlFunctionRegistry, verifier);
         analyzer.analyze(preAnalyzer.preAnalyze(parser.createStatement(query), index));
         return metrics.stats();
     }
@@ -189,9 +191,7 @@ public class VerifierMetricsTests extends ESTestCase {
 
         MetricsHolder() {
             this.metrics = new long[FeatureMetric.values().length];
-            for (int i = 0; i < this.metrics.length; i++) {
-                this.metrics[i] = 0;
-            }
+            Arrays.fill(this.metrics, 0);
         }
 
         void set(Set<FeatureMetric> metricSet) {

+ 19 - 1
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/analyzer/AnalyzerRules.java

@@ -14,6 +14,7 @@ import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic;
 import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals;
 import org.elasticsearch.xpack.ql.plan.logical.Filter;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.ql.rule.ParameterizedRule;
 import org.elasticsearch.xpack.ql.rule.Rule;
 
 import static java.util.Arrays.asList;
@@ -66,7 +67,6 @@ public final class AnalyzerRules {
             return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t));
         }
 
-        @Override
         protected abstract LogicalPlan rule(SubPlan plan);
 
         protected boolean skipResolved() {
@@ -74,6 +74,24 @@ public final class AnalyzerRules {
         }
     }
 
+    public abstract static class ParameterizedAnalyzerRule<SubPlan extends LogicalPlan, P> extends ParameterizedRule<
+        SubPlan,
+        LogicalPlan,
+        P> {
+
+        // transformUp (post-order) - that is first children and then the node
+        // but with a twist; only if the tree is not resolved or analyzed
+        public final LogicalPlan apply(LogicalPlan plan, P context) {
+            return plan.transformUp(typeToken(), t -> t.analyzed() || skipResolved() && t.resolved() ? t : rule(t, context));
+        }
+
+        protected abstract LogicalPlan rule(SubPlan plan, P context);
+
+        protected boolean skipResolved() {
+            return true;
+        }
+    }
+
     public abstract static class BaseAnalyzerRule extends AnalyzerRule<LogicalPlan> {
 
         @Override

+ 1 - 6
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/optimizer/OptimizerRules.java

@@ -1619,7 +1619,6 @@ public final class OptimizerRules {
             return rule(plan);
         }
 
-        @Override
         protected final LogicalPlan rule(LogicalPlan plan) {
             // eliminate redundant casts
             return plan.transformExpressionsUp(castType, this::maybePruneCast);
@@ -1766,12 +1765,10 @@ public final class OptimizerRules {
             return plan;
         }
 
-        @Override
-        protected LogicalPlan rule(LogicalPlan plan) {
+        private void rule(LogicalPlan plan) {
             if (plan.optimized() == false) {
                 plan.setOptimized();
             }
-            return plan;
         }
     }
 
@@ -1794,7 +1791,6 @@ public final class OptimizerRules {
                 : plan.transformUp(typeToken(), this::rule);
         }
 
-        @Override
         protected abstract LogicalPlan rule(SubPlan plan);
     }
 
@@ -1817,7 +1813,6 @@ public final class OptimizerRules {
                 : plan.transformExpressionsUp(expressionTypeToken, this::rule);
         }
 
-        @Override
         protected LogicalPlan rule(LogicalPlan plan) {
             return plan;
         }

+ 19 - 0
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/ParameterizedRule.java

@@ -0,0 +1,19 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.ql.rule;
+
+import org.elasticsearch.xpack.ql.tree.Node;
+
+public abstract class ParameterizedRule<E extends T, T extends Node<T>, P> extends Rule<E, T> {
+
+    public abstract T apply(T t, P p);
+
+    public T apply(T t) {
+        throw new RuleExecutionException("Cannot call parameterized rule without parameter");
+    }
+}

+ 31 - 0
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/ParameterizedRuleExecutor.java

@@ -0,0 +1,31 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.ql.rule;
+
+import org.elasticsearch.xpack.ql.tree.Node;
+
+import java.util.function.Function;
+
+public abstract class ParameterizedRuleExecutor<TreeType extends Node<TreeType>, Context> extends RuleExecutor<TreeType> {
+
+    private final Context context;
+
+    protected ParameterizedRuleExecutor(Context context) {
+        this.context = context;
+    }
+
+    protected Context context() {
+        return context;
+    }
+
+    @Override
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    protected Function<TreeType, TreeType> transform(Rule<?, TreeType> rule) {
+        return (rule instanceof ParameterizedRule pr) ? t -> (TreeType) pr.apply(t, context) : t -> rule.apply(t);
+    }
+}

+ 4 - 6
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/Rule.java

@@ -11,17 +11,15 @@ import org.apache.logging.log4j.Logger;
 import org.elasticsearch.xpack.ql.tree.Node;
 import org.elasticsearch.xpack.ql.util.ReflectionUtils;
 
-import java.util.function.UnaryOperator;
-
 /**
  * Rules that apply transformation to a tree. In addition, performs
  * type filtering so that a rule that the rule implementation doesn't
  * have to manually filter.
  * <p>
  * Rules <strong>could</strong> could be built as lambdas but most
- * rules are much larger so we keep them as full blown subclasses.
+ * rules are much larger, so we keep them as full-blown subclasses.
  */
-public abstract class Rule<E extends T, T extends Node<T>> implements UnaryOperator<T> {
+public abstract class Rule<E extends T, T extends Node<T>> {
 
     protected Logger log = LogManager.getLogger(getClass());
 
@@ -44,10 +42,10 @@ public abstract class Rule<E extends T, T extends Node<T>> implements UnaryOpera
         return name;
     }
 
-    protected abstract T rule(E e);
-
     @Override
     public String toString() {
         return name();
     }
+
+    public abstract T apply(T t);
 }

+ 23 - 18
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/rule/RuleExecutor.java

@@ -16,6 +16,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.function.Function;
 
 public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
 
@@ -45,7 +46,7 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
         }
     }
 
-    public class Batch {
+    public static class Batch<TreeType extends Node<TreeType>> {
         private final String name;
         private final Rule<?, TreeType>[] rules;
         private final Limiter limit;
@@ -68,19 +69,19 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
         }
     }
 
-    private final Iterable<Batch> batches = batches();
+    private final Iterable<Batch<TreeType>> batches = batches();
 
-    protected abstract Iterable<RuleExecutor<TreeType>.Batch> batches();
+    protected abstract Iterable<RuleExecutor.Batch<TreeType>> batches();
 
     public class Transformation {
         private final TreeType before, after;
-        private final Rule<?, TreeType> rule;
+        private final String name;
         private Boolean lazyHasChanged;
 
-        Transformation(TreeType plan, Rule<?, TreeType> rule) {
-            this.rule = rule;
-            before = plan;
-            after = rule.apply(before);
+        Transformation(String name, TreeType plan, Function<TreeType, TreeType> transform) {
+            this.name = name;
+            this.before = plan;
+            this.after = transform.apply(before);
         }
 
         public boolean hasChanged() {
@@ -90,8 +91,8 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
             return lazyHasChanged;
         }
 
-        public String ruleName() {
-            return rule.name();
+        public String name() {
+            return name;
         }
 
         public TreeType before() {
@@ -106,9 +107,9 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
     public class ExecutionInfo {
 
         private final TreeType before, after;
-        private final Map<Batch, List<Transformation>> transformations;
+        private final Map<Batch<TreeType>, List<Transformation>> transformations;
 
-        ExecutionInfo(TreeType before, TreeType after, Map<Batch, List<Transformation>> transformations) {
+        ExecutionInfo(TreeType before, TreeType after, Map<Batch<TreeType>, List<Transformation>> transformations) {
             this.before = before;
             this.after = after;
             this.transformations = transformations;
@@ -122,23 +123,23 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
             return after;
         }
 
-        public Map<Batch, List<Transformation>> transformations() {
+        public Map<Batch<TreeType>, List<Transformation>> transformations() {
             return transformations;
         }
     }
 
-    protected TreeType execute(TreeType plan) {
+    protected final TreeType execute(TreeType plan) {
         return executeWithInfo(plan).after;
     }
 
-    protected ExecutionInfo executeWithInfo(TreeType plan) {
+    protected final ExecutionInfo executeWithInfo(TreeType plan) {
         TreeType currentPlan = plan;
 
         long totalDuration = 0;
 
-        Map<Batch, List<Transformation>> transformations = new LinkedHashMap<>();
+        Map<Batch<TreeType>, List<Transformation>> transformations = new LinkedHashMap<>();
 
-        for (Batch batch : batches) {
+        for (Batch<TreeType> batch : batches) {
             int batchRuns = 0;
             List<Transformation> tfs = new ArrayList<>();
             transformations.put(batch, tfs);
@@ -156,7 +157,7 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
                     if (log.isTraceEnabled()) {
                         log.trace("About to apply rule {}", rule);
                     }
-                    Transformation tf = new Transformation(currentPlan, rule);
+                    Transformation tf = new Transformation(rule.name(), currentPlan, transform(rule));
                     tfs.add(tf);
                     currentPlan = tf.after;
 
@@ -198,4 +199,8 @@ public abstract class RuleExecutor<TreeType extends Node<TreeType>> {
 
         return new ExecutionInfo(plan, currentPlan, transformations);
     }
+
+    protected Function<TreeType, TreeType> transform(Rule<?, TreeType> rule) {
+        return rule::apply;
+    }
 }

+ 39 - 42
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java

@@ -9,6 +9,7 @@ package org.elasticsearch.xpack.sql.analysis.analyzer;
 import org.elasticsearch.common.logging.LoggerMessageFormat;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AddMissingEqualsToBoolField;
+import org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.ParameterizedAnalyzerRule;
 import org.elasticsearch.xpack.ql.capabilities.Resolvables;
 import org.elasticsearch.xpack.ql.common.Failure;
 import org.elasticsearch.xpack.ql.expression.Alias;
@@ -28,7 +29,6 @@ import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute;
 import org.elasticsearch.xpack.ql.expression.UnresolvedStar;
 import org.elasticsearch.xpack.ql.expression.function.Function;
 import org.elasticsearch.xpack.ql.expression.function.FunctionDefinition;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.expression.function.FunctionResolutionStrategy;
 import org.elasticsearch.xpack.ql.expression.function.Functions;
 import org.elasticsearch.xpack.ql.expression.function.UnresolvedFunction;
@@ -43,6 +43,7 @@ import org.elasticsearch.xpack.ql.plan.logical.OrderBy;
 import org.elasticsearch.xpack.ql.plan.logical.Project;
 import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan;
 import org.elasticsearch.xpack.ql.plan.logical.UnresolvedRelation;
+import org.elasticsearch.xpack.ql.rule.ParameterizedRuleExecutor;
 import org.elasticsearch.xpack.ql.rule.RuleExecutor;
 import org.elasticsearch.xpack.ql.type.DataType;
 import org.elasticsearch.xpack.ql.type.DataTypes;
@@ -58,7 +59,6 @@ import org.elasticsearch.xpack.sql.plan.logical.LocalRelation;
 import org.elasticsearch.xpack.sql.plan.logical.Pivot;
 import org.elasticsearch.xpack.sql.plan.logical.SubQueryAlias;
 import org.elasticsearch.xpack.sql.plan.logical.With;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.type.SqlDataTypeConverter;
 
 import java.util.ArrayList;
@@ -79,36 +79,13 @@ import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.AnalyzerRule;
 import static org.elasticsearch.xpack.ql.analyzer.AnalyzerRules.BaseAnalyzerRule;
 import static org.elasticsearch.xpack.ql.util.CollectionUtils.combine;
 
-public class Analyzer extends RuleExecutor<LogicalPlan> {
-    /**
-     * Valid functions.
-     */
-    private final FunctionRegistry functionRegistry;
-    /**
-     * Information about the index against which the SQL is being analyzed.
-     */
-    private final IndexResolution indexResolution;
-    /**
-     * Per-request specific settings needed in some of the functions (timezone, username and clustername),
-     * to which they are attached.
-     */
-    private final SqlConfiguration configuration;
-    /**
-     * The verifier has the role of checking the analyzed tree for failures and build a list of failures.
-     */
-    private final Verifier verifier;
+public class Analyzer extends ParameterizedRuleExecutor<LogicalPlan, AnalyzerContext> {
 
-    public Analyzer(SqlConfiguration configuration, FunctionRegistry functionRegistry, IndexResolution results, Verifier verifier) {
-        this.configuration = configuration;
-        this.functionRegistry = functionRegistry;
-        this.indexResolution = results;
-        this.verifier = verifier;
-    }
+    private static final Iterable<RuleExecutor.Batch<LogicalPlan>> rules;
 
-    @Override
-    protected Iterable<RuleExecutor<LogicalPlan>.Batch> batches() {
-        Batch substitution = new Batch("Substitution", new CTESubstitution());
-        Batch resolution = new Batch(
+    static {
+        var substitution = new Batch<>("Substitution", new CTESubstitution());
+        var resolution = new Batch<>(
             "Resolution",
             new ResolveTable(),
             new ResolveRefs(),
@@ -123,14 +100,30 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
             new ResolveAggsInOrderBy()
             // new ImplicitCasting()
         );
-        Batch finish = new Batch(
+        var finish = new Batch<>(
             "Finish Analysis",
             new ReplaceSubQueryAliases(), // Should be run before pruning SubqueryAliases
             new PruneSubQueryAliases(),
             new AddMissingEqualsToBoolField(),
             CleanAliases.INSTANCE
         );
-        return Arrays.asList(substitution, resolution, finish);
+        rules = Arrays.asList(substitution, resolution, finish);
+    }
+
+    /**
+     * The verifier has the role of checking the analyzed tree for failures and build a list of failures.
+     */
+    private final Verifier verifier;
+
+    public Analyzer(AnalyzerContext context, Verifier verifier) {
+        super(context);
+        context.analyzeWithoutVerify().set(this::execute);
+        this.verifier = verifier;
+    }
+
+    @Override
+    protected Iterable<RuleExecutor.Batch<LogicalPlan>> batches() {
+        return rules;
     }
 
     public LogicalPlan analyze(LogicalPlan plan) {
@@ -149,7 +142,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
     }
 
     public LogicalPlan verify(LogicalPlan plan) {
-        Collection<Failure> failures = verifier.verify(plan, configuration.version());
+        Collection<Failure> failures = verifier.verify(plan, context().configuration().version());
         if (failures.isEmpty() == false) {
             throw new VerificationException(failures);
         }
@@ -310,9 +303,10 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
         }
     }
 
-    private class ResolveTable extends AnalyzerRule<UnresolvedRelation> {
-        @Override
-        protected LogicalPlan rule(UnresolvedRelation plan) {
+    private static class ResolveTable extends ParameterizedAnalyzerRule<UnresolvedRelation, AnalyzerContext> {
+
+        protected LogicalPlan rule(UnresolvedRelation plan, AnalyzerContext context) {
+            IndexResolution indexResolution = context.indexResolution();
             if (indexResolution.isValid() == false) {
                 return plan.unresolvedMessage().equals(indexResolution.toString())
                     ? plan
@@ -339,7 +333,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
         }
     }
 
-    private class ResolveRefs extends BaseAnalyzerRule {
+    private static class ResolveRefs extends BaseAnalyzerRule {
 
         @Override
         protected LogicalPlan doRule(LogicalPlan plan) {
@@ -901,10 +895,12 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
         }
     }
 
-    private class ResolveFunctions extends AnalyzerRule<LogicalPlan> {
+    private static class ResolveFunctions extends ParameterizedAnalyzerRule<LogicalPlan, AnalyzerContext> {
 
         @Override
-        protected LogicalPlan rule(LogicalPlan plan) {
+        protected LogicalPlan rule(LogicalPlan plan, AnalyzerContext context) {
+            var functionRegistry = context.functionRegistry();
+            var configuration = context.configuration();
             return plan.transformExpressionsUp(UnresolvedFunction.class, uf -> {
                 if (uf.analyzed()) {
                     return uf;
@@ -1063,7 +1059,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
     // Handle aggs in HAVING. To help folding any aggs not found in Aggregation
     // will be pushed down to the Aggregate and then projected. This also simplifies the Verifier's job.
     //
-    private class ResolveAggsInHaving extends AnalyzerRule<Filter> {
+    private static class ResolveAggsInHaving extends ParameterizedAnalyzerRule<Filter, AnalyzerContext> {
 
         @Override
         protected boolean skipResolved() {
@@ -1071,7 +1067,7 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
         }
 
         @Override
-        protected LogicalPlan rule(Filter f) {
+        protected LogicalPlan rule(Filter f, AnalyzerContext context) {
             // HAVING = Filter followed by an Agg
             // tag::noformat - https://bugs.eclipse.org/bugs/show_bug.cgi?id=574437
             if (f.child() instanceof Aggregate agg && agg.resolved()) {
@@ -1092,7 +1088,8 @@ public class Analyzer extends RuleExecutor<LogicalPlan> {
                         combine(agg.aggregates(), new Alias(f.source(), ".having", condition))
                     );
 
-                    tryResolvingCondition = (Aggregate) analyze(tryResolvingCondition, false);
+                    var analyze = context.analyzeWithoutVerify().get();
+                    tryResolvingCondition = (Aggregate) analyze.apply(tryResolvingCondition);
 
                     // if it got resolved
                     if (tryResolvingCondition.resolved()) {

+ 27 - 0
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerContext.java

@@ -0,0 +1,27 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.sql.analysis.analyzer;
+
+import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
+import org.elasticsearch.xpack.ql.index.IndexResolution;
+import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
+import org.elasticsearch.xpack.ql.util.Holder;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
+
+import java.util.function.Function;
+
+public record AnalyzerContext(
+    SqlConfiguration configuration,
+    FunctionRegistry functionRegistry,
+    IndexResolution indexResolution,
+    Holder<Function<LogicalPlan, LogicalPlan>> analyzeWithoutVerify
+) {
+    public AnalyzerContext(SqlConfiguration configuration, FunctionRegistry functionRegistry, IndexResolution indexResolution) {
+        this(configuration, functionRegistry, indexResolution, new Holder<>());
+    }
+}

+ 7 - 9
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/optimizer/Optimizer.java

@@ -121,8 +121,8 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<LogicalPlan>.Batch> batches() {
-        Batch substitutions = new Batch(
+    protected Iterable<RuleExecutor.Batch<LogicalPlan>> batches() {
+        var substitutions = new Batch<>(
             "Substitutions",
             Limiter.ONCE,
             new RewritePivot(),
@@ -130,9 +130,9 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new ReplaceAggregatesWithLiterals()
         );
 
-        Batch refs = new Batch("Replace References", Limiter.ONCE, new ReplaceReferenceAttributeWithSource());
+        var refs = new Batch<>("Replace References", Limiter.ONCE, new ReplaceReferenceAttributeWithSource());
 
-        Batch operators = new Batch(
+        var operators = new Batch<>(
             "Operator Optimization",
             // combining
             new CombineProjections(),
@@ -166,7 +166,7 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new PushDownAndCombineFilters()
         );
 
-        Batch aggregate = new Batch(
+        var aggregate = new Batch<>(
             "Aggregation Rewrite",
             new ReplaceMinMaxWithTopHits(),
             new ReplaceAggsWithMatrixStats(),
@@ -178,7 +178,7 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new ReplaceAggsWithPercentileRanks()
         );
 
-        Batch local = new Batch(
+        var local = new Batch<>(
             "Skip Elasticsearch",
             new SkipQueryOnLimitZero(),
             new SkipQueryForLiteralAggregations(),
@@ -188,7 +188,7 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             new PruneLiteralsInGroupBy()
         );
 
-        Batch label = new Batch("Set as Optimized", Limiter.ONCE, CleanAliases.INSTANCE, new SetAsOptimized());
+        var label = new Batch<>("Set as Optimized", Limiter.ONCE, CleanAliases.INSTANCE, new SetAsOptimized());
 
         return Arrays.asList(substitutions, refs, operators, aggregate, local, label);
     }
@@ -588,7 +588,6 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
             return rule(plan);
         }
 
-        @Override
         protected LogicalPlan rule(LogicalPlan plan) {
             Map<Attribute, Alias> aliases = new LinkedHashMap<>();
             List<Attribute> attrs = new ArrayList<>();
@@ -1262,7 +1261,6 @@ public class Optimizer extends RuleExecutor<LogicalPlan> {
         @Override
         public abstract LogicalPlan apply(LogicalPlan plan);
 
-        @Override
         protected LogicalPlan rule(LogicalPlan plan) {
             return plan;
         }

+ 2 - 2
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/Debug.java

@@ -106,7 +106,7 @@ public class Debug extends Command {
                     sb.append(entry.getKey().name());
                     sb.append("***");
                     for (Transformation tf : entry.getValue()) {
-                        sb.append(tf.ruleName());
+                        sb.append(tf.name());
                         sb.append("\n");
                         sb.append(NodeUtils.diffString(tf.before(), tf.after()));
                         sb.append("\n");
@@ -127,7 +127,7 @@ public class Debug extends Command {
                     int counter = 0;
                     for (Transformation tf : entry.getValue()) {
                         if (tf.hasChanged()) {
-                            plans.put(tf.ruleName() + "#" + ++counter, tf.after());
+                            plans.put(tf.name() + "#" + ++counter, tf.after());
                         }
                     }
                 }

+ 2 - 3
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/Mapper.java

@@ -45,8 +45,8 @@ class Mapper extends RuleExecutor<PhysicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<PhysicalPlan>.Batch> batches() {
-        Batch conversion = new Batch("Mapping", new JoinMapper(), new SimpleExecMapper());
+    protected Iterable<RuleExecutor.Batch<PhysicalPlan>> batches() {
+        var conversion = new Batch<>("Mapping", new JoinMapper(), new SimpleExecMapper());
 
         return Arrays.asList(conversion);
     }
@@ -136,7 +136,6 @@ class Mapper extends RuleExecutor<PhysicalPlan> {
         }
 
         @SuppressWarnings("unchecked")
-        @Override
         protected final PhysicalPlan rule(UnplannedExec plan) {
             LogicalPlan subPlan = plan.plan();
             if (subPlanToken.isInstance(subPlan)) {

+ 4 - 6
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java

@@ -116,8 +116,8 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
     }
 
     @Override
-    protected Iterable<RuleExecutor<PhysicalPlan>.Batch> batches() {
-        Batch rollup = new Batch(
+    protected Iterable<RuleExecutor.Batch<PhysicalPlan>> batches() {
+        var rollup = new Batch<>(
             "Fold queries",
             new FoldPivot(),
             new FoldAggregate(),
@@ -127,9 +127,8 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
             new FoldLimit()
         );
 
-        Batch local = new Batch("Local queries", new LocalLimit(), new PropagateEmptyLocal());
-
-        Batch finish = new Batch("Finish query", Limiter.ONCE, new PlanOutputToQueryRef());
+        var local = new Batch<>("Local queries", new LocalLimit(), new PropagateEmptyLocal());
+        var finish = new Batch<>("Finish query", Limiter.ONCE, new PlanOutputToQueryRef());
 
         return Arrays.asList(rollup, local, finish);
     }
@@ -943,7 +942,6 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
             return plan.transformUp(typeToken(), this::rule);
         }
 
-        @Override
         protected abstract PhysicalPlan rule(SubPlan plan);
     }
 }

+ 6 - 4
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/SqlSession.java

@@ -20,6 +20,7 @@ import org.elasticsearch.xpack.ql.plan.TableIdentifier;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.ql.rule.RuleExecutor;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
+import org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerContext;
 import org.elasticsearch.xpack.sql.analysis.analyzer.PreAnalyzer;
 import org.elasticsearch.xpack.sql.analysis.analyzer.PreAnalyzer.PreAnalysis;
 import org.elasticsearch.xpack.sql.analysis.analyzer.TableInfo;
@@ -116,12 +117,12 @@ public class SqlSession implements Session {
         }
 
         preAnalyze(parsed, r -> {
-            Analyzer analyzer = new Analyzer(
+            AnalyzerContext context = new AnalyzerContext(
                 configuration,
                 functionRegistry,
-                IndexCompatibility.compatible(r, Version.fromId(configuration.version().id)),
-                verifier
+                IndexCompatibility.compatible(r, Version.fromId(configuration.version().id))
             );
+            Analyzer analyzer = new Analyzer(context, verifier);
             return analyzer.analyze(parsed, verify);
         }, listener);
     }
@@ -133,7 +134,8 @@ public class SqlSession implements Session {
         }
 
         preAnalyze(parsed, r -> {
-            Analyzer analyzer = new Analyzer(configuration, functionRegistry, r, verifier);
+            AnalyzerContext context = new AnalyzerContext(configuration, functionRegistry, r);
+            Analyzer analyzer = new Analyzer(context, verifier);
             return analyzer.debugAnalyze(parsed);
         }, listener);
     }

+ 46 - 0
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerTestUtils.java

@@ -0,0 +1,46 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.sql.analysis.analyzer;
+
+import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
+import org.elasticsearch.xpack.ql.index.IndexResolution;
+import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
+import org.elasticsearch.xpack.sql.stats.Metrics;
+
+import static org.elasticsearch.xpack.sql.SqlTestUtils.TEST_CFG;
+
+public final class AnalyzerTestUtils {
+
+    private AnalyzerTestUtils() {}
+
+    public static Analyzer analyzer(IndexResolution resolution) {
+        return analyzer(TEST_CFG, new SqlFunctionRegistry(), resolution);
+    }
+
+    public static Analyzer analyzer(IndexResolution resolution, Verifier verifier) {
+        return analyzer(TEST_CFG, new SqlFunctionRegistry(), resolution, verifier);
+    }
+
+    public static Analyzer analyzer(SqlConfiguration configuration, IndexResolution resolution) {
+        return analyzer(configuration, new SqlFunctionRegistry(), resolution);
+    }
+
+    public static Analyzer analyzer(SqlConfiguration configuration, FunctionRegistry registry, IndexResolution resolution) {
+        return analyzer(configuration, registry, resolution, new Verifier(new Metrics()));
+    }
+
+    public static Analyzer analyzer(
+        SqlConfiguration configuration,
+        FunctionRegistry registry,
+        IndexResolution resolution,
+        Verifier verifier
+    ) {
+        return new Analyzer(new AnalyzerContext(configuration, registry, resolution), verifier);
+    }
+}

+ 4 - 3
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/AnalyzerTests.java

@@ -35,13 +35,14 @@ import static org.hamcrest.Matchers.startsWith;
 public class AnalyzerTests extends ESTestCase {
 
     private final SqlParser parser = new SqlParser();
-    private final Analyzer analyzer = new Analyzer(
+    private final AnalyzerContext context = new AnalyzerContext(
         SqlTestUtils.TEST_CFG,
         new SqlFunctionRegistry(),
-        IndexResolution.valid(new EsIndex("test", loadMapping("mapping-basic.json"))),
-        new Verifier(new Metrics())
+        IndexResolution.valid(new EsIndex("test", loadMapping("mapping-basic.json")))
     );
 
+    private final Analyzer analyzer = new Analyzer(context, new Verifier(new Metrics()));
+
     private LogicalPlan analyze(String sql) {
         return analyzer.analyze(parser.createStatement(sql), false);
     }

+ 28 - 15
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/FieldAttributeTests.java

@@ -76,7 +76,7 @@ public class FieldAttributeTests extends ESTestCase {
 
         EsIndex test = new EsIndex("test", mapping);
         getIndexResult = IndexResolution.valid(test);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
+        analyzer = analyzer(functionRegistry, getIndexResult, verifier);
     }
 
     private LogicalPlan plan(String sql) {
@@ -197,7 +197,7 @@ public class FieldAttributeTests extends ESTestCase {
 
         EsIndex index = new EsIndex("test", mapping);
         getIndexResult = IndexResolution.valid(index);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
+        analyzer = analyzer(functionRegistry, getIndexResult, verifier);
 
         VerificationException ex = expectThrows(VerificationException.class, () -> plan("SELECT test.bar FROM test"));
         assertEquals(
@@ -232,7 +232,7 @@ public class FieldAttributeTests extends ESTestCase {
         Map<String, EsField> mapping = TypesTests.loadMapping("mapping-basic.json");
         EsIndex index = new EsIndex("test", mapping);
         getIndexResult = IndexResolution.valid(index);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
+        analyzer = analyzer(functionRegistry, getIndexResult, verifier);
 
         LogicalPlan plan = plan("SELECT sum(salary) AS s FROM test");
         assertThat(plan, instanceOf(Aggregate.class));
@@ -265,7 +265,7 @@ public class FieldAttributeTests extends ESTestCase {
         Map<String, EsField> mapping = TypesTests.loadMapping("mapping-basic.json");
         EsIndex index = new EsIndex("test", mapping);
         getIndexResult = IndexResolution.valid(index);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
+        analyzer = analyzer(functionRegistry, getIndexResult, verifier);
 
         VerificationException ex = expectThrows(
             VerificationException.class,
@@ -323,7 +323,7 @@ public class FieldAttributeTests extends ESTestCase {
         SqlConfiguration sqlConfig = SqlTestUtils.randomConfiguration(SqlVersion.fromId(preUnsignedLong.id));
 
         for (String sql : List.of(query, queryWithLiteral, queryWithCastLiteral, queryWithAlias, queryWithArithmetic, queryWithCast)) {
-            analyzer = new Analyzer(
+            analyzer = analyzer(
                 sqlConfig,
                 functionRegistry,
                 loadCompatibleIndexResolution("mapping-numeric.json", preUnsignedLong),
@@ -333,7 +333,7 @@ public class FieldAttributeTests extends ESTestCase {
             assertThat(ex.getMessage(), containsString("Found 1 problem\nline 1:8: Cannot use field [unsigned_long]"));
 
             for (Version v : List.of(INTRODUCING_UNSIGNED_LONG, postUnsignedLong)) {
-                analyzer = new Analyzer(
+                analyzer = analyzer(
                     SqlTestUtils.randomConfiguration(SqlVersion.fromId(v.id)),
                     functionRegistry,
                     loadCompatibleIndexResolution("mapping-numeric.json", v),
@@ -362,7 +362,7 @@ public class FieldAttributeTests extends ESTestCase {
         SqlConfiguration sqlConfig = SqlTestUtils.randomConfiguration(SqlVersion.fromId(preVersion.id));
 
         for (String sql : List.of(query, queryWithCastLiteral, queryWithAlias, queryWithCast)) {
-            analyzer = new Analyzer(
+            analyzer = analyzer(
                 sqlConfig,
                 functionRegistry,
                 loadCompatibleIndexResolution("mapping-version.json", preVersion),
@@ -372,7 +372,7 @@ public class FieldAttributeTests extends ESTestCase {
             assertThat(ex.getMessage(), containsString("Cannot use field [version_number]"));
 
             for (Version v : List.of(INTRODUCING_VERSION_FIELD_TYPE, postVersion)) {
-                analyzer = new Analyzer(
+                analyzer = analyzer(
                     SqlTestUtils.randomConfiguration(SqlVersion.fromId(v.id)),
                     functionRegistry,
                     loadCompatibleIndexResolution("mapping-version.json", v),
@@ -393,7 +393,7 @@ public class FieldAttributeTests extends ESTestCase {
     public void testNonProjectedUnsignedLongVersionCompatibility() {
         Version preUnsignedLong = Version.fromId(INTRODUCING_UNSIGNED_LONG.id - SqlVersion.MINOR_MULTIPLIER);
         SqlConfiguration sqlConfig = SqlTestUtils.randomConfiguration(SqlVersion.fromId(preUnsignedLong.id));
-        analyzer = new Analyzer(
+        analyzer = analyzer(
             sqlConfig,
             functionRegistry,
             loadCompatibleIndexResolution("mapping-numeric.json", preUnsignedLong),
@@ -427,7 +427,7 @@ public class FieldAttributeTests extends ESTestCase {
         String sql = "SELECT container.ul as unsigned_long FROM test";
 
         Version preUnsignedLong = Version.fromId(INTRODUCING_UNSIGNED_LONG.id - SqlVersion.MINOR_MULTIPLIER);
-        analyzer = new Analyzer(
+        analyzer = analyzer(
             SqlTestUtils.randomConfiguration(SqlVersion.fromId(preUnsignedLong.id)),
             functionRegistry,
             compatibleIndexResolution(props, preUnsignedLong),
@@ -438,7 +438,7 @@ public class FieldAttributeTests extends ESTestCase {
 
         Version postUnsignedLong = Version.fromId(INTRODUCING_UNSIGNED_LONG.id + SqlVersion.MINOR_MULTIPLIER);
         for (Version v : List.of(INTRODUCING_UNSIGNED_LONG, postUnsignedLong)) {
-            analyzer = new Analyzer(
+            analyzer = analyzer(
                 SqlTestUtils.randomConfiguration(SqlVersion.fromId(v.id)),
                 functionRegistry,
                 compatibleIndexResolution(props, v),
@@ -463,7 +463,7 @@ public class FieldAttributeTests extends ESTestCase {
         for (SqlVersion version : List.of(preUnsignedLong, SqlVersion.fromId(INTRODUCING_UNSIGNED_LONG.id), postUnsignedLong)) {
             SqlConfiguration config = SqlTestUtils.randomConfiguration(version);
             // the mapping is mutated when making it "compatible", so it needs to be reloaded inside the loop.
-            analyzer = new Analyzer(
+            analyzer = analyzer(
                 config,
                 functionRegistry,
                 loadCompatibleIndexResolution("mapping-numeric.json", Version.fromId(version.id)),
@@ -481,7 +481,7 @@ public class FieldAttributeTests extends ESTestCase {
     }
 
     public void testFunctionOverNonExistingFieldAsArgumentAndSameAlias() throws Exception {
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, loadIndexResolution("mapping-basic.json"), verifier);
+        analyzer = analyzer(SqlTestUtils.TEST_CFG, functionRegistry, loadIndexResolution("mapping-basic.json"), verifier);
 
         VerificationException ex = expectThrows(
             VerificationException.class,
@@ -491,7 +491,7 @@ public class FieldAttributeTests extends ESTestCase {
     }
 
     public void testFunctionWithExpressionOverNonExistingFieldAsArgumentAndSameAlias() throws Exception {
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, loadIndexResolution("mapping-basic.json"), verifier);
+        analyzer = analyzer(SqlTestUtils.TEST_CFG, functionRegistry, loadIndexResolution("mapping-basic.json"), verifier);
 
         VerificationException ex = expectThrows(
             VerificationException.class,
@@ -503,7 +503,7 @@ public class FieldAttributeTests extends ESTestCase {
     public void testExpandStarOnIndexWithoutColumns() {
         EsIndex test = new EsIndex("test", Collections.emptyMap());
         getIndexResult = IndexResolution.valid(test);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
+        analyzer = analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, verifier);
 
         LogicalPlan plan = plan("SELECT * FROM test");
 
@@ -529,4 +529,17 @@ public class FieldAttributeTests extends ESTestCase {
         EsIndex index = new EsIndex("test", mapping);
         return IndexCompatibility.compatible(IndexResolution.valid(index), version);
     }
+
+    private static Analyzer analyzer(
+        SqlConfiguration configuration,
+        FunctionRegistry functionRegistry,
+        IndexResolution resolution,
+        Verifier verifier
+    ) {
+        return new Analyzer(new AnalyzerContext(configuration, functionRegistry, resolution), verifier);
+    }
+
+    private static Analyzer analyzer(FunctionRegistry functionRegistry, IndexResolution resolution, Verifier verifier) {
+        return analyzer(SqlTestUtils.TEST_CFG, functionRegistry, resolution, verifier);
+    }
 }

+ 3 - 5
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java

@@ -12,7 +12,6 @@ import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.analysis.index.IndexResolverTests;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.First;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.Last;
 import org.elasticsearch.xpack.sql.expression.function.scalar.math.Round;
@@ -25,7 +24,6 @@ import org.elasticsearch.xpack.sql.expression.predicate.conditional.IfNull;
 import org.elasticsearch.xpack.sql.expression.predicate.conditional.Least;
 import org.elasticsearch.xpack.sql.expression.predicate.conditional.NullIf;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 
 import java.util.Arrays;
 import java.util.HashMap;
@@ -39,7 +37,7 @@ import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonMap;
 import static org.elasticsearch.xpack.ql.type.DataTypes.KEYWORD;
 import static org.elasticsearch.xpack.ql.type.DataTypes.OBJECT;
-import static org.elasticsearch.xpack.sql.SqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.types.SqlTypesTests.loadMapping;
 
 public class VerifierErrorMessagesTests extends ESTestCase {
@@ -54,7 +52,7 @@ public class VerifierErrorMessagesTests extends ESTestCase {
     }
 
     private String error(IndexResolution getIndexResult, String sql) {
-        Analyzer analyzer = new Analyzer(TEST_CFG, new SqlFunctionRegistry(), getIndexResult, new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(getIndexResult);
         VerificationException e = expectThrows(VerificationException.class, () -> analyzer.analyze(parser.createStatement(sql), true));
         String message = e.getMessage();
         assertTrue(message.startsWith("Found "));
@@ -74,7 +72,7 @@ public class VerifierErrorMessagesTests extends ESTestCase {
     }
 
     private LogicalPlan accept(IndexResolution resolution, String sql) {
-        Analyzer analyzer = new Analyzer(TEST_CFG, new SqlFunctionRegistry(), resolution, new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(resolution);
         return analyzer.analyze(parser.createStatement(sql), true);
     }
 

+ 3 - 4
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/DatabaseFunctionTests.java

@@ -15,15 +15,14 @@ import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.plan.logical.Project;
 import org.elasticsearch.xpack.sql.action.Protocol;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.proto.Mode;
 import org.elasticsearch.xpack.sql.session.SqlConfiguration;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
+
 public class DatabaseFunctionTests extends ESTestCase {
 
     public void testDatabaseFunctionOutput() {
@@ -49,7 +48,7 @@ public class DatabaseFunctionTests extends ESTestCase {
             null,
             randomBoolean()
         );
-        Analyzer analyzer = new Analyzer(sqlConfig, new SqlFunctionRegistry(), IndexResolution.valid(test), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(sqlConfig, IndexResolution.valid(test));
 
         Project result = (Project) analyzer.analyze(parser.createStatement("SELECT DATABASE()"), true);
         NamedExpression ne = result.projections().get(0);

+ 3 - 4
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/UserFunctionTests.java

@@ -15,15 +15,14 @@ import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.plan.logical.Project;
 import org.elasticsearch.xpack.sql.action.Protocol;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.proto.Mode;
 import org.elasticsearch.xpack.sql.session.SqlConfiguration;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
+
 public class UserFunctionTests extends ESTestCase {
 
     public void testNoUsernameFunctionOutput() {
@@ -48,7 +47,7 @@ public class UserFunctionTests extends ESTestCase {
             null,
             randomBoolean()
         );
-        Analyzer analyzer = new Analyzer(sqlConfig, new SqlFunctionRegistry(), IndexResolution.valid(test), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(sqlConfig, IndexResolution.valid(test));
 
         Project result = (Project) analyzer.analyze(parser.createStatement("SELECT USER()"), true);
         NamedExpression ne = result.projections().get(0);

+ 2 - 4
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentDateTimeTests.java

@@ -15,11 +15,8 @@ import org.elasticsearch.xpack.ql.session.Configuration;
 import org.elasticsearch.xpack.ql.tree.AbstractNodeTestCase;
 import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.ParsingException;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 
 import java.time.ZoneId;
@@ -29,6 +26,7 @@ import java.util.Objects;
 
 import static org.elasticsearch.xpack.ql.tree.Source.EMPTY;
 import static org.elasticsearch.xpack.sql.SqlTestUtils.literal;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 
 public class CurrentDateTimeTests extends AbstractNodeTestCase<CurrentDateTime, Expression> {
 
@@ -93,7 +91,7 @@ public class CurrentDateTimeTests extends AbstractNodeTestCase<CurrentDateTime,
             new EsIndex("test", SqlTypesTests.loadMapping("mapping-multi-field-with-nested.json"))
         );
 
-        Analyzer analyzer = new Analyzer(SqlTestUtils.TEST_CFG, new SqlFunctionRegistry(), indexResolution, new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(indexResolution);
         ParsingException e = expectThrows(
             ParsingException.class,
             () -> analyzer.analyze(parser.createStatement("SELECT CURRENT_TIMESTAMP(100000000000000)"), true)

+ 2 - 4
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/CurrentTimeTests.java

@@ -15,11 +15,8 @@ import org.elasticsearch.xpack.ql.session.Configuration;
 import org.elasticsearch.xpack.ql.tree.AbstractNodeTestCase;
 import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.ParsingException;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 
 import java.time.OffsetTime;
@@ -30,6 +27,7 @@ import java.util.Objects;
 
 import static org.elasticsearch.xpack.ql.tree.Source.EMPTY;
 import static org.elasticsearch.xpack.sql.SqlTestUtils.literal;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 
 public class CurrentTimeTests extends AbstractNodeTestCase<CurrentTime, Expression> {
 
@@ -94,7 +92,7 @@ public class CurrentTimeTests extends AbstractNodeTestCase<CurrentTime, Expressi
             new EsIndex("test", SqlTypesTests.loadMapping("mapping-multi-field-with-nested.json"))
         );
 
-        Analyzer analyzer = new Analyzer(SqlTestUtils.TEST_CFG, new SqlFunctionRegistry(), indexResolution, new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(indexResolution);
         ParsingException e = expectThrows(
             ParsingException.class,
             () -> analyzer.analyze(parser.createStatement("SELECT CURRENT_TIME(100000000000000)"), true)

+ 2 - 7
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerRunTests.java

@@ -12,7 +12,6 @@ import org.elasticsearch.xpack.ql.expression.Expression;
 import org.elasticsearch.xpack.ql.expression.FieldAttribute;
 import org.elasticsearch.xpack.ql.expression.Literal;
 import org.elasticsearch.xpack.ql.expression.UnresolvedAttribute;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison;
 import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.Equals;
 import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.GreaterThan;
@@ -28,11 +27,8 @@ import org.elasticsearch.xpack.ql.plan.logical.Filter;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.ql.plan.logical.UnaryPlan;
 import org.elasticsearch.xpack.ql.type.EsField;
-import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 
 import java.time.ZonedDateTime;
@@ -48,12 +44,12 @@ import static org.elasticsearch.xpack.ql.expression.predicate.operator.compariso
 import static org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparisonProcessor.BinaryComparisonOperation.LTE;
 import static org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparisonProcessor.BinaryComparisonOperation.NEQ;
 import static org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparisonProcessor.BinaryComparisonOperation.NULLEQ;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 
 public class OptimizerRunTests extends ESTestCase {
 
     private final SqlParser parser;
     private final IndexResolution getIndexResult;
-    private final FunctionRegistry functionRegistry;
     private final Analyzer analyzer;
     private final Optimizer optimizer;
     private static final Map<String, Class<? extends BinaryComparison>> COMPARISONS = new HashMap<>() {
@@ -71,13 +67,12 @@ public class OptimizerRunTests extends ESTestCase {
 
     public OptimizerRunTests() {
         parser = new SqlParser();
-        functionRegistry = new FunctionRegistry();
 
         Map<String, EsField> mapping = SqlTypesTests.loadMapping("mapping-multi-field-variation.json");
 
         EsIndex test = new EsIndex("test", mapping);
         getIndexResult = IndexResolution.valid(test);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, functionRegistry, getIndexResult, new Verifier(new Metrics()));
+        analyzer = analyzer(getIndexResult);
         optimizer = new Optimizer();
     }
 

+ 2 - 4
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumnsTests.java

@@ -10,7 +10,6 @@ import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexCompatibility;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
@@ -18,7 +17,6 @@ import org.elasticsearch.xpack.ql.index.IndexResolver;
 import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.action.Protocol;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.plan.logical.command.Command;
 import org.elasticsearch.xpack.sql.proto.Mode;
@@ -28,7 +26,6 @@ import org.elasticsearch.xpack.sql.session.Cursor;
 import org.elasticsearch.xpack.sql.session.SchemaRowSet;
 import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.SqlSession;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 
 import java.sql.Types;
@@ -50,6 +47,7 @@ import static org.elasticsearch.xpack.ql.index.VersionCompatibilityChecks.INTROD
 import static org.elasticsearch.xpack.ql.index.VersionCompatibilityChecks.isTypeSupportedInVersion;
 import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG;
 import static org.elasticsearch.xpack.ql.type.DataTypes.VERSION;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.proto.Mode.isDriver;
 import static org.elasticsearch.xpack.sql.types.SqlTypesTests.loadMapping;
 import static org.mockito.ArgumentMatchers.any;
@@ -392,7 +390,7 @@ public class SysColumnsTests extends ESTestCase {
         Map<String, EsField> mapping
     ) {
         EsIndex test = new EsIndex("test", mapping);
-        Analyzer analyzer = new Analyzer(config, new FunctionRegistry(), IndexResolution.valid(test), new Verifier(new Metrics()));
+        Analyzer analyzer = analyzer(config, IndexResolution.valid(test));
         Command cmd = (Command) analyzer.analyze(parser.createStatement(sql, params, UTC), true);
 
         IndexResolver resolver = mock(IndexResolver.class);

+ 2 - 9
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java

@@ -9,7 +9,6 @@ package org.elasticsearch.xpack.sql.plan.logical.command.sys;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.index.IndexResolver;
@@ -20,7 +19,6 @@ import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.action.Protocol;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.plan.logical.command.Command;
 import org.elasticsearch.xpack.sql.proto.Mode;
@@ -28,7 +26,6 @@ import org.elasticsearch.xpack.sql.proto.SqlTypedParamValue;
 import org.elasticsearch.xpack.sql.session.SchemaRowSet;
 import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.SqlSession;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 
@@ -44,6 +41,7 @@ import static java.util.Collections.emptyList;
 import static org.elasticsearch.action.ActionListener.wrap;
 import static org.elasticsearch.xpack.ql.index.IndexResolver.SQL_TABLE;
 import static org.elasticsearch.xpack.ql.index.IndexResolver.SQL_VIEW;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
@@ -368,12 +366,7 @@ public class SysTablesTests extends ESTestCase {
 
     private Tuple<Command, SqlSession> sql(String sql, List<SqlTypedParamValue> params, SqlConfiguration cfg) {
         EsIndex test = new EsIndex("test", mapping);
-        Analyzer analyzer = new Analyzer(
-            SqlTestUtils.TEST_CFG,
-            new FunctionRegistry(),
-            IndexResolution.valid(test),
-            new Verifier(new Metrics())
-        );
+        Analyzer analyzer = analyzer(IndexResolution.valid(test));
         Command cmd = (Command) analyzer.analyze(parser.createStatement(sql, params, cfg.zoneId()), true);
 
         IndexResolver resolver = mock(IndexResolver.class);

+ 2 - 2
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTypesTests.java

@@ -9,7 +9,6 @@ package org.elasticsearch.xpack.sql.plan.logical.command.sys;
 import org.elasticsearch.Version;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.xpack.ql.expression.function.FunctionRegistry;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.index.IndexResolver;
@@ -38,6 +37,7 @@ import static org.elasticsearch.action.ActionListener.wrap;
 import static org.elasticsearch.xpack.ql.index.VersionCompatibilityChecks.isTypeSupportedInVersion;
 import static org.elasticsearch.xpack.ql.type.DataTypes.UNSIGNED_LONG;
 import static org.elasticsearch.xpack.ql.type.DataTypes.VERSION;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.plan.logical.command.sys.SysColumnsTests.UNSIGNED_LONG_TEST_VERSIONS;
 import static org.elasticsearch.xpack.sql.plan.logical.command.sys.SysColumnsTests.VERSION_FIELD_TEST_VERSIONS;
 import static org.mockito.Mockito.mock;
@@ -67,7 +67,7 @@ public class SysTypesTests extends ESTestCase {
             false
         );
         EsIndex test = new EsIndex("test", SqlTypesTests.loadMapping("mapping-multi-field-with-nested.json", true));
-        Analyzer analyzer = new Analyzer(configuration, new FunctionRegistry(), IndexResolution.valid(test), null);
+        Analyzer analyzer = analyzer(configuration, IndexResolution.valid(test));
         Command cmd = (Command) analyzer.analyze(parser.createStatement(sql), false);
 
         IndexResolver resolver = mock(IndexResolver.class);

+ 2 - 3
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java

@@ -19,7 +19,6 @@ import org.elasticsearch.xpack.ql.querydsl.container.Sort;
 import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
 import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.optimizer.Optimizer;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
@@ -31,7 +30,6 @@ import org.elasticsearch.xpack.sql.querydsl.container.PivotColumnRef;
 import org.elasticsearch.xpack.sql.querydsl.container.QueryContainer;
 import org.elasticsearch.xpack.sql.session.EmptyExecutable;
 import org.elasticsearch.xpack.sql.session.SingletonExecutable;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -42,6 +40,7 @@ import java.util.Map;
 
 import static java.util.Arrays.asList;
 import static java.util.stream.Collectors.toList;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.endsWith;
 import static org.hamcrest.Matchers.startsWith;
@@ -60,7 +59,7 @@ public class QueryFolderTests extends ESTestCase {
         Map<String, EsField> mapping = SqlTypesTests.loadMapping("mapping-multi-field-variation.json");
         EsIndex test = new EsIndex("test", mapping);
         IndexResolution getIndexResult = IndexResolution.valid(test);
-        analyzer = new Analyzer(SqlTestUtils.TEST_CFG, new SqlFunctionRegistry(), getIndexResult, new Verifier(new Metrics()));
+        analyzer = analyzer(getIndexResult);
         optimizer = new Optimizer();
         planner = new Planner();
     }

+ 2 - 5
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorSpecTests.java

@@ -16,13 +16,10 @@ import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.optimizer.Optimizer;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.plan.physical.EsQueryExec;
 import org.elasticsearch.xpack.sql.plan.physical.PhysicalPlan;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 import org.hamcrest.Matcher;
@@ -32,7 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static org.elasticsearch.xpack.sql.SqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 
 public class QueryTranslatorSpecTests extends ESTestCase {
 
@@ -49,7 +46,7 @@ public class QueryTranslatorSpecTests extends ESTestCase {
             Map<String, EsField> mapping = SqlTypesTests.loadMapping(mappingFile);
             EsIndex test = new EsIndex("test", mapping);
             IndexResolution getIndexResult = IndexResolution.valid(test);
-            analyzer = new Analyzer(TEST_CFG, new SqlFunctionRegistry(), getIndexResult, new Verifier(new Metrics()));
+            analyzer = analyzer(getIndexResult);
             optimizer = new Optimizer();
             planner = new Planner();
         }

+ 2 - 3
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java

@@ -48,7 +48,6 @@ import org.elasticsearch.xpack.ql.querydsl.query.TermQuery;
 import org.elasticsearch.xpack.ql.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.ql.type.EsField;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
 import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.ExtendedStatsEnclosed;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.MatrixStatsEnclosed;
@@ -75,7 +74,6 @@ import org.elasticsearch.xpack.sql.querydsl.agg.AggFilter;
 import org.elasticsearch.xpack.sql.querydsl.agg.GroupByDateHistogram;
 import org.elasticsearch.xpack.sql.querydsl.container.MetricAggRef;
 import org.elasticsearch.xpack.sql.session.SingletonExecutable;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 import org.hamcrest.Matcher;
@@ -102,6 +100,7 @@ import static org.elasticsearch.xpack.ql.type.DataTypes.INTEGER;
 import static org.elasticsearch.xpack.ql.type.DataTypes.TEXT;
 import static org.elasticsearch.xpack.sql.SqlTestUtils.TEST_CFG;
 import static org.elasticsearch.xpack.sql.SqlTestUtils.literal;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation.E;
 import static org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation.PI;
 import static org.elasticsearch.xpack.sql.planner.QueryTranslator.DATE_FORMAT;
@@ -130,7 +129,7 @@ public class QueryTranslatorTests extends ESTestCase {
             Map<String, EsField> mapping = SqlTypesTests.loadMapping(mappingFile);
             EsIndex test = new EsIndex("test", mapping);
             IndexResolution getIndexResult = IndexResolution.valid(test);
-            analyzer = new Analyzer(TEST_CFG, sqlFunctionRegistry, getIndexResult, new Verifier(new Metrics()));
+            analyzer = analyzer(getIndexResult);
             optimizer = new Optimizer();
             planner = new Planner();
         }

+ 2 - 5
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/VerifierTests.java

@@ -11,14 +11,11 @@ import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
-import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.plan.physical.EsQueryExec;
 import org.elasticsearch.xpack.sql.plan.physical.PhysicalPlan;
-import org.elasticsearch.xpack.sql.stats.Metrics;
 
-import static org.elasticsearch.xpack.sql.SqlTestUtils.TEST_CFG;
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.types.SqlTypesTests.loadMapping;
 
 public class VerifierTests extends ESTestCase {
@@ -27,7 +24,7 @@ public class VerifierTests extends ESTestCase {
     private final IndexResolution indexResolution = IndexResolution.valid(
         new EsIndex("test", loadMapping("mapping-multi-field-with-nested.json"))
     );
-    private final Analyzer analyzer = new Analyzer(TEST_CFG, new SqlFunctionRegistry(), indexResolution, new Verifier(new Metrics()));
+    private final Analyzer analyzer = analyzer(indexResolution);
     private final Planner planner = new Planner();
 
     private PhysicalPlan verify(String sql) {

+ 2 - 3
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/stats/VerifierMetricsTests.java

@@ -12,15 +12,14 @@ import org.elasticsearch.xpack.core.watcher.common.stats.Counters;
 import org.elasticsearch.xpack.ql.index.EsIndex;
 import org.elasticsearch.xpack.ql.index.IndexResolution;
 import org.elasticsearch.xpack.ql.type.EsField;
-import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Analyzer;
 import org.elasticsearch.xpack.sql.analysis.analyzer.Verifier;
-import org.elasticsearch.xpack.sql.expression.function.SqlFunctionRegistry;
 import org.elasticsearch.xpack.sql.parser.SqlParser;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
 
 import java.util.Map;
 
+import static org.elasticsearch.xpack.sql.analysis.analyzer.AnalyzerTestUtils.analyzer;
 import static org.elasticsearch.xpack.sql.stats.FeatureMetric.COMMAND;
 import static org.elasticsearch.xpack.sql.stats.FeatureMetric.GROUPBY;
 import static org.elasticsearch.xpack.sql.stats.FeatureMetric.HAVING;
@@ -258,7 +257,7 @@ public class VerifierMetricsTests extends ESTestCase {
             verifier = new Verifier(metrics);
         }
 
-        Analyzer analyzer = new Analyzer(SqlTestUtils.TEST_CFG, new SqlFunctionRegistry(), IndexResolution.valid(test), verifier);
+        Analyzer analyzer = analyzer(IndexResolution.valid(test), verifier);
         analyzer.analyze(parser.createStatement(sql), true);
 
         return metrics == null ? null : metrics.stats();