Copilot commented on code in PR #56706:
URL: https://github.com/apache/doris/pull/56706#discussion_r2404268829


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsage.java:
##########
@@ -0,0 +1,135 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.analysis;
+
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.rules.Rule;
+import org.apache.doris.nereids.rules.RuleType;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.SearchExpression;
+import org.apache.doris.nereids.trees.expressions.functions.scalar.Search;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
+import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
+import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Check search expression usage - search() can only be used in WHERE filters 
on single-table OLAP scans.
+ * This rule validates that search() expressions only appear in supported 
contexts.
+ * Must run in analysis phase before search() gets optimized away.
+ */
+public class CheckSearchUsage implements AnalysisRuleFactory {
+
+    @Override
+    public List<Rule> buildRules() {
+        return ImmutableList.of(
+            any().thenApply(ctx -> {
+                Plan plan = ctx.root;
+                checkPlanRecursively(plan);
+                return plan;
+            }).toRule(RuleType.CHECK_SEARCH_USAGE)
+        );
+    }
+
+    private void checkPlanRecursively(Plan plan) {
+        // Check if current plan node contains search expressions
+        if (containsSearchInPlanExpressions(plan)) {
+            validateSearchUsage(plan);
+        }
+
+        // Check aggregate nodes specifically for GROUP BY usage
+        if (plan instanceof LogicalAggregate) {
+            LogicalAggregate<?> agg = (LogicalAggregate<?>) plan;
+            for (Expression expr : agg.getGroupByExpressions()) {
+                if (containsSearchExpression(expr)) {
+                    throw new AnalysisException("search() cannot appear in 
GROUP BY expressions; "
+                            + "search predicates are only supported in WHERE 
filters on single-table scans");
+                }
+            }
+            for (Expression expr : agg.getOutputExpressions()) {
+                if (containsSearchExpression(expr)) {
+                    throw new AnalysisException("search() cannot appear in 
aggregate output expressions; "
+                            + "search predicates are only supported in WHERE 
filters on single-table scans");
+                }
+            }
+        }
+
+        // Check project nodes
+        if (plan instanceof LogicalProject) {
+            LogicalProject<?> project = (LogicalProject<?>) plan;
+            for (Expression expr : project.getProjects()) {
+                if (containsSearchExpression(expr)) {
+                    // Only allow if it's the project directly above a 
filter->scan pattern
+                    Plan child = plan.child(0);
+                    if (!(child instanceof LogicalFilter
+                            && child.child(0) instanceof LogicalOlapScan)) {
+                        throw new AnalysisException("search() can only appear 
in WHERE filters on OLAP scans; "
+                                + "projection of search() is not supported");
+                    }
+                    break;

Review Comment:
   This logic allows search() in projections when there's a direct filter->scan 
pattern, but based on the PR description and test cases, search() should only 
be allowed in WHERE clauses. This creates an inconsistency where `SELECT 
search('title:hello') FROM table WHERE id > 1` would be allowed if there's a 
filter present.
   ```suggestion
                       throw new AnalysisException("search() can only appear in 
WHERE filters on OLAP scans; "
                               + "projection of search() is not supported");
   ```



##########
fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsageTest.java:
##########
@@ -0,0 +1,300 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.analysis;
+
+import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.util.MemoPatternMatchSupported;
+import org.apache.doris.nereids.util.PlanChecker;
+import org.apache.doris.nereids.util.PlanConstructor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.utframe.TestWithFeService;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Unit tests for CheckSearchUsage rule.
+ * This test validates that search() function can only be used in WHERE clauses
+ * on single-table OLAP scans, and is rejected in other contexts.
+ */
+public class CheckSearchUsageTest extends TestWithFeService implements 
MemoPatternMatchSupported {
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDatabase("test");
+        connectContext.setDatabase("test");
+
+        // Create test table with inverted index
+        createTable("CREATE TABLE test_search_table (\n"
+                + "  id INT,\n"
+                + "  title VARCHAR(255),\n"
+                + "  content TEXT,\n"
+                + "  category VARCHAR(100),\n"
+                + "  INDEX idx_title(title) USING INVERTED,\n"
+                + "  INDEX idx_content(content) USING INVERTED\n"
+                + ") ENGINE=OLAP\n"
+                + "DUPLICATE KEY(id)\n"
+                + "DISTRIBUTED BY HASH(id) BUCKETS 3\n"
+                + "PROPERTIES ('replication_num' = '1');");
+    }
+
+    @Test
+    public void testSearchInWhereClauseAllowed() {
+        // Valid usage: search() in WHERE clause on single table
+        String sql = "SELECT id, title FROM test_search_table WHERE 
search('title:hello')";
+
+        // This should NOT throw exception
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInWhereWithAndAllowed() {
+        // Valid usage: search() combined with other predicates
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello') AND id > 10";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInGroupByRejected() {
+        // Invalid: search() directly in GROUP BY expression
+        String sql = "SELECT count(*) FROM test_search_table GROUP BY 
search('title:hello')";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("GROUP BY")
+                    || exception.getMessage().contains("WHERE filters")
+                    || exception.getMessage().contains("single-table")),
+                "Expected error about search() usage restrictions, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInGroupByWithAliasRejected() {
+        // Invalid: search() in SELECT, then GROUP BY alias
+        String sql = "SELECT search('title:hello') as s, count(*) FROM 
test_search_table GROUP BY s";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("projection")
+                    || exception.getMessage().contains("WHERE")),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInSelectWithoutGroupByRejected() {
+        // Invalid: search() in SELECT projection without WHERE
+        String sql = "SELECT search('title:hello'), title FROM 
test_search_table";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("projection")
+                    || exception.getMessage().contains("WHERE")),
+                "Expected error about search() in projection, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInAggregateOutputRejected() {
+        // Invalid: search() wrapped in aggregate function
+        // Note: This might be caught by other checks, but we test it anyway
+        String sql = "SELECT count(search('title:hello')) FROM 
test_search_table";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                || exception.getMessage().contains("WHERE"),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInHavingRejected() {
+        // Invalid: search() in HAVING clause
+        String sql = "SELECT category, count(*) FROM test_search_table "
+                + "GROUP BY category HAVING search('title:hello')";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                || exception.getMessage().contains("WHERE")
+                || exception.getMessage().contains("HAVING"),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchWithJoinRejected() {
+        // Create second table for join test
+        try {
+            createTable("CREATE TABLE test_search_table2 (\n"
+                    + "  id INT,\n"
+                    + "  name VARCHAR(255)\n"
+                    + ") ENGINE=OLAP\n"
+                    + "DUPLICATE KEY(id)\n"
+                    + "DISTRIBUTED BY HASH(id) BUCKETS 3\n"
+                    + "PROPERTIES ('replication_num' = '1');");
+        } catch (Exception e) {
+            // Table might already exist from previous test
+        }
+
+        // Invalid: search() in WHERE with JOIN
+        String sql = "SELECT t1.id FROM test_search_table t1 "
+                + "JOIN test_search_table2 t2 ON t1.id = t2.id "

Review Comment:
   Using a try-catch block to handle table creation is not a reliable test 
pattern. Consider using `@BeforeEach` to ensure clean test state, or use a 
unique table name per test, or check if the table exists before creating it.
   ```suggestion
           // Use a unique table name for join test to avoid conflicts
           String tableName = "test_search_table2_" + 
System.currentTimeMillis();
           createTable("CREATE TABLE " + tableName + " (\n"
                   + "  id INT,\n"
                   + "  name VARCHAR(255)\n"
                   + ") ENGINE=OLAP\n"
                   + "DUPLICATE KEY(id)\n"
                   + "DISTRIBUTED BY HASH(id) BUCKETS 3\n"
                   + "PROPERTIES ('replication_num' = '1');");
   
           // Invalid: search() in WHERE with JOIN
           String sql = "SELECT t1.id FROM test_search_table t1 "
                   + "JOIN " + tableName + " t2 ON t1.id = t2.id "
   ```



##########
fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsageTest.java:
##########
@@ -0,0 +1,300 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.nereids.rules.analysis;
+
+import org.apache.doris.nereids.CascadesContext;
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.util.MemoPatternMatchSupported;
+import org.apache.doris.nereids.util.PlanChecker;
+import org.apache.doris.nereids.util.PlanConstructor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.utframe.TestWithFeService;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Unit tests for CheckSearchUsage rule.
+ * This test validates that search() function can only be used in WHERE clauses
+ * on single-table OLAP scans, and is rejected in other contexts.
+ */
+public class CheckSearchUsageTest extends TestWithFeService implements 
MemoPatternMatchSupported {
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        createDatabase("test");
+        connectContext.setDatabase("test");
+
+        // Create test table with inverted index
+        createTable("CREATE TABLE test_search_table (\n"
+                + "  id INT,\n"
+                + "  title VARCHAR(255),\n"
+                + "  content TEXT,\n"
+                + "  category VARCHAR(100),\n"
+                + "  INDEX idx_title(title) USING INVERTED,\n"
+                + "  INDEX idx_content(content) USING INVERTED\n"
+                + ") ENGINE=OLAP\n"
+                + "DUPLICATE KEY(id)\n"
+                + "DISTRIBUTED BY HASH(id) BUCKETS 3\n"
+                + "PROPERTIES ('replication_num' = '1');");
+    }
+
+    @Test
+    public void testSearchInWhereClauseAllowed() {
+        // Valid usage: search() in WHERE clause on single table
+        String sql = "SELECT id, title FROM test_search_table WHERE 
search('title:hello')";
+
+        // This should NOT throw exception
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInWhereWithAndAllowed() {
+        // Valid usage: search() combined with other predicates
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello') AND id > 10";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInGroupByRejected() {
+        // Invalid: search() directly in GROUP BY expression
+        String sql = "SELECT count(*) FROM test_search_table GROUP BY 
search('title:hello')";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("GROUP BY")
+                    || exception.getMessage().contains("WHERE filters")
+                    || exception.getMessage().contains("single-table")),
+                "Expected error about search() usage restrictions, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInGroupByWithAliasRejected() {
+        // Invalid: search() in SELECT, then GROUP BY alias
+        String sql = "SELECT search('title:hello') as s, count(*) FROM 
test_search_table GROUP BY s";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("projection")
+                    || exception.getMessage().contains("WHERE")),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInSelectWithoutGroupByRejected() {
+        // Invalid: search() in SELECT projection without WHERE
+        String sql = "SELECT search('title:hello'), title FROM 
test_search_table";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && (exception.getMessage().contains("projection")
+                    || exception.getMessage().contains("WHERE")),
+                "Expected error about search() in projection, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInAggregateOutputRejected() {
+        // Invalid: search() wrapped in aggregate function
+        // Note: This might be caught by other checks, but we test it anyway
+        String sql = "SELECT count(search('title:hello')) FROM 
test_search_table";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                || exception.getMessage().contains("WHERE"),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInHavingRejected() {
+        // Invalid: search() in HAVING clause
+        String sql = "SELECT category, count(*) FROM test_search_table "
+                + "GROUP BY category HAVING search('title:hello')";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                || exception.getMessage().contains("WHERE")
+                || exception.getMessage().contains("HAVING"),
+                "Expected error about search() usage, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchWithJoinRejected() {
+        // Create second table for join test
+        try {
+            createTable("CREATE TABLE test_search_table2 (\n"
+                    + "  id INT,\n"
+                    + "  name VARCHAR(255)\n"
+                    + ") ENGINE=OLAP\n"
+                    + "DUPLICATE KEY(id)\n"
+                    + "DISTRIBUTED BY HASH(id) BUCKETS 3\n"
+                    + "PROPERTIES ('replication_num' = '1');");
+        } catch (Exception e) {
+            // Table might already exist from previous test
+        }
+
+        // Invalid: search() in WHERE with JOIN
+        String sql = "SELECT t1.id FROM test_search_table t1 "
+                + "JOIN test_search_table2 t2 ON t1.id = t2.id "
+                + "WHERE search('title:hello')";
+
+        AnalysisException exception = 
Assertions.assertThrows(AnalysisException.class, () -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+
+        Assertions.assertTrue(
+                exception.getMessage().contains("search()")
+                && exception.getMessage().contains("single"),
+                "Expected error about single table, got: " + 
exception.getMessage());
+    }
+
+    @Test
+    public void testSearchInSubqueryRejected() {
+        // Invalid: search() in subquery
+        String sql = "SELECT * FROM (SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello')) t WHERE id > 10";
+
+        // The search() in subquery's WHERE should be allowed, but when 
projected up it might fail
+        // Let's test if it works in subquery

Review Comment:
   The comment suggests uncertainty about the expected behavior ('might fail', 
'Let's test if it works'). This test should have a clear assertion about 
whether search() in subqueries is allowed or not, and the comment should 
reflect the definitive expected behavior.
   ```suggestion
           // The search() function is allowed in the WHERE clause of a 
subquery over a single table.
           // This test verifies that such usage does not throw an exception.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to