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


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SearchDslParser.java:
##########
@@ -0,0 +1,346 @@
+// 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.trees.expressions.functions.scalar;
+
+import org.apache.doris.nereids.search.SearchLexer;
+import org.apache.doris.nereids.search.SearchParser;
+import org.apache.doris.nereids.search.SearchParserBaseVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Search DSL Parser using ANTLR-generated parser.
+ * Parses DSL syntax and extracts field bindings for FE->BE communication.
+ * <p>
+ * Supported syntax:
+ * - field:term
+ * - field:"quoted term"
+ * - field:prefix*
+ * - field:*wildcard*
+ * - field:/regexp/
+ * - AND/OR/NOT operators
+ * - Parentheses for grouping
+ * - Range queries: field:[1 TO 10], field:{1 TO 10}
+ * - List queries: field:IN(value1 value2)
+ * - Any/All queries: field:ANY(value), field:ALL(value)
+ */
+public class SearchDslParser {
+    private static final Logger LOG = 
LogManager.getLogger(SearchDslParser.class);
+    private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
+
+    /**
+     * Parse DSL string and return intermediate representation
+     */
+    public static QsPlan parseDsl(String dsl) {
+        if (dsl == null || dsl.trim().isEmpty()) {
+            return new QsPlan(new QsNode(QsClauseType.TERM, "error", 
"empty_dsl"), new ArrayList<>());
+        }
+
+        try {
+            // Create ANTLR lexer and parser
+            SearchLexer lexer = new SearchLexer(new ANTLRInputStream(dsl));
+            CommonTokenStream tokens = new CommonTokenStream(lexer);
+            SearchParser parser = new SearchParser(tokens);
+
+            // Parse the search query
+            ParseTree tree = parser.search();
+
+            // Build AST using visitor pattern
+            QsAstBuilder visitor = new QsAstBuilder();
+            QsNode root = visitor.visit(tree);
+
+            // Extract field bindings
+            Set<String> fieldNames = visitor.getFieldNames();
+            List<QsFieldBinding> bindings = new ArrayList<>();
+            int slotIndex = 0;
+            for (String fieldName : fieldNames) {
+                bindings.add(new QsFieldBinding(fieldName, slotIndex++));
+            }
+
+            return new QsPlan(root, bindings);
+
+        } catch (Exception e) {
+            LOG.error("Failed to parse search DSL: '{}'", dsl, e);
+            throw new RuntimeException("Invalid search DSL syntax: " + dsl + 
". Error: " + e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Clause types supported
+     */
+    public enum QsClauseType {
+        TERM,       // field:value
+        PHRASE,     // field:"phrase search"
+        PREFIX,     // field:prefix*
+        WILDCARD,   // field:*wild*card*
+        REGEXP,     // field:/pattern/
+        RANGE,      // field:[1 TO 10] or field:{1 TO 10}
+        LIST,       // field:IN(value1 value2)
+        ANY,        // field:ANY(value) - any match
+        ALL,        // field:ALL(value) - all match
+        AND,        // clause1 AND clause2
+        OR,         // clause1 OR clause2
+        NOT         // NOT clause
+    }
+
+    /**
+     * ANTLR visitor to build QsNode AST from parse tree
+     */
+    private static class QsAstBuilder extends SearchParserBaseVisitor<QsNode> {
+        private final Set<String> fieldNames = new HashSet<>();
+
+        public Set<String> getFieldNames() {
+            return fieldNames;
+        }
+
+        @Override
+        public QsNode visitSearch(SearchParser.SearchContext ctx) {
+            return visit(ctx.clause());
+        }
+
+        @Override
+        public QsNode visitOrClause(SearchParser.OrClauseContext ctx) {
+            if (ctx.andClause().size() == 1) {
+                return visit(ctx.andClause(0));
+            }
+
+            List<QsNode> children = new ArrayList<>();
+            for (SearchParser.AndClauseContext andCtx : ctx.andClause()) {
+                children.add(visit(andCtx));
+            }
+            return new QsNode(QsClauseType.OR, children);
+        }
+
+        @Override
+        public QsNode visitAndClause(SearchParser.AndClauseContext ctx) {
+            if (ctx.notClause().size() == 1) {
+                return visit(ctx.notClause(0));
+            }
+
+            List<QsNode> children = new ArrayList<>();
+            for (SearchParser.NotClauseContext notCtx : ctx.notClause()) {
+                children.add(visit(notCtx));
+            }
+            return new QsNode(QsClauseType.AND, children);
+        }
+
+        @Override
+        public QsNode visitNotClause(SearchParser.NotClauseContext ctx) {
+            if (ctx.NOT() != null) {
+                QsNode child = visit(ctx.atomClause());
+                List<QsNode> children = new ArrayList<>();
+                children.add(child);
+                return new QsNode(QsClauseType.NOT, children);
+            }
+            return visit(ctx.atomClause());
+        }
+
+        @Override
+        public QsNode visitAtomClause(SearchParser.AtomClauseContext ctx) {
+            if (ctx.clause() != null) {
+                // Parenthesized clause
+                return visit(ctx.clause());
+            }
+            return visit(ctx.fieldQuery());
+        }
+
+        @Override
+        public QsNode visitFieldQuery(SearchParser.FieldQueryContext ctx) {
+            String fieldName = ctx.fieldName().getText();
+            if (fieldName.startsWith("\"") && fieldName.endsWith("\"")) {
+                fieldName = fieldName.substring(1, fieldName.length() - 1);
+            }
+            fieldNames.add(fieldName);
+
+            return visit(ctx.searchValue());
+        }
+
+        @Override
+        public QsNode visitSearchValue(SearchParser.SearchValueContext ctx) {
+            String fieldName = getCurrentFieldName();
+            if (ctx.TERM() != null) {
+                return new QsNode(QsClauseType.TERM, fieldName, 
ctx.TERM().getText());
+            } else
+                if (ctx.PREFIX() != null) {
+                    return new QsNode(QsClauseType.PREFIX, fieldName, 
ctx.PREFIX().getText());
+                } else
+                    if (ctx.WILDCARD() != null) {
+                        return new QsNode(QsClauseType.WILDCARD, fieldName, 
ctx.WILDCARD().getText());
+                    } else
+                        if (ctx.REGEXP() != null) {
+                            String regexp = ctx.REGEXP().getText();
+                            // Remove surrounding slashes
+                            if (regexp.startsWith("/") && 
regexp.endsWith("/")) {
+                                regexp = regexp.substring(1, regexp.length() - 
1);
+                            }
+                            return new QsNode(QsClauseType.REGEXP, fieldName, 
regexp);
+                        } else
+                            if (ctx.QUOTED() != null) {
+                                String quoted = ctx.QUOTED().getText();
+                                // Remove surrounding quotes
+                                if (quoted.startsWith("\"") && 
quoted.endsWith("\"")) {
+                                    quoted = quoted.substring(1, 
quoted.length() - 1);
+                                }
+                                return new QsNode(QsClauseType.PHRASE, 
fieldName, quoted);
+                            } else
+                                if (ctx.rangeValue() != null) {
+                                    return new QsNode(QsClauseType.RANGE, 
fieldName, ctx.rangeValue().getText());
+                                } else
+                                    if (ctx.listValue() != null) {
+                                        return new QsNode(QsClauseType.LIST, 
fieldName, ctx.listValue().getText());
+                                    } else
+                                        if (ctx.anyAllValue() != null) {
+                                            String anyAllText = 
ctx.anyAllValue().getText();
+                                            String innerContent = "";
+
+                                            // Extract content between 
parentheses
+                                            int openParen = 
anyAllText.indexOf('(');
+                                            int closeParen = 
anyAllText.lastIndexOf(')');
+                                            if (openParen >= 0 && closeParen > 
openParen) {
+                                                innerContent = 
anyAllText.substring(openParen + 1, closeParen).trim();
+                                            }
+
+                                            if 
(anyAllText.toUpperCase().startsWith("ANY(") || anyAllText.toLowerCase()
+                                                    .startsWith("any(")) {
+                                                return new 
QsNode(QsClauseType.ANY, fieldName, innerContent);
+                                            } else
+                                                if 
(anyAllText.toUpperCase().startsWith("ALL(")
+                                                        || 
anyAllText.toLowerCase().startsWith("all(")) {
+                                                    return new 
QsNode(QsClauseType.ALL, fieldName, innerContent);
+                                                } else {
+                                                    // Fallback to ANY for 
unknown cases
+                                                    return new 
QsNode(QsClauseType.ANY, fieldName, innerContent);
+                                                }
+                                        }

Review Comment:
   The nested if-else chain creates deeply indented code that is difficult to 
read and maintain. Consider refactoring this into a switch statement or 
separate methods to improve readability.
   ```suggestion
               }
               if (ctx.PREFIX() != null) {
                   return new QsNode(QsClauseType.PREFIX, fieldName, 
ctx.PREFIX().getText());
               }
               if (ctx.WILDCARD() != null) {
                   return new QsNode(QsClauseType.WILDCARD, fieldName, 
ctx.WILDCARD().getText());
               }
               if (ctx.REGEXP() != null) {
                   String regexp = ctx.REGEXP().getText();
                   // Remove surrounding slashes
                   if (regexp.startsWith("/") && regexp.endsWith("/")) {
                       regexp = regexp.substring(1, regexp.length() - 1);
                   }
                   return new QsNode(QsClauseType.REGEXP, fieldName, regexp);
               }
               if (ctx.QUOTED() != null) {
                   String quoted = ctx.QUOTED().getText();
                   // Remove surrounding quotes
                   if (quoted.startsWith("\"") && quoted.endsWith("\"")) {
                       quoted = quoted.substring(1, quoted.length() - 1);
                   }
                   return new QsNode(QsClauseType.PHRASE, fieldName, quoted);
               }
               if (ctx.rangeValue() != null) {
                   return new QsNode(QsClauseType.RANGE, fieldName, 
ctx.rangeValue().getText());
               }
               if (ctx.listValue() != null) {
                   return new QsNode(QsClauseType.LIST, fieldName, 
ctx.listValue().getText());
               }
               if (ctx.anyAllValue() != null) {
                   String anyAllText = ctx.anyAllValue().getText();
                   String innerContent = "";
   
                   // Extract content between parentheses
                   int openParen = anyAllText.indexOf('(');
                   int closeParen = anyAllText.lastIndexOf(')');
                   if (openParen >= 0 && closeParen > openParen) {
                       innerContent = anyAllText.substring(openParen + 1, 
closeParen).trim();
                   }
   
                   if (anyAllText.toUpperCase().startsWith("ANY(") || 
anyAllText.toLowerCase().startsWith("any(")) {
                       return new QsNode(QsClauseType.ANY, fieldName, 
innerContent);
                   }
                   if (anyAllText.toUpperCase().startsWith("ALL(") || 
anyAllText.toLowerCase().startsWith("all(")) {
                       return new QsNode(QsClauseType.ALL, fieldName, 
innerContent);
                   }
                   // Fallback to ANY for unknown cases
                   return new QsNode(QsClauseType.ANY, fieldName, innerContent);
               }
   ```



##########
be/src/olap/rowset/segment_v2/inverted_index_iterator.cpp:
##########
@@ -137,9 +137,9 @@ Result<InvertedIndexReaderPtr> 
InvertedIndexIterator::_select_best_reader(
             preferred_type = InvertedIndexReaderType::STRING_TYPE;
         }
     }
-    DBUG_EXECUTE_IF("inverted_index_reader._select_best_reader", {
+    DBUG_EXECUTE_IF("inverted_index_reader.select_best_reader", {
         auto type = 
DebugPoints::instance()->get_debug_param_or_default<int32_t>(
-                "inverted_index_reader._select_best_reader", "type", -1);
+                "inverted_index_reader.select_best_reader", "type", -1);

Review Comment:
   The debug point name has been updated from `_select_best_reader` to 
`select_best_reader` but the string literal still uses the old name with 
underscore prefix. This inconsistency could cause debug functionality to not 
work properly.



##########
be/src/vec/functions/function_search.cpp:
##########
@@ -0,0 +1,548 @@
+// 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.
+
+#include "vec/functions/function_search.h"
+
+#include <CLucene/config/repl_wchar.h>
+#include <CLucene/search/Scorer.h>
+#include <glog/logging.h>
+
+#include <memory>
+#include <roaring/roaring.hh>
+#include <set>
+#include <sstream>
+
+#include "common/status.h"
+#include "gen_cpp/Exprs_types.h"
+#include "olap/rowset/segment_v2/index_file_reader.h"
+#include "olap/rowset/segment_v2/index_query_context.h"
+#include "olap/rowset/segment_v2/inverted_index/analyzer/analyzer.h"
+#include "olap/rowset/segment_v2/inverted_index/query/conjunction_query.h"
+#include 
"olap/rowset/segment_v2/inverted_index/query_v2/boolean_query/boolean_query.h"
+#include "olap/rowset/segment_v2/inverted_index/query_v2/composite_reader.h"
+#include "olap/rowset/segment_v2/inverted_index/query_v2/operator.h"
+#include 
"olap/rowset/segment_v2/inverted_index/query_v2/term_query/term_query.h"
+#include "olap/rowset/segment_v2/inverted_index/util/string_helper.h"
+#include "olap/rowset/segment_v2/inverted_index_iterator.h"
+#include "olap/rowset/segment_v2/inverted_index_reader.h"
+#include "vec/columns/column_const.h"
+#include "vec/core/columns_with_type_and_name.h"
+#include "vec/data_types/data_type_string.h"
+#include "vec/functions/simple_function_factory.h"
+
+namespace doris::vectorized {
+
+using doris::segment_v2::InvertedIndexQueryType;
+using namespace doris::segment_v2;
+using namespace doris::segment_v2::inverted_index;
+using namespace doris::segment_v2::inverted_index::query_v2;
+using namespace lucene::analysis;
+using namespace lucene::index;
+using namespace lucene::util;
+using namespace lucene::search;
+
+Status FunctionSearch::execute_impl(FunctionContext* /*context*/, Block& 
/*block*/,
+                                    const ColumnNumbers& /*arguments*/, 
uint32_t /*result*/,
+                                    size_t /*input_rows_count*/) const {
+    return Status::RuntimeError("only inverted index queries are supported");
+}
+
+// Enhanced implementation: Handle new parameter structure (DSL + 
SlotReferences)
+Status FunctionSearch::evaluate_inverted_index(
+        const ColumnsWithTypeAndName& arguments,
+        const std::vector<vectorized::IndexFieldNameAndTypePair>& 
data_type_with_names,
+        std::vector<segment_v2::IndexIterator*> iterators, uint32_t num_rows,
+        segment_v2::InvertedIndexResultBitmap& bitmap_result) const {
+    return Status::OK();
+}
+
+// Helper function to build BooleanQuery from compound TSearchClause 
(AND/OR/NOT)
+std::shared_ptr<BooleanQuery> FunctionSearch::build_query_from_clause(
+        const TSearchClause& clause, const 
std::shared_ptr<segment_v2::IndexQueryContext>& context,
+        const std::unordered_map<std::string, 
vectorized::IndexFieldNameAndTypePair>&
+                data_type_with_names,
+        const std::unordered_map<std::string, segment_v2::IndexIterator*>& 
iterators) const {
+    const std::string& clause_type = clause.clause_type;
+
+    // This method should only be called for compound queries
+    if (clause_type != "AND" && clause_type != "OR" && clause_type != "NOT") {
+        LOG(WARNING) << "build_query_from_clause called with non-compound 
clause type: "
+                     << clause_type;
+        return nullptr;
+    }
+
+    // Determine operator type
+    OperatorType op;
+    if (clause_type == "AND") {
+        op = OperatorType::OP_AND;
+    } else if (clause_type == "OR") {
+        op = OperatorType::OP_OR;
+    } else { // NOT
+        op = OperatorType::OP_NOT;
+    }
+
+    BooleanQuery::Builder builder(op);
+
+    if (clause.__isset.children && !clause.children.empty()) {
+        for (const auto& child_clause : clause.children) {
+            const std::string& child_type = child_clause.clause_type;
+
+            if (child_type == "AND" || child_type == "OR" || child_type == 
"NOT") {
+                // Recursive call for compound child
+                auto child_boolean_query = 
build_query_from_clause(child_clause, context,
+                                                                   
data_type_with_names, iterators);
+                if (child_boolean_query) {
+                    
builder.add(std::static_pointer_cast<query_v2::Query>(child_boolean_query));
+                }
+            } else {
+                // Leaf child - build appropriate leaf query with proper 
iterator access
+                auto child_leaf_query =
+                        build_leaf_query(child_clause, context, 
data_type_with_names, iterators);
+                if (child_leaf_query) {
+                    builder.add(child_leaf_query);
+                }
+            }
+        }
+    }
+
+    return builder.build();
+}
+
+// Helper function to build leaf queries
+std::shared_ptr<query_v2::Query> FunctionSearch::build_leaf_query(
+        const TSearchClause& clause, const 
std::shared_ptr<segment_v2::IndexQueryContext>& context,
+        const std::unordered_map<std::string, 
vectorized::IndexFieldNameAndTypePair>&
+                data_type_with_names,
+        const std::unordered_map<std::string, segment_v2::IndexIterator*>& 
iterators) const {
+    if (!clause.__isset.field_name || !clause.__isset.value) {
+        LOG(WARNING) << "search: Leaf clause missing field_name or value";
+        return nullptr;
+    }
+
+    const std::string& field_name = clause.field_name;
+    const std::string& value = clause.value;
+    const std::string& clause_type = clause.clause_type;
+
+    // Find field in data_type_with_names
+    auto field_iter = data_type_with_names.find(field_name);
+    if (field_iter == data_type_with_names.end()) {
+        LOG(WARNING) << "search: Field '" << field_name << "' not found in 
data_type_with_names";
+        return nullptr;
+    }
+
+    const std::string& real_field_name = field_iter->second.first;
+    std::wstring field_wstr = StringHelper::to_wstring(real_field_name);
+    std::wstring value_wstr = StringHelper::to_wstring(value);
+
+    // Build appropriate query based on clause type using DRY principle
+    ClauseTypeCategory category = get_clause_type_category(clause_type);
+
+    // Handle different clause types based on tokenization needs (most are 
TODO and use TermQuery for now)
+    if (clause_type == "TERM") {
+        return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+    } else if (category == ClauseTypeCategory::TOKENIZED) {
+        // Tokenized queries: PHRASE, MATCH, ANY, ALL - need tokenization
+        if (clause_type == "PHRASE") {
+            // TODO: Implement PhraseQuery
+            LOG(INFO) << "TODO: PHRASE query type not yet implemented, using 
TermQuery";
+            return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+        } else if (clause_type == "MATCH") {
+            // TODO: Implement MatchQuery
+            LOG(INFO) << "TODO: MATCH query type not yet implemented, using 
TermQuery";
+            return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+        } else if (clause_type == "ANY") {
+            // ANY query: field:ANY(value1 value2 ...) - any match (OR 
operation)
+            auto index_properties = get_field_index_properties(field_name, 
iterators);
+            if (index_properties.empty()) {
+                LOG(WARNING) << "Failed to get index properties for ANY query 
field: "
+                             << field_name;
+                return std::make_shared<query_v2::TermQuery>(context, 
field_wstr, value_wstr);
+            }
+
+            // Parse using proper tokenization
+            std::vector<segment_v2::TermInfo> term_infos =
+                    
segment_v2::inverted_index::InvertedIndexAnalyzer::get_analyse_result(
+                            value, index_properties);
+            if (term_infos.empty()) {
+                LOG(WARNING) << "No terms found after tokenization for ANY 
query: field="
+                             << field_name << ", value='" << value << "'";
+                return nullptr;
+            }
+
+            if (term_infos.size() == 1) {
+                // Single term - use TermQuery directly
+                std::wstring term_wstr = 
StringHelper::to_wstring(term_infos[0].get_single_term());
+                return std::make_shared<query_v2::TermQuery>(context, 
field_wstr, term_wstr);
+            }
+
+            // Multiple terms - create BooleanQuery with OR operation
+            BooleanQuery::Builder builder(OperatorType::OP_OR);
+            for (const auto& term_info : term_infos) {
+                std::wstring term_wstr = 
StringHelper::to_wstring(term_info.get_single_term());
+                auto term_query =
+                        std::make_shared<query_v2::TermQuery>(context, 
field_wstr, term_wstr);
+                
builder.add(std::static_pointer_cast<query_v2::Query>(term_query));
+            }
+
+            LOG(INFO) << "Built ANY query with " << term_infos.size()
+                      << " OR conditions for field: " << field_name;
+            return std::static_pointer_cast<query_v2::Query>(builder.build());
+
+        } else if (clause_type == "ALL") {
+            // ALL query: field:ALL(value1 value2 ...) - all match (AND 
operation)
+            auto index_properties = get_field_index_properties(field_name, 
iterators);
+            if (index_properties.empty()) {
+                LOG(WARNING) << "Failed to get index properties for ALL query 
field: "
+                             << field_name;
+                return std::make_shared<query_v2::TermQuery>(context, 
field_wstr, value_wstr);
+            }
+
+            // Parse using proper tokenization
+            std::vector<segment_v2::TermInfo> term_infos =
+                    
segment_v2::inverted_index::InvertedIndexAnalyzer::get_analyse_result(
+                            value, index_properties);
+            if (term_infos.empty()) {
+                LOG(WARNING) << "No terms found after tokenization for ALL 
query: field="
+                             << field_name << ", value='" << value << "'";
+                return nullptr;
+            }
+
+            if (term_infos.size() == 1) {
+                // Single term - use TermQuery directly
+                std::wstring term_wstr = 
StringHelper::to_wstring(term_infos[0].get_single_term());
+                return std::make_shared<query_v2::TermQuery>(context, 
field_wstr, term_wstr);
+            }
+
+            // Multiple terms - create BooleanQuery with AND operation
+            BooleanQuery::Builder builder(OperatorType::OP_AND);
+            for (const auto& term_info : term_infos) {
+                std::wstring term_wstr = 
StringHelper::to_wstring(term_info.get_single_term());
+                auto term_query =
+                        std::make_shared<query_v2::TermQuery>(context, 
field_wstr, term_wstr);
+                
builder.add(std::static_pointer_cast<query_v2::Query>(term_query));
+            }
+
+            LOG(INFO) << "Built ALL query with " << term_infos.size()
+                      << " AND conditions for field: " << field_name;
+            return std::static_pointer_cast<query_v2::Query>(builder.build());
+        }
+        return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+    } else if (category == ClauseTypeCategory::NON_TOKENIZED) {
+        // Non-tokenized queries based on QueryParser.g4: exact/pattern 
matching, range, list operations
+        if (clause_type == "PREFIX") {
+            // TODO: Implement PrefixQuery
+            LOG(INFO) << "TODO: PREFIX query type not yet implemented, using 
TermQuery";
+        } else if (clause_type == "WILDCARD") {
+            // TODO: Implement WildcardQuery
+            LOG(INFO) << "TODO: WILDCARD query type not yet implemented, using 
TermQuery";
+        } else if (clause_type == "REGEXP") {
+            // TODO: Implement RegexpQuery
+            LOG(INFO) << "TODO: REGEXP query type not yet implemented, using 
TermQuery";
+        } else if (clause_type == "RANGE") {
+            // TODO: Implement RangeQuery
+            LOG(INFO) << "TODO: RANGE query type not yet implemented, using 
TermQuery";
+        } else if (clause_type == "LIST") {
+            // TODO: Implement ListQuery (IN operation)
+            LOG(INFO) << "TODO: LIST query type not yet implemented, using 
TermQuery";
+        }
+        return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+    } else {
+        // Default to TermQuery (should not happen due to 
get_clause_type_category logic)
+        LOG(WARNING) << "Unexpected clause category for leaf query type '" << 
clause_type
+                     << "', using TermQuery";
+        return std::make_shared<query_v2::TermQuery>(context, field_wstr, 
value_wstr);
+    }
+}
+
+Status FunctionSearch::evaluate_inverted_index_with_search_param(
+        const TSearchParam& search_param,
+        const std::unordered_map<std::string, 
vectorized::IndexFieldNameAndTypePair>&
+                data_type_with_names,
+        std::unordered_map<std::string, segment_v2::IndexIterator*> iterators, 
uint32_t num_rows,
+        segment_v2::InvertedIndexResultBitmap& bitmap_result) const {
+    LOG(INFO) << "search: Processing structured query with DSL: " << 
search_param.original_dsl
+              << ", available " << data_type_with_names.size() << " indexed 
columns, "
+              << iterators.size() << " iterators";
+
+    if (iterators.empty() || data_type_with_names.empty()) {
+        LOG(INFO) << "No indexed columns or iterators available, returning 
empty result";
+        return Status::OK();
+    }
+
+    // Create IndexQueryContext
+    auto context = std::make_shared<segment_v2::IndexQueryContext>();
+    context->collection_statistics = std::make_shared<CollectionStatistics>();
+    context->collection_similarity = std::make_shared<CollectionSimilarity>();
+
+    const std::string& root_clause_type = search_param.root.clause_type;
+    std::shared_ptr<query_v2::Query> root_query = nullptr;
+
+    // Check if root is compound node (AND/OR/NOT) or leaf node
+    if (root_clause_type == "AND" || root_clause_type == "OR" || 
root_clause_type == "NOT") {
+        // Compound node - build BooleanQuery
+        auto boolean_query = build_query_from_clause(search_param.root, 
context,
+                                                     data_type_with_names, 
iterators);
+        if (!boolean_query) {
+            LOG(WARNING) << "search: Failed to build BooleanQuery from 
compound clause";
+            return Status::InternalError("Failed to build BooleanQuery from 
compound DSL clause");
+        }
+        root_query = std::static_pointer_cast<query_v2::Query>(boolean_query);
+        LOG(INFO) << "search: Built BooleanQuery for compound clause type: " 
<< root_clause_type;
+    } else {
+        // Leaf node - build appropriate leaf query directly
+        root_query = build_leaf_query(search_param.root, context, 
data_type_with_names, iterators);
+        if (!root_query) {
+            LOG(WARNING) << "search: Failed to build leaf query for clause 
type: "
+                         << root_clause_type;
+            return Status::InternalError("Failed to build leaf query from DSL 
clause");
+        }
+        LOG(INFO) << "search: Built leaf query for clause type: " << 
root_clause_type;
+    }
+    // Analyze query type to determine appropriate InvertedIndexQueryType for 
reader selection
+    InvertedIndexQueryType query_type = analyze_query_type(search_param.root);
+    // Initialize result bitmaps
+    std::shared_ptr<roaring::Roaring> roaring = 
std::make_shared<roaring::Roaring>();
+    std::shared_ptr<roaring::Roaring> null_bitmap = 
std::make_shared<roaring::Roaring>();
+
+    try {
+        // Create CompositeReader to manage multiple index readers
+        auto composite_reader = std::make_unique<query_v2::CompositeReader>();
+
+        // Build individual IndexReaders for each field
+        std::vector<std::unique_ptr<lucene::index::IndexReader>> index_readers;
+        index_readers.reserve(data_type_with_names.size());
+
+        for (const auto& field_pair : data_type_with_names) {
+            const std::string& field_name = field_pair.first;
+            const std::string& real_field_name = field_pair.second.first;
+            const vectorized::DataTypePtr& column_type = 
field_pair.second.second;
+            std::wstring field_wstr = 
StringHelper::to_wstring(real_field_name);
+
+            // Use helper function to get inverted reader - eliminates ~25 
lines of duplicate code
+            auto inverted_reader =
+                    get_field_inverted_reader(field_name, iterators, 
column_type, query_type);
+            if (!inverted_reader) {
+                LOG(INFO) << "search: Could not get inverted reader for field: 
" << field_name;
+                continue;
+            }
+
+            // Directly access the index file and create directory
+            auto index_file_reader = inverted_reader->get_index_file_reader();
+            if (!index_file_reader) {
+                LOG(WARNING) << "search: No index file reader for field: " << 
field_name;
+                continue;
+            }
+
+            // Initialize index file reader
+            auto st = 
index_file_reader->init(config::inverted_index_read_buffer_size,
+                                              context->io_ctx);
+            if (!st.ok()) {
+                LOG(WARNING) << "search: Failed to init index file reader for 
field: " << field_name
+                             << ", error: " << st.to_string();
+                continue;
+            }
+
+            // Open directory directly
+            auto directory = DORIS_TRY(
+                    
index_file_reader->open(&inverted_reader->get_index_meta(), context->io_ctx));

Review Comment:
   The DORIS_TRY macro will return from the function on error, but the 
subsequent code assumes `directory` is always valid. This could lead to 
accessing a null pointer if the macro doesn't behave as expected.
   ```suggestion
                       
index_file_reader->open(&inverted_reader->get_index_meta(), context->io_ctx));
               if (!directory) {
                   LOG(WARNING) << "search: Failed to open directory for field: 
" << field_name;
                   continue;
               }
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/SearchDslParser.java:
##########
@@ -0,0 +1,346 @@
+// 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.trees.expressions.functions.scalar;
+
+import org.apache.doris.nereids.search.SearchLexer;
+import org.apache.doris.nereids.search.SearchParser;
+import org.apache.doris.nereids.search.SearchParserBaseVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Search DSL Parser using ANTLR-generated parser.
+ * Parses DSL syntax and extracts field bindings for FE->BE communication.
+ * <p>
+ * Supported syntax:
+ * - field:term
+ * - field:"quoted term"
+ * - field:prefix*
+ * - field:*wildcard*
+ * - field:/regexp/
+ * - AND/OR/NOT operators
+ * - Parentheses for grouping
+ * - Range queries: field:[1 TO 10], field:{1 TO 10}
+ * - List queries: field:IN(value1 value2)
+ * - Any/All queries: field:ANY(value), field:ALL(value)
+ */
+public class SearchDslParser {
+    private static final Logger LOG = 
LogManager.getLogger(SearchDslParser.class);
+    private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
+
+    /**
+     * Parse DSL string and return intermediate representation
+     */
+    public static QsPlan parseDsl(String dsl) {
+        if (dsl == null || dsl.trim().isEmpty()) {
+            return new QsPlan(new QsNode(QsClauseType.TERM, "error", 
"empty_dsl"), new ArrayList<>());
+        }
+
+        try {
+            // Create ANTLR lexer and parser
+            SearchLexer lexer = new SearchLexer(new ANTLRInputStream(dsl));
+            CommonTokenStream tokens = new CommonTokenStream(lexer);
+            SearchParser parser = new SearchParser(tokens);
+
+            // Parse the search query
+            ParseTree tree = parser.search();
+
+            // Build AST using visitor pattern
+            QsAstBuilder visitor = new QsAstBuilder();
+            QsNode root = visitor.visit(tree);
+
+            // Extract field bindings
+            Set<String> fieldNames = visitor.getFieldNames();
+            List<QsFieldBinding> bindings = new ArrayList<>();
+            int slotIndex = 0;
+            for (String fieldName : fieldNames) {
+                bindings.add(new QsFieldBinding(fieldName, slotIndex++));
+            }
+
+            return new QsPlan(root, bindings);
+
+        } catch (Exception e) {
+            LOG.error("Failed to parse search DSL: '{}'", dsl, e);
+            throw new RuntimeException("Invalid search DSL syntax: " + dsl + 
". Error: " + e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Clause types supported
+     */
+    public enum QsClauseType {
+        TERM,       // field:value
+        PHRASE,     // field:"phrase search"
+        PREFIX,     // field:prefix*
+        WILDCARD,   // field:*wild*card*
+        REGEXP,     // field:/pattern/
+        RANGE,      // field:[1 TO 10] or field:{1 TO 10}
+        LIST,       // field:IN(value1 value2)
+        ANY,        // field:ANY(value) - any match
+        ALL,        // field:ALL(value) - all match
+        AND,        // clause1 AND clause2
+        OR,         // clause1 OR clause2
+        NOT         // NOT clause
+    }
+
+    /**
+     * ANTLR visitor to build QsNode AST from parse tree
+     */
+    private static class QsAstBuilder extends SearchParserBaseVisitor<QsNode> {
+        private final Set<String> fieldNames = new HashSet<>();
+
+        public Set<String> getFieldNames() {
+            return fieldNames;
+        }
+
+        @Override
+        public QsNode visitSearch(SearchParser.SearchContext ctx) {
+            return visit(ctx.clause());
+        }
+
+        @Override
+        public QsNode visitOrClause(SearchParser.OrClauseContext ctx) {
+            if (ctx.andClause().size() == 1) {
+                return visit(ctx.andClause(0));
+            }
+
+            List<QsNode> children = new ArrayList<>();
+            for (SearchParser.AndClauseContext andCtx : ctx.andClause()) {
+                children.add(visit(andCtx));
+            }
+            return new QsNode(QsClauseType.OR, children);
+        }
+
+        @Override
+        public QsNode visitAndClause(SearchParser.AndClauseContext ctx) {
+            if (ctx.notClause().size() == 1) {
+                return visit(ctx.notClause(0));
+            }
+
+            List<QsNode> children = new ArrayList<>();
+            for (SearchParser.NotClauseContext notCtx : ctx.notClause()) {
+                children.add(visit(notCtx));
+            }
+            return new QsNode(QsClauseType.AND, children);
+        }
+
+        @Override
+        public QsNode visitNotClause(SearchParser.NotClauseContext ctx) {
+            if (ctx.NOT() != null) {
+                QsNode child = visit(ctx.atomClause());
+                List<QsNode> children = new ArrayList<>();
+                children.add(child);
+                return new QsNode(QsClauseType.NOT, children);
+            }
+            return visit(ctx.atomClause());
+        }
+
+        @Override
+        public QsNode visitAtomClause(SearchParser.AtomClauseContext ctx) {
+            if (ctx.clause() != null) {
+                // Parenthesized clause
+                return visit(ctx.clause());
+            }
+            return visit(ctx.fieldQuery());
+        }
+
+        @Override
+        public QsNode visitFieldQuery(SearchParser.FieldQueryContext ctx) {
+            String fieldName = ctx.fieldName().getText();
+            if (fieldName.startsWith("\"") && fieldName.endsWith("\"")) {
+                fieldName = fieldName.substring(1, fieldName.length() - 1);
+            }
+            fieldNames.add(fieldName);
+
+            return visit(ctx.searchValue());
+        }
+
+        @Override
+        public QsNode visitSearchValue(SearchParser.SearchValueContext ctx) {
+            String fieldName = getCurrentFieldName();
+            if (ctx.TERM() != null) {
+                return new QsNode(QsClauseType.TERM, fieldName, 
ctx.TERM().getText());
+            } else
+                if (ctx.PREFIX() != null) {
+                    return new QsNode(QsClauseType.PREFIX, fieldName, 
ctx.PREFIX().getText());
+                } else
+                    if (ctx.WILDCARD() != null) {
+                        return new QsNode(QsClauseType.WILDCARD, fieldName, 
ctx.WILDCARD().getText());
+                    } else
+                        if (ctx.REGEXP() != null) {
+                            String regexp = ctx.REGEXP().getText();
+                            // Remove surrounding slashes
+                            if (regexp.startsWith("/") && 
regexp.endsWith("/")) {
+                                regexp = regexp.substring(1, regexp.length() - 
1);
+                            }
+                            return new QsNode(QsClauseType.REGEXP, fieldName, 
regexp);
+                        } else
+                            if (ctx.QUOTED() != null) {
+                                String quoted = ctx.QUOTED().getText();
+                                // Remove surrounding quotes
+                                if (quoted.startsWith("\"") && 
quoted.endsWith("\"")) {
+                                    quoted = quoted.substring(1, 
quoted.length() - 1);
+                                }
+                                return new QsNode(QsClauseType.PHRASE, 
fieldName, quoted);
+                            } else
+                                if (ctx.rangeValue() != null) {
+                                    return new QsNode(QsClauseType.RANGE, 
fieldName, ctx.rangeValue().getText());
+                                } else
+                                    if (ctx.listValue() != null) {
+                                        return new QsNode(QsClauseType.LIST, 
fieldName, ctx.listValue().getText());
+                                    } else
+                                        if (ctx.anyAllValue() != null) {
+                                            String anyAllText = 
ctx.anyAllValue().getText();
+                                            String innerContent = "";
+
+                                            // Extract content between 
parentheses
+                                            int openParen = 
anyAllText.indexOf('(');
+                                            int closeParen = 
anyAllText.lastIndexOf(')');
+                                            if (openParen >= 0 && closeParen > 
openParen) {
+                                                innerContent = 
anyAllText.substring(openParen + 1, closeParen).trim();
+                                            }
+
+                                            if 
(anyAllText.toUpperCase().startsWith("ANY(") || anyAllText.toLowerCase()
+                                                    .startsWith("any(")) {
+                                                return new 
QsNode(QsClauseType.ANY, fieldName, innerContent);
+                                            } else
+                                                if 
(anyAllText.toUpperCase().startsWith("ALL(")
+                                                        || 
anyAllText.toLowerCase().startsWith("all(")) {
+                                                    return new 
QsNode(QsClauseType.ALL, fieldName, innerContent);
+                                                } else {
+                                                    // Fallback to ANY for 
unknown cases
+                                                    return new 
QsNode(QsClauseType.ANY, fieldName, innerContent);
+                                                }
+                                        }
+
+            return new QsNode(QsClauseType.TERM, fieldName, ctx.getText());
+        }
+
+        private String getCurrentFieldName() {
+            // This is a simplified approach - in a real implementation,
+            // we'd need to track context properly
+            return fieldNames.isEmpty() ? "_all" : 
fieldNames.iterator().next();
+        }

Review Comment:
   The getCurrentFieldName method returns arbitrary field name from a Set which 
has no guaranteed ordering. This could lead to inconsistent field name 
resolution. Consider maintaining a proper context stack or field name 
resolution mechanism.



-- 
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