Merge pull request #68523 from rschu1ze/improve_upper_lower-2

Improve compatibility of `upper/lowerUTF8` with Spark
This commit is contained in:
Robert Schulze 2024-08-20 10:30:06 +00:00 committed by GitHub
commit 67344be073
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
23 changed files with 161 additions and 263 deletions

2
.gitmodules vendored
View File

@ -108,7 +108,7 @@
url = https://github.com/ClickHouse/icudata
[submodule "contrib/icu"]
path = contrib/icu
url = https://github.com/unicode-org/icu
url = https://github.com/ClickHouse/icu
[submodule "contrib/flatbuffers"]
path = contrib/flatbuffers
url = https://github.com/ClickHouse/flatbuffers

2
contrib/icu vendored

@ -1 +1 @@
Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625
Subproject commit 4216173eeeb39c1d4caaa54a68860e800412d273

View File

@ -92,3 +92,8 @@ endif()
clickhouse_add_executable (check_pointer_valid check_pointer_valid.cpp)
target_link_libraries (check_pointer_valid PRIVATE clickhouse_common_io clickhouse_common_config)
if (TARGET ch_contrib::icu)
clickhouse_add_executable (utf8_upper_lower utf8_upper_lower.cpp)
target_link_libraries (utf8_upper_lower PRIVATE ch_contrib::icu)
endif ()

View File

@ -0,0 +1,27 @@
#include <iostream>
#include <unicode/unistr.h>
std::string utf8_to_lower(const std::string & input)
{
icu::UnicodeString unicodeInput(input.c_str(), "UTF-8");
unicodeInput.toLower();
std::string output;
unicodeInput.toUTF8String(output);
return output;
}
std::string utf8_to_upper(const std::string & input)
{
icu::UnicodeString unicodeInput(input.c_str(), "UTF-8");
unicodeInput.toUpper();
std::string output;
unicodeInput.toUTF8String(output);
return output;
}
int main()
{
std::string input = "ır";
std::cout << "upper:" << utf8_to_upper(input) << std::endl;
return 0;
}

View File

@ -1,7 +1,6 @@
#pragma once
#include <Columns/ColumnString.h>
namespace DB
{

View File

@ -1,15 +1,14 @@
#pragma once
#include "config.h"
#if USE_ICU
#include <Columns/ColumnString.h>
#include <Functions/LowerUpperImpl.h>
#include <base/defines.h>
#include <Poco/UTF8Encoding.h>
#include <base/find_symbols.h>
#include <unicode/unistr.h>
#include <Common/StringUtils.h>
#include <Common/UTF8Helpers.h>
#ifdef __SSE2__
#include <emmintrin.h>
#endif
namespace DB
{
@ -19,71 +18,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
/// xor or do nothing
template <bool>
UInt8 xor_or_identity(const UInt8 c, const int mask)
{
return c ^ mask;
}
template <>
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
{
return c;
}
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
template <bool to_lower>
inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst)
{
if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// ѐёђѓєѕіїјљњћќѝўџ
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// А
*dst++ = *src++;
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu))
{
/// а-п
*dst++ = *src++;
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu))
{
/// Р
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// р
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
}
/** If the string contains UTF-8 encoded text, convert it to the lower (upper) case.
* Note: It is assumed that after the character is converted to another case,
* the length of its multibyte sequence in UTF-8 does not change.
* Otherwise, the behavior is undefined.
*/
template <char not_case_lower_bound,
char not_case_upper_bound,
int to_case(int),
void cyrillic_to_case(const UInt8 *&, UInt8 *&)>
template <char not_case_lower_bound, char not_case_upper_bound, bool upper>
struct LowerUpperUTF8Impl
{
static void vector(
@ -103,180 +38,46 @@ struct LowerUpperUTF8Impl
return;
}
res_data.resize_exact(data.size());
res_offsets.assign(offsets);
array(data.data(), data.data() + data.size(), offsets, res_data.data());
res_data.resize(data.size());
res_offsets.resize_exact(offsets.size());
String output;
size_t curr_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
const auto * data_start = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
size_t size = offsets[i] - offsets[i - 1];
icu::UnicodeString input(data_start, static_cast<int32_t>(size), "UTF-8");
if constexpr (upper)
input.toUpper();
else
input.toLower();
output.clear();
input.toUTF8String(output);
/// For valid UTF-8 input strings, ICU sometimes produces output with extra '\0's at the end. Only the data before the first
/// '\0' is valid. It the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this
/// case, the behavior is also reasonable.
const char * res_end = find_last_not_symbols_or_null<'\0'>(output.data(), output.data() + output.size());
size_t valid_size = res_end ? res_end - output.data() + 1 : 0;
res_data.resize(curr_offset + valid_size + 1);
memcpy(&res_data[curr_offset], output.data(), valid_size);
res_data[curr_offset + valid_size] = 0;
curr_offset += valid_size + 1;
res_offsets[i] = curr_offset;
}
}
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument");
}
/** Converts a single code point starting at `src` to desired case, storing result starting at `dst`.
* `src` and `dst` are incremented by corresponding sequence lengths. */
static bool toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst, bool partial)
{
if (src[0] <= ascii_upper_bound)
{
if (*src >= not_case_lower_bound && *src <= not_case_upper_bound)
*dst++ = *src++ ^ flip_case_mask;
else
*dst++ = *src++;
}
else if (src + 1 < src_end
&& ((src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0xBFu)) || (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x9Fu))))
{
cyrillic_to_case(src, dst);
}
else if (src + 1 < src_end && src[0] == 0xC2u)
{
/// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
*dst++ = *src++;
*dst++ = *src++;
}
else if (src + 2 < src_end && src[0] == 0xE2u)
{
/// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
*dst++ = *src++;
*dst++ = *src++;
}
else
{
size_t src_sequence_length = UTF8::seqLength(*src);
/// In case partial buffer was passed (due to SSE optimization)
/// we cannot convert it with current src_end, but we may have more
/// bytes to convert and eventually got correct symbol.
if (partial && src_sequence_length > static_cast<size_t>(src_end - src))
return false;
auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src);
if (src_code_point)
{
int dst_code_point = to_case(*src_code_point);
if (dst_code_point > 0)
{
size_t dst_sequence_length = UTF8::convertCodePointToUTF8(dst_code_point, dst, src_end - src);
assert(dst_sequence_length <= 4);
/// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8.
/// As an example, this happens for ß and ẞ.
if (dst_sequence_length == src_sequence_length)
{
src += dst_sequence_length;
dst += dst_sequence_length;
return true;
}
}
}
*dst = *src;
++dst;
++src;
}
return true;
}
private:
static constexpr auto ascii_upper_bound = '\x7f';
static constexpr auto flip_case_mask = 'A' ^ 'a';
static void array(const UInt8 * src, const UInt8 * src_end, const ColumnString::Offsets & offsets, UInt8 * dst)
{
const auto * offset_it = offsets.begin();
const UInt8 * begin = src;
#ifdef __SSE2__
static constexpr auto bytes_sse = sizeof(__m128i);
/// If we are before this position, we can still read at least bytes_sse.
const auto * src_end_sse = src_end - bytes_sse + 1;
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
const auto v_zero = _mm_setzero_si128();
const auto v_not_case_lower_bound = _mm_set1_epi8(not_case_lower_bound - 1);
const auto v_not_case_upper_bound = _mm_set1_epi8(not_case_upper_bound + 1);
const auto v_flip_case_mask = _mm_set1_epi8(flip_case_mask);
while (src < src_end_sse)
{
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(src));
/// check for ASCII
const auto is_not_ascii = _mm_cmplt_epi8(chars, v_zero);
const auto mask_is_not_ascii = _mm_movemask_epi8(is_not_ascii);
/// ASCII
if (mask_is_not_ascii == 0)
{
const auto is_not_case
= _mm_and_si128(_mm_cmpgt_epi8(chars, v_not_case_lower_bound), _mm_cmplt_epi8(chars, v_not_case_upper_bound));
const auto mask_is_not_case = _mm_movemask_epi8(is_not_case);
/// everything in correct case ASCII
if (mask_is_not_case == 0)
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), chars);
else
{
/// ASCII in mixed case
/// keep `flip_case_mask` only where necessary, zero out elsewhere
const auto xor_mask = _mm_and_si128(v_flip_case_mask, is_not_case);
/// flip case by applying calculated mask
const auto cased_chars = _mm_xor_si128(chars, xor_mask);
/// store result back to destination
_mm_storeu_si128(reinterpret_cast<__m128i *>(dst), cased_chars);
}
src += bytes_sse;
dst += bytes_sse;
}
else
{
/// UTF-8
/// Find the offset of the next string after src
size_t offset_from_begin = src - begin;
while (offset_from_begin >= *offset_it)
++offset_it;
/// Do not allow one row influence another (since row may have invalid sequence, and break the next)
const UInt8 * row_end = begin + *offset_it;
chassert(row_end >= src);
const UInt8 * expected_end = std::min(src + bytes_sse, row_end);
while (src < expected_end)
{
if (!toCase(src, expected_end, dst, /* partial= */ true))
{
/// Fallback to handling byte by byte.
src_end_sse = src;
break;
}
}
}
}
/// Find the offset of the next string after src
size_t offset_from_begin = src - begin;
while (offset_it != offsets.end() && offset_from_begin >= *offset_it)
++offset_it;
#endif
/// handle remaining symbols, row by row (to avoid influence of bad UTF8 symbols from one row, to another)
while (src < src_end)
{
const UInt8 * row_end = begin + *offset_it;
chassert(row_end >= src);
while (src < row_end)
toCase(src, row_end, dst, /* partial= */ false);
++offset_it;
}
}
};
}
#endif

View File

@ -1,9 +1,8 @@
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionStringToString.h>
#include <Functions/LowerUpperUTF8Impl.h>
#include <Functions/FunctionFactory.h>
#include <Poco/Unicode.h>
#include <Common/UTF8Helpers.h>
namespace DB
{

View File

@ -1,9 +1,10 @@
#include <DataTypes/DataTypeString.h>
#include "config.h"
#if USE_ICU
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringToString.h>
#include <Functions/LowerUpperUTF8Impl.h>
#include <Functions/FunctionFactory.h>
#include <Poco/Unicode.h>
namespace DB
{
@ -15,13 +16,25 @@ struct NameLowerUTF8
static constexpr auto name = "lowerUTF8";
};
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>;
using FunctionLowerUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', false>, NameLowerUTF8>;
}
REGISTER_FUNCTION(LowerUTF8)
{
factory.registerFunction<FunctionLowerUTF8>();
FunctionDocumentation::Description description
= R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)";
FunctionDocumentation::Syntax syntax = "lowerUTF8(input)";
FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}};
FunctionDocumentation::ReturnedValue returned_value = "A String data type value";
FunctionDocumentation::Examples examples = {
{"first", "SELECT lowerUTF8('München') as Lowerutf8;", "münchen"},
};
FunctionDocumentation::Categories categories = {"String"};
factory.registerFunction<FunctionLowerUTF8>({description, syntax, arguments, returned_value, examples, categories});
}
}
#endif

View File

@ -1,8 +1,10 @@
#include "config.h"
#if USE_ICU
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringToString.h>
#include <Functions/LowerUpperUTF8Impl.h>
#include <Functions/FunctionFactory.h>
#include <Poco/Unicode.h>
namespace DB
{
@ -14,13 +16,25 @@ struct NameUpperUTF8
static constexpr auto name = "upperUTF8";
};
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>;
using FunctionUpperUTF8 = FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', true>, NameUpperUTF8>;
}
REGISTER_FUNCTION(UpperUTF8)
{
factory.registerFunction<FunctionUpperUTF8>();
FunctionDocumentation::Description description
= R"(Converts a string to lowercase, assuming that the string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined.)";
FunctionDocumentation::Syntax syntax = "upperUTF8(input)";
FunctionDocumentation::Arguments arguments = {{"input", "Input with String type"}};
FunctionDocumentation::ReturnedValue returned_value = "A String data type value";
FunctionDocumentation::Examples examples = {
{"first", "SELECT upperUTF8('München') as Upperutf8;", "MÜNCHEN"},
};
FunctionDocumentation::Categories categories = {"String"};
factory.registerFunction<FunctionUpperUTF8>({description, syntax, arguments, returned_value, examples, categories});
}
}
#endif

View File

@ -22,3 +22,7 @@
1
1
1
1
1
1
1

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
select lower('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str;
select lowerUTF8('aaaaaaaaaaaaaaa012345789,.!aaaa' as str) = str;
select lower('AaAaAaAaAaAaAaA012345789,.!aAaA') = 'aaaaaaaaaaaaaaa012345789,.!aaaa';
@ -27,3 +30,11 @@ select sum(lower(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaАБВ
select sum(upper(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() from system.one array join range(16384) as n;
select sum(lowerUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('aaaaабвгaaaaaaaa')) = count() from system.one array join range(16384) as n;
select sum(upperUTF8(materialize('aaaaАБВГAAAAaaAA')) = materialize('AAAAАБВГAAAAAAAA')) = count() from system.one array join range(16384) as n;
-- Turkish language
select upperUTF8('ır') = 'IR';
select lowerUTF8('ır') = 'ır';
-- German language
select upper('öäüß') = 'öäüß';
select lower('ÖÄÜẞ') = 'ÖÄÜẞ';

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
SET send_logs_level = 'fatal';
select 1 = position('', '');

View File

@ -1 +1,4 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
SELECT lowerUTF8('\xF0') = lowerUTF8('\xF0');

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
SELECT randomStringUTF8('string'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT lengthUTF8(randomStringUTF8(100));
SELECT toTypeName(randomStringUTF8(10));

View File

@ -1,2 +1,2 @@
FF
FF
EFBFBD
EFBFBD

View File

@ -1,2 +1,5 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
SELECT hex(lowerUTF8('\xFF'));
SELECT hex(upperUTF8('\xFF'));

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
--
-- countSubstrings
--

View File

@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo
-- MONOGRAM FOR YANG
with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_
select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV;
1,"0xE2","0xE2"
1,"<EFBFBD>","<22>"
15,"foo⚊barbazbam","FOO⚊BARBAZBAM"
1,"0xE2","0xE2"
1,"<EFBFBD>","<22>"
15,"foo⚊barbazbam","FOO⚊BARBAZBAM"
-- NOTE: regression test for introduced bug
-- https://github.com/ClickHouse/ClickHouse/issues/42756

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
drop table if exists utf8_overlap;
create table utf8_overlap (str String) engine=Memory();

View File

@ -416,7 +416,6 @@ logTrace
lowCardinalityIndices
lowCardinalityKeys
lower
lowerUTF8
makeDate
makeDate32
makeDateTime
@ -897,7 +896,6 @@ tupleToNameValuePairs
unbin
unhex
upper
upperUTF8
uptime
validateNestedArraySizes
version

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
create table if not exists t (`arr.key` Array(LowCardinality(String)), `arr.value` Array(LowCardinality(String))) engine = Memory;
insert into t (`arr.key`, `arr.value`) values (['a'], ['b']);
select if(true, if(lowerUTF8(arr.key) = 'a', 1, 2), 3) as x from t left array join arr;

View File

@ -1,2 +1,5 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
SELECT lowerUTF8(arrayJoin(['©--------------------------------------', '©--------------------'])) ORDER BY 1;
SELECT upperUTF8(materialize('aaaaАБВГaaaaaaaaaaaaАБВГAAAAaaAA')) FROM numbers(2);

View File

@ -1,3 +1,6 @@
-- Tags: no-fasttest
-- no-fasttest: upper/lowerUTF8 use ICU
DROP TABLE IF EXISTS test_data;
CREATE TABLE test_data