You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/25 14:38:01 UTC

[doris] 04/05: [Fix](inverted index) fix inverted query cache for chinese tokenizer (#21106)

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

kxiao pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c1c068982107852c3be5431c41fd5993a09a5e45
Author: airborne12 <ai...@gmail.com>
AuthorDate: Sun Jun 25 22:04:02 2023 +0800

    [Fix](inverted index) fix inverted query cache for chinese tokenizer (#21106)
    
    1. query cache for chinese tokenizer is confusing when just converting w_char to char.
    2. seperate query_type from inverted_index_reader to clean code.
---
 .../rowset/segment_v2/inverted_index_cache.cpp     |  6 ++
 .../olap/rowset/segment_v2/inverted_index_cache.h  | 12 +++-
 .../rowset/segment_v2/inverted_index_query_type.h  | 71 ++++++++++++++++++++++
 .../rowset/segment_v2/inverted_index_reader.cpp    |  1 +
 .../olap/rowset/segment_v2/inverted_index_reader.h | 13 +---
 .../inverted_index_p0/test_chinese_analyzer.out    | 38 ++++++++++++
 .../inverted_index_p0/test_chinese_analyzer.groovy | 33 ++++++++++
 7 files changed, 159 insertions(+), 15 deletions(-)

diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp
index 7db99e26e9..243a738da9 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.cpp
@@ -238,6 +238,9 @@ Cache::Handle* InvertedIndexSearcherCache::_insert(const InvertedIndexSearcherCa
 InvertedIndexQueryCache* InvertedIndexQueryCache::_s_instance = nullptr;
 
 bool InvertedIndexQueryCache::lookup(const CacheKey& key, InvertedIndexQueryCacheHandle* handle) {
+    if (key.encode().empty()) {
+        return false;
+    }
     auto lru_handle = _cache->lookup(key.encode());
     if (lru_handle == nullptr) {
         return false;
@@ -257,6 +260,9 @@ void InvertedIndexQueryCache::insert(const CacheKey& key, std::shared_ptr<roarin
     cache_value_ptr->last_visit_time = UnixMillis();
     cache_value_ptr->bitmap = bitmap;
     cache_value_ptr->size = bitmap->getSizeInBytes();
+    if (key.encode().empty()) {
+        return;
+    }
 
     auto lru_handle = _cache->insert(key.encode(), (void*)cache_value_ptr.release(),
                                      bitmap->getSizeInBytes(), deleter, CachePriority::NORMAL);
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_cache.h b/be/src/olap/rowset/segment_v2/inverted_index_cache.h
index 7f9b7e8c14..0ab2b18af4 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_cache.h
+++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.h
@@ -18,6 +18,7 @@
 #pragma once
 
 #include <CLucene.h> // IWYU pragma: keep
+#include <CLucene/config/repl_wchar.h>
 #include <CLucene/util/Misc.h>
 #include <butil/macros.h>
 #include <glog/logging.h>
@@ -35,6 +36,7 @@
 #include "io/fs/file_system.h"
 #include "io/fs/path.h"
 #include "olap/lru_cache.h"
+#include "olap/rowset/segment_v2/inverted_index_query_type.h"
 #include "runtime/memory/mem_tracker.h"
 #include "util/slice.h"
 #include "util/time.h"
@@ -183,7 +185,6 @@ private:
     DISALLOW_COPY_AND_ASSIGN(InvertedIndexCacheHandle);
 };
 
-enum class InvertedIndexQueryType;
 class InvertedIndexQueryCacheHandle;
 
 class InvertedIndexQueryCache {
@@ -201,9 +202,14 @@ public:
             key_buf.append("/");
             key_buf.append(column_name);
             key_buf.append("/");
-            key_buf.append(1, static_cast<char>(query_type));
+            auto query_type_str = InvertedIndexQueryType_toString(query_type);
+            if (query_type_str.empty()) {
+                return "";
+            }
+            key_buf.append(query_type_str);
             key_buf.append("/");
-            key_buf.append(lucene::util::Misc::toString(value.c_str()));
+            auto str = lucene_wcstoutf8string(value.c_str(), value.length());
+            key_buf.append(str);
             return key_buf;
         }
     };
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_query_type.h b/be/src/olap/rowset/segment_v2/inverted_index_query_type.h
new file mode 100644
index 0000000000..1ebfe63591
--- /dev/null
+++ b/be/src/olap/rowset/segment_v2/inverted_index_query_type.h
@@ -0,0 +1,71 @@
+// 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.
+
+#pragma once
+
+#include <string>
+
+namespace doris {
+namespace segment_v2 {
+
+enum class InvertedIndexQueryType {
+    UNKNOWN_QUERY = -1,
+    EQUAL_QUERY = 0,
+    LESS_THAN_QUERY = 1,
+    LESS_EQUAL_QUERY = 2,
+    GREATER_THAN_QUERY = 3,
+    GREATER_EQUAL_QUERY = 4,
+    MATCH_ANY_QUERY = 5,
+    MATCH_ALL_QUERY = 6,
+    MATCH_PHRASE_QUERY = 7,
+};
+
+inline std::string InvertedIndexQueryType_toString(InvertedIndexQueryType query_type) {
+    switch (query_type) {
+    case InvertedIndexQueryType::UNKNOWN_QUERY: {
+        return "UNKNOWN";
+    }
+    case InvertedIndexQueryType::EQUAL_QUERY: {
+        return "EQ";
+    }
+    case InvertedIndexQueryType::LESS_THAN_QUERY: {
+        return "LT";
+    }
+    case InvertedIndexQueryType::LESS_EQUAL_QUERY: {
+        return "LE";
+    }
+    case InvertedIndexQueryType::GREATER_THAN_QUERY: {
+        return "GT";
+    }
+    case InvertedIndexQueryType::GREATER_EQUAL_QUERY: {
+        return "GE";
+    }
+    case InvertedIndexQueryType::MATCH_ANY_QUERY: {
+        return "MANY";
+    }
+    case InvertedIndexQueryType::MATCH_ALL_QUERY: {
+        return "MALL";
+    }
+    case InvertedIndexQueryType::MATCH_PHRASE_QUERY: {
+        return "MPHRASE";
+    }
+    default:
+        return "";
+    }
+}
+} // namespace segment_v2
+} // namespace doris
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
index c311bcec8f..fae74fc883 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
@@ -344,6 +344,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string
                 InvertedIndexQueryCache::CacheKey cache_key {index_file_path, column_name,
                                                              InvertedIndexQueryType::EQUAL_QUERY,
                                                              token_ws};
+                VLOG_DEBUG << "cache_key:" << cache_key.encode();
                 InvertedIndexQueryCacheHandle cache_handle;
                 if (cache->lookup(cache_key, &cache_handle)) {
                     stats->inverted_index_query_cache_hit++;
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_reader.h b/be/src/olap/rowset/segment_v2/inverted_index_reader.h
index ac7280ed54..1b30286a15 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_reader.h
+++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.h
@@ -30,6 +30,7 @@
 #include "io/fs/path.h"
 #include "olap/inverted_index_parser.h"
 #include "olap/rowset/segment_v2/inverted_index_compound_reader.h"
+#include "olap/rowset/segment_v2/inverted_index_query_type.h"
 #include "olap/tablet_schema.h"
 
 namespace lucene {
@@ -63,18 +64,6 @@ enum class InvertedIndexReaderType {
     BKD = 2,
 };
 
-enum class InvertedIndexQueryType {
-    UNKNOWN_QUERY = -1,
-    EQUAL_QUERY = 0,
-    LESS_THAN_QUERY = 1,
-    LESS_EQUAL_QUERY = 2,
-    GREATER_THAN_QUERY = 3,
-    GREATER_EQUAL_QUERY = 4,
-    MATCH_ANY_QUERY = 5,
-    MATCH_ALL_QUERY = 6,
-    MATCH_PHRASE_QUERY = 7,
-};
-
 class InvertedIndexReader {
 public:
     explicit InvertedIndexReader(io::FileSystemSPtr fs, const std::string& path,
diff --git a/regression-test/data/inverted_index_p0/test_chinese_analyzer.out b/regression-test/data/inverted_index_p0/test_chinese_analyzer.out
index dfb1cd2ccb..91045120ab 100644
--- a/regression-test/data/inverted_index_p0/test_chinese_analyzer.out
+++ b/regression-test/data/inverted_index_p0/test_chinese_analyzer.out
@@ -27,3 +27,41 @@
 -- !sql --
 3	人民可以得到更多实惠
 
+-- !sql --
+2	我爱你中国
+
+-- !sql --
+2	我爱你中国
+
+-- !sql --
+1	我来到北京清华大学
+
+-- !sql --
+1	我来到北京清华大学
+
+-- !sql --
+1	我来到北京清华大学
+
+-- !sql --
+3	人民可以得到更多实惠
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
+-- !sql --
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
+-- !sql --
+4	陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.
+
diff --git a/regression-test/suites/inverted_index_p0/test_chinese_analyzer.groovy b/regression-test/suites/inverted_index_p0/test_chinese_analyzer.groovy
index 1acf8ffa6a..dd375f3894 100644
--- a/regression-test/suites/inverted_index_p0/test_chinese_analyzer.groovy
+++ b/regression-test/suites/inverted_index_p0/test_chinese_analyzer.groovy
@@ -77,4 +77,37 @@ suite("test_chinese_analyzer"){
     qt_sql "SELECT * FROM $indexTblName2 WHERE c MATCH '大学' ORDER BY id;"
     qt_sql "SELECT * FROM $indexTblName2 WHERE c MATCH '清华大学' ORDER BY id;"
     qt_sql "SELECT * FROM $indexTblName2 WHERE c MATCH '人民' ORDER BY id;"
+
+    def indexTblName3 = "chinese_analyzer_test3"
+
+    sql "DROP TABLE IF EXISTS ${indexTblName3}"
+    // create 1 replica table
+    sql """
+	CREATE TABLE IF NOT EXISTS ${indexTblName3}(
+		`id`int(11)NULL,
+		`c` text NULL,
+		INDEX c_idx(`c`) USING INVERTED PROPERTIES("parser"="unicode", "support_phrase"="true") COMMENT ''
+	) ENGINE=OLAP
+	DUPLICATE KEY(`id`)
+	COMMENT 'OLAP'
+	DISTRIBUTED BY HASH(`id`) BUCKETS 1
+	PROPERTIES(
+                "replication_allocation" = "tag.location.default: 1"
+        );
+    """
+
+    sql "INSERT INTO $indexTblName3 VALUES (1, '我来到北京清华大学'), (2, '我爱你中国'), (3, '人民可以得到更多实惠'), (4, '陕西省西安市高新区创业大厦A座,我的手机号码是12345678901,邮箱是12345678@qq.com,,ip是1.1.1.1,this information is created automatically.');"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_PHRASE '我爱你' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL'我爱你' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '清华' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '大学' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '清华大学' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '人民' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '陕西' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '12345678901' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_ALL '12345678' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_PHRASE '1.1.1.1' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_PHRASE '陕西西安' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH_PHRASE '陕西省西安市' ORDER BY id;"
+    qt_sql "SELECT * FROM $indexTblName3 WHERE c MATCH 'information' ORDER BY id;"
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org