Skip to content

Commit 8e653fe

Browse files
committed
update fix
format update format fix update fix fix
1 parent b884481 commit 8e653fe

File tree

9 files changed

+69
-62
lines changed

9 files changed

+69
-62
lines changed

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

Lines changed: 3 additions & 4 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_one(
118+
nullable.get_null_map_data().data() + 1, nullable.size() - 1);
119119
}
120120
}
121121
}
@@ -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_one(filter.data() + offset, end - offset);
103101
}
104102
end = offset;
105103
}

be/src/util/simd/bits.h

Lines changed: 41 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -221,14 +221,6 @@ static size_t find_byte(const T* data, size_t start, size_t end, T byte) {
221221
return (T*)p - data;
222222
}
223223

224-
template <typename T>
225-
bool contain_byte(const T* __restrict data, const size_t length, const signed char byte) {
226-
if (length == 0) {
227-
return false;
228-
}
229-
return nullptr != std::memchr(reinterpret_cast<const void*>(data), byte, length);
230-
}
231-
232224
inline size_t find_one(const std::vector<uint8_t>& vec, size_t start) {
233225
return find_byte<uint8_t>(vec, start, 1);
234226
}
@@ -243,24 +235,55 @@ inline size_t find_zero(const std::vector<uint8_t>& vec, size_t start) {
243235

244236
inline bool contain_one(const uint8_t* __restrict data, size_t size) {
245237
size_t i = 0;
246-
#if defined(__SSE2__)
247-
{
248-
const __m128i zero = _mm_setzero_si128();
249-
__m128i acc = zero;
250-
for (; i + 15 < size; i += 16) {
251-
acc = _mm_or_si128(acc, _mm_loadu_si128(reinterpret_cast<const __m128i*>(data + i)));
238+
#if defined(__AVX2__)
239+
for (; i + 32 <= size; i += 32) {
240+
__m256i chunk = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(data + i));
241+
if (!_mm256_testz_si256(chunk, chunk)) {
242+
return true;
243+
}
244+
}
245+
#elif defined(__SSE2__)
246+
const __m128i zero = _mm_setzero_si128();
247+
for (; i + 16 <= size; i += 16) {
248+
__m128i chunk = _mm_loadu_si128(reinterpret_cast<const __m128i*>(data + i));
249+
if (_mm_movemask_epi8(_mm_cmpeq_epi8(chunk, zero)) != 0xFFFF) {
250+
return true;
252251
}
252+
}
253+
#endif
254+
for (; i < size; ++i) {
255+
if (data[i]) {
256+
return true;
257+
}
258+
}
259+
return false;
260+
}
253261

254-
if (_mm_movemask_epi8(_mm_cmpeq_epi8(acc, zero)) != 0xFFFF) {
262+
inline bool contain_zero(const uint8_t* __restrict data, size_t size) {
263+
size_t i = 0;
264+
#if defined(__AVX2__)
265+
const __m256i zero = _mm256_setzero_si256();
266+
for (; i + 32 <= size; i += 32) {
267+
__m256i chunk = _mm256_loadu_si256(reinterpret_cast<const __m256i*>(data + i));
268+
if (_mm256_movemask_epi8(_mm256_cmpeq_epi8(chunk, zero)) != 0) {
269+
return true;
270+
}
271+
}
272+
#elif defined(__SSE2__)
273+
const __m128i zero = _mm_setzero_si128();
274+
for (; i + 16 <= size; i += 16) {
275+
__m128i chunk = _mm_loadu_si128(reinterpret_cast<const __m128i*>(data + i));
276+
if (_mm_movemask_epi8(_mm_cmpeq_epi8(chunk, zero)) != 0) {
255277
return true;
256278
}
257279
}
258280
#endif
259-
uint8_t tail_acc = 0;
260281
for (; i < size; ++i) {
261-
tail_acc |= data[i];
282+
if (!data[i]) {
283+
return true;
284+
}
262285
}
263-
return tail_acc != 0;
286+
return false;
264287
}
265288

266289
} // namespace doris::simd

be/src/vec/columns/column_nullable.cpp

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

264264
void ColumnNullable::serialize(StringRef* keys, size_t num_rows) const {
265-
const bool has_null = simd::contain_byte(get_null_map_data().data(), num_rows, 1);
265+
const bool has_null = simd::contain_one(get_null_map_data().data(), num_rows);
266266
const auto* __restrict null_map =
267267
assert_cast<const ColumnUInt8&>(get_null_map_column()).get_data().data();
268268
_nested_column->serialize_with_nullable(keys, num_rows, has_null, null_map);
@@ -580,11 +580,11 @@ void ColumnNullable::sort_column(const ColumnSorter* sorter, EqualFlags& flags,
580580
}
581581

582582
bool ColumnNullable::only_null() const {
583-
return !simd::contain_byte(get_null_map_data().data(), size(), 0);
583+
return !simd::contain_zero(get_null_map_data().data(), size());
584584
}
585585

586586
bool ColumnNullable::has_null(size_t begin, size_t end) const {
587-
return simd::contain_byte(get_null_map_data().data() + begin, end - begin, 1);
587+
return simd::contain_one(get_null_map_data().data() + begin, end - begin);
588588
}
589589

590590
bool ColumnNullable::has_null() const {

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

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,10 @@ inline HashKeyType get_hash_key_type_with_fixed(size_t size) {
8080
}
8181

8282
inline HashKeyType get_hash_key_type_fixed(const std::vector<vectorized::DataTypePtr>& data_types) {
83+
if (data_types.size() >= vectorized::BITSIZE) {
84+
return HashKeyType::serialized;
85+
}
86+
8387
bool has_null = false;
8488
size_t key_byte_size = 0;
8589

@@ -94,8 +98,7 @@ inline HashKeyType get_hash_key_type_fixed(const std::vector<vectorized::DataTyp
9498
}
9599
}
96100

97-
size_t bitmap_size = has_null ? vectorized::get_bitmap_size(data_types.size()) : 0;
98-
return get_hash_key_type_with_fixed(bitmap_size + key_byte_size);
101+
return get_hash_key_type_with_fixed(has_null + key_byte_size);
99102
}
100103

101104
inline HashKeyType get_hash_key_type(const std::vector<vectorized::DataTypePtr>& data_types) {

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

Lines changed: 10 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
#pragma once
1919

20+
#include <algorithm>
2021
#include <cstdint>
2122
#include <type_traits>
2223
#include <utility>
@@ -291,10 +292,6 @@ struct MethodSerialized : public MethodBase<TData> {
291292
}
292293
};
293294

294-
inline size_t get_bitmap_size(size_t key_number) {
295-
return (key_number + BITSIZE - 1) / BITSIZE;
296-
}
297-
298295
template <typename TData>
299296
struct MethodStringNoCache : public MethodBase<TData> {
300297
using Base = MethodBase<TData>;
@@ -562,7 +559,7 @@ struct MethodKeysFixed : public MethodBase<TData> {
562559
template <typename T>
563560
void pack_fixeds(size_t row_numbers, const ColumnRawPtrs& key_columns,
564561
const ColumnRawPtrs& nullmap_columns, DorisVector<T>& result) {
565-
size_t bitmap_size = get_bitmap_size(nullmap_columns.size());
562+
size_t bitmap_size = nullmap_columns.empty() ? 0 : 1;
566563
if (bitmap_size) {
567564
// set size to 0 at first, then use resize to call default constructor on index included from [0, row_numbers) to reset all memory
568565
// only need to reset the memory used to bitmap
@@ -587,16 +584,12 @@ struct MethodKeysFixed : public MethodBase<TData> {
587584
has_null_column[j] = simd::contain_one(data, row_numbers);
588585
if (has_null_column[j]) {
589586
nullmap_datas.emplace_back(data);
590-
bit_offsets.emplace_back(j % BITSIZE);
587+
bit_offsets.emplace_back(j);
591588
}
592589
}
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-
590+
constexpr_int_match<1, BITSIZE, PackNullmapsReducer>::run(
591+
int(nullmap_datas.size()), nullmap_datas.data(), bit_offsets.data(),
592+
row_numbers, sizeof(T), reinterpret_cast<uint8_t*>(result_data));
600593
offset += bitmap_size;
601594
}
602595

@@ -657,6 +650,7 @@ struct MethodKeysFixed : public MethodBase<TData> {
657650
void init_serialized_keys(const ColumnRawPtrs& key_columns, uint32_t num_rows,
658651
const uint8_t* null_map = nullptr, bool is_join = false,
659652
bool is_build = false, uint32_t bucket_size = 0) override {
653+
CHECK(key_columns.size() <= BITSIZE);
660654
ColumnRawPtrs actual_columns;
661655
ColumnRawPtrs null_maps;
662656
actual_columns.reserve(key_columns.size());
@@ -694,14 +688,8 @@ struct MethodKeysFixed : public MethodBase<TData> {
694688

695689
void insert_keys_into_columns(std::vector<typename Base::Key>& input_keys,
696690
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.
698-
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-
}
691+
size_t pos = std::ranges::any_of(key_columns,
692+
[](const auto& col) { return col->is_nullable(); });
705693

706694
for (size_t i = 0; i < key_columns.size(); ++i) {
707695
size_t size = key_sizes[i];
@@ -720,11 +708,8 @@ struct MethodKeysFixed : public MethodBase<TData> {
720708

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

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_one(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
}

be/test/vec/columns/column_nullable_test.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -37,8 +37,8 @@ TEST(ColumnNullableTest, NullTest) {
3737
ColumnNullable::MutablePtr null_col = create_column_nullable<TYPE_BIGINT>(500, true);
3838
EXPECT_TRUE(null_col->has_null());
3939

40-
ColumnNullable::MutablePtr dst_col =
41-
ColumnNullable::create(create_nested_column<TYPE_BIGINT>(10), ColumnUInt8::create(10));
40+
ColumnNullable::MutablePtr dst_col = ColumnNullable::create(
41+
create_nested_column<TYPE_BIGINT>(10), ColumnUInt8::create(10, 0));
4242
EXPECT_FALSE(dst_col->has_null());
4343

4444
ColumnInt64::MutablePtr source_col = ColumnInt64::create();
@@ -49,7 +49,7 @@ TEST(ColumnNullableTest, NullTest) {
4949
dst_col->clear();
5050
EXPECT_FALSE(dst_col->has_null());
5151
dst_col->insert_range_from(
52-
*ColumnNullable::create(std::move(source_col), ColumnUInt8::create(10)), 5, 5);
52+
*ColumnNullable::create(std::move(source_col), ColumnUInt8::create(10, 0)), 5, 5);
5353
EXPECT_FALSE(dst_col->has_null());
5454

5555
dst_col->clear();

0 commit comments

Comments
 (0)