You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/10/13 01:12:09 UTC

[doris] branch master updated: [improvement](olap) cache value of has_null in ColumnNullable (#13289)

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

yiguolei 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 9b590ac4cb [improvement](olap) cache value of has_null in ColumnNullable (#13289)
9b590ac4cb is described below

commit 9b590ac4cb907b9efee3b4dca2c2e0a038a187e0
Author: Jerry Hu <mr...@gmail.com>
AuthorDate: Thu Oct 13 09:12:02 2022 +0800

    [improvement](olap) cache value of has_null in ColumnNullable (#13289)
---
 be/src/olap/like_column_predicate.cpp              |  66 +++++---
 be/src/olap/null_predicate.cpp                     |  38 +++--
 be/src/util/simd/bits.h                            |   5 +
 be/src/vec/columns/column_nullable.cpp             |  43 ++++-
 be/src/vec/columns/column_nullable.h               |  51 +++---
 .../data/correctness_p0/test_null_predicate.out    | 176 +++++++++++++++++++++
 .../correctness_p0/test_null_predicate.groovy      |  90 +++++++++++
 7 files changed, 400 insertions(+), 69 deletions(-)

diff --git a/be/src/olap/like_column_predicate.cpp b/be/src/olap/like_column_predicate.cpp
index 12c17fda3e..a3546d3546 100644
--- a/be/src/olap/like_column_predicate.cpp
+++ b/be/src/olap/like_column_predicate.cpp
@@ -80,37 +80,61 @@ uint16_t LikeColumnPredicate<is_vectorized>::evaluate(const vectorized::IColumn&
                 auto* nested_col_ptr = vectorized::check_and_get_column<
                         vectorized::ColumnDictionary<vectorized::Int32>>(nested_col);
                 auto& data_array = nested_col_ptr->get_data();
-                for (uint16_t i = 0; i != size; i++) {
-                    uint16_t idx = sel[i];
-                    sel[new_size] = idx;
-                    if (null_map_data[idx]) {
-                        new_size += _opposite;
-                        continue;
+                if (!nullable_col->has_null()) {
+                    for (uint16_t i = 0; i != size; i++) {
+                        uint16_t idx = sel[i];
+                        sel[new_size] = idx;
+                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
+                        unsigned char flag = 0;
+                        (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
+                                           cell_value, pattern, &flag);
+                        new_size += _opposite ^ flag;
                     }
+                } else {
+                    for (uint16_t i = 0; i != size; i++) {
+                        uint16_t idx = sel[i];
+                        sel[new_size] = idx;
+                        if (null_map_data[idx]) {
+                            new_size += _opposite;
+                            continue;
+                        }
 
-                    StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
-                    unsigned char flag = 0;
-                    (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                       cell_value, pattern, &flag);
-                    new_size += _opposite ^ flag;
+                        StringValue cell_value = nested_col_ptr->get_shrink_value(data_array[idx]);
+                        unsigned char flag = 0;
+                        (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
+                                           cell_value, pattern, &flag);
+                        new_size += _opposite ^ flag;
+                    }
                 }
             } else {
                 auto* data_array = vectorized::check_and_get_column<
                                            vectorized::PredicateColumnType<TYPE_STRING>>(nested_col)
                                            ->get_data()
                                            .data();
-                for (uint16_t i = 0; i != size; i++) {
-                    uint16_t idx = sel[i];
-                    sel[new_size] = idx;
-                    if (null_map_data[idx]) {
-                        new_size += _opposite;
-                        continue;
+                if (!nullable_col->has_null()) {
+                    for (uint16_t i = 0; i != size; i++) {
+                        uint16_t idx = sel[i];
+                        sel[new_size] = idx;
+
+                        unsigned char flag = 0;
+                        (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
+                                           data_array[idx], pattern, &flag);
+                        new_size += _opposite ^ flag;
                     }
+                } else {
+                    for (uint16_t i = 0; i != size; i++) {
+                        uint16_t idx = sel[i];
+                        sel[new_size] = idx;
+                        if (null_map_data[idx]) {
+                            new_size += _opposite;
+                            continue;
+                        }
 
-                    unsigned char flag = 0;
-                    (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
-                                       data_array[idx], pattern, &flag);
-                    new_size += _opposite ^ flag;
+                        unsigned char flag = 0;
+                        (_state->function)(const_cast<vectorized::LikeSearchState*>(&_like_state),
+                                           data_array[idx], pattern, &flag);
+                        new_size += _opposite ^ flag;
+                    }
                 }
             }
         } else {
diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp
index ade037374e..04dacde7bf 100644
--- a/be/src/olap/null_predicate.cpp
+++ b/be/src/olap/null_predicate.cpp
@@ -90,6 +90,9 @@ uint16_t NullPredicate::evaluate(const vectorized::IColumn& column, uint16_t* se
                                  uint16_t size) const {
     uint16_t new_size = 0;
     if (auto* nullable = check_and_get_column<ColumnNullable>(column)) {
+        if (!nullable->has_null()) {
+            return _is_null ? 0 : size;
+        }
         auto& null_map = nullable->get_null_map_data();
         for (uint16_t i = 0; i < size; ++i) {
             uint16_t idx = sel[i];
@@ -106,11 +109,17 @@ uint16_t NullPredicate::evaluate(const vectorized::IColumn& column, uint16_t* se
 void NullPredicate::evaluate_or(const IColumn& column, const uint16_t* sel, uint16_t size,
                                 bool* flags) const {
     if (auto* nullable = check_and_get_column<ColumnNullable>(column)) {
-        auto& null_map = nullable->get_null_map_data();
-        for (uint16_t i = 0; i < size; ++i) {
-            if (flags[i]) continue;
-            uint16_t idx = sel[i];
-            flags[i] |= (null_map[idx] == _is_null);
+        if (!nullable->has_null()) {
+            if (!_is_null) {
+                memset(flags, true, size);
+            }
+        } else {
+            auto& null_map = nullable->get_null_map_data();
+            for (uint16_t i = 0; i < size; ++i) {
+                if (flags[i]) continue;
+                uint16_t idx = sel[i];
+                flags[i] |= (null_map[idx] == _is_null);
+            }
         }
     } else {
         if (!_is_null) memset(flags, true, size);
@@ -120,11 +129,17 @@ void NullPredicate::evaluate_or(const IColumn& column, const uint16_t* sel, uint
 void NullPredicate::evaluate_and(const IColumn& column, const uint16_t* sel, uint16_t size,
                                  bool* flags) const {
     if (auto* nullable = check_and_get_column<ColumnNullable>(column)) {
-        auto& null_map = nullable->get_null_map_data();
-        for (uint16_t i = 0; i < size; ++i) {
-            if (flags[i]) continue;
-            uint16_t idx = sel[i];
-            flags[i] &= (null_map[idx] == _is_null);
+        if (!nullable->has_null()) {
+            if (_is_null) {
+                memset(flags, false, size);
+            }
+        } else {
+            auto& null_map = nullable->get_null_map_data();
+            for (uint16_t i = 0; i < size; ++i) {
+                if (flags[i]) continue;
+                uint16_t idx = sel[i];
+                flags[i] &= (null_map[idx] == _is_null);
+            }
         }
     } else {
         if (_is_null) memset(flags, false, size);
@@ -134,6 +149,9 @@ void NullPredicate::evaluate_and(const IColumn& column, const uint16_t* sel, uin
 void NullPredicate::evaluate_vec(const vectorized::IColumn& column, uint16_t size,
                                  bool* flags) const {
     if (auto* nullable = check_and_get_column<ColumnNullable>(column)) {
+        if (!nullable->has_null()) {
+            memset(flags, !_is_null, size);
+        }
         auto& null_map = nullable->get_null_map_data();
         for (uint16_t i = 0; i < size; ++i) {
             flags[i] = (null_map[i] == _is_null);
diff --git a/be/src/util/simd/bits.h b/be/src/util/simd/bits.h
index df91e63c61..4f4fe65414 100644
--- a/be/src/util/simd/bits.h
+++ b/be/src/util/simd/bits.h
@@ -112,6 +112,11 @@ inline static size_t find_byte(const std::vector<T>& vec, size_t start, T byte)
     return (T*)p - vec.data();
 }
 
+template <typename T>
+inline bool contain_byte(const T* __restrict data, const size_t length, const signed char byte) {
+    return nullptr != std::memchr(reinterpret_cast<const void*>(data), byte, length);
+}
+
 inline size_t find_one(const std::vector<uint8_t>& vec, size_t start) {
     return find_byte<uint8_t>(vec, start, 1);
 }
diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp
index ab37eb3f59..401ee02a9e 100644
--- a/be/src/vec/columns/column_nullable.cpp
+++ b/be/src/vec/columns/column_nullable.cpp
@@ -42,6 +42,8 @@ ColumnNullable::ColumnNullable(MutableColumnPtr&& nested_column_, MutableColumnP
     if (is_column_const(*null_map)) {
         LOG(FATAL) << "ColumnNullable cannot have constant null map";
     }
+
+    _update_has_null();
 }
 
 MutableColumnPtr ColumnNullable::get_shrinked_column() {
@@ -62,7 +64,7 @@ void ColumnNullable::update_hashes_with_value(std::vector<SipHash>& hashes,
     auto s = hashes.size();
     DCHECK(s == size());
     auto* __restrict real_null_data = assert_cast<const ColumnUInt8&>(*null_map).get_data().data();
-    if (doris::simd::count_zero_num(reinterpret_cast<const int8_t*>(real_null_data), s) == s) {
+    if (!_has_null) {
         nested_column->update_hashes_with_value(hashes, nullptr);
     } else {
         for (int i = 0; i < s; ++i) {
@@ -96,7 +98,7 @@ void ColumnNullable::update_hashes_with_value(uint64_t* __restrict hashes,
     DCHECK(null_data == nullptr);
     auto s = size();
     auto* __restrict real_null_data = assert_cast<const ColumnUInt8&>(*null_map).get_data().data();
-    if (doris::simd::count_zero_num(reinterpret_cast<const int8_t*>(real_null_data), s) == s) {
+    if (!_has_null) {
         nested_column->update_hashes_with_value(hashes, nullptr);
     } else {
         for (int i = 0; i < s; ++i) {
@@ -146,6 +148,7 @@ void ColumnNullable::insert_data(const char* pos, size_t length) {
     if (pos == nullptr) {
         get_nested_column().insert_default();
         get_null_map_data().push_back(1);
+        _has_null = true;
     } else {
         get_nested_column().insert_data(pos, length);
         get_null_map_data().push_back(0);
@@ -159,6 +162,7 @@ void ColumnNullable::insert_many_strings(const StringRef* strings, size_t num) {
         if (strings[i].data == nullptr) {
             nested_column.insert_default();
             null_map_data.push_back(1);
+            _has_null = true;
         } else {
             nested_column.insert_data(strings[i].data, strings[i].size);
             null_map_data.push_back(0);
@@ -188,10 +192,12 @@ const char* ColumnNullable::deserialize_and_insert_from_arena(const char* pos) {
 
     get_null_map_data().push_back(val);
 
-    if (val == 0)
+    if (val == 0) {
         pos = get_nested_column().deserialize_and_insert_from_arena(pos);
-    else
+    } else {
         get_nested_column().insert_default();
+        _has_null = true;
+    }
 
     return pos;
 }
@@ -223,6 +229,7 @@ void ColumnNullable::deserialize_vec(std::vector<StringRef>& keys, const size_t
     for (size_t i = 0; i != num_rows; ++i) {
         UInt8 val = *reinterpret_cast<const UInt8*>(keys[i].data);
         null_map_data[i] = val;
+        _has_null |= val;
         keys[i].data += sizeof(val);
         keys[i].size -= sizeof(val);
     }
@@ -233,6 +240,8 @@ void ColumnNullable::insert_range_from(const IColumn& src, size_t start, size_t
     const ColumnNullable& nullable_col = assert_cast<const ColumnNullable&>(src);
     get_null_map_column().insert_range_from(*nullable_col.null_map, start, length);
     get_nested_column().insert_range_from(*nullable_col.nested_column, start, length);
+    auto& src_null_map_data = nullable_col.get_null_map_data();
+    _has_null |= simd::contain_byte(src_null_map_data.data() + start, length, 1);
 }
 
 void ColumnNullable::insert_indices_from(const IColumn& src, const int* indices_begin,
@@ -242,12 +251,14 @@ void ColumnNullable::insert_indices_from(const IColumn& src, const int* indices_
                                             indices_end);
     get_null_map_column().insert_indices_from(src_concrete.get_null_map_column(), indices_begin,
                                               indices_end);
+    _update_has_null();
 }
 
 void ColumnNullable::insert(const Field& x) {
     if (x.is_null()) {
         get_nested_column().insert_default();
         get_null_map_data().push_back(1);
+        _has_null = true;
     } else {
         get_nested_column().insert(x);
         get_null_map_data().push_back(0);
@@ -257,7 +268,9 @@ void ColumnNullable::insert(const Field& x) {
 void ColumnNullable::insert_from(const IColumn& src, size_t n) {
     const ColumnNullable& src_concrete = assert_cast<const ColumnNullable&>(src);
     get_nested_column().insert_from(src_concrete.get_nested_column(), n);
-    get_null_map_data().push_back(src_concrete.get_null_map_data()[n]);
+    auto is_null = src_concrete.get_null_map_data()[n];
+    _has_null |= is_null;
+    get_null_map_data().push_back(is_null);
 }
 
 void ColumnNullable::insert_from_not_nullable(const IColumn& src, size_t n) {
@@ -281,6 +294,7 @@ void ColumnNullable::insert_many_from_not_nullable(const IColumn& src, size_t po
 void ColumnNullable::pop_back(size_t n) {
     get_nested_column().pop_back(n);
     get_null_map_column().pop_back(n);
+    _update_has_null();
 }
 
 ColumnPtr ColumnNullable::filter(const Filter& filt, ssize_t result_size_hint) const {
@@ -297,6 +311,7 @@ Status ColumnNullable::filter_by_selector(const uint16_t* sel, size_t sel_size,
             sel, sel_size, const_cast<doris::vectorized::IColumn*>(nest_col_ptr.get())));
     RETURN_IF_ERROR(get_null_map_column().filter_by_selector(
             sel, sel_size, const_cast<doris::vectorized::IColumn*>(null_map_ptr.get())));
+    _update_has_null();
     return Status::OK();
 }
 
@@ -400,8 +415,13 @@ void ColumnNullable::reserve(size_t n) {
 }
 
 void ColumnNullable::resize(size_t n) {
+    auto& null_map_data = get_null_map_data();
+    auto old_size = null_map_data.size();
     get_nested_column().resize(n);
-    get_null_map_data().resize(n);
+    null_map_data.resize(n);
+    if (UNLIKELY(old_size > n)) {
+        _update_has_null();
+    }
 }
 
 size_t ColumnNullable::byte_size() const {
@@ -554,6 +574,17 @@ void ColumnNullable::sort_column(const ColumnSorter* sorter, EqualFlags& flags,
                         last_column);
 }
 
+void ColumnNullable::_update_has_null() {
+    const UInt8* null_pos = get_null_map_data().data();
+    _has_null = simd::contain_byte(null_pos, get_null_map_data().size(), 1);
+    _need_update_has_null = false;
+}
+
+bool ColumnNullable::has_null(size_t size) const {
+    const UInt8* null_pos = get_null_map_data().data();
+    return simd::contain_byte(null_pos, size, 1);
+}
+
 ColumnPtr make_nullable(const ColumnPtr& column, bool is_nullable) {
     if (is_column_nullable(*column)) return column;
 
diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h
index ea9acfce18..fdfa880e1d 100644
--- a/be/src/vec/columns/column_nullable.h
+++ b/be/src/vec/columns/column_nullable.h
@@ -133,16 +133,19 @@ public:
     void insert_default() override {
         get_nested_column().insert_default();
         get_null_map_data().push_back(1);
+        _has_null = true;
     }
 
     void insert_many_defaults(size_t length) override {
         get_nested_column().insert_many_defaults(length);
         get_null_map_data().resize_fill(get_null_map_data().size() + length, 1);
+        _has_null = true;
     }
 
     void insert_null_elements(int num) {
         get_nested_column().insert_many_defaults(num);
         get_null_map_column().fill(1, num);
+        _has_null = true;
     }
 
     void pop_back(size_t n) override;
@@ -232,9 +235,14 @@ public:
     void clear() override {
         null_map->clear();
         nested_column->clear();
+        _has_null = false;
+    }
+
+    NullMap& get_null_map_data() {
+        _need_update_has_null = true;
+        return get_null_map_column().get_data();
     }
 
-    NullMap& get_null_map_data() { return get_null_map_column().get_data(); }
     const NullMap& get_null_map_data() const { return get_null_map_column().get_data(); }
 
     /// Apply the null byte map of a specified nullable column onto the
@@ -249,40 +257,15 @@ public:
     /// Check that size of null map equals to size of nested column.
     void check_consistency() const;
 
-    bool has_null() const override { return has_null(get_null_map_data().size()); }
-
-    bool has_null(size_t size) const override {
-        const UInt8* null_pos = get_null_map_data().data();
-        const UInt8* null_pos_end = get_null_map_data().data() + size;
-#if defined(__SSE2__) || defined(__aarch64__)
-        /** A slightly more optimized version.
-        * Based on the assumption that often pieces of consecutive values
-        *  completely pass or do not pass the filter.
-        * Therefore, we will optimistically check the parts of `SIMD_BYTES` values.
-        */
-        static constexpr size_t SIMD_BYTES = 16;
-        const __m128i zero16 = _mm_setzero_si128();
-        const UInt8* null_end_sse = null_pos + size / SIMD_BYTES * SIMD_BYTES;
-
-        while (null_pos < null_end_sse) {
-            int mask = _mm_movemask_epi8(_mm_cmpgt_epi8(
-                    _mm_loadu_si128(reinterpret_cast<const __m128i*>(null_pos)), zero16));
-
-            if (0 != mask) {
-                return true;
-            }
-            null_pos += SIMD_BYTES;
-        }
-#endif
-        while (null_pos < null_pos_end) {
-            if (*null_pos != 0) {
-                return true;
-            }
-            null_pos++;
+    bool has_null() const override {
+        if (UNLIKELY(_need_update_has_null)) {
+            const_cast<ColumnNullable*>(this)->_update_has_null();
         }
-        return false;
+        return _has_null;
     }
 
+    bool has_null(size_t size) const override;
+
     void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override {
         DCHECK(size() > self_row);
         const ColumnNullable& nullable_rhs = assert_cast<const ColumnNullable&>(rhs);
@@ -319,6 +302,10 @@ private:
     WrappedPtr nested_column;
     WrappedPtr null_map;
 
+    bool _need_update_has_null = false;
+    bool _has_null;
+
+    void _update_has_null();
     template <bool negative>
     void apply_null_map_impl(const ColumnUInt8& map);
 };
diff --git a/regression-test/data/correctness_p0/test_null_predicate.out b/regression-test/data/correctness_p0/test_null_predicate.out
new file mode 100644
index 0000000000..8de8224c73
--- /dev/null
+++ b/regression-test/data/correctness_p0/test_null_predicate.out
@@ -0,0 +1,176 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select1 --
+100	name#100
+101	\N
+102	name#102
+103	\N
+104	name#104
+105	\N
+106	name#106
+107	\N
+108	\N
+109	name#109
+110	name#110
+111	\N
+112	name#112
+113	\N
+114	\N
+115	\N
+116	name#116
+117	name#117
+118	name#118
+119	\N
+120	name#120
+121	name#121
+122	name#122
+123	\N
+
+-- !select2 --
+24
+
+-- !select3 --
+101	\N
+103	\N
+105	\N
+107	\N
+108	\N
+111	\N
+113	\N
+114	\N
+115	\N
+119	\N
+123	\N
+
+-- !select4 --
+101	\N
+103	\N
+105	\N
+107	\N
+108	\N
+
+-- !select5 --
+111	\N
+113	\N
+114	\N
+115	\N
+119	\N
+123	\N
+
+-- !select6 --
+100	name#100
+101	\N
+102	name#102
+103	\N
+104	name#104
+105	\N
+106	name#106
+107	\N
+108	\N
+109	name#109
+111	\N
+113	\N
+114	\N
+115	\N
+119	\N
+123	\N
+
+-- !select7 --
+101	\N
+103	\N
+105	\N
+107	\N
+108	\N
+110	name#110
+111	\N
+112	name#112
+113	\N
+114	\N
+115	\N
+116	name#116
+117	name#117
+118	name#118
+119	\N
+120	name#120
+121	name#121
+122	name#122
+123	\N
+
+-- !select8 --
+11
+
+-- !select9 --
+100	name#100
+102	name#102
+104	name#104
+106	name#106
+109	name#109
+110	name#110
+112	name#112
+116	name#116
+117	name#117
+118	name#118
+120	name#120
+121	name#121
+122	name#122
+
+-- !select10 --
+100	name#100
+102	name#102
+104	name#104
+106	name#106
+109	name#109
+
+-- !select11 --
+110	name#110
+112	name#112
+116	name#116
+117	name#117
+118	name#118
+120	name#120
+121	name#121
+122	name#122
+
+-- !select12 --
+100	name#100
+101	\N
+102	name#102
+103	\N
+104	name#104
+105	\N
+106	name#106
+107	\N
+108	\N
+109	name#109
+110	name#110
+112	name#112
+116	name#116
+117	name#117
+118	name#118
+120	name#120
+121	name#121
+122	name#122
+
+-- !select13 --
+100	name#100
+102	name#102
+104	name#104
+106	name#106
+109	name#109
+110	name#110
+111	\N
+112	name#112
+113	\N
+114	\N
+115	\N
+116	name#116
+117	name#117
+118	name#118
+119	\N
+120	name#120
+121	name#121
+122	name#122
+123	\N
+
+-- !select14 --
+13
+
diff --git a/regression-test/suites/correctness_p0/test_null_predicate.groovy b/regression-test/suites/correctness_p0/test_null_predicate.groovy
new file mode 100644
index 0000000000..7c9ff05469
--- /dev/null
+++ b/regression-test/suites/correctness_p0/test_null_predicate.groovy
@@ -0,0 +1,90 @@
+// 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_null_predicate") {
+    def tableName = "test_null_predicate"
+
+
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+    sql """
+            CREATE TABLE ${tableName} (
+            `id` INT,
+            `name` STRING NULL
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`id`)
+            COMMENT "OLAP"
+            DISTRIBUTED BY HASH(`id`) BUCKETS 4
+            PROPERTIES (
+            "replication_num" = "1",
+            "in_memory" = "false",
+            "storage_format" = "V2"
+            );
+    """
+
+    // bucket1: 102 106 108 114 116 120
+    // bucket2: 101 105 111 113 119 123  -- all null
+    // bucket3: 100 104 110 112 118 122  -- no null
+    // bucket4: 103 107 109 115 117 121
+
+    sql """ INSERT INTO ${tableName} VALUES 
+            (100, "name#100"),
+            (101, null),
+            (102, "name#102"),
+            (103, null),
+
+            (104, "name#104"),
+            (105, null),
+            (106, "name#106"),
+            (107, null),
+
+            (108, null),
+            (109, "name#109"),
+            (110, "name#110"),
+            (111, null),
+
+            (112, "name#112"),
+            (113, null),
+            (114, null),
+            (115, null),
+
+            (116, "name#116"),
+            (117, "name#117"),
+            (118, "name#118"),
+            (119, null),
+
+            (120, "name#120"),
+            (121, "name#121"),
+            (122, "name#122"),
+            (123, null); """
+
+    sql """ set enable_vectorized_engine = true; """
+
+    qt_select1 """ select id, name from ${tableName} order by id, name; """
+    qt_select2 """ select count(1) from ${tableName}; """
+    qt_select3 """ select id, name from ${tableName} where name is null order by id; """
+    qt_select4 """ select id, name from ${tableName} where id < 110 and name is null order by id; """
+    qt_select5 """ select id, name from ${tableName} where id > 109 and name is null order by id; """
+    qt_select6 """ select id, name from ${tableName} where id < 110 or name is null order by id; """
+    qt_select7 """ select id, name from ${tableName} where id > 109 or name is null order by id; """
+    qt_select8 """ select count(1) from ${tableName} where name is null; """
+    qt_select9 """ select id, name from ${tableName} where name is not null order by id, name; """
+    qt_select10 """ select id, name from ${tableName} where id < 110 and name is not null order by id, name; """
+    qt_select11 """ select id, name from ${tableName} where id > 109 and name is not null order by id, name; """
+    qt_select12 """ select id, name from ${tableName} where id < 110 or name is not null order by id, name; """
+    qt_select13 """ select id, name from ${tableName} where id > 109 or name is not null order by id, name; """
+    qt_select14 """ select count(1) from ${tableName} where name is not null; """
+}


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