Jelajahi Sumber

ESQL: Push down StartsWith and EndsWith functions to Lucene (#123381)

Fixes https://github.com/elastic/elasticsearch/issues/123067

Just like WildcardLike and RLike, some functions can be converted to Lucene queries. Here it's those two, which are nearly identical to WildcardLike

This, like some other functions, needs a FoldContext. I'm using the static method for this here, but it's fixed in https://github.com/elastic/elasticsearch/pull/123398, which I kept separated as it changes many files
Iván Cea Fontenla 7 bulan lalu
induk
melakukan
2fff041077

+ 6 - 0
docs/changelog/123381.yaml

@@ -0,0 +1,6 @@
+pr: 123381
+summary: Push down `StartsWith` and `EndsWith` functions to Lucene
+area: ES|QL
+type: enhancement
+issues:
+ - 123067

+ 172 - 0
x-pack/plugin/esql/qa/testFixtures/src/main/resources/string.csv-spec

@@ -954,6 +954,46 @@ false                  | null
 false                  | null
 ;
 
+startsWithLucenePushdown
+
+from hosts
+| where starts_with(host, "bet") and starts_with(host_group, "Kuber")
+| keep host, host_group
+| sort host, host_group;
+
+host:keyword | host_group:text
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+;
+
+startsWithLuceneDisabledPushdown
+
+from hosts
+| where host == "unknown host" or (starts_with(host, "bet") and starts_with(host_group, "Kuber"))
+| keep host, host_group
+| sort host, host_group;
+
+host:keyword | host_group:text
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+;
+
+startsWithLucenePushdownIgnoreMultivalues
+
+from hosts
+| where starts_with(description, "epsilon")
+| keep description
+| sort description;
+
+warning:Line 2:9: evaluation of [starts_with(description, \"epsilon\")] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:9: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+description:text
+epsilon gw instance
+;
+
 substringOfText
 
 from hosts | where host=="epsilon" | eval l1 = substring(host_group, 0, 5), l2 = substring(description, 0, 5) | keep l1, l2;
@@ -1199,6 +1239,138 @@ Bernatsky      |false
 ;
 
 
+endsWithLucenePushdown
+
+from hosts
+| where ends_with(host, "ta") and ends_with(host_group, "cluster")
+| keep host, host_group
+| sort host, host_group;
+
+host:keyword | host_group:text
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+;
+
+endsWithLuceneDisabledPushdown
+
+from hosts
+| where host == "unknown host" or (ends_with(host, "ta") and ends_with(host_group, "cluster"))
+| keep host, host_group
+| sort host, host_group;
+
+host:keyword | host_group:text
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+;
+
+endsWithLucenePushdownIgnoreMultivalues
+
+from hosts
+| where ends_with(description, "host")
+| keep description
+| sort description;
+
+warning:Line 2:9: evaluation of [ends_with(description, \"host\")] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:9: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+description:text
+;
+
+
+lucenePushdownMultipleWhere
+
+from hosts
+| where starts_with(host, "bet")
+| keep host, host_group
+| sort host, host_group
+| where ends_with(host_group, "cluster");
+
+host:keyword | host_group:text
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+beta         | Kubernetes cluster
+;
+
+lucenePushdownMultipleIndices
+
+from airports* metadata _index
+| where starts_with(name::keyword, "Sahn") and ends_with(abbrev, "UH")
+| keep abbrev, name, _index
+| sort abbrev, name, _index;
+
+abbrev:keyword | name:text | _index:keyword
+LUH            | Sahnewal  | airports
+LUH            | Sahnewal  | airports_mp
+LUH            | Sahnewal  | airports_no_doc_values
+LUH            | Sahnewal  | airports_not_indexed
+LUH            | Sahnewal  | airports_not_indexed_nor_doc_values
+LUH            | Sahnewal  | airports_web
+;
+
+lucenePushdownOr
+
+from airports
+| where starts_with(name::keyword, "Sahn") or ends_with(abbrev, "UH")
+| keep abbrev, name
+| sort abbrev, name;
+
+abbrev:keyword | name:text
+AUH            | Abu Dhabi Int'l  
+LUH            | Sahnewal         
+RUH            | King Khalid Int'l
+;
+
+lucenePushdownMultipleOr
+
+from airports
+| where starts_with(name::keyword, "Sahn") or ends_with(abbrev, "UH") or starts_with(abbrev, "OOL")
+| keep abbrev, name
+| sort abbrev, name;
+
+abbrev:keyword | name:text
+AUH            | Abu Dhabi Int'l
+LUH            | Sahnewal
+OOL            | Gold Coast
+RUH            | King Khalid Int'l
+;
+
+lucenePushdownMultipleAnd
+
+from airports metadata _index
+| where starts_with(name::keyword, "Sahn") and ends_with(abbrev, "UH")
+| where ends_with(name::keyword, "al")
+| keep abbrev, name, _index
+| sort abbrev, name, _index;
+
+abbrev:keyword | name:text | _index:keyword
+LUH            | Sahnewal  | airports
+;
+
+lucenePushdownMixAndOr
+
+from airports
+| where starts_with(name::keyword, "Sahn") and (starts_with(name::keyword, "Abc") or ends_with(abbrev, "UH"))
+| keep abbrev, name, scalerank
+| sort abbrev, name;
+
+abbrev:keyword | name:text | scalerank:integer
+LUH            | Sahnewal  | 9
+;
+
+lucenePushdownMixOrAnd
+
+from airports* metadata _index
+| where starts_with(name::keyword, "Sahn") or (starts_with(abbrev, "G") and ends_with(name::keyword, "Falls Int'l"))
+| where ends_with(_index, "airports")
+| keep abbrev, name, scalerank, _index
+| sort abbrev;
+
+abbrev:keyword | name:text         | scalerank:integer | _index:keyword
+GTF            | Great Falls Int'l | 8                 | airports      
+LUH            | Sahnewal          | 9                 | airports
+;
 
 toLowerRow#[skip:-8.12.99]
 // tag::to_lower[]

+ 31 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/EndsWith.java

@@ -7,13 +7,20 @@
 
 package org.elasticsearch.xpack.esql.expression.function.scalar.string;
 
+import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.lucene.BytesRefs;
 import org.elasticsearch.compute.ann.Evaluator;
 import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
+import org.elasticsearch.xpack.esql.capabilities.TranslationAware;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.FoldContext;
+import org.elasticsearch.xpack.esql.core.querydsl.query.Query;
+import org.elasticsearch.xpack.esql.core.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
@@ -22,6 +29,8 @@ import org.elasticsearch.xpack.esql.expression.function.FunctionInfo;
 import org.elasticsearch.xpack.esql.expression.function.Param;
 import org.elasticsearch.xpack.esql.expression.function.scalar.EsqlScalarFunction;
 import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.planner.TranslatorHandler;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -31,7 +40,7 @@ import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.Param
 import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND;
 import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString;
 
-public class EndsWith extends EsqlScalarFunction {
+public class EndsWith extends EsqlScalarFunction implements TranslationAware.SingleValueTranslationAware {
     public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "EndsWith", EndsWith::new);
 
     private final Expression str;
@@ -129,6 +138,27 @@ public class EndsWith extends EsqlScalarFunction {
         return new EndsWithEvaluator.Factory(source(), toEvaluator.apply(str), toEvaluator.apply(suffix));
     }
 
+    @Override
+    public boolean translatable(LucenePushdownPredicates pushdownPredicates) {
+        return pushdownPredicates.isPushableAttribute(str) && suffix.foldable();
+    }
+
+    @Override
+    public Query asQuery(TranslatorHandler handler) {
+        LucenePushdownPredicates.checkIsPushableAttribute(str);
+        var fieldName = handler.nameOf(str instanceof FieldAttribute fa ? fa.exactAttribute() : str);
+
+        // TODO: Get the real FoldContext here
+        var wildcardQuery = "*" + QueryParser.escape(BytesRefs.toString(suffix.fold(FoldContext.small())));
+
+        return new WildcardQuery(source(), fieldName, wildcardQuery);
+    }
+
+    @Override
+    public Expression singleValueField() {
+        return str;
+    }
+
     Expression str() {
         return str;
     }

+ 31 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWith.java

@@ -7,13 +7,20 @@
 
 package org.elasticsearch.xpack.esql.expression.function.scalar.string;
 
+import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.lucene.BytesRefs;
 import org.elasticsearch.compute.ann.Evaluator;
 import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator;
+import org.elasticsearch.xpack.esql.capabilities.TranslationAware;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.FoldContext;
+import org.elasticsearch.xpack.esql.core.querydsl.query.Query;
+import org.elasticsearch.xpack.esql.core.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
@@ -22,6 +29,8 @@ import org.elasticsearch.xpack.esql.expression.function.FunctionInfo;
 import org.elasticsearch.xpack.esql.expression.function.Param;
 import org.elasticsearch.xpack.esql.expression.function.scalar.EsqlScalarFunction;
 import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.planner.TranslatorHandler;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -31,7 +40,7 @@ import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.Param
 import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND;
 import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString;
 
-public class StartsWith extends EsqlScalarFunction {
+public class StartsWith extends EsqlScalarFunction implements TranslationAware.SingleValueTranslationAware {
     public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(
         Expression.class,
         "StartsWith",
@@ -126,6 +135,27 @@ public class StartsWith extends EsqlScalarFunction {
         return new StartsWithEvaluator.Factory(source(), toEvaluator.apply(str), toEvaluator.apply(prefix));
     }
 
+    @Override
+    public boolean translatable(LucenePushdownPredicates pushdownPredicates) {
+        return pushdownPredicates.isPushableAttribute(str) && prefix.foldable();
+    }
+
+    @Override
+    public Query asQuery(TranslatorHandler handler) {
+        LucenePushdownPredicates.checkIsPushableAttribute(str);
+        var fieldName = handler.nameOf(str instanceof FieldAttribute fa ? fa.exactAttribute() : str);
+
+        // TODO: Get the real FoldContext here
+        var wildcardQuery = QueryParser.escape(BytesRefs.toString(prefix.fold(FoldContext.small()))) + "*";
+
+        return new WildcardQuery(source(), fieldName, wildcardQuery);
+    }
+
+    @Override
+    public Expression singleValueField() {
+        return str;
+    }
+
     Expression str() {
         return str;
     }

+ 41 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/EndsWithTests.java

@@ -12,14 +12,21 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.type.EsField;
 import org.elasticsearch.xpack.esql.expression.function.AbstractScalarFunctionTestCase;
 import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.planner.TranslatorHandler;
 import org.hamcrest.Matcher;
 
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Supplier;
 
 import static org.hamcrest.Matchers.equalTo;
@@ -98,4 +105,38 @@ public class EndsWithTests extends AbstractScalarFunctionTestCase {
     protected Expression build(Source source, List<Expression> args) {
         return new EndsWith(source, args.get(0), args.get(1));
     }
+
+    public void testLuceneQuery_AllLiterals_NonTranslatable() {
+        var function = new EndsWith(
+            Source.EMPTY,
+            new Literal(Source.EMPTY, "test", DataType.KEYWORD),
+            new Literal(Source.EMPTY, "test", DataType.KEYWORD)
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(false));
+    }
+
+    public void testLuceneQuery_NonFoldableSuffix_NonTranslatable() {
+        var function = new EndsWith(
+            Source.EMPTY,
+            new FieldAttribute(Source.EMPTY, "field", new EsField("field", DataType.KEYWORD, Map.of(), true)),
+            new FieldAttribute(Source.EMPTY, "field", new EsField("suffix", DataType.KEYWORD, Map.of(), true))
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(false));
+    }
+
+    public void testLuceneQuery_NonFoldableSuffix_Translatable() {
+        var function = new EndsWith(
+            Source.EMPTY,
+            new FieldAttribute(Source.EMPTY, "field", new EsField("suffix", DataType.KEYWORD, Map.of(), true)),
+            new Literal(Source.EMPTY, "a*b?c\\", DataType.KEYWORD)
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(true));
+
+        var query = function.asQuery(TranslatorHandler.TRANSLATOR_HANDLER);
+
+        assertThat(query, equalTo(new WildcardQuery(Source.EMPTY, "field", "*a\\*b\\?c\\\\")));
+    }
 }

+ 41 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/StartsWithTests.java

@@ -12,13 +12,20 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 
 import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.type.EsField;
 import org.elasticsearch.xpack.esql.expression.function.AbstractScalarFunctionTestCase;
 import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.planner.TranslatorHandler;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.function.Supplier;
 
 import static org.hamcrest.Matchers.equalTo;
@@ -58,4 +65,38 @@ public class StartsWithTests extends AbstractScalarFunctionTestCase {
     protected Expression build(Source source, List<Expression> args) {
         return new StartsWith(source, args.get(0), args.get(1));
     }
+
+    public void testLuceneQuery_AllLiterals_NonTranslatable() {
+        var function = new StartsWith(
+            Source.EMPTY,
+            new Literal(Source.EMPTY, "test", DataType.KEYWORD),
+            new Literal(Source.EMPTY, "test", DataType.KEYWORD)
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(false));
+    }
+
+    public void testLuceneQuery_NonFoldablePrefix_NonTranslatable() {
+        var function = new StartsWith(
+            Source.EMPTY,
+            new FieldAttribute(Source.EMPTY, "field", new EsField("field", DataType.KEYWORD, Map.of(), true)),
+            new FieldAttribute(Source.EMPTY, "field", new EsField("prefix", DataType.KEYWORD, Map.of(), true))
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(false));
+    }
+
+    public void testLuceneQuery_NonFoldablePrefix_Translatable() {
+        var function = new StartsWith(
+            Source.EMPTY,
+            new FieldAttribute(Source.EMPTY, "field", new EsField("prefix", DataType.KEYWORD, Map.of(), true)),
+            new Literal(Source.EMPTY, "a*b?c\\", DataType.KEYWORD)
+        );
+
+        assertThat(function.translatable(LucenePushdownPredicates.DEFAULT), equalTo(true));
+
+        var query = function.asQuery(TranslatorHandler.TRANSLATOR_HANDLER);
+
+        assertThat(query, equalTo(new WildcardQuery(Source.EMPTY, "field", "a\\*b\\?c\\\\*")));
+    }
 }

+ 65 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java

@@ -1132,6 +1132,71 @@ public class PhysicalPlanOptimizerTests extends ESTestCase {
         assertThat(rq.to(), nullValue());
     }
 
+    /**
+     * Expects
+     *
+     * LimitExec[1000[INTEGER]]
+     * \_ExchangeExec[[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, gender{f}#7,
+     *    hire_date{f}#12, job{f}#13, job.raw{f}#14, languages{f}#8, last_name{f}#9,
+     *    long_noidx{f}#15, salary{f}#10],false]
+     *   \_ProjectExec[[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, gender{f}#7,
+     *      hire_date{f}#12, job{f}#13, job.raw{f}#14, languages{f}#8, last_name{f}#9,
+     *      long_noidx{f}#15, salary{f}#10]]
+     *     \_FieldExtractExec[_meta_field{f}#11, emp_no{f}#5, first_name{f}#6, ge..]
+     *       \_EsQueryExec[test], indexMode[standard], query[
+     *         {"bool":{"must":[
+     *           {"bool":{"should":[
+     *             {"esql_single_value":{"field":"first_name","next":
+     *               {"wildcard":{"first_name":{"wildcard":"\\*Firs*","boost":1.0}}}
+     *               "source":"starts_with(first_name, \"*Firs\")@2:9"}},
+     *             {"esql_single_value":{"field":"first_name","next":
+     *               {"wildcard":{"first_name":{"wildcard":"*irst\\*","boost":1.0}}},
+     *               "source":"ends_with(first_name, \"irst*\")@2:45"}}],"boost":1.0}},
+     *             {"esql_single_value":{"field":"last_name","next":
+     *               {"wildcard":{"last_name":{"wildcard":"*ast","boost":1.0}}},
+     *               "source":"ends_with(last_name, \"ast\")@3:9"}}
+     *           ],"boost":1.0}}
+     *       ][_doc{f}#27], limit[1000], sort[] estimatedRowSize[332]
+     */
+    public void testPushMultipleFunctions() {
+        var plan = physicalPlan("""
+            from airports
+            | where starts_with(first_name, "*Firs") or ends_with(first_name, "irst*")
+            | where ends_with(last_name, "ast")
+            """);
+
+        var optimized = optimizedPlan(plan);
+        var topLimit = as(optimized, LimitExec.class);
+        var exchange = asRemoteExchange(topLimit.child());
+        var project = as(exchange.child(), ProjectExec.class);
+        var fieldExtract = as(project.child(), FieldExtractExec.class);
+        var source = source(fieldExtract.child());
+        assertThat(source.estimatedRowSize(), equalTo(allFieldRowSize + Integer.BYTES));
+
+        var andBool = as(source.query(), BoolQueryBuilder.class);
+        assertThat(andBool.must(), hasSize(2));
+        assertThat(andBool.should(), hasSize(0));
+
+        var orBool = as(andBool.must().get(0), BoolQueryBuilder.class);
+        assertThat(orBool.should(), hasSize(2));
+        assertThat(orBool.must(), hasSize(0));
+
+        var orStartsWith = as(sv(orBool.should().get(0), "first_name"), WildcardQueryBuilder.class);
+        assertThat(orStartsWith.fieldName(), equalTo("first_name"));
+        assertThat(orStartsWith.caseInsensitive(), equalTo(false));
+        assertThat(orStartsWith.value(), equalTo("\\*Firs*"));
+
+        var orEndsWith = as(sv(orBool.should().get(1), "first_name"), WildcardQueryBuilder.class);
+        assertThat(orEndsWith.fieldName(), equalTo("first_name"));
+        assertThat(orEndsWith.caseInsensitive(), equalTo(false));
+        assertThat(orEndsWith.value(), equalTo("*irst\\*"));
+
+        var andEndsWith = as(sv(andBool.must().get(1), "last_name"), WildcardQueryBuilder.class);
+        assertThat(andEndsWith.fieldName(), equalTo("last_name"));
+        assertThat(andEndsWith.caseInsensitive(), equalTo(false));
+        assertThat(andEndsWith.value(), equalTo("*ast"));
+    }
+
     public void testLimit() {
         var optimized = optimizedPlan(physicalPlan("""
             from test