Merge pull request #61632 from bigo-sg/opt_string_ascii

Process string functions XXXUTF8 'asciily' if input strings are all ascii chars
This commit is contained in:
Michael Kolupaev 2024-05-13 22:13:01 +00:00 committed by GitHub
commit 0fd52c36c6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 255 additions and 82 deletions

View File

@ -1,9 +1,13 @@
#include <Common/UTF8Helpers.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/TargetSpecific.h>
#include <Common/UTF8Helpers.h>
#include <widechar_width.h>
#include <bit>
#if USE_MULTITARGET_CODE
#include <immintrin.h>
#endif
namespace DB
{
@ -215,5 +219,71 @@ size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, s
return computeWidthImpl<BytesBeforeLimit>(data, size, prefix, limit);
}
DECLARE_DEFAULT_CODE(
bool isAllASCII(const UInt8 * data, size_t size)
{
UInt8 mask = 0;
for (size_t i = 0; i < size; ++i)
mask |= data[i];
return !(mask & 0x80);
})
DECLARE_SSE42_SPECIFIC_CODE(
/// Copy from https://github.com/lemire/fastvalidate-utf-8/blob/master/include/simdasciicheck.h
bool isAllASCII(const UInt8 * data, size_t size)
{
__m128i masks = _mm_setzero_si128();
size_t i = 0;
for (; i + 16 <= size; i += 16)
{
__m128i bytes = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data + i));
masks = _mm_or_si128(masks, bytes);
}
int mask = _mm_movemask_epi8(masks);
UInt8 tail_mask = 0;
for (; i < size; i++)
tail_mask |= data[i];
mask |= (tail_mask & 0x80);
return !mask;
})
DECLARE_AVX2_SPECIFIC_CODE(
bool isAllASCII(const UInt8 * data, size_t size)
{
__m256i masks = _mm256_setzero_si256();
size_t i = 0;
for (; i + 32 <= size; i += 32)
{
__m256i bytes = _mm256_loadu_si256(reinterpret_cast<const __m256i *>(data + i));
masks = _mm256_or_si256(masks, bytes);
}
int mask = _mm256_movemask_epi8(masks);
UInt8 tail_mask = 0;
for (; i < size; i++)
tail_mask |= data[i];
mask |= (tail_mask & 0x80);
return !mask;
})
bool isAllASCII(const UInt8* data, size_t size)
{
#if USE_MULTITARGET_CODE
if (isArchSupported(TargetArch::AVX2))
return TargetSpecific::AVX2::isAllASCII(data, size);
if (isArchSupported(TargetArch::SSE42))
return TargetSpecific::SSE42::isAllASCII(data, size);
#endif
return TargetSpecific::Default::isAllASCII(data, size);
}
}
}

View File

@ -136,7 +136,10 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept
*/
size_t computeBytesBeforeWidth(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept;
}
/// If all the characters in the string are ASCII, return true.
bool isAllASCII(const UInt8* data, size_t size);
}
}

View File

@ -48,7 +48,7 @@ struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
NumericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size)
: elements(assert_cast<ColVecType&>(elements_).getData()), offsets(offsets_)
{
offsets.resize(column_size);
offsets.resize_exact(column_size);
}
void next()
@ -69,7 +69,7 @@ struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
void reserve(size_t num_elements)
{
elements.reserve(num_elements);
elements.reserve_exact(num_elements);
}
};
@ -85,7 +85,7 @@ struct StringSink
StringSink(ColumnString & col, size_t column_size)
: elements(col.getChars()), offsets(col.getOffsets())
{
offsets.resize(column_size);
offsets.resize_exact(column_size);
}
void ALWAYS_INLINE next()
@ -108,7 +108,7 @@ struct StringSink
void reserve(size_t num_elements)
{
elements.reserve(num_elements);
elements.reserve_exact(num_elements);
}
};
@ -125,7 +125,7 @@ struct FixedStringSink
FixedStringSink(ColumnFixedString & col, size_t column_size)
: elements(col.getChars()), string_size(col.getN()), total_rows(column_size)
{
elements.resize(column_size * string_size);
elements.resize_exact(column_size * string_size);
}
void next()
@ -146,7 +146,7 @@ struct FixedStringSink
void reserve(size_t num_elements)
{
elements.reserve(num_elements);
elements.reserve_exact(num_elements);
}
};
@ -165,7 +165,7 @@ struct GenericArraySink : public ArraySinkImpl<GenericArraySink>
GenericArraySink(IColumn & elements_, ColumnArray::Offsets & offsets_, size_t column_size)
: elements(elements_), offsets(offsets_)
{
offsets.resize(column_size);
offsets.resize_exact(column_size);
}
void next()
@ -210,7 +210,7 @@ struct NullableArraySink : public ArraySink
void reserve(size_t num_elements)
{
ArraySink::reserve(num_elements);
null_map.reserve(num_elements);
null_map.reserve_exact(num_elements);
}
};

View File

@ -323,6 +323,8 @@ struct StringSource
return {&elements[prev_offset], length + elem_size > offset ? std::min(elem_size, length + elem_size - offset) : 0};
return {&elements[prev_offset + elem_size - offset], std::min(length, offset)};
}
const ColumnString::Chars & getElements() const { return elements; }
};
/// Treats Enum values as Strings, modeled after StringSource
@ -517,11 +519,12 @@ struct FixedStringSource
const UInt8 * pos;
const UInt8 * end;
size_t string_size;
const typename ColumnString::Chars & elements;
size_t row_num = 0;
size_t column_size = 0;
explicit FixedStringSource(const ColumnFixedString & col)
: string_size(col.getN())
explicit FixedStringSource(const ColumnFixedString & col) : string_size(col.getN()), elements(col.getChars())
{
const auto & chars = col.getChars();
pos = chars.data();
@ -592,6 +595,8 @@ struct FixedStringSource
return {pos, length + string_size > offset ? std::min(string_size, length + string_size - offset) : 0};
return {pos + string_size - offset, std::min(length, offset)};
}
const ColumnString::Chars & getElements() const { return elements; }
};

View File

@ -13,14 +13,14 @@ struct LowerUpperImpl
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
res_data.resize(data.size());
res_data.resize_exact(data.size());
res_offsets.assign(offsets);
array(data.data(), data.data() + data.size(), res_data.data());
}
static void vectorFixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data)
{
res_data.resize(data.size());
res_data.resize_exact(data.size());
array(data.data(), data.data() + data.size(), res_data.data());
}

View File

@ -1,8 +1,9 @@
#pragma once
#include <Columns/ColumnString.h>
#include <Functions/LowerUpperImpl.h>
#include <base/defines.h>
#include <Poco/UTF8Encoding.h>
#include <Common/UTF8Helpers.h>
#include <base/defines.h>
#ifdef __SSE2__
#include <emmintrin.h>
@ -92,7 +93,15 @@ struct LowerUpperUTF8Impl
{
if (data.empty())
return;
res_data.resize(data.size());
bool all_ascii = UTF8::isAllASCII(data.data(), data.size());
if (all_ascii)
{
LowerUpperImpl<not_case_lower_bound, not_case_upper_bound>::vector(data, offsets, res_data, res_offsets);
return;
}
res_data.resize_exact(data.size());
res_offsets.assign(offsets);
array(data.data(), data.data() + data.size(), offsets, res_data.data());
}

View File

@ -210,19 +210,18 @@ namespace
pad_string = column_pad_const->getValue<String>();
}
PaddingChars<is_utf8> padding_chars{pad_string};
auto col_res = ColumnString::create();
StringSink res_sink{*col_res, input_rows_count};
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
executeForSource(StringSource{*col}, column_length, padding_chars, res_sink);
executeForSource(StringSource{*col}, column_length, pad_string, res_sink);
else if (const ColumnFixedString * col_fixed = checkAndGetColumn<ColumnFixedString>(column_string.get()))
executeForSource(FixedStringSource{*col_fixed}, column_length, padding_chars, res_sink);
executeForSource(FixedStringSource{*col_fixed}, column_length, pad_string, res_sink);
else if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
executeForSource(ConstSource<StringSource>{*col_const}, column_length, padding_chars, res_sink);
executeForSource(ConstSource<StringSource>{*col_const}, column_length, pad_string, res_sink);
else if (const ColumnConst * col_const_fixed = checkAndGetColumnConst<ColumnFixedString>(column_string.get()))
executeForSource(ConstSource<FixedStringSource>{*col_const_fixed}, column_length, padding_chars, res_sink);
executeForSource(ConstSource<FixedStringSource>{*col_const_fixed}, column_length, pad_string, res_sink);
else
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
@ -235,23 +234,40 @@ namespace
private:
template <typename SourceStrings>
void executeForSource(
SourceStrings && strings,
const ColumnPtr & column_length,
const PaddingChars<is_utf8> & padding_chars,
StringSink & res_sink) const
void executeForSource(SourceStrings && strings, const ColumnPtr & column_length, const String & pad_string, StringSink & res_sink) const
{
if (const auto * col_const = checkAndGetColumn<ColumnConst>(column_length.get()))
executeForSourceAndLength(std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
const auto & chars = strings.getElements();
bool all_ascii = UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(pad_string.data()), pad_string.size())
&& UTF8::isAllASCII(chars.data(), chars.size());
bool is_actually_utf8 = is_utf8 && !all_ascii;
if (!is_actually_utf8)
{
PaddingChars<false> padding_chars{pad_string};
if (const auto * col_const = checkAndGetColumn<ColumnConst>(column_length.get()))
executeForSourceAndLength<false>(
std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
else
executeForSourceAndLength<false>(
std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
}
else
executeForSourceAndLength(std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
{
PaddingChars<true> padding_chars{pad_string};
if (const auto * col_const = checkAndGetColumn<ColumnConst>(column_length.get()))
executeForSourceAndLength<true>(
std::forward<SourceStrings>(strings), ConstSource<GenericValueSource>{*col_const}, padding_chars, res_sink);
else
executeForSourceAndLength<true>(
std::forward<SourceStrings>(strings), GenericValueSource{*column_length}, padding_chars, res_sink);
}
}
template <typename SourceStrings, typename SourceLengths>
template <bool is_actually_utf8, typename SourceStrings, typename SourceLengths>
void executeForSourceAndLength(
SourceStrings && strings,
SourceLengths && lengths,
const PaddingChars<is_utf8> & padding_chars,
const PaddingChars<is_actually_utf8> & padding_chars,
StringSink & res_sink) const
{
bool is_const_new_length = lengths.isConst();
@ -263,7 +279,7 @@ namespace
for (; !res_sink.isEnd(); res_sink.next(), strings.next(), lengths.next())
{
auto str = strings.getWhole();
ssize_t current_length = getLengthOfSlice<is_utf8>(str);
ssize_t current_length = getLengthOfSlice<is_actually_utf8>(str);
if (!res_sink.rowNum() || !is_const_new_length)
{
@ -293,7 +309,7 @@ namespace
}
else if (new_length < current_length)
{
str = removeSuffixFromSlice<is_utf8>(str, current_length - new_length);
str = removeSuffixFromSlice<is_actually_utf8>(str, current_length - new_length);
writeSlice(str, res_sink);
}
else if (new_length > current_length)

View File

@ -1,10 +1,10 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <base/map.h>
#include "reverse.h"
namespace DB
@ -17,42 +17,6 @@ namespace ErrorCodes
namespace
{
/** Reverse the string as a sequence of bytes.
*/
struct ReverseImpl
{
static void vector(const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
res_data.resize(data.size());
res_offsets.assign(offsets);
size_t size = offsets.size();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (size_t j = prev_offset; j < offsets[i] - 1; ++j)
res_data[j] = data[offsets[i] + prev_offset - 2 - j];
res_data[offsets[i] - 1] = 0;
prev_offset = offsets[i];
}
}
static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data)
{
res_data.resize(data.size());
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
for (size_t j = i * n; j < (i + 1) * n; ++j)
res_data[j] = data[(i * 2 + 1) * n - j - 1];
}
};
class FunctionReverse : public IFunction
{
public:

42
src/Functions/reverse.h Normal file
View File

@ -0,0 +1,42 @@
#pragma once
#include <Columns/ColumnString.h>
namespace DB
{
/** Reverse the string as a sequence of bytes.
*/
struct ReverseImpl
{
static void vector(const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
res_data.resize_exact(data.size());
res_offsets.assign(offsets);
size_t size = offsets.size();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (size_t j = prev_offset; j < offsets[i] - 1; ++j)
res_data[j] = data[offsets[i] + prev_offset - 2 - j];
res_data[offsets[i] - 1] = 0;
prev_offset = offsets[i];
}
}
static void vectorFixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data)
{
res_data.resize_exact(data.size());
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
for (size_t j = i * n; j < (i + 1) * n; ++j)
res_data[j] = data[(i * 2 + 1) * n - j - 1];
}
};
}

View File

@ -1,7 +1,9 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringToString.h>
#include <Common/UTF8Helpers.h>
#include "reverse.h"
namespace DB
@ -25,10 +27,18 @@ struct ReverseUTF8Impl
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
bool all_ascii = UTF8::isAllASCII(data.data(), data.size());
if (all_ascii)
{
ReverseImpl::vector(data, offsets, res_data, res_offsets);
return;
}
res_data.resize(data.size());
res_offsets.assign(offsets);
size_t size = offsets.size();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{

View File

@ -148,9 +148,23 @@ public:
if constexpr (is_utf8)
{
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count);
{
bool all_ascii = UTF8::isAllASCII(col->getChars().data(), col->getChars().size());
if (all_ascii)
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, StringSource(*col), input_rows_count);
else
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, UTF8StringSource(*col), input_rows_count);
}
if (const ColumnConst * col_const = checkAndGetColumnConst<ColumnString>(column_string.get()))
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
{
StringRef str_ref = col_const->getDataAt(0);
bool all_ascii = UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(str_ref.data), str_ref.size);
if (all_ascii)
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<StringSource>(*col_const), input_rows_count);
else
return executeForSource(column_offset, column_length, column_offset_const, column_length_const, offset, length, ConstSource<UTF8StringSource>(*col_const), input_rows_count);
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
}
else

View File

@ -129,8 +129,10 @@ namespace
res_data.reserve(str_column->getChars().size() / 2);
res_offsets.reserve(rows);
bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
for (size_t i = 0; i < rows; ++i)
{
@ -140,10 +142,12 @@ namespace
StringRef res_ref;
if constexpr (!is_utf8)
res_ref = substringIndex(str_ref, delim[0], count);
else if (all_ascii)
res_ref = substringIndex(str_ref, delim[0], count);
else
res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count);
appendToResultColumn(res_ref, res_data, res_offsets);
appendToResultColumn<true>(res_ref, res_data, res_offsets);
}
}
@ -158,8 +162,10 @@ namespace
res_data.reserve(str_column->getChars().size() / 2);
res_offsets.reserve(rows);
bool all_ascii = UTF8::isAllASCII(str_column->getChars().data(), str_column->getChars().size())
&& UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
for (size_t i = 0; i < rows; ++i)
{
@ -168,10 +174,12 @@ namespace
StringRef res_ref;
if constexpr (!is_utf8)
res_ref = substringIndex(str_ref, delim[0], count);
else if (all_ascii)
res_ref = substringIndex(str_ref, delim[0], count);
else
res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count);
appendToResultColumn(res_ref, res_data, res_offsets);
appendToResultColumn<true>(res_ref, res_data, res_offsets);
}
}
@ -186,8 +194,10 @@ namespace
res_data.reserve(str.size() * rows / 2);
res_offsets.reserve(rows);
bool all_ascii = UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(str.data()), str.size())
&& UTF8::isAllASCII(reinterpret_cast<const UInt8 *>(delim.data()), delim.size());
std::unique_ptr<PositionCaseSensitiveUTF8::SearcherInBigHaystack> searcher
= !is_utf8 ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
= !is_utf8 || all_ascii ? nullptr : std::make_unique<PositionCaseSensitiveUTF8::SearcherInBigHaystack>(delim.data(), delim.size());
StringRef str_ref{str.data(), str.size()};
for (size_t i = 0; i < rows; ++i)
@ -197,18 +207,26 @@ namespace
StringRef res_ref;
if constexpr (!is_utf8)
res_ref = substringIndex(str_ref, delim[0], count);
else if (all_ascii)
res_ref = substringIndex(str_ref, delim[0], count);
else
res_ref = substringIndexUTF8(searcher.get(), str_ref, delim, count);
appendToResultColumn(res_ref, res_data, res_offsets);
appendToResultColumn<false>(res_ref, res_data, res_offsets);
}
}
template <bool padded>
static void appendToResultColumn(const StringRef & res_ref, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets)
{
size_t res_offset = res_data.size();
res_data.resize(res_offset + res_ref.size + 1);
memcpy(&res_data[res_offset], res_ref.data, res_ref.size);
if constexpr (padded)
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], res_ref.data, res_ref.size);
else
memcpy(&res_data[res_offset], res_ref.data, res_ref.size);
res_offset += res_ref.size;
res_data[res_offset] = 0;
++res_offset;

View File

@ -46,8 +46,8 @@ public:
ColumnString::Offsets & res_offsets)
{
size_t size = offsets.size();
res_offsets.resize(size);
res_data.reserve(data.size());
res_offsets.resize_exact(size);
res_data.reserve_exact(data.size());
size_t prev_offset = 0;
size_t res_offset = 0;

View File

@ -0,0 +1,22 @@
<test>
<query>select substringUTF8(materialize('hello world'), 2, 5) from numbers(10000000)</query>
<query>select substringUTF8(materialize('hello 世界'), 2, 5) from numbers(10000000)</query>
<query>select substringIndexUTF8(materialize('www.clickhouse.com'), '.', 2) from numbers(10000000)</query>
<query>select substringIndexUTF8(materialize('官网www.clickhouse.com'), '.', 2) from numbers(10000000)</query>
<query>select reverseUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select reverseUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select lowerUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select lowerUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select upperUTF8(materialize('hello world')) from numbers(10000000)</query>
<query>select upperUTF8(materialize('hello 世界')) from numbers(10000000)</query>
<query>select leftPadUTF8(materialize('hello '), 10, ',') from numbers(10000000)</query>
<query>select leftPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000)</query>
<query>select rightPadUTF8(materialize('hello '), 10, ',') from numbers(10000000)</query>
<query>select rightPadUTF8(materialize('hello '), 10, '世界') from numbers(10000000)</query>
</test>