You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by da...@apache.org on 2022/05/27 12:44:31 UTC

[incubator-doris] branch master updated: [fix] Fix bug of bloom filter hash value calculation error (#9802)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new af2cfa2db4 [fix] Fix bug of bloom filter hash value calculation error (#9802)
af2cfa2db4 is described below

commit af2cfa2db48034170dec4972f3ef0ceb1ade498b
Author: Luwei <81...@qq.com>
AuthorDate: Fri May 27 20:44:26 2022 +0800

    [fix] Fix bug of bloom filter hash value calculation error (#9802)
    
    * Fix bug of bloom filter hash value calculation error
    
    * fix code style
---
 be/src/olap/bloom_filter_predicate.h   |  5 +++--
 be/src/olap/schema.cpp                 |  2 +-
 be/src/vec/columns/column_dictionary.h | 24 +++++++++++++++++++++---
 3 files changed, 25 insertions(+), 6 deletions(-)

diff --git a/be/src/olap/bloom_filter_predicate.h b/be/src/olap/bloom_filter_predicate.h
index a28d31249c..d9bf34d2ce 100644
--- a/be/src/olap/bloom_filter_predicate.h
+++ b/be/src/olap/bloom_filter_predicate.h
@@ -120,7 +120,8 @@ void BloomFilterColumnPredicate<T>::evaluate(vectorized::IColumn& column, uint16
         if (nullable_col->get_nested_column().is_column_dictionary()) {
             auto* dict_col = vectorized::check_and_get_column<vectorized::ColumnDictI32>(
                     nullable_col->get_nested_column());
-            const_cast<vectorized::ColumnDictI32*>(dict_col)->generate_hash_values();
+            const_cast<vectorized::ColumnDictI32*>(dict_col)
+                    ->generate_hash_values_for_runtime_filter();
             for (uint16_t i = 0; i < *size; i++) {
                 uint16_t idx = sel[i];
                 sel[new_size] = idx;
@@ -140,7 +141,7 @@ void BloomFilterColumnPredicate<T>::evaluate(vectorized::IColumn& column, uint16
         }
     } else if (column.is_column_dictionary()) {
         auto* dict_col = vectorized::check_and_get_column<vectorized::ColumnDictI32>(column);
-        const_cast<vectorized::ColumnDictI32*>(dict_col)->generate_hash_values();
+        const_cast<vectorized::ColumnDictI32*>(dict_col)->generate_hash_values_for_runtime_filter();
         for (uint16_t i = 0; i < *size; i++) {
             uint16_t idx = sel[i];
             sel[new_size] = idx;
diff --git a/be/src/olap/schema.cpp b/be/src/olap/schema.cpp
index 305cdb0ad7..8c218ab8ed 100644
--- a/be/src/olap/schema.cpp
+++ b/be/src/olap/schema.cpp
@@ -160,7 +160,7 @@ vectorized::IColumn::MutablePtr Schema::get_predicate_column_ptr(FieldType type)
     case OLAP_FIELD_TYPE_VARCHAR:
     case OLAP_FIELD_TYPE_STRING:
         if (config::enable_low_cardinality_optimize) {
-            return doris::vectorized::ColumnDictionary<doris::vectorized::Int32>::create();
+            return doris::vectorized::ColumnDictionary<doris::vectorized::Int32>::create(type);
         }
         return doris::vectorized::PredicateColumnType<StringValue>::create();
 
diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h
index f3d07a6ad3..a45171f204 100644
--- a/be/src/vec/columns/column_dictionary.h
+++ b/be/src/vec/columns/column_dictionary.h
@@ -62,6 +62,7 @@ private:
     ColumnDictionary() {}
     ColumnDictionary(const size_t n) : _codes(n) {}
     ColumnDictionary(const ColumnDictionary& src) : _codes(src._codes.begin(), src._codes.end()) {}
+    ColumnDictionary(FieldType type) : _type(type) {}
 
 public:
     using Self = ColumnDictionary;
@@ -251,7 +252,9 @@ public:
         return _dict.find_code_by_bound(value, greater, eq);
     }
 
-    void generate_hash_values() { _dict.generate_hash_values(); }
+    void generate_hash_values_for_runtime_filter() {
+        _dict.generate_hash_values_for_runtime_filter(_type);
+    }
 
     uint32_t get_hash_value(uint32_t idx) const { return _dict.get_hash_value(_codes[idx]); }
 
@@ -305,12 +308,26 @@ public:
             return code >= _dict_data.size() ? _null_value : _dict_data[code];
         }
 
-        inline void generate_hash_values() {
+        // The function is only used in the runtime filter feature
+        inline void generate_hash_values_for_runtime_filter(FieldType type) {
             if (_hash_values.empty()) {
                 _hash_values.resize(_dict_data.size());
                 for (size_t i = 0; i < _dict_data.size(); i++) {
                     auto& sv = _dict_data[i];
-                    uint32_t hash_val = HashUtil::murmur_hash3_32(sv.ptr, sv.len, 0);
+                    // The char data is stored in the disk with the schema length,
+                    // and zeros are filled if the length is insufficient
+
+                    // When reading data, use shrink_char_type_column_suffix_zero(_char_type_idx)
+                    // Remove the suffix 0
+                    // When writing data, use the CharField::consume function to fill in the trailing 0.
+
+                    // For dictionary data of char type, sv.len is the schema length,
+                    // so use strnlen to remove the 0 at the end to get the actual length.
+                    int32_t len = sv.len;
+                    if (type == OLAP_FIELD_TYPE_CHAR) {
+                        len = strnlen(sv.ptr, sv.len);
+                    }
+                    uint32_t hash_val = HashUtil::murmur_hash3_32(sv.ptr, len, 0);
                     _hash_values[i] = hash_val;
                 }
             }
@@ -404,6 +421,7 @@ private:
     bool _dict_code_converted = false;
     Dictionary _dict;
     Container _codes;
+    FieldType _type;
 };
 
 template class ColumnDictionary<int32_t>;


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