Skip to content

Commit 7e3530f

Browse files
committed
update fix
format update format
1 parent ce36e33 commit 7e3530f

File tree

11 files changed

+217
-70
lines changed

11 files changed

+217
-70
lines changed

be/src/pipeline/exec/hashjoin_build_sink.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -192,7 +192,7 @@ struct ProcessHashTableBuild {
192192
bool* has_null_key) {
193193
if (null_map) {
194194
// first row is mocked and is null
195-
if (simd::contain_one(null_map->data() + 1, _rows - 1)) {
195+
if (simd::contain_non_zero(null_map->data() + 1, _rows - 1)) {
196196
*has_null_key = true;
197197
}
198198
if (short_circuit_for_null && *has_null_key) {

be/src/pipeline/exec/join/process_hash_table_probe_impl.h

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -114,8 +114,8 @@ void ProcessHashTableProbe<JoinOpType>::build_side_output_column(vectorized::Mut
114114
_build_column_has_null[i] = false;
115115
if (_right_output_slot_flags[i] && column.is_nullable()) {
116116
const auto& nullable = assert_cast<const vectorized::ColumnNullable&>(column);
117-
_build_column_has_null[i] = !simd::contain_byte(
118-
nullable.get_null_map_data().data() + 1, nullable.size() - 1, 1);
117+
_build_column_has_null[i] = !simd::contain_non_zero(
118+
nullable.get_null_map_data().data() + 1, nullable.size() - 1);
119119
}
120120
}
121121
}
@@ -192,7 +192,7 @@ typename HashTableType::State ProcessHashTableProbe<JoinOpType>::_init_probe_sid
192192
hash_table_ctx.arena.clear();
193193
// In order to make the null keys equal when using single null eq, all null keys need to be set to default value.
194194
if (_parent->_probe_columns.size() == 1 && null_map) {
195-
if (simd::contain_one(null_map, probe_rows)) {
195+
if (simd::contain_non_zero(null_map, probe_rows)) {
196196
_parent->_probe_columns[0]->assume_mutable()->replace_column_null_data(null_map);
197197
}
198198
}
@@ -368,8 +368,7 @@ Status ProcessHashTableProbe<JoinOpType>::finalize_block_with_filter(
368368
}
369369
const auto& column_filter =
370370
assert_cast<const vectorized::ColumnUInt8*>(filter_ptr.get())->get_data();
371-
bool need_filter =
372-
simd::count_zero_num((int8_t*)column_filter.data(), column_filter.size()) != 0;
371+
bool need_filter = simd::contain_zero(column_filter.data(), column_filter.size());
373372
if (need_filter) {
374373
row_indexs.filter(column_filter);
375374
}

be/src/pipeline/exec/nested_loop_join_probe_operator.h

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -97,9 +97,7 @@ class NestedLoopJoinProbeLocalState final
9797
}
9898
if (!_cur_probe_row_visited_flags[i]) {
9999
_cur_probe_row_visited_flags[i] =
100-
simd::contain_byte<uint8_t>(filter.data() + offset, end - offset, 1)
101-
? 1
102-
: 0;
100+
simd::contain_non_zero(filter.data() + offset, end - offset);
103101
}
104102
end = offset;
105103
}

be/src/util/simd/bits.h

Lines changed: 54 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -261,14 +261,6 @@ static size_t find_byte(const T* data, size_t start, size_t end, T byte) {
261261
return (T*)p - data;
262262
}
263263

264-
template <typename T>
265-
bool contain_byte(const T* __restrict data, const size_t length, const signed char byte) {
266-
if (length == 0) {
267-
return false;
268-
}
269-
return nullptr != std::memchr(reinterpret_cast<const void*>(data), byte, length);
270-
}
271-
272264
inline size_t find_one(const std::vector<uint8_t>& vec, size_t start) {
273265
return find_byte<uint8_t>(vec, start, 1);
274266
}
@@ -281,26 +273,69 @@ inline size_t find_zero(const std::vector<uint8_t>& vec, size_t start) {
281273
return find_byte<uint8_t>(vec, start, 0);
282274
}
283275

284-
inline bool contain_one(const uint8_t* __restrict data, size_t size) {
276+
namespace detail {
277+
// Checker for contain_non_zero: detect if any non-zero value exists
278+
struct ContainNonZeroChecker {
279+
static inline bool check_simd_result(uint32_t mask) {
280+
#if defined(__AVX2__)
281+
return mask != 0xFFFFFFFF;
282+
#else
283+
return mask != 0xFFFF;
284+
#endif
285+
}
286+
287+
static inline bool check_tail(uint8_t byte) { return byte != 0; }
288+
};
289+
290+
// Checker for contain_zero: detect if any zero value exists
291+
struct ContainZeroChecker {
292+
static inline bool check_simd_result(uint32_t mask) { return mask != 0; }
293+
294+
static inline bool check_tail(uint8_t byte) { return byte == 0; }
295+
};
296+
297+
template <typename Checker>
298+
inline bool contain_value(const uint8_t* __restrict data, size_t size) {
285299
size_t i = 0;
286-
#if defined(__SSE2__)
300+
#if defined(__AVX2__)
301+
{
302+
const __m256i zero = _mm256_setzero_si256();
303+
for (; i + 31 < size; i += 32) {
304+
__m256i chunk = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(data + i));
305+
uint32_t mask = _mm256_movemask_epi8(_mm256_cmpeq_epi8(chunk, zero));
306+
if (Checker::check_simd_result(mask)) {
307+
return true;
308+
}
309+
}
310+
}
311+
#elif defined(__SSE2__)
287312
{
288313
const __m128i zero = _mm_setzero_si128();
289-
__m128i acc = zero;
290314
for (; i + 15 < size; i += 16) {
291-
acc = _mm_or_si128(acc, _mm_loadu_si128(reinterpret_cast<const __m128i*>(data + i)));
292-
}
293-
294-
if (_mm_movemask_epi8(_mm_cmpeq_epi8(acc, zero)) != 0xFFFF) {
295-
return true;
315+
__m128i chunk = _mm_loadu_si128(reinterpret_cast<const __m128i*>(data + i));
316+
uint32_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8(chunk, zero));
317+
if (Checker::check_simd_result(mask)) {
318+
return true;
319+
}
296320
}
297321
}
298322
#endif
299-
uint8_t tail_acc = 0;
300323
for (; i < size; ++i) {
301-
tail_acc |= data[i];
324+
if (Checker::check_tail(data[i])) {
325+
return true;
326+
}
302327
}
303-
return tail_acc != 0;
328+
return false;
329+
}
330+
} // namespace detail
331+
332+
// Check if there is any non-zero value in the data
333+
inline bool contain_non_zero(const uint8_t* __restrict data, size_t size) {
334+
return detail::contain_value<detail::ContainNonZeroChecker>(data, size);
335+
}
336+
337+
inline bool contain_zero(const uint8_t* __restrict data, size_t size) {
338+
return detail::contain_value<detail::ContainZeroChecker>(data, size);
304339
}
305340

306341
} // namespace doris::simd

be/src/vec/columns/column_nullable.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -280,7 +280,7 @@ size_t ColumnNullable::serialize_impl(char* pos, const size_t row) const {
280280
}
281281

282282
void ColumnNullable::serialize(StringRef* keys, size_t num_rows) const {
283-
const bool has_null = simd::contain_byte(get_null_map_data().data(), num_rows, 1);
283+
const bool has_null = simd::contain_non_zero(get_null_map_data().data(), num_rows);
284284
const auto* __restrict null_map =
285285
assert_cast<const ColumnUInt8&>(get_null_map_column()).get_data().data();
286286
_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,
598598
}
599599

600600
bool ColumnNullable::only_null() const {
601-
return !simd::contain_byte(get_null_map_data().data(), size(), 0);
601+
return !simd::contain_zero(get_null_map_data().data(), size());
602602
}
603603

604604
bool ColumnNullable::has_null(size_t begin, size_t end) const {
605-
return simd::contain_byte(get_null_map_data().data() + begin, end - begin, 1);
605+
return simd::contain_non_zero(get_null_map_data().data() + begin, end - begin);
606606
}
607607

608608
bool ColumnNullable::has_null() const {

be/src/vec/common/hash_table/hash_key_type.h

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -94,12 +94,12 @@ inline HashKeyType get_hash_key_type_fixed(const std::vector<vectorized::DataTyp
9494
}
9595
}
9696

97-
size_t bitmap_size = has_null ? vectorized::get_bitmap_size(data_types.size()) : 0;
97+
size_t bitmap_size = has_null ? 1 : 0;
9898
return get_hash_key_type_with_fixed(bitmap_size + key_byte_size);
9999
}
100100

101101
inline HashKeyType get_hash_key_type(const std::vector<vectorized::DataTypePtr>& data_types) {
102-
if (data_types.size() > 1) {
102+
if (data_types.size() > 1 && data_types.size() <= vectorized::BITSIZE) {
103103
return get_hash_key_type_fixed(data_types);
104104
}
105105
if (data_types.empty()) {

be/src/vec/common/hash_table/hash_map_context.h

Lines changed: 7 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -291,10 +291,6 @@ struct MethodSerialized : public MethodBase<TData> {
291291
}
292292
};
293293

294-
inline size_t get_bitmap_size(size_t key_number) {
295-
return (key_number + BITSIZE - 1) / BITSIZE;
296-
}
297-
298294
template <typename TData>
299295
struct MethodStringNoCache : public MethodBase<TData> {
300296
using Base = MethodBase<TData>;
@@ -562,7 +558,7 @@ struct MethodKeysFixed : public MethodBase<TData> {
562558
template <typename T>
563559
void pack_fixeds(size_t row_numbers, const ColumnRawPtrs& key_columns,
564560
const ColumnRawPtrs& nullmap_columns, DorisVector<T>& result) {
565-
size_t bitmap_size = get_bitmap_size(nullmap_columns.size());
561+
size_t bitmap_size = nullmap_columns.empty() ? 0 : 1;
566562
if (bitmap_size) {
567563
// set size to 0 at first, then use resize to call default constructor on index included from [0, row_numbers) to reset all memory
568564
// only need to reset the memory used to bitmap
@@ -584,19 +580,15 @@ struct MethodKeysFixed : public MethodBase<TData> {
584580
const uint8_t* __restrict data =
585581
assert_cast<const ColumnUInt8&>(*nullmap_columns[j]).get_data().data();
586582

587-
has_null_column[j] = simd::contain_one(data, row_numbers);
583+
has_null_column[j] = simd::contain_non_zero(data, row_numbers);
588584
if (has_null_column[j]) {
589585
nullmap_datas.emplace_back(data);
590586
bit_offsets.emplace_back(j % BITSIZE);
591587
}
592588
}
593-
for (size_t j = 0, bucket = 0; j < nullmap_datas.size(); j += BITSIZE, bucket++) {
594-
int column_batch = std::min(BITSIZE, (int)(nullmap_datas.size() - j));
595-
constexpr_int_match<1, BITSIZE, PackNullmapsReducer>::run(
596-
column_batch, nullmap_datas.data() + j, bit_offsets.data() + j, row_numbers,
597-
sizeof(T), reinterpret_cast<uint8_t*>(result_data + bucket));
598-
}
599-
589+
constexpr_int_match<1, BITSIZE, PackNullmapsReducer>::run(
590+
int(nullmap_datas.size()), nullmap_datas.data(), bit_offsets.data(),
591+
row_numbers, sizeof(T), reinterpret_cast<uint8_t*>(result_data));
600592
offset += bitmap_size;
601593
}
602594

@@ -657,6 +649,7 @@ struct MethodKeysFixed : public MethodBase<TData> {
657649
void init_serialized_keys(const ColumnRawPtrs& key_columns, uint32_t num_rows,
658650
const uint8_t* null_map = nullptr, bool is_join = false,
659651
bool is_build = false, uint32_t bucket_size = 0) override {
652+
CHECK(key_columns.size() <= BITSIZE);
660653
ColumnRawPtrs actual_columns;
661654
ColumnRawPtrs null_maps;
662655
actual_columns.reserve(key_columns.size());
@@ -694,14 +687,7 @@ struct MethodKeysFixed : public MethodBase<TData> {
694687

695688
void insert_keys_into_columns(std::vector<typename Base::Key>& input_keys,
696689
MutableColumns& key_columns, const uint32_t num_rows) override {
697-
// In any hash key value, column values to be read start just after the bitmap, if it exists.
698690
size_t pos = 0;
699-
for (size_t i = 0; i < key_columns.size(); ++i) {
700-
if (key_columns[i]->is_nullable()) {
701-
pos = get_bitmap_size(key_columns.size());
702-
break;
703-
}
704-
}
705691

706692
for (size_t i = 0; i < key_columns.size(); ++i) {
707693
size_t size = key_sizes[i];
@@ -720,11 +706,8 @@ struct MethodKeysFixed : public MethodBase<TData> {
720706

721707
// The current column is nullable. Check if the value of the
722708
// corresponding key is nullable. Update the null map accordingly.
723-
size_t bucket = i / BITSIZE;
724-
size_t offset = i % BITSIZE;
725709
for (size_t j = 0; j < num_rows; j++) {
726-
nullmap[j] =
727-
(reinterpret_cast<const UInt8*>(&input_keys[j])[bucket] >> offset) & 1;
710+
nullmap[j] = (*reinterpret_cast<const UInt8*>(&input_keys[j]) >> i) & 1;
728711
}
729712
} else {
730713
// key_columns is a mutable element. However, when accessed through get_raw_data().data,

be/src/vec/core/column_with_type_and_name.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,7 @@ ColumnWithTypeAndName ColumnWithTypeAndName::unnest_nullable(
129129
const auto& null_map = source_column->get_null_map_data();
130130
// only need to mutate nested column, avoid to copy nullmap
131131
auto mutable_nested_col = (*std::move(nested_column)).mutate();
132-
if (simd::contain_one(null_map.data(), null_map.size())) {
132+
if (simd::contain_non_zero(null_map.data(), null_map.size())) {
133133
mutable_nested_col->replace_column_null_data(null_map.data());
134134
}
135135

be/src/vec/data_types/serde/data_type_map_serde.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -352,8 +352,8 @@ Status DataTypeMapSerDe::write_column_to_arrow(const IColumn& column, const Null
352352
if ((null_map && (*null_map)[r])) {
353353
RETURN_IF_ERROR(checkArrowStatus(builder.AppendNull(), column.get_name(),
354354
array_builder->type()->name()));
355-
} else if (simd::contain_byte(keys_nullmap_data + offsets[r - 1],
356-
offsets[r] - offsets[r - 1], 1)) {
355+
} else if (simd::contain_non_zero(keys_nullmap_data + offsets[r - 1],
356+
offsets[r] - offsets[r - 1])) {
357357
// arrow do not support key is null, so we ignore the null key-value
358358
MutableColumnPtr key_mutable_data = nested_keys_column.clone_empty();
359359
MutableColumnPtr value_mutable_data = nested_values_column.clone_empty();

be/src/vec/functions/function_jsonb.cpp

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -727,8 +727,7 @@ class FunctionJsonbExtractPath : public IFunction {
727727
VectorizedUtils::update_null_map(*result_null_map, *path_null_map, path_const);
728728
}
729729

730-
if (0 == simd::count_zero_num(reinterpret_cast<const int8_t*>(result_null_map->data()),
731-
input_rows_count)) {
730+
if (!simd::contain_zero(result_null_map->data(), input_rows_count)) {
732731
return create_all_null_result();
733732
}
734733
}

0 commit comments

Comments
 (0)