You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ji...@apache.org on 2023/04/15 05:20:33 UTC

[doris] branch master updated: [bugfix](inverted index) fix inverted index to support NULL value filter (#18302)

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

jianliangqi 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 98b8bef05b [bugfix](inverted index) fix inverted index to support NULL value filter (#18302)
98b8bef05b is described below

commit 98b8bef05b17ef09ef18d8f6dea49a215b9240a0
Author: Kang <kx...@gmail.com>
AuthorDate: Sat Apr 15 13:20:26 2023 +0800

    [bugfix](inverted index) fix inverted index to support NULL value filter (#18302)
---
 be/src/olap/comparison_predicate.h                 |  10 ++
 be/src/olap/in_list_predicate.h                    |  11 ++
 be/src/olap/match_predicate.cpp                    |  11 ++
 be/src/olap/null_predicate.cpp                     |  23 +++++
 be/src/olap/null_predicate.h                       |   3 +
 .../olap/rowset/segment_v2/inverted_index_cache.h  |   5 +-
 .../olap/rowset/segment_v2/inverted_index_desc.h   |   1 +
 .../rowset/segment_v2/inverted_index_reader.cpp    |  86 ++++++++++++++-
 .../olap/rowset/segment_v2/inverted_index_reader.h |   9 ++
 .../rowset/segment_v2/inverted_index_writer.cpp    |  26 ++++-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   3 +-
 .../inverted_index_p0/test_inverted_index_null.out | 115 +++++++++++++++++++++
 .../test_inverted_index_null.groovy                |  89 ++++++++++++++++
 13 files changed, 383 insertions(+), 9 deletions(-)

diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index e2b85f38b5..0371b27282 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -108,6 +108,16 @@ public:
         RETURN_IF_ERROR(iterator->read_from_inverted_index(column_name, &_value, query_type,
                                                            num_rows, &roaring));
 
+        // mask out null_bitmap, since NULL cmp VALUE will produce NULL
+        //  and be treated as false in WHERE
+        // keep it after query, since query will try to read null_bitmap and put it to cache
+        InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+        RETURN_IF_ERROR(iterator->read_null_bitmap(&null_bitmap_cache_handle));
+        roaring::Roaring* null_bitmap = null_bitmap_cache_handle.get_bitmap();
+        if (null_bitmap) {
+            *bitmap -= *null_bitmap;
+        }
+
         if constexpr (PT == PredicateType::NE) {
             *bitmap -= roaring;
         } else {
diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h
index 825bdb8c02..7d0001cf1c 100644
--- a/be/src/olap/in_list_predicate.h
+++ b/be/src/olap/in_list_predicate.h
@@ -244,6 +244,17 @@ public:
             indices |= index;
             iter->next();
         }
+
+        // mask out null_bitmap, since NULL cmp VALUE will produce NULL
+        //  and be treated as false in WHERE
+        // keep it after query, since query will try to read null_bitmap and put it to cache
+        InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+        RETURN_IF_ERROR(iterator->read_null_bitmap(&null_bitmap_cache_handle));
+        roaring::Roaring* null_bitmap = null_bitmap_cache_handle.get_bitmap();
+        if (null_bitmap) {
+            *result -= *null_bitmap;
+        }
+
         if constexpr (PT == PredicateType::IN_LIST) {
             *result &= indices;
         } else {
diff --git a/be/src/olap/match_predicate.cpp b/be/src/olap/match_predicate.cpp
index e9b68e3b3e..22d56cd1ff 100644
--- a/be/src/olap/match_predicate.cpp
+++ b/be/src/olap/match_predicate.cpp
@@ -62,6 +62,17 @@ Status MatchPredicate::evaluate(const Schema& schema, InvertedIndexIterator* ite
         s = iterator->read_from_inverted_index(column_desc->name(), buf, inverted_index_query_type,
                                                num_rows, &roaring, true);
     }
+
+    // mask out null_bitmap, since NULL cmp VALUE will produce NULL
+    //  and be treated as false in WHERE
+    // keep it after query, since query will try to read null_bitmap and put it to cache
+    InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+    RETURN_IF_ERROR(iterator->read_null_bitmap(&null_bitmap_cache_handle));
+    roaring::Roaring* null_bitmap = null_bitmap_cache_handle.get_bitmap();
+    if (null_bitmap) {
+        *bitmap -= *null_bitmap;
+    }
+
     *bitmap &= roaring;
     return s;
 }
diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp
index fe081d9be4..e9a9bf9705 100644
--- a/be/src/olap/null_predicate.cpp
+++ b/be/src/olap/null_predicate.cpp
@@ -46,6 +46,29 @@ Status NullPredicate::evaluate(BitmapIndexIterator* iterator, uint32_t num_rows,
     return Status::OK();
 }
 
+Status NullPredicate::evaluate(const Schema& schema, InvertedIndexIterator* iterator,
+                               uint32_t num_rows, roaring::Roaring* bitmap) const {
+    // mask out null_bitmap, since NULL cmp VALUE will produce NULL
+    //  and be treated as false in WHERE
+    InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+    RETURN_IF_ERROR(iterator->read_null_bitmap(&null_bitmap_cache_handle));
+    roaring::Roaring* null_bitmap = null_bitmap_cache_handle.get_bitmap();
+    if (null_bitmap) {
+        if (_is_null) {
+            *bitmap &= *null_bitmap;
+        } else {
+            *bitmap -= *null_bitmap;
+        }
+    } else {
+        // all rows not null
+        if (_is_null) {
+            *bitmap -= *bitmap;
+        }
+    }
+
+    return Status::OK();
+}
+
 uint16_t NullPredicate::evaluate(const vectorized::IColumn& column, uint16_t* sel,
                                  uint16_t size) const {
     uint16_t new_size = 0;
diff --git a/be/src/olap/null_predicate.h b/be/src/olap/null_predicate.h
index 7424548054..e878907228 100644
--- a/be/src/olap/null_predicate.h
+++ b/be/src/olap/null_predicate.h
@@ -36,6 +36,9 @@ public:
     Status evaluate(BitmapIndexIterator* iterator, uint32_t num_rows,
                     roaring::Roaring* roaring) const override;
 
+    Status evaluate(const Schema& schema, InvertedIndexIterator* iterator, uint32_t num_rows,
+                    roaring::Roaring* bitmap) const override;
+
     uint16_t evaluate(const vectorized::IColumn& column, uint16_t* sel,
                       uint16_t size) const override;
 
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 cf88c17b02..77d428aa42 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_cache.h
+++ b/be/src/olap/rowset/segment_v2/inverted_index_cache.h
@@ -249,7 +249,10 @@ public:
     Cache* cache() const { return _cache; }
     Slice data() const { return _cache->value_slice(_handle); }
 
-    InvertedIndexQueryCache::CacheValue* match_bitmap() const {
+    InvertedIndexQueryCache::CacheValue* get_bitmap() const {
+        if (!_cache) {
+            return nullptr;
+        }
         return ((InvertedIndexQueryCache::CacheValue*)_cache->value(_handle));
     }
 
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_desc.h b/be/src/olap/rowset/segment_v2/inverted_index_desc.h
index f3796668d4..0da5dfddbb 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_desc.h
+++ b/be/src/olap/rowset/segment_v2/inverted_index_desc.h
@@ -29,6 +29,7 @@ class InvertedIndexDescriptor {
 public:
     static std::string get_temporary_index_path(const std::string& segment_path, uint32_t uuid);
     static std::string get_index_file_name(const std::string& path, uint32_t uuid);
+    static const std::string get_temporary_null_bitmap_file_name() { return "null_bitmap"; }
     static const std::string get_temporary_bkd_index_data_file_name() { return "bkd"; }
     static const std::string get_temporary_bkd_index_meta_file_name() { return "bkd_meta"; }
     static const std::string get_temporary_bkd_index_file_name() { return "bkd_index"; }
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 47207a917e..9a7520dc57 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_reader.cpp
@@ -32,9 +32,21 @@
 #include "olap/rowset/segment_v2/inverted_index_desc.h"
 #include "olap/tablet_schema.h"
 #include "olap/utils.h"
+#include "util/faststring.h"
 #include "util/time.h"
 #include "vec/common/string_ref.h"
 
+#define FINALIZE_INPUT(x) \
+    if (x != nullptr) {   \
+        x->close();       \
+        _CLDELETE(x);     \
+    }
+#define FINALLY_FINALIZE_INPUT(x) \
+    try {                         \
+        FINALIZE_INPUT(x)         \
+    } catch (...) {               \
+    }
+
 namespace doris {
 namespace segment_v2 {
 
@@ -50,6 +62,61 @@ bool InvertedIndexReader::indexExists(io::Path& index_file_path) {
     return exists;
 }
 
+Status InvertedIndexReader::read_null_bitmap(InvertedIndexQueryCacheHandle* cache_handle,
+                                             lucene::store::Directory* dir) {
+    lucene::store::IndexInput* null_bitmap_in = nullptr;
+    bool owned_dir = false;
+    try {
+        // try to get query bitmap result from cache and return immediately on cache hit
+        io::Path path(_path);
+        auto index_dir = path.parent_path();
+        auto index_file_name =
+                InvertedIndexDescriptor::get_index_file_name(path.filename(), _index_id);
+        auto index_file_path = index_dir / index_file_name;
+        InvertedIndexQueryCache::CacheKey cache_key {
+                index_file_path, "", InvertedIndexQueryType::UNKNOWN_QUERY, L"null_bitmap"};
+        auto cache = InvertedIndexQueryCache::instance();
+        if (cache->lookup(cache_key, cache_handle)) {
+            return Status::OK();
+        }
+
+        if (!dir) {
+            dir = new DorisCompoundReader(
+                    DorisCompoundDirectory::getDirectory(_fs, index_dir.c_str()),
+                    index_file_name.c_str(), config::inverted_index_read_buffer_size);
+            owned_dir = true;
+        }
+
+        // ownership of null_bitmap and its deletion will be transfered to cache
+        roaring::Roaring* null_bitmap = new roaring::Roaring();
+        auto null_bitmap_file_name = InvertedIndexDescriptor::get_temporary_null_bitmap_file_name();
+        if (dir->fileExists(null_bitmap_file_name.c_str())) {
+            null_bitmap_in = dir->openInput(null_bitmap_file_name.c_str());
+            size_t null_bitmap_size = null_bitmap_in->length();
+            faststring buf;
+            buf.resize(null_bitmap_size);
+            null_bitmap_in->readBytes(reinterpret_cast<uint8_t*>(buf.data()), null_bitmap_size);
+            *null_bitmap = roaring::Roaring::read(reinterpret_cast<char*>(buf.data()), false);
+            null_bitmap->runOptimize();
+            cache->insert(cache_key, null_bitmap, cache_handle);
+            FINALIZE_INPUT(null_bitmap_in);
+        }
+        if (owned_dir) {
+            FINALIZE_INPUT(dir);
+        }
+    } catch (CLuceneError& e) {
+        FINALLY_FINALIZE_INPUT(null_bitmap_in);
+        if (owned_dir) {
+            FINALLY_FINALIZE_INPUT(dir);
+        }
+        LOG(WARNING) << "Inverted index read null bitmap error occurred: " << e.what();
+        return Status::Error<doris::ErrorCode::INVERTED_INDEX_CLUCENE_ERROR>(
+                "Inverted index read null bitmap error occurred");
+    }
+
+    return Status::OK();
+}
+
 std::vector<std::wstring> FullTextIndexReader::get_analyse_result(
         const std::wstring& field_name, const std::string& value, InvertedIndexQueryType query_type,
         InvertedIndexParserType analyser_type) {
@@ -135,6 +202,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string
 
         roaring::Roaring query_match_bitmap;
         bool first = true;
+        bool null_bitmap_already_read = false;
         for (auto token_ws : analyse_result) {
             roaring::Roaring* term_match_bitmap = nullptr;
 
@@ -146,7 +214,7 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string
             InvertedIndexQueryCacheHandle cache_handle;
             if (cache->lookup(cache_key, &cache_handle)) {
                 stats->inverted_index_query_cache_hit++;
-                term_match_bitmap = cache_handle.match_bitmap();
+                term_match_bitmap = cache_handle.get_bitmap();
             } else {
                 stats->inverted_index_query_cache_miss++;
                 term_match_bitmap = new roaring::Roaring();
@@ -162,6 +230,15 @@ Status FullTextIndexReader::query(OlapReaderStatistics* stats, const std::string
                         stats);
                 auto index_searcher = inverted_index_cache_handle.get_index_searcher();
 
+                // try to reuse index_searcher's directory to read null_bitmap to cache
+                // to avoid open directory additionally for null_bitmap
+                if (!null_bitmap_already_read) {
+                    InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+                    read_null_bitmap(&null_bitmap_cache_handle,
+                                     index_searcher->getReader()->directory());
+                    null_bitmap_already_read = true;
+                }
+
                 try {
                     SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer);
                     index_searcher->_search(
@@ -266,7 +343,7 @@ Status StringTypeInvertedIndexReader::query(OlapReaderStatistics* stats,
     if (cache->lookup(cache_key, &cache_handle)) {
         stats->inverted_index_query_cache_hit++;
         SCOPED_RAW_TIMER(&stats->inverted_index_query_bitmap_copy_timer);
-        *bit_map = *cache_handle.match_bitmap();
+        *bit_map = *cache_handle.get_bitmap();
         return Status::OK();
     } else {
         stats->inverted_index_query_cache_miss++;
@@ -312,6 +389,11 @@ Status StringTypeInvertedIndexReader::query(OlapReaderStatistics* stats,
             _fs, index_dir.c_str(), index_file_name, &inverted_index_cache_handle, stats);
     auto index_searcher = inverted_index_cache_handle.get_index_searcher();
 
+    // try to reuse index_searcher's directory to read null_bitmap to cache
+    // to avoid open directory additionally for null_bitmap
+    InvertedIndexQueryCacheHandle null_bitmap_cache_handle;
+    read_null_bitmap(&null_bitmap_cache_handle, index_searcher->getReader()->directory());
+
     try {
         SCOPED_RAW_TIMER(&stats->inverted_index_searcher_search_timer);
         index_searcher->_search(query.get(),
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 81fc38aeb1..731b6ca6b2 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 "olap/inverted_index_parser.h"
 #include "olap/olap_common.h"
 #include "olap/rowset/segment_v2/common.h"
+#include "olap/rowset/segment_v2/inverted_index_cache.h"
 #include "olap/rowset/segment_v2/inverted_index_compound_reader.h"
 #include "olap/tablet_schema.h"
 
@@ -77,6 +78,9 @@ public:
                              const void* query_value, InvertedIndexQueryType query_type,
                              InvertedIndexParserType analyser_type, uint32_t* count) = 0;
 
+    Status read_null_bitmap(InvertedIndexQueryCacheHandle* cache_handle,
+                            lucene::store::Directory* dir = nullptr);
+
     virtual InvertedIndexReaderType type() = 0;
     bool indexExists(io::Path& index_file_path);
 
@@ -219,6 +223,11 @@ public:
     Status try_read_from_inverted_index(const std::string& column_name, const void* query_value,
                                         InvertedIndexQueryType query_type, uint32_t* count);
 
+    Status read_null_bitmap(InvertedIndexQueryCacheHandle* cache_handle,
+                            lucene::store::Directory* dir = nullptr) {
+        return _reader->read_null_bitmap(cache_handle, dir);
+    }
+
     InvertedIndexParserType get_inverted_index_analyser_type() const;
 
     InvertedIndexReaderType get_inverted_index_reader_type() const;
diff --git a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp
index 77a95bb505..ae95408648 100644
--- a/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/inverted_index_writer.cpp
@@ -30,6 +30,7 @@
 #include "olap/rowset/segment_v2/inverted_index_compound_directory.h"
 #include "olap/rowset/segment_v2/inverted_index_desc.h"
 #include "olap/tablet_schema.h"
+#include "util/faststring.h"
 #include "util/string_util.h"
 
 #define FINALIZE_OUTPUT(x) \
@@ -183,6 +184,7 @@ public:
     }
 
     Status add_nulls(uint32_t count) override {
+        _null_bitmap.addRange(_rid, _rid + count);
         _rid += count;
         if constexpr (field_is_slice_type(field_type)) {
             if (_field == nullptr || _index_writer == nullptr) {
@@ -321,15 +323,30 @@ public:
     }
 
     Status finish() override {
-        lucene::store::Directory* dir = nullptr;
+        auto index_path = InvertedIndexDescriptor::get_temporary_index_path(
+                _directory + "/" + _segment_file_name, _index_meta->index_id());
+        lucene::store::Directory* dir =
+                DorisCompoundDirectory::getDirectory(_fs, index_path.c_str(), true);
+        lucene::store::IndexOutput* null_bitmap_out = nullptr;
         lucene::store::IndexOutput* data_out = nullptr;
         lucene::store::IndexOutput* index_out = nullptr;
         lucene::store::IndexOutput* meta_out = nullptr;
         try {
+            // write null_bitmap file
+            _null_bitmap.runOptimize();
+            size_t size = _null_bitmap.getSizeInBytes(false);
+            if (size > 0) {
+                null_bitmap_out = dir->createOutput(
+                        InvertedIndexDescriptor::get_temporary_null_bitmap_file_name().c_str());
+                faststring buf;
+                buf.resize(size);
+                _null_bitmap.write(reinterpret_cast<char*>(buf.data()), false);
+                null_bitmap_out->writeBytes(reinterpret_cast<uint8_t*>(buf.data()), size);
+                FINALIZE_OUTPUT(null_bitmap_out)
+            }
+
+            // write bkd file
             if constexpr (field_is_numeric_type(field_type)) {
-                auto index_path = InvertedIndexDescriptor::get_temporary_index_path(
-                        _directory + "/" + _segment_file_name, _index_meta->index_id());
-                dir = DorisCompoundDirectory::getDirectory(_fs, index_path.c_str(), true);
                 _bkd_writer->max_doc_ = _rid;
                 _bkd_writer->docs_seen_ = _row_ids_seen_for_bkd;
                 data_out = dir->createOutput(
@@ -350,6 +367,7 @@ public:
                 close();
             }
         } catch (CLuceneError& e) {
+            FINALLY_FINALIZE_OUTPUT(null_bitmap_out)
             FINALLY_FINALIZE_OUTPUT(meta_out)
             FINALLY_FINALIZE_OUTPUT(data_out)
             FINALLY_FINALIZE_OUTPUT(index_out)
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 874908c416..4d4aa4ae56 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -721,8 +721,7 @@ bool SegmentIterator::_column_has_fulltext_index(int32_t unique_id) {
 }
 
 inline bool SegmentIterator::_inverted_index_not_support_pred_type(const PredicateType& type) {
-    return type == PredicateType::IS_NULL || type == PredicateType::IS_NOT_NULL ||
-           type == PredicateType::BF || type == PredicateType::BITMAP_FILTER;
+    return type == PredicateType::BF || type == PredicateType::BITMAP_FILTER;
 }
 
 #define all_predicates_are_range_predicate(predicate_set)   \
diff --git a/regression-test/data/inverted_index_p0/test_inverted_index_null.out b/regression-test/data/inverted_index_p0/test_inverted_index_null.out
new file mode 100644
index 0000000000..3512facd94
--- /dev/null
+++ b/regression-test/data/inverted_index_p0/test_inverted_index_null.out
@@ -0,0 +1,115 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_0 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+6	shanghai	hehe	\N	haha	\N
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_is_null_1 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+
+-- !select_is_null_2 --
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+6	shanghai	hehe	\N	haha	\N
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_is_null_3 --
+
+-- !select_is_null_4 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+6	shanghai	hehe	\N	haha	\N
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_is_null_5 --
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+6	shanghai	hehe	\N	haha	\N
+7	tengxun	qie	addr gg	lj	\N
+
+-- !select_is_null_6 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+3	beijing	addr xuanwu	wugui	\N	300
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+8	tengxun2	qie	\N	lj	800
+
+-- !select_compare_11 --
+6	shanghai	hehe	\N	haha	\N
+
+-- !select_compare_12 --
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_compare_13 --
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+6	shanghai	hehe	\N	haha	\N
+
+-- !select_compare_14 --
+6	shanghai	hehe	\N	haha	\N
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_compare_21 --
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+
+-- !select_compare_22 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+3	beijing	addr xuanwu	wugui	\N	300
+8	tengxun2	qie	\N	lj	800
+
+-- !select_compare_23 --
+1	\N	addr qie3	yy	lj	100
+2	\N	hehe	\N	lala	200
+3	beijing	addr xuanwu	wugui	\N	300
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+
+-- !select_compare_24 --
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+8	tengxun2	qie	\N	lj	800
+
+-- !select_in_1 --
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+6	shanghai	hehe	\N	haha	\N
+
+-- !select_in_2 --
+7	tengxun	qie	addr gg	lj	\N
+8	tengxun2	qie	\N	lj	800
+
+-- !select_in_3 --
+1	\N	addr qie3	yy	lj	100
+3	beijing	addr xuanwu	wugui	\N	300
+
+-- !select_in_4 --
+2	\N	hehe	\N	lala	200
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+8	tengxun2	qie	\N	lj	800
+
+-- !select_match_1 --
+1	\N	addr qie3	yy	lj	100
+3	beijing	addr xuanwu	wugui	\N	300
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+5	beijing	addr chaoyang	wangjing	donghuqu	500
+
+-- !select_match_2 --
+4	beijing	addr fengtai	fengtai1	fengtai2	\N
+
diff --git a/regression-test/suites/inverted_index_p0/test_inverted_index_null.groovy b/regression-test/suites/inverted_index_p0/test_inverted_index_null.groovy
new file mode 100644
index 0000000000..43cf22d572
--- /dev/null
+++ b/regression-test/suites/inverted_index_p0/test_inverted_index_null.groovy
@@ -0,0 +1,89 @@
+// 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_inverted_index_null") {
+
+    def table1 = "test_inverted_index_null"
+
+    sql "drop table if exists ${table1}"
+
+    sql """
+       CREATE TABLE IF NOT EXISTS `${table1}` (
+      `id` int NULL COMMENT "",
+      `city` varchar(20) NULL COMMENT "",
+      `addr` varchar(20) NULL COMMENT "",
+      `name` varchar(20) NULL COMMENT "",
+      `compy` varchar(20) NULL COMMENT "",
+      `n` int NULL COMMENT "",
+      INDEX idx_city(city) USING INVERTED,
+      INDEX idx_addr(addr) USING INVERTED PROPERTIES("parser"="english"),
+      INDEX idx_n(n) USING INVERTED
+    ) ENGINE=OLAP
+    DUPLICATE KEY(`id`)
+    COMMENT "OLAP"
+    DISTRIBUTED BY HASH(`id`) BUCKETS 1
+    PROPERTIES (
+    "replication_allocation" = "tag.location.default: 1",
+    "in_memory" = "false",
+    "storage_format" = "V2"
+    )
+    """
+
+    sql """insert into ${table1} values
+            (1,null,'addr qie3','yy','lj',100),
+            (2,null,'hehe',null,'lala',200),
+            (3,'beijing','addr xuanwu','wugui',null,300),
+            (4,'beijing','addr fengtai','fengtai1','fengtai2',null),
+            (5,'beijing','addr chaoyang','wangjing','donghuqu',500),
+            (6,'shanghai','hehe',null,'haha',null),
+            (7,'tengxun','qie','addr gg','lj',null),
+            (8,'tengxun2','qie',null,'lj',800)
+    """
+
+    // select all data
+    qt_select_0 "SELECT * FROM ${table1} ORDER BY id"
+
+    // test IS NULL , IS NOT NULL
+    qt_select_is_null_1 "SELECT * FROM ${table1} WHERE city IS NULL ORDER BY id"
+    qt_select_is_null_2 "SELECT * FROM ${table1} WHERE city IS NOT NULL ORDER BY id"
+    qt_select_is_null_3 "SELECT * FROM ${table1} WHERE addr IS NULL ORDER BY id"
+    qt_select_is_null_4 "SELECT * FROM ${table1} WHERE addr IS NOT NULL ORDER BY id"
+    qt_select_is_null_5 "SELECT * FROM ${table1} WHERE n IS NULL ORDER BY id"
+    qt_select_is_null_6 "SELECT * FROM ${table1} WHERE n IS NOT NULL ORDER BY id"
+
+    // test compare predicate
+    qt_select_compare_11 "SELECT * FROM ${table1} WHERE city  = 'shanghai' ORDER BY id"
+    qt_select_compare_12 "SELECT * FROM ${table1} WHERE city != 'shanghai' ORDER BY id"
+    qt_select_compare_13 "SELECT * FROM ${table1} WHERE city <= 'shanghai' ORDER BY id"
+    qt_select_compare_14 "SELECT * FROM ${table1} WHERE city >= 'shanghai' ORDER BY id"
+
+    qt_select_compare_21 "SELECT * FROM ${table1} WHERE n  = 500 ORDER BY id"
+    qt_select_compare_22 "SELECT * FROM ${table1} WHERE n != 500 ORDER BY id"
+    qt_select_compare_23 "SELECT * FROM ${table1} WHERE n <= 500 ORDER BY id"
+    qt_select_compare_24 "SELECT * FROM ${table1} WHERE n >= 500 ORDER BY id"
+
+    // test in predicates
+    qt_select_in_1 "SELECT * FROM ${table1} WHERE city IN ('shanghai', 'beijing') ORDER BY id"
+    qt_select_in_2 "SELECT * FROM ${table1} WHERE city NOT IN ('shanghai', 'beijing') ORDER BY id"
+    qt_select_in_3 "SELECT * FROM ${table1} WHERE n IN (100, 300) ORDER BY id"
+    qt_select_in_4 "SELECT * FROM ${table1} WHERE n NOT IN (100, 300) ORDER BY id"
+
+    // test match predicates
+    qt_select_match_1 "SELECT * FROM ${table1} WHERE addr MATCH_ANY 'addr fengtai' ORDER BY id"
+    qt_select_match_2 "SELECT * FROM ${table1} WHERE addr MATCH_ALL 'addr fengtai' ORDER BY id"
+}
+


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