diff --git a/be/src/pipeline/exec/hashjoin_build_sink.h b/be/src/pipeline/exec/hashjoin_build_sink.h index 607d8f0cbf5e29..45ab687a631837 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.h +++ b/be/src/pipeline/exec/hashjoin_build_sink.h @@ -192,11 +192,8 @@ struct ProcessHashTableBuild { bool* has_null_key) { if (null_map) { // first row is mocked and is null - // TODO: Need to test the for loop. break may better - for (uint32_t i = 1; i < _rows; i++) { - if ((*null_map)[i]) { - *has_null_key = true; - } + if (simd::contain_one(null_map->data() + 1, _rows - 1)) { + *has_null_key = true; } if (short_circuit_for_null && *has_null_key) { return Status::OK(); @@ -208,7 +205,7 @@ struct ProcessHashTableBuild { _rows, _batch_size, *has_null_key, hash_table_ctx.direct_mapping_range()); // In order to make the null keys equal when using single null eq, all null keys need to be set to default value. - if (_build_raw_ptrs.size() == 1 && null_map) { + if (_build_raw_ptrs.size() == 1 && null_map && *has_null_key) { _build_raw_ptrs[0]->assume_mutable()->replace_column_null_data(null_map->data()); } diff --git a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h index 60261223cdf6e0..bf0d4cd79733cb 100644 --- a/be/src/pipeline/exec/join/process_hash_table_probe_impl.h +++ b/be/src/pipeline/exec/join/process_hash_table_probe_impl.h @@ -114,8 +114,8 @@ void ProcessHashTableProbe::build_side_output_column(vectorized::Mut _build_column_has_null[i] = false; if (_right_output_slot_flags[i] && column.is_nullable()) { const auto& nullable = assert_cast(column); - _build_column_has_null[i] = !simd::contain_byte( - nullable.get_null_map_data().data() + 1, nullable.size() - 1, 1); + _build_column_has_null[i] = !simd::contain_one( + nullable.get_null_map_data().data() + 1, nullable.size() - 1); } } } @@ -192,7 +192,9 @@ typename HashTableType::State ProcessHashTableProbe::_init_probe_sid hash_table_ctx.arena.clear(); // In order to make the null keys equal when using single null eq, all null keys need to be set to default value. if (_parent->_probe_columns.size() == 1 && null_map) { - _parent->_probe_columns[0]->assume_mutable()->replace_column_null_data(null_map); + if (simd::contain_one(null_map, probe_rows)) { + _parent->_probe_columns[0]->assume_mutable()->replace_column_null_data(null_map); + } } hash_table_ctx.init_serialized_keys(_parent->_probe_columns, probe_rows, null_map, true, @@ -366,8 +368,7 @@ Status ProcessHashTableProbe::finalize_block_with_filter( } const auto& column_filter = assert_cast(filter_ptr.get())->get_data(); - bool need_filter = - simd::count_zero_num((int8_t*)column_filter.data(), column_filter.size()) != 0; + bool need_filter = simd::contain_zero(column_filter.data(), column_filter.size()); if (need_filter) { row_indexs.filter(column_filter); } diff --git a/be/src/pipeline/exec/nested_loop_join_probe_operator.h b/be/src/pipeline/exec/nested_loop_join_probe_operator.h index 945a964672b92f..72f419a936cb94 100644 --- a/be/src/pipeline/exec/nested_loop_join_probe_operator.h +++ b/be/src/pipeline/exec/nested_loop_join_probe_operator.h @@ -97,9 +97,7 @@ class NestedLoopJoinProbeLocalState final } if (!_cur_probe_row_visited_flags[i]) { _cur_probe_row_visited_flags[i] = - simd::contain_byte(filter.data() + offset, end - offset, 1) - ? 1 - : 0; + simd::contain_one(filter.data() + offset, end - offset); } end = offset; } diff --git a/be/src/util/simd/bits.h b/be/src/util/simd/bits.h index 01ac30d45baaac..334e0b99fe52ad 100644 --- a/be/src/util/simd/bits.h +++ b/be/src/util/simd/bits.h @@ -261,14 +261,6 @@ static size_t find_byte(const T* data, size_t start, size_t end, T byte) { return (T*)p - data; } -template -bool contain_byte(const T* __restrict data, const size_t length, const signed char byte) { - if (length == 0) { - return false; - } - return nullptr != std::memchr(reinterpret_cast(data), byte, length); -} - inline size_t find_one(const std::vector& vec, size_t start) { return find_byte(vec, start, 1); } @@ -281,5 +273,58 @@ inline size_t find_zero(const std::vector& vec, size_t start) { return find_byte(vec, start, 0); } +inline bool contain_one(const uint8_t* __restrict data, size_t size) { + size_t i = 0; +#if defined(__AVX2__) + for (; i + 32 <= size; i += 32) { + __m256i chunk = _mm256_loadu_si256(reinterpret_cast(data + i)); + if (!_mm256_testz_si256(chunk, chunk)) { + return true; + } + } +#elif defined(__SSE2__) + const __m128i zero = _mm_setzero_si128(); + for (; i + 16 <= size; i += 16) { + __m128i chunk = _mm_loadu_si128(reinterpret_cast(data + i)); + if (_mm_movemask_epi8(_mm_cmpeq_epi8(chunk, zero)) != 0xFFFF) { + return true; + } + } +#endif + for (; i < size; ++i) { + if (data[i]) { + return true; + } + } + return false; +} + +inline bool contain_zero(const uint8_t* __restrict data, size_t size) { + size_t i = 0; +#if defined(__AVX2__) + const __m256i zero = _mm256_setzero_si256(); + for (; i + 32 <= size; i += 32) { + __m256i chunk = _mm256_loadu_si256(reinterpret_cast(data + i)); + if (_mm256_movemask_epi8(_mm256_cmpeq_epi8(chunk, zero)) != 0) { + return true; + } + } +#elif defined(__SSE2__) + const __m128i zero = _mm_setzero_si128(); + for (; i + 16 <= size; i += 16) { + __m128i chunk = _mm_loadu_si128(reinterpret_cast(data + i)); + if (_mm_movemask_epi8(_mm_cmpeq_epi8(chunk, zero)) != 0) { + return true; + } + } +#endif + for (; i < size; ++i) { + if (!data[i]) { + return true; + } + } + return false; +} + } // namespace doris::simd #include "common/compile_check_end.h" \ No newline at end of file diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index 782514fcc6f185..543b4a8418acb8 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -523,10 +523,6 @@ void ColumnDecimal::compare_internal(size_t rhs_row_id, const IColumn& rhs, template void ColumnDecimal::replace_column_null_data(const uint8_t* __restrict null_map) { auto s = size(); - size_t null_count = s - simd::count_zero_num((const int8_t*)null_map, s); - if (0 == null_count) { - return; - } for (size_t i = 0; i < s; ++i) { data[i] = null_map[i] ? value_type() : data[i]; } diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 5c78fa456c7820..57152e1eee0412 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -280,7 +280,7 @@ size_t ColumnNullable::serialize_impl(char* pos, const size_t row) const { } void ColumnNullable::serialize(StringRef* keys, size_t num_rows) const { - const bool has_null = simd::contain_byte(get_null_map_data().data(), num_rows, 1); + const bool has_null = simd::contain_one(get_null_map_data().data(), num_rows); const auto* __restrict null_map = assert_cast(get_null_map_column()).get_data().data(); _nested_column->serialize_with_nullable(keys, num_rows, has_null, null_map); @@ -598,11 +598,11 @@ void ColumnNullable::sort_column(const ColumnSorter* sorter, EqualFlags& flags, } bool ColumnNullable::only_null() const { - return !simd::contain_byte(get_null_map_data().data(), size(), 0); + return !simd::contain_zero(get_null_map_data().data(), size()); } bool ColumnNullable::has_null(size_t begin, size_t end) const { - return simd::contain_byte(get_null_map_data().data() + begin, end - begin, 1); + return simd::contain_one(get_null_map_data().data() + begin, end - begin); } bool ColumnNullable::has_null() const { diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index a5367ab8634545..e75aed76b8a523 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -525,12 +525,9 @@ MutableColumnPtr ColumnVector::permute(const IColumn::Permutation& perm, size template void ColumnVector::replace_column_null_data(const uint8_t* __restrict null_map) { auto s = size(); - size_t null_count = s - simd::count_zero_num((const int8_t*)null_map, s); - if (0 == null_count) { - return; - } + auto value = default_value(); for (size_t i = 0; i < s; ++i) { - data[i] = null_map[i] ? default_value() : data[i]; + data[i] = null_map[i] ? value : data[i]; } } diff --git a/be/src/vec/common/hash_table/hash_key_type.h b/be/src/vec/common/hash_table/hash_key_type.h index 025af1bdc40948..8a264c9ddb9687 100644 --- a/be/src/vec/common/hash_table/hash_key_type.h +++ b/be/src/vec/common/hash_table/hash_key_type.h @@ -80,6 +80,10 @@ inline HashKeyType get_hash_key_type_with_fixed(size_t size) { } inline HashKeyType get_hash_key_type_fixed(const std::vector& data_types) { + if (data_types.size() >= vectorized::BITSIZE) { + return HashKeyType::serialized; + } + bool has_null = false; size_t key_byte_size = 0; @@ -94,8 +98,7 @@ inline HashKeyType get_hash_key_type_fixed(const std::vector& data_types) { diff --git a/be/src/vec/common/hash_table/hash_map_context.h b/be/src/vec/common/hash_table/hash_map_context.h index 0140a30653fa35..e7a5af1706e99b 100644 --- a/be/src/vec/common/hash_table/hash_map_context.h +++ b/be/src/vec/common/hash_table/hash_map_context.h @@ -17,11 +17,13 @@ #pragma once +#include #include #include #include #include "common/compiler_util.h" +#include "util/simd/bits.h" #include "vec/columns/column_array.h" #include "vec/columns/column_nullable.h" #include "vec/common/arena.h" @@ -31,6 +33,7 @@ #include "vec/common/hash_table/string_hash_map.h" #include "vec/common/string_ref.h" #include "vec/core/types.h" +#include "vec/utils/template_helpers.hpp" namespace doris::vectorized { #include "common/compile_check_begin.h" @@ -289,10 +292,6 @@ struct MethodSerialized : public MethodBase { } }; -inline size_t get_bitmap_size(size_t key_number) { - return (key_number + BITSIZE - 1) / BITSIZE; -} - template struct MethodStringNoCache : public MethodBase { using Base = MethodBase; @@ -476,6 +475,69 @@ struct MethodOneNumberDirect : public MethodOneNumber { } }; +template +void pack_nullmaps_interleaved(const uint8_t* const* datas, const uint8_t* bit_offsets, + size_t row_numbers, size_t stride, uint8_t* __restrict out) { + static_assert(N >= 1 && N <= BITSIZE); + + const uint8_t* __restrict p0 = (N > 0) ? datas[0] : nullptr; + const uint8_t* __restrict p1 = (N > 1) ? datas[1] : nullptr; + const uint8_t* __restrict p2 = (N > 2) ? datas[2] : nullptr; + const uint8_t* __restrict p3 = (N > 3) ? datas[3] : nullptr; + const uint8_t* __restrict p4 = (N > 4) ? datas[4] : nullptr; + const uint8_t* __restrict p5 = (N > 5) ? datas[5] : nullptr; + const uint8_t* __restrict p6 = (N > 6) ? datas[6] : nullptr; + const uint8_t* __restrict p7 = (N > 7) ? datas[7] : nullptr; + + const uint8_t m0 = (N > 0) ? bit_offsets[0] : 0; + const uint8_t m1 = (N > 1) ? bit_offsets[1] : 0; + const uint8_t m2 = (N > 2) ? bit_offsets[2] : 0; + const uint8_t m3 = (N > 3) ? bit_offsets[3] : 0; + const uint8_t m4 = (N > 4) ? bit_offsets[4] : 0; + const uint8_t m5 = (N > 5) ? bit_offsets[5] : 0; + const uint8_t m6 = (N > 6) ? bit_offsets[6] : 0; + const uint8_t m7 = (N > 7) ? bit_offsets[7] : 0; + + for (size_t i = 0; i < row_numbers; ++i) { + uint8_t byte = 0; + + if constexpr (N > 0) { + byte |= p0[i] << m0; + } + if constexpr (N > 1) { + byte |= p1[i] << m1; + } + if constexpr (N > 2) { + byte |= p2[i] << m2; + } + if constexpr (N > 3) { + byte |= p3[i] << m3; + } + if constexpr (N > 4) { + byte |= p4[i] << m4; + } + if constexpr (N > 5) { + byte |= p5[i] << m5; + } + if constexpr (N > 6) { + byte |= p6[i] << m6; + } + if constexpr (N > 7) { + byte |= p7[i] << m7; + } + + out[i * stride] |= byte; + } +} + +template +struct PackNullmapsReducer { + static void run(const uint8_t* const* datas, const uint8_t* coefficients, size_t row_numbers, + size_t stride, uint8_t* __restrict out) { + pack_nullmaps_interleaved(datas, coefficients, row_numbers, stride, out); + } +}; + template struct MethodKeysFixed : public MethodBase { using Base = MethodBase; @@ -497,47 +559,56 @@ struct MethodKeysFixed : public MethodBase { template void pack_fixeds(size_t row_numbers, const ColumnRawPtrs& key_columns, const ColumnRawPtrs& nullmap_columns, DorisVector& result) { - size_t bitmap_size = get_bitmap_size(nullmap_columns.size()); - // set size to 0 at first, then use resize to call default constructor on index included from [0, row_numbers) to reset all memory - result.clear(); + size_t bitmap_size = nullmap_columns.empty() ? 0 : 1; + if (bitmap_size) { + // set size to 0 at first, then use resize to call default constructor on index included from [0, row_numbers) to reset all memory + // only need to reset the memory used to bitmap + result.clear(); + } result.resize(row_numbers); + auto* __restrict result_data = reinterpret_cast(result.data()); + size_t offset = 0; + std::vector has_null_column(nullmap_columns.size(), false); if (bitmap_size > 0) { + std::vector nullmap_datas; + std::vector bit_offsets; for (size_t j = 0; j < nullmap_columns.size(); j++) { if (!nullmap_columns[j]) { continue; } - size_t bucket = j / BITSIZE; - size_t local_offset = j % BITSIZE; - const auto& data = + const uint8_t* __restrict data = assert_cast(*nullmap_columns[j]).get_data().data(); - for (size_t i = 0; i < row_numbers; ++i) { - *((char*)(&result[i]) + bucket) |= data[i] << local_offset; + + has_null_column[j] = simd::contain_one(data, row_numbers); + if (has_null_column[j]) { + nullmap_datas.emplace_back(data); + bit_offsets.emplace_back(j); } } + constexpr_int_match<1, BITSIZE, PackNullmapsReducer>::run( + int(nullmap_datas.size()), nullmap_datas.data(), bit_offsets.data(), + row_numbers, sizeof(T), reinterpret_cast(result_data)); offset += bitmap_size; } for (size_t j = 0; j < key_columns.size(); ++j) { - const char* data = key_columns[j]->get_raw_data().data; + const char* __restrict data = key_columns[j]->get_raw_data().data; auto foo = [&](Fixed zero) { CHECK_EQ(sizeof(Fixed), key_sizes[j]); - if (!nullmap_columns.empty() && nullmap_columns[j]) { - const auto& nullmap = + if (has_null_column.size() && has_null_column[j]) { + const auto* nullmap = assert_cast(*nullmap_columns[j]).get_data().data(); - for (size_t i = 0; i < row_numbers; ++i) { - // make sure null cell is filled by 0x0 - memcpy_fixed( - (char*)(&result[i]) + offset, - nullmap[i] ? (char*)&zero : data + i * sizeof(Fixed)); - } - } else { - for (size_t i = 0; i < row_numbers; ++i) { - memcpy_fixed((char*)(&result[i]) + offset, - data + i * sizeof(Fixed)); - } + // make sure null cell is filled by 0x0 + key_columns[j]->assume_mutable()->replace_column_null_data(nullmap); + } + auto* __restrict current = result_data + offset; + for (size_t i = 0; i < row_numbers; ++i) { + memcpy_fixed(current, data); + current += sizeof(T); + data += sizeof(Fixed); } }; @@ -579,6 +650,7 @@ struct MethodKeysFixed : public MethodBase { void init_serialized_keys(const ColumnRawPtrs& key_columns, uint32_t num_rows, const uint8_t* null_map = nullptr, bool is_join = false, bool is_build = false, uint32_t bucket_size = 0) override { + CHECK(key_columns.size() <= BITSIZE); ColumnRawPtrs actual_columns; ColumnRawPtrs null_maps; actual_columns.reserve(key_columns.size()); @@ -616,14 +688,8 @@ struct MethodKeysFixed : public MethodBase { void insert_keys_into_columns(std::vector& input_keys, MutableColumns& key_columns, const uint32_t num_rows) override { - // In any hash key value, column values to be read start just after the bitmap, if it exists. - size_t pos = 0; - for (size_t i = 0; i < key_columns.size(); ++i) { - if (key_columns[i]->is_nullable()) { - pos = get_bitmap_size(key_columns.size()); - break; - } - } + size_t pos = std::ranges::any_of(key_columns, + [](const auto& col) { return col->is_nullable(); }); for (size_t i = 0; i < key_columns.size(); ++i) { size_t size = key_sizes[i]; @@ -642,11 +708,8 @@ struct MethodKeysFixed : public MethodBase { // The current column is nullable. Check if the value of the // corresponding key is nullable. Update the null map accordingly. - size_t bucket = i / BITSIZE; - size_t offset = i % BITSIZE; for (size_t j = 0; j < num_rows; j++) { - nullmap[j] = - (reinterpret_cast(&input_keys[j])[bucket] >> offset) & 1; + nullmap[j] = (*reinterpret_cast(&input_keys[j]) >> i) & 1; } } else { // key_columns is a mutable element. However, when accessed through get_raw_data().data, diff --git a/be/src/vec/core/column_with_type_and_name.cpp b/be/src/vec/core/column_with_type_and_name.cpp index 0428d583247f8d..6478ad2bdec629 100644 --- a/be/src/vec/core/column_with_type_and_name.cpp +++ b/be/src/vec/core/column_with_type_and_name.cpp @@ -27,6 +27,7 @@ #include #include +#include "util/simd/bits.h" #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/columns/column_nothing.h" @@ -128,7 +129,9 @@ ColumnWithTypeAndName ColumnWithTypeAndName::unnest_nullable( const auto& null_map = source_column->get_null_map_data(); // only need to mutate nested column, avoid to copy nullmap auto mutable_nested_col = (*std::move(nested_column)).mutate(); - mutable_nested_col->replace_column_null_data(null_map.data()); + if (simd::contain_one(null_map.data(), null_map.size())) { + mutable_nested_col->replace_column_null_data(null_map.data()); + } return {std::move(mutable_nested_col), nested_type, ""}; } diff --git a/be/src/vec/data_types/serde/data_type_map_serde.cpp b/be/src/vec/data_types/serde/data_type_map_serde.cpp index b0dad43148acca..37937ef48cbd8b 100644 --- a/be/src/vec/data_types/serde/data_type_map_serde.cpp +++ b/be/src/vec/data_types/serde/data_type_map_serde.cpp @@ -352,8 +352,8 @@ Status DataTypeMapSerDe::write_column_to_arrow(const IColumn& column, const Null if ((null_map && (*null_map)[r])) { RETURN_IF_ERROR(checkArrowStatus(builder.AppendNull(), column.get_name(), array_builder->type()->name())); - } else if (simd::contain_byte(keys_nullmap_data + offsets[r - 1], - offsets[r] - offsets[r - 1], 1)) { + } else if (simd::contain_one(keys_nullmap_data + offsets[r - 1], + offsets[r] - offsets[r - 1])) { // arrow do not support key is null, so we ignore the null key-value MutableColumnPtr key_mutable_data = nested_keys_column.clone_empty(); MutableColumnPtr value_mutable_data = nested_values_column.clone_empty(); diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp index 4594fd6e668faa..b5af6c1ec8e7de 100644 --- a/be/src/vec/functions/function_jsonb.cpp +++ b/be/src/vec/functions/function_jsonb.cpp @@ -727,8 +727,7 @@ class FunctionJsonbExtractPath : public IFunction { VectorizedUtils::update_null_map(*result_null_map, *path_null_map, path_const); } - if (0 == simd::count_zero_num(reinterpret_cast(result_null_map->data()), - input_rows_count)) { + if (!simd::contain_zero(result_null_map->data(), input_rows_count)) { return create_all_null_result(); } } diff --git a/be/test/util/simd/bits_test.cpp b/be/test/util/simd/bits_test.cpp index 7892a54195031b..cc85a92186f103 100644 --- a/be/test/util/simd/bits_test.cpp +++ b/be/test/util/simd/bits_test.cpp @@ -148,14 +148,147 @@ TEST(BitsTest, FindByte) { } TEST(BitsTest, ContainByte) { - std::vector v = {5, 0, 1, 7, 1, 9, 0, 3}; - const uint8_t* data = v.data(); - EXPECT_TRUE(contain_byte(data, v.size(), static_cast(5))); - EXPECT_TRUE(contain_byte(data, v.size(), static_cast(0))); - EXPECT_TRUE(contain_byte(data, v.size(), static_cast(1))); - EXPECT_TRUE(contain_byte(data, v.size(), static_cast(3))); - EXPECT_FALSE(contain_byte(data, v.size(), static_cast(42))); - EXPECT_FALSE(contain_byte(data, 0, static_cast(5))); + // Case 1: all zeros + { + std::vector v = {0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_FALSE(contain_one(data, v.size())); + } + + // Case 2: all ones + { + std::vector v = {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}; + const uint8_t* data = v.data(); + EXPECT_FALSE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 3: single zero at start + { + std::vector v = {0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 4: single one at start + { + std::vector v = {1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 5: single zero at end + { + std::vector v = {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 6: single one at end + { + std::vector v = {0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 7: alternating zeros and ones + { + std::vector v = {0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1, 0, 1}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 8: small size zeros (less than SIMD block) + { + std::vector v = {0, 0, 0}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_FALSE(contain_one(data, v.size())); + } + + // Case 9: small size ones + { + std::vector v = {1, 1, 1}; + const uint8_t* data = v.data(); + EXPECT_FALSE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 10: size = 1, zero + { + std::vector v = {0}; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_FALSE(contain_one(data, v.size())); + } + + // Case 11: size = 1, one + { + std::vector v = {1}; + const uint8_t* data = v.data(); + EXPECT_FALSE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 12: large size with zero in middle + { + std::vector v(100, 1); + v[50] = 0; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 13: large size with one in middle (all zeros except one) + { + std::vector v(100, 0); + v[50] = 1; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 14: SIMD block boundary (32 bytes for AVX2) + { + std::vector v(32, 1); + const uint8_t* data = v.data(); + EXPECT_FALSE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 15: SIMD block + 1 byte + { + std::vector v(33, 0); + v[32] = 1; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } + + // Case 16: multiple SIMD blocks all zeros + { + std::vector v(128, 0); + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_FALSE(contain_one(data, v.size())); + } + + // Case 17: multiple SIMD blocks with one at different position + { + std::vector v(128, 0); + v[64] = 1; + const uint8_t* data = v.data(); + EXPECT_TRUE(contain_zero(data, v.size())); + EXPECT_TRUE(contain_one(data, v.size())); + } } TEST(BitsTest, FindOne) { diff --git a/be/test/vec/columns/column_nullable_test.cpp b/be/test/vec/columns/column_nullable_test.cpp index 624a8b2d9790bc..3bcf18d306b763 100644 --- a/be/test/vec/columns/column_nullable_test.cpp +++ b/be/test/vec/columns/column_nullable_test.cpp @@ -37,8 +37,8 @@ TEST(ColumnNullableTest, NullTest) { ColumnNullable::MutablePtr null_col = create_column_nullable(500, true); EXPECT_TRUE(null_col->has_null()); - ColumnNullable::MutablePtr dst_col = - ColumnNullable::create(create_nested_column(10), ColumnUInt8::create(10)); + ColumnNullable::MutablePtr dst_col = ColumnNullable::create( + create_nested_column(10), ColumnUInt8::create(10, 0)); EXPECT_FALSE(dst_col->has_null()); ColumnInt64::MutablePtr source_col = ColumnInt64::create(); @@ -49,7 +49,7 @@ TEST(ColumnNullableTest, NullTest) { dst_col->clear(); EXPECT_FALSE(dst_col->has_null()); dst_col->insert_range_from( - *ColumnNullable::create(std::move(source_col), ColumnUInt8::create(10)), 5, 5); + *ColumnNullable::create(std::move(source_col), ColumnUInt8::create(10, 0)), 5, 5); EXPECT_FALSE(dst_col->has_null()); dst_col->clear();