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]