This is an automated email from the ASF dual-hosted git repository.

airborne pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 155cd67531d [fix](search) add restriction for search function (#56706)
155cd67531d is described below

commit 155cd67531d49e525e43dcf4d8c8a00d3b9cf20a
Author: Jack <[email protected]>
AuthorDate: Sun Oct 5 21:09:26 2025 +0800

    [fix](search) add restriction for search function (#56706)
    
    ### What problem does this PR solve?
    
    Issue Number: close #xxx
    
    Related PR: #56139
    
    Problem Summary:
    This PR adds restrictions for the search() function to ensure it can
    only be used in WHERE clauses on single-table OLAP scans. The
    implementation includes validation rules that reject search() usage in
    other contexts like SELECT projections, GROUP BY clauses, HAVING
    clauses, and multi-table scenarios.
---
 .../doris/nereids/jobs/executor/Analyzer.java      |   3 +
 .../org/apache/doris/nereids/rules/RuleType.java   |   1 +
 .../nereids/rules/analysis/CheckSearchUsage.java   | 130 +++++++++
 .../rules/analysis/CheckSearchUsageTest.java       | 295 +++++++++++++++++++++
 .../data/search/test_search_usage_restrictions.out |  20 ++
 .../search/test_search_usage_restrictions.groovy   | 176 ++++++++++++
 6 files changed, 625 insertions(+)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
index beb423a0460..358e9faeb06 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java
@@ -29,6 +29,7 @@ import org.apache.doris.nereids.rules.analysis.BindSkewExpr;
 import org.apache.doris.nereids.rules.analysis.CheckAfterBind;
 import org.apache.doris.nereids.rules.analysis.CheckAnalysis;
 import org.apache.doris.nereids.rules.analysis.CheckPolicy;
+import org.apache.doris.nereids.rules.analysis.CheckSearchUsage;
 import org.apache.doris.nereids.rules.analysis.CollectJoinConstraint;
 import org.apache.doris.nereids.rules.analysis.CollectSubQueryAlias;
 import org.apache.doris.nereids.rules.analysis.CompressedMaterialize;
@@ -177,6 +178,8 @@ public class Analyzer extends AbstractBatchJobExecutor {
             // @t_zone must be replaced as 'GMT' before 
EliminateGroupByConstant and NormalizeAggregate rule.
             // So need run VariableToLiteral rule before the two rules.
             topDown(new VariableToLiteral()),
+            // run CheckSearchUsage before CheckAnalysis to detect search() in 
GROUP BY before it gets optimized
+            bottomUp(new CheckSearchUsage()),
             // run CheckAnalysis before EliminateGroupByConstant in order to 
report error message correctly like bellow
             // select SUM(lo_tax) FROM lineorder group by 1;
             // errCode = 2, detailMessage = GROUP BY expression must not 
contain aggregate functions: sum(lo_tax)
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
index 849913fc6fc..37e357d8f0d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java
@@ -135,6 +135,7 @@ public enum RuleType {
     CHECK_AND_STANDARDIZE_WINDOW_FUNCTION_AND_FRAME(RuleTypeClass.REWRITE),
     CHECK_MATCH_EXPRESSION(RuleTypeClass.REWRITE),
     REWRITE_SEARCH_TO_SLOTS(RuleTypeClass.REWRITE),
+    CHECK_SEARCH_USAGE(RuleTypeClass.REWRITE),
     CREATE_PARTITION_TOPN_FOR_WINDOW(RuleTypeClass.REWRITE),
     AGGREGATE_DISASSEMBLE(RuleTypeClass.REWRITE),
     SIMPLIFY_AGG_GROUP_BY(RuleTypeClass.REWRITE),
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsage.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsage.java
new file mode 100644
index 00000000000..0fd66ef0618
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsage.java
@@ -0,0 +1,130 @@
+// 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
+                    throw new AnalysisException("search() can only appear in 
WHERE filters on OLAP scans; "
+                            + "projection of search() is not supported");
+                }
+            }
+        }
+
+        // Recursively check children
+        for (Plan child : plan.children()) {
+            checkPlanRecursively(child);
+        }
+    }
+
+    private void validateSearchUsage(Plan plan) {
+        if (plan instanceof LogicalFilter) {
+            Plan child = plan.child(0);
+            if (!(child instanceof LogicalOlapScan)) {
+                throw new AnalysisException("search() predicate only supports 
filtering directly on a single "
+                        + "table scan; remove joins, subqueries, or additional 
operators between search() "
+                        + "and the target table");
+            }
+        } else if (!(plan instanceof LogicalProject)) {
+            // search() can only appear in LogicalFilter or specific 
LogicalProject nodes
+            throw new AnalysisException("search() predicates are only 
supported inside WHERE filters on "
+                    + "single-table scans");
+        }
+    }
+
+    private boolean containsSearchInPlanExpressions(Plan plan) {
+        for (Expression expr : plan.getExpressions()) {
+            if (containsSearchExpression(expr)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private boolean containsSearchExpression(Expression expression) {
+        if (expression instanceof Search || expression instanceof 
SearchExpression) {
+            return true;
+        }
+        for (Expression child : expression.children()) {
+            if (containsSearchExpression(child)) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsageTest.java
 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsageTest.java
new file mode 100644
index 00000000000..ab9a98660e8
--- /dev/null
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckSearchUsageTest.java
@@ -0,0 +1,295 @@
+// 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.util.MemoPatternMatchSupported;
+import org.apache.doris.nereids.util.PlanChecker;
+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() 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.
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+    }
+
+    @Test
+    public void testSearchWithMultipleFieldsAllowed() {
+        // Valid: search() with multiple fields in WHERE
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello AND content:world')";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInOrderByRejected() {
+        // Invalid: search() in ORDER BY
+        String sql = "SELECT id, title FROM test_search_table ORDER BY 
search('title:hello')";
+
+        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 testSearchInCaseWhenRejected() {
+        // Invalid: search() in CASE WHEN (outside WHERE)
+        String sql = "SELECT CASE WHEN search('title:hello') THEN 1 ELSE 0 END 
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 testSearchWithComplexWhereAllowed() {
+        // Valid: search() in complex WHERE clause
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE (search('title:hello') OR id = 1) AND category = 
'tech'";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testMultipleSearchInWhereAllowed() {
+        // Valid: multiple search() functions in WHERE
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello') AND search('content:world')";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext)
+                    .analyze(sql)
+                    .matches(logicalFilter());
+        });
+    }
+
+    @Test
+    public void testSearchInWhereWithLimitAllowed() {
+        // Valid: search() in WHERE with LIMIT
+        String sql = "SELECT id, title FROM test_search_table "
+                + "WHERE search('title:hello') LIMIT 10";
+
+        Assertions.assertDoesNotThrow(() -> {
+            PlanChecker.from(connectContext).analyze(sql);
+        });
+    }
+
+    @Test
+    public void testRuleTypeCorrect() {
+        CheckSearchUsage rule = new CheckSearchUsage();
+        Assertions.assertNotNull(rule);
+        Assertions.assertNotNull(rule.buildRules());
+        Assertions.assertEquals(1, rule.buildRules().size());
+        Assertions.assertEquals(
+                org.apache.doris.nereids.rules.RuleType.CHECK_SEARCH_USAGE,
+                rule.buildRules().get(0).getRuleType());
+    }
+}
diff --git a/regression-test/data/search/test_search_usage_restrictions.out 
b/regression-test/data/search/test_search_usage_restrictions.out
new file mode 100644
index 00000000000..6fc83963beb
--- /dev/null
+++ b/regression-test/data/search/test_search_usage_restrictions.out
@@ -0,0 +1,20 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !valid_where --
+
+-- !valid_where_complex --
+
+-- !valid_multiple_search --
+1
+
+-- !valid_with_limit --
+
+-- !valid_subquery --
+
+-- !valid_multi_field --
+
+-- !valid_complex_where --
+1
+
+-- !valid_union --
+1
+
diff --git 
a/regression-test/suites/search/test_search_usage_restrictions.groovy 
b/regression-test/suites/search/test_search_usage_restrictions.groovy
new file mode 100644
index 00000000000..ea31a4eb998
--- /dev/null
+++ b/regression-test/suites/search/test_search_usage_restrictions.groovy
@@ -0,0 +1,176 @@
+// 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.
+
+suite("test_search_usage_restrictions") {
+    def tableName = "search_usage_test_table"
+    def tableName2 = "search_usage_test_table2"
+
+    sql "DROP TABLE IF EXISTS ${tableName}"
+    sql "DROP TABLE IF EXISTS ${tableName2}"
+
+    // Create test table with inverted index
+    sql """
+        CREATE TABLE ${tableName} (
+            id INT,
+            title VARCHAR(255),
+            content TEXT,
+            category VARCHAR(100),
+            INDEX idx_title (title) USING INVERTED,
+            INDEX idx_content (content) USING INVERTED PROPERTIES("parser" = 
"english"),
+            INDEX idx_category (category) USING INVERTED
+        ) ENGINE=OLAP
+        DUPLICATE KEY(id)
+        DISTRIBUTED BY HASH(id) BUCKETS 3
+        PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1"
+        )
+    """
+
+    // Create second table for join tests
+    sql """
+        CREATE TABLE ${tableName2} (
+            id INT,
+            name VARCHAR(255)
+        ) ENGINE=OLAP
+        DUPLICATE KEY(id)
+        DISTRIBUTED BY HASH(id) BUCKETS 3
+        PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1"
+        )
+    """
+
+    // Insert test data
+    sql """INSERT INTO ${tableName} VALUES
+        (1, 'Machine Learning', 'AI and ML tutorial', 'Technology'),
+        (2, 'Deep Learning', 'Neural networks guide', 'Technology'),
+        (3, 'Python Guide', 'Python programming', 'Programming'),
+        (4, 'Data Science', 'Data analysis methods', 'Science'),
+        (5, 'Web Development', 'Web dev tips', 'Technology')
+    """
+
+    sql """INSERT INTO ${tableName2} VALUES (1, 'Test'), (2, 'Example')"""
+
+    // Wait for data
+    Thread.sleep(5000)
+
+    // ============ Valid Usage Tests ============
+
+    // Test 1: search() in WHERE clause is allowed
+    qt_valid_where "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ id, 
title FROM ${tableName} WHERE search('title:Learning') ORDER BY id"
+
+    // Test 2: search() with AND/OR in WHERE is allowed
+    qt_valid_where_complex "SELECT 
/*+SET_VAR(enable_common_expr_pushdown=true) */ id FROM ${tableName} WHERE 
search('title:Learning') AND id > 1 ORDER BY id"
+
+    // Test 3: Multiple search() in WHERE is allowed
+    qt_valid_multiple_search "SELECT 
/*+SET_VAR(enable_common_expr_pushdown=true) */ id FROM ${tableName} WHERE 
search('title:Learning') OR search('content:tutorial') ORDER BY id"
+
+    // Test 4: search() with LIMIT is allowed
+    qt_valid_with_limit "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) 
*/ id FROM ${tableName} WHERE search('title:Learning') LIMIT 2"
+
+    // ============ Invalid Usage Tests - Should Fail ============
+
+    // Test 5: search() in GROUP BY should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ count(*) 
FROM ${tableName} GROUP BY search('title:Learning')"
+        exception "predicates are only supported inside WHERE filters on 
single-table scans"
+    }
+
+    // Test 6: search() in SELECT then GROUP BY alias should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ 
search('title:Learning') as s, count(*) FROM ${tableName} GROUP BY s"
+        exception "search()"
+    }
+
+    // Test 7: search() in SELECT projection (without WHERE) should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ 
search('title:Learning'), title FROM ${tableName}"
+        exception "search()"
+    }
+
+    // Test 8: search() in aggregate output should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ 
count(search('title:Learning')) FROM ${tableName}"
+        exception "search()"
+    }
+
+    // Test 9: search() in HAVING clause should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ category, 
count(*) FROM ${tableName} GROUP BY category HAVING search('title:Learning')"
+        exception "predicates are only supported inside WHERE filters on 
single-table scans"
+    }
+
+    // Test 10: search() with JOIN should fail (not single table)
+    test {
+        sql """
+            SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ t1.id FROM 
${tableName} t1
+            JOIN ${tableName2} t2 ON t1.id = t2.id
+            WHERE search('title:Learning')
+        """
+        exception "single"
+    }
+
+    // Test 11: search() in ORDER BY should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ id, title 
FROM ${tableName} ORDER BY search('title:Learning')"
+        exception "search()"
+    }
+
+    // Test 12: search() in CASE WHEN (outside WHERE) should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ CASE WHEN 
search('title:Learning') THEN 1 ELSE 0 END FROM ${tableName}"
+        exception "search()"
+    }
+
+    // Test 13: search() in aggregate function context should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ 
max(search('title:Learning')) FROM ${tableName}"
+        exception "search()"
+    }
+
+    // Test 14: search() in window function should fail
+    test {
+        sql "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ id, 
row_number() OVER (ORDER BY search('title:Learning')) FROM ${tableName}"
+        exception "search()"
+    }
+
+    // ============ Edge Cases ============
+
+    // Test 15: search() in subquery WHERE is allowed (subquery is single 
table)
+    qt_valid_subquery """
+        SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ * FROM (
+            SELECT id, title FROM ${tableName} WHERE search('title:Learning')
+        ) t WHERE id > 1 ORDER BY id
+    """
+
+    // Test 16: Multiple fields in search() DSL is allowed
+    qt_valid_multi_field "SELECT /*+SET_VAR(enable_common_expr_pushdown=true) 
*/ id FROM ${tableName} WHERE search('title:Learning AND content:tutorial') 
ORDER BY id"
+
+    // Test 17: search() with complex boolean logic in WHERE is allowed
+    qt_valid_complex_where """
+        SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ id FROM 
${tableName}
+        WHERE (search('title:Learning') OR id = 1) AND category = 'Technology'
+        ORDER BY id
+    """
+
+    // Test 18: search() in UNION queries (each part valid) should work
+    qt_valid_union """
+        SELECT /*+SET_VAR(enable_common_expr_pushdown=true) */ id FROM 
${tableName} WHERE search('title:Learning')
+        UNION ALL
+        SELECT id FROM ${tableName} WHERE search('content:tutorial')
+        ORDER BY id
+    """
+}


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

Reply via email to