From d3c047695edc1130370719f618a2018908cb5761 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 10 Sep 2018 02:36:06 +0300 Subject: [PATCH] Every function in its own file, part 3 --- dbms/src/Functions/EmptyImpl.h | 59 + dbms/src/Functions/FunctionStartsEndsWith.h | 137 ++ dbms/src/Functions/FunctionStringOrArrayToT.h | 100 ++ dbms/src/Functions/FunctionStringToString.h | 75 + dbms/src/Functions/FunctionsConditional.cpp | 49 +- dbms/src/Functions/FunctionsString.cpp | 1283 ----------------- dbms/src/Functions/FunctionsString.h | 204 --- dbms/src/Functions/LowerUpperImpl.h | 67 + dbms/src/Functions/LowerUpperUTF8Impl.h | 228 +++ .../Functions/appendTrailingCharIfAbsent.cpp | 115 ++ dbms/src/Functions/concat.cpp | 151 ++ dbms/src/Functions/empty.cpp | 22 + dbms/src/Functions/endsWith.cpp | 16 + dbms/src/Functions/length.cpp | 53 + dbms/src/Functions/lengthUTF8.cpp | 68 + dbms/src/Functions/lower.cpp | 21 + dbms/src/Functions/lowerUTF8.cpp | 23 + dbms/src/Functions/notEmpty.cpp | 21 + .../src/Functions/registerFunctionsString.cpp | 44 + dbms/src/Functions/reverse.cpp | 121 ++ dbms/src/Functions/reverseUTF8.cpp | 81 ++ dbms/src/Functions/startsWith.cpp | 16 + dbms/src/Functions/substring.cpp | 168 +++ dbms/src/Functions/substringUTF8.cpp | 166 +++ dbms/src/Functions/upper.cpp | 21 + dbms/src/Functions/upperUTF8.cpp | 23 + 26 files changed, 1820 insertions(+), 1512 deletions(-) create mode 100644 dbms/src/Functions/EmptyImpl.h create mode 100644 dbms/src/Functions/FunctionStartsEndsWith.h create mode 100644 dbms/src/Functions/FunctionStringOrArrayToT.h create mode 100644 dbms/src/Functions/FunctionStringToString.h delete mode 100644 dbms/src/Functions/FunctionsString.cpp delete mode 100644 dbms/src/Functions/FunctionsString.h create mode 100644 dbms/src/Functions/LowerUpperImpl.h create mode 100644 dbms/src/Functions/LowerUpperUTF8Impl.h create mode 100644 dbms/src/Functions/appendTrailingCharIfAbsent.cpp create mode 100644 dbms/src/Functions/concat.cpp create mode 100644 dbms/src/Functions/empty.cpp create mode 100644 dbms/src/Functions/endsWith.cpp create mode 100644 dbms/src/Functions/length.cpp create mode 100644 dbms/src/Functions/lengthUTF8.cpp create mode 100644 dbms/src/Functions/lower.cpp create mode 100644 dbms/src/Functions/lowerUTF8.cpp create mode 100644 dbms/src/Functions/notEmpty.cpp create mode 100644 dbms/src/Functions/registerFunctionsString.cpp create mode 100644 dbms/src/Functions/reverse.cpp create mode 100644 dbms/src/Functions/reverseUTF8.cpp create mode 100644 dbms/src/Functions/startsWith.cpp create mode 100644 dbms/src/Functions/substring.cpp create mode 100644 dbms/src/Functions/substringUTF8.cpp create mode 100644 dbms/src/Functions/upper.cpp create mode 100644 dbms/src/Functions/upperUTF8.cpp diff --git a/dbms/src/Functions/EmptyImpl.h b/dbms/src/Functions/EmptyImpl.h new file mode 100644 index 00000000000..b32d49e388d --- /dev/null +++ b/dbms/src/Functions/EmptyImpl.h @@ -0,0 +1,59 @@ +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +template +struct EmptyImpl +{ + /// If the function will return constant value for FixedString data type. + static constexpr auto is_fixed_to_constant = false; + + static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + ColumnString::Offset prev_offset = 1; + for (size_t i = 0; i < size; ++i) + { + res[i] = negative ^ (offsets[i] == prev_offset); + prev_offset = offsets[i] + 1; + } + } + + /// Only make sense if is_fixed_to_constant. + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/) + { + throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR); + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + { + std::vector empty_chars(n); + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n)); + } + + static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = negative ^ (offsets[i] == prev_offset); + prev_offset = offsets[i]; + } + } +}; + +} diff --git a/dbms/src/Functions/FunctionStartsEndsWith.h b/dbms/src/Functions/FunctionStartsEndsWith.h new file mode 100644 index 00000000000..f7b18b81465 --- /dev/null +++ b/dbms/src/Functions/FunctionStartsEndsWith.h @@ -0,0 +1,137 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct NameStartsWith +{ + static constexpr auto name = "startsWith"; +}; +struct NameEndsWith +{ + static constexpr auto name = "endsWith"; +}; + +template +class FunctionStartsEndsWith : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 2; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isStringOrFixedString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); + const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + vec_res.resize(input_rows_count); + + if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) + dispatch(StringSource(*haystack), needle_column, vec_res); + else if (const ColumnFixedString * haystack = checkAndGetColumn(haystack_column)) + dispatch(FixedStringSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) + dispatch>(ConstSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) + dispatch>(ConstSource(*haystack), needle_column, vec_res); + else + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + block.getByPosition(result).column = std::move(col_res); + } + +private: + template + void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + { + if (const ColumnString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, StringSource(*needle), res_data); + else if (const ColumnFixedString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, FixedStringSource(*needle), res_data); + else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle), res_data); + else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle), res_data); + else + throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + template + static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) + { + size_t row_num = 0; + + while (!haystack_source.isEnd()) + { + auto haystack = haystack_source.getWhole(); + auto needle = needle_source.getWhole(); + + if (needle.size > haystack.size) + { + res_data[row_num] = false; + } + else + { + if constexpr (std::is_same_v) + { + res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); + } + else /// endsWith + { + res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); + } + } + + haystack_source.next(); + needle_source.next(); + ++row_num; + } + } +}; + +} diff --git a/dbms/src/Functions/FunctionStringOrArrayToT.h b/dbms/src/Functions/FunctionStringOrArrayToT.h new file mode 100644 index 00000000000..f94dcfe7647 --- /dev/null +++ b/dbms/src/Functions/FunctionStringOrArrayToT.h @@ -0,0 +1,100 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringOrArrayToT : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0]) + && !isArray(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared>(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::vector(col->getChars(), col->getOffsets(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + if (Impl::is_fixed_to_constant) + { + ResultType res = 0; + Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res); + + block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res)); + } + else + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + } + else if (const ColumnArray * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnVector::create(); + + typename ColumnVector::Container & vec_res = col_res->getData(); + vec_res.resize(col->size()); + Impl::array(col->getOffsets(), vec_res); + + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/FunctionStringToString.h b/dbms/src/Functions/FunctionStringToString.h new file mode 100644 index 00000000000..a32a5279762 --- /dev/null +++ b/dbms/src/Functions/FunctionStringToString.h @@ -0,0 +1,75 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +template +class FunctionStringToString : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isInjective(const Block &) override + { + return is_injective; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnFixedString::create(col->getN()); + Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/dbms/src/Functions/FunctionsConditional.cpp b/dbms/src/Functions/FunctionsConditional.cpp index acfb57c439e..92ae0a5b0fb 100644 --- a/dbms/src/Functions/FunctionsConditional.cpp +++ b/dbms/src/Functions/FunctionsConditional.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include @@ -250,22 +252,15 @@ DataTypePtr FunctionCaseWithExpression::getReturnTypeImpl(const DataTypes & args /// See the comments in executeImpl() to understand why we actually have to /// get the return type of a transform function. - /// Get the return types of the arrays that we pass to the transform function. - ColumnsWithTypeAndName src_array_types; - ColumnsWithTypeAndName dst_array_types; + /// Get the types of the arrays that we pass to the transform function. + DataTypes src_array_types; + DataTypes dst_array_types; - for (size_t i = 1; i < (args.size() - 1); ++i) - { - if ((i % 2) != 0) - src_array_types.push_back({nullptr, args[i], {}}); - else - dst_array_types.push_back({nullptr, args[i], {}}); - } + for (size_t i = 1; i < args.size() - 1; ++i) + ((i % 2) ? src_array_types : dst_array_types).push_back(args[i]); - FunctionArray fun_array{context}; - - DataTypePtr src_array_type = fun_array.getReturnType(src_array_types); - DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types); + DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); + DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); /// Finally get the return type of the transform function. FunctionTransform fun_transform; @@ -290,29 +285,31 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers /// Create the arrays required by the transform function. ColumnNumbers src_array_args; - ColumnsWithTypeAndName src_array_types; + ColumnsWithTypeAndName src_array_elems; + DataTypes src_array_types; ColumnNumbers dst_array_args; - ColumnsWithTypeAndName dst_array_types; + ColumnsWithTypeAndName dst_array_elems; + DataTypes dst_array_types; for (size_t i = 1; i < (args.size() - 1); ++i) { - if ((i % 2) != 0) + if (i % 2) { src_array_args.push_back(args[i]); - src_array_types.push_back(block.getByPosition(args[i])); + src_array_elems.push_back(block.getByPosition(args[i])); + src_array_types.push_back(block.getByPosition(args[i]).type); } else { dst_array_args.push_back(args[i]); - dst_array_types.push_back(block.getByPosition(args[i])); + dst_array_elems.push_back(block.getByPosition(args[i])); + dst_array_types.push_back(block.getByPosition(args[i]).type); } } - FunctionArray fun_array{context}; - - DataTypePtr src_array_type = fun_array.getReturnType(src_array_types); - DataTypePtr dst_array_type = fun_array.getReturnType(dst_array_types); + DataTypePtr src_array_type = std::make_shared(getLeastSupertype(src_array_types)); + DataTypePtr dst_array_type = std::make_shared(getLeastSupertype(dst_array_types)); Block temp_block = block; @@ -322,8 +319,10 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers size_t dst_array_pos = temp_block.columns(); temp_block.insert({nullptr, dst_array_type, ""}); - fun_array.execute(temp_block, src_array_args, src_array_pos, input_rows_count); - fun_array.execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); + auto fun_array = FunctionFactory::instance().get("array", context); + + fun_array->build(src_array_elems)->execute(temp_block, src_array_args, src_array_pos, input_rows_count); + fun_array->build(dst_array_elems)->execute(temp_block, dst_array_args, dst_array_pos, input_rows_count); /// Execute transform. FunctionTransform fun_transform; diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp deleted file mode 100644 index 61542b7ea36..00000000000 --- a/dbms/src/Functions/FunctionsString.cpp +++ /dev/null @@ -1,1283 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include - - -#if __SSE2__ -#include -#endif - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ZERO_ARRAY_OR_TUPLE_INDEX; - extern const int LOGICAL_ERROR; -} - -using namespace GatherUtils; - -template -struct EmptyImpl -{ - /// If the function will return constant value for FixedString data type. - static constexpr auto is_fixed_to_constant = false; - - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - ColumnString::Offset prev_offset = 1; - for (size_t i = 0; i < size; ++i) - { - res[i] = negative ^ (offsets[i] == prev_offset); - prev_offset = offsets[i] + 1; - } - } - - /// Only make sense if is_fixed_to_constant. - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/) - { - throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR); - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) - { - std::vector empty_chars(n); - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - res[i] = negative ^ (0 == memcmp(&data[i * size], empty_chars.data(), n)); - } - - static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - res[i] = negative ^ (offsets[i] == prev_offset); - prev_offset = offsets[i]; - } - } -}; - - -/** Calculates the length of a string in bytes. - */ -struct LengthImpl -{ - static constexpr auto is_fixed_to_constant = true; - - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); - } - - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res) - { - res = n; - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) - { - } - - static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - for (size_t i = 0; i < size; ++i) - res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]); - } -}; - - -/** If the string is UTF-8 encoded text, it returns the length of the text in code points. - * (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization) - * (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization) - * Otherwise, the behavior is undefined. - */ -struct LengthUTF8Impl -{ - static constexpr auto is_fixed_to_constant = false; - - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) - { - size_t size = offsets.size(); - - ColumnString::Offset prev_offset = 0; - for (size_t i = 0; i < size; ++i) - { - res[i] = UTF8::countCodePoints(&data[prev_offset], offsets[i] - prev_offset - 1); - prev_offset = offsets[i]; - } - } - - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/) - { - } - - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) - { - size_t size = data.size() / n; - - for (size_t i = 0; i < size; ++i) - { - res[i] = UTF8::countCodePoints(&data[i * n], n); - } - } - - static void array(const ColumnString::Offsets &, PaddedPODArray &) - { - throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } -}; - - -template -struct LowerUpperImpl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.resize(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), res_data.data()); - } - - static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) - { - res_data.resize(data.size()); - array(data.data(), data.data() + data.size(), res_data.data()); - } - -private: - static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) - { - const auto flip_case_mask = 'A' ^ 'a'; - -#if __SSE2__ - const auto bytes_sse = sizeof(__m128i); - const auto src_end_sse = src_end - (src_end - src) % bytes_sse; - - 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); - - for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse) - { - /// load 16 sequential 8-bit characters - const auto chars = _mm_loadu_si128(reinterpret_cast(src)); - - /// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound] - 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)); - - /// 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); - } -#endif - - for (; src < src_end; ++src, ++dst) - if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) - *dst = *src ^ flip_case_mask; - else - *dst = *src; - } -}; - -/** Expands the string in bytes. - */ -struct ReverseImpl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & 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 vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & 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]; - } -}; - - -/** Expands the sequence of code points in a UTF-8 encoded string. - * The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols. - * If the string is not encoded in UTF-8, then the behavior is undefined. - */ -struct ReverseUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & 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) - { - ColumnString::Offset j = prev_offset; - while (j < offsets[i] - 1) - { - if (data[j] < 0xBF) - { - res_data[offsets[i] + prev_offset - 2 - j] = data[j]; - j += 1; - } - else if (data[j] < 0xE0) - { - memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 1], &data[j], 2); - j += 2; - } - else if (data[j] < 0xF0) - { - memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 2], &data[j], 3); - j += 3; - } - else - { - res_data[offsets[i] + prev_offset - 2 - j] = data[j]; - j += 1; - } - } - - res_data[offsets[i] - 1] = 0; - prev_offset = offsets[i]; - } - } - - static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) - { - throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -template -void LowerUpperUTF8Impl::vector(const ColumnString::Chars_t & data, - const IColumn::Offsets & offsets, - ColumnString::Chars_t & res_data, - IColumn::Offsets & res_offsets) -{ - res_data.resize(data.size()); - res_offsets.assign(offsets); - array(data.data(), data.data() + data.size(), res_data.data()); -} - -template -void LowerUpperUTF8Impl::vector_fixed( - const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) -{ - res_data.resize(data.size()); - array(data.data(), data.data() + data.size(), res_data.data()); -} - -template -void LowerUpperUTF8Impl::constant( - const std::string & data, std::string & res_data) -{ - res_data.resize(data.size()); - array(reinterpret_cast(data.data()), - reinterpret_cast(data.data() + data.size()), - reinterpret_cast(res_data.data())); -} - -template -void LowerUpperUTF8Impl::toCase( - const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) -{ - 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 - { - static const Poco::UTF8Encoding utf8; - - if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src)) - { - src += chars; - dst += chars; - } - else - { - ++src; - ++dst; - } - } -} - -template -void LowerUpperUTF8Impl::array( - const UInt8 * src, const UInt8 * src_end, UInt8 * dst) -{ -#if __SSE2__ - const auto bytes_sse = sizeof(__m128i); - auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; - - /// 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(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 - const auto expected_end = src + bytes_sse; - - while (src < expected_end) - toCase(src, src_end, dst); - - /// adjust src_end_sse by pushing it forward or backward - const auto diff = src - expected_end; - if (diff != 0) - { - if (src_end_sse + diff < src_end) - src_end_sse += diff; - else - src_end_sse -= bytes_sse - diff; - } - } - } -#endif - /// handle remaining symbols - while (src < src_end) - toCase(src, src_end, dst); -} - - -/** If the string is encoded in UTF-8, then it selects a substring of code points in it. - * Otherwise, the behavior is undefined. - */ -struct SubstringUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - size_t start, - size_t length, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets) - { - res_data.reserve(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); - - ColumnString::Offset prev_offset = 0; - ColumnString::Offset res_offset = 0; - for (size_t i = 0; i < size; ++i) - { - ColumnString::Offset j = prev_offset; - ColumnString::Offset pos = 1; - ColumnString::Offset bytes_start = 0; - ColumnString::Offset bytes_length = 0; - while (j < offsets[i] - 1) - { - if (pos == start) - bytes_start = j - prev_offset + 1; - - if (data[j] < 0xBF) - j += 1; - else if (data[j] < 0xE0) - j += 2; - else if (data[j] < 0xF0) - j += 3; - else - j += 1; - - if (pos >= start && pos < start + length) - bytes_length = j - prev_offset + 1 - bytes_start; - else if (pos >= start + length) - break; - - ++pos; - } - - if (bytes_start == 0) - { - res_data.resize(res_data.size() + 1); - res_data[res_offset] = 0; - ++res_offset; - } - else - { - size_t bytes_to_copy = std::min(offsets[i] - prev_offset - bytes_start, bytes_length); - res_data.resize(res_data.size() + bytes_to_copy + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + bytes_start - 1], bytes_to_copy); - res_offset += bytes_to_copy + 1; - res_data[res_offset - 1] = 0; - } - res_offsets[i] = res_offset; - prev_offset = offsets[i]; - } - } -}; - - -template -class FunctionStringOrArrayToT : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0]) - && !isArray(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared>(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::vector(col->getChars(), col->getOffsets(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - if (Impl::is_fixed_to_constant) - { - ResultType res = 0; - Impl::vector_fixed_to_constant(col->getChars(), col->getN(), res); - - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col->size(), toField(res)); - } - else - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::vector_fixed_to_vector(col->getChars(), col->getN(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - } - else if (const ColumnArray * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnVector::create(); - - typename ColumnVector::Container & vec_res = col_res->getData(); - vec_res.resize(col->size()); - Impl::array(col->getOffsets(), vec_res); - - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -/// Also works with arrays. -class FunctionReverse : public IFunction -{ -public: - static constexpr auto name = "reverse"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isInjective(const Block &) override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0]) - && !isArray(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnFixedString::create(col->getN()); - ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); - block.getByPosition(result).column = std::move(col_res); - } - else if (checkColumn(column.get())) - { - FunctionArrayReverse().execute(block, arguments, result, input_rows_count); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -template -class ConcatImpl : public IFunction -{ -public: - static constexpr auto name = Name::name; - ConcatImpl(const Context & context) : context(context) {} - static FunctionPtr create(const Context & context) - { - return std::make_shared(context); - } - - String getName() const override - { - return name; - } - - bool isVariadic() const override - { - return true; - } - - size_t getNumberOfArguments() const override - { - return 0; - } - - bool isInjective(const Block &) override - { - return is_injective; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!is_injective && !arguments.empty() && isArray(arguments[0])) - return FunctionArrayConcat(context).getReturnTypeImpl(arguments); - - if (arguments.size() < 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) - + ", should be at least 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - for (const auto arg_idx : ext::range(0, arguments.size())) - { - const auto arg = arguments[arg_idx].get(); - if (!isStringOrFixedString(arg)) - throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - if (!is_injective && !arguments.empty() && isArray(block.getByPosition(arguments[0]).type)) - return FunctionArrayConcat(context).executeImpl(block, arguments, result, input_rows_count); - - if (arguments.size() == 2) - executeBinary(block, arguments, result, input_rows_count); - else - executeNAry(block, arguments, result, input_rows_count); - } - -private: - const Context & context; - - void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) - { - const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); - const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); - - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); - const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); - const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); - - auto c_res = ColumnString::create(); - - if (c0_string && c1_string) - concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); - else if (c0_string && c1_const_string) - concat(StringSource(*c0_string), ConstSource(*c1_const_string), StringSink(*c_res, c0->size())); - else if (c0_const_string && c1_string) - concat(ConstSource(*c0_const_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); - else - { - /// Fallback: use generic implementation for not very important cases. - executeNAry(block, arguments, result, input_rows_count); - return; - } - - block.getByPosition(result).column = std::move(c_res); - } - - void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) - { - size_t num_sources = arguments.size(); - StringSources sources(num_sources); - - for (size_t i = 0; i < num_sources; ++i) - sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column); - - auto c_res = ColumnString::create(); - concat(sources, StringSink(*c_res, input_rows_count)); - block.getByPosition(result).column = std::move(c_res); - } -}; - - -class FunctionSubstring : public IFunction -{ -public: - static constexpr auto name = "substring"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - - bool useDefaultImplementationForConstants() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - size_t number_of_arguments = arguments.size(); - - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(number_of_arguments) + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!isStringOrFixedString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isNumber(arguments[1])) - throw Exception("Illegal type " + arguments[1]->getName() - + " of second argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (number_of_arguments == 3 && !isNumber(arguments[2])) - throw Exception("Illegal type " + arguments[2]->getName() - + " of second argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - template - void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length, - const ColumnConst * column_start_const, const ColumnConst * column_length_const, - Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source, - size_t input_rows_count) - { - auto col_res = ColumnString::create(); - - if (!column_length) - { - if (column_start_const) - { - if (start_value > 0) - sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1); - else if (start_value < 0) - sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value); - else - throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - } - else - sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); - } - else - { - if (column_start_const && column_length_const) - { - if (start_value > 0) - sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value); - else if (start_value < 0) - sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value); - else - throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - } - else - sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length); - } - - block.getByPosition(result).column = std::move(col_res); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - size_t number_of_arguments = arguments.size(); - - ColumnPtr column_string = block.getByPosition(arguments[0]).column; - ColumnPtr column_start = block.getByPosition(arguments[1]).column; - ColumnPtr column_length; - - if (number_of_arguments == 3) - column_length = block.getByPosition(arguments[2]).column; - - const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); - const ColumnConst * column_length_const = nullptr; - - if (number_of_arguments == 3) - column_length_const = checkAndGetColumn(column_length.get()); - - Int64 start_value = 0; - Int64 length_value = 0; - - if (column_start_const) - { - start_value = column_start_const->getInt(0); - } - if (column_length_const) - { - length_value = column_length_const->getInt(0); - if (length_value < 0) - throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } - - if (const ColumnString * col = checkAndGetColumn(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, StringSource(*col), input_rows_count); - else if (const ColumnFixedString * col = checkAndGetColumn(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, FixedStringSource(*col), input_rows_count); - else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, ConstSource(*col), input_rows_count); - else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) - executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, - length_value, block, result, ConstSource(*col), input_rows_count); - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -class FunctionSubstringUTF8 : public IFunction -{ -public: - static constexpr auto name = "substringUTF8"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 3; - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isNumber(arguments[1]) || !isNumber(arguments[2])) - throw Exception("Illegal type " + (isNumber(arguments[1]) ? arguments[2]->getName() : arguments[1]->getName()) - + " of argument of function " - + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column_string = block.getByPosition(arguments[0]).column; - const ColumnPtr column_start = block.getByPosition(arguments[1]).column; - const ColumnPtr column_length = block.getByPosition(arguments[2]).column; - - if (!column_start->isColumnConst() || !column_length->isColumnConst()) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); - - Field start_field = (*block.getByPosition(arguments[1]).column)[0]; - Field length_field = (*block.getByPosition(arguments[2]).column)[0]; - - if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type."); - - UInt64 start = start_field.get(); - UInt64 length = length_field.get(); - - if (start == 0) - throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - /// Otherwise may lead to overflow and pass bounds check inside inner loop. - if (start >= 0x8000000000000000ULL || length >= 0x8000000000000000ULL) - throw Exception("Too large values of 2nd or 3rd argument provided for function substring.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (const ColumnString * col = checkAndGetColumn(column_string.get())) - { - auto col_res = ColumnString::create(); - SubstringUTF8Impl::vector(col->getChars(), col->getOffsets(), start, length, col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - - -class FunctionAppendTrailingCharIfAbsent : public IFunction -{ -public: - static constexpr auto name = "appendTrailingCharIfAbsent"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - -private: - size_t getNumberOfArguments() const override - { - return 2; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[0])) - throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - if (!isString(arguments[1])) - throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto & column = block.getByPosition(arguments[0]).column; - const auto & column_char = block.getByPosition(arguments[1]).column; - - if (!checkColumnConst(column_char.get())) - throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; - - String trailing_char_str = static_cast(*column_char).getValue(); - - if (trailing_char_str.size() != 1) - throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS}; - - if (const auto col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - - const auto & src_data = col->getChars(); - const auto & src_offsets = col->getOffsets(); - - auto & dst_data = col_res->getChars(); - auto & dst_offsets = col_res->getOffsets(); - - const auto size = src_offsets.size(); - dst_data.resize(src_data.size() + size); - dst_offsets.resize(size); - - ColumnString::Offset src_offset{}; - ColumnString::Offset dst_offset{}; - - for (const auto i : ext::range(0, size)) - { - const auto src_length = src_offsets[i] - src_offset; - memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length); - src_offset = src_offsets[i]; - dst_offset += src_length; - - if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) - { - dst_data[dst_offset - 1] = trailing_char_str.front(); - dst_data[dst_offset] = 0; - ++dst_offset; - } - - dst_offsets[i] = dst_offset; - } - - dst_data.resize_assume_reserved(dst_offset); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN}; - } -}; - - -struct NameStartsWith -{ - static constexpr auto name = "startsWith"; -}; -struct NameEndsWith -{ - static constexpr auto name = "endsWith"; -}; - -template -class FunctionStartsEndsWith : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 2; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isStringOrFixedString(arguments[1])) - throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared>(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const IColumn * haystack_column = block.getByPosition(arguments[0]).column.get(); - const IColumn * needle_column = block.getByPosition(arguments[1]).column.get(); - - auto col_res = ColumnVector::create(); - typename ColumnVector::Container & vec_res = col_res->getData(); - - vec_res.resize(input_rows_count); - - if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) - dispatch(StringSource(*haystack), needle_column, vec_res); - else if (const ColumnFixedString * haystack = checkAndGetColumn(haystack_column)) - dispatch(FixedStringSource(*haystack), needle_column, vec_res); - else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) - dispatch>(ConstSource(*haystack), needle_column, vec_res); - else if (const ColumnConst * haystack = checkAndGetColumnConst(haystack_column)) - dispatch>(ConstSource(*haystack), needle_column, vec_res); - else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - - block.getByPosition(result).column = std::move(col_res); - } - -private: - template - void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const - { - if (const ColumnString * needle = checkAndGetColumn(needle_column)) - execute(haystack_source, StringSource(*needle), res_data); - else if (const ColumnFixedString * needle = checkAndGetColumn(needle_column)) - execute(haystack_source, FixedStringSource(*needle), res_data); - else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) - execute>(haystack_source, ConstSource(*needle), res_data); - else if (const ColumnConst * needle = checkAndGetColumnConst(needle_column)) - execute>(haystack_source, ConstSource(*needle), res_data); - else - throw Exception("Illegal combination of columns as arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - } - - template - static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) - { - size_t row_num = 0; - - while (!haystack_source.isEnd()) - { - auto haystack = haystack_source.getWhole(); - auto needle = needle_source.getWhole(); - - if (needle.size > haystack.size) - { - res_data[row_num] = false; - } - else - { - if constexpr (std::is_same_v) - { - res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); - } - else /// endsWith - { - res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); - } - } - - haystack_source.next(); - needle_source.next(); - ++row_num; - } - } -}; - - -struct NameEmpty -{ - static constexpr auto name = "empty"; -}; -struct NameNotEmpty -{ - static constexpr auto name = "notEmpty"; -}; -struct NameLength -{ - static constexpr auto name = "length"; -}; -struct NameLengthUTF8 -{ - static constexpr auto name = "lengthUTF8"; -}; -struct NameLower -{ - static constexpr auto name = "lower"; -}; -struct NameUpper -{ - static constexpr auto name = "upper"; -}; -struct NameReverseUTF8 -{ - static constexpr auto name = "reverseUTF8"; -}; -struct NameConcat -{ - static constexpr auto name = "concat"; -}; -struct NameConcatAssumeInjective -{ - static constexpr auto name = "concatAssumeInjective"; -}; - - -using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8>; -using FunctionNotEmpty = FunctionStringOrArrayToT, NameNotEmpty, UInt8>; -using FunctionLength = FunctionStringOrArrayToT; -using FunctionLengthUTF8 = FunctionStringOrArrayToT; -using FunctionLower = FunctionStringToString, NameLower>; -using FunctionUpper = FunctionStringToString, NameUpper>; -using FunctionReverseUTF8 = FunctionStringToString; -using FunctionConcat = ConcatImpl; -using FunctionConcatAssumeInjective = ConcatImpl; -using FunctionStartsWith = FunctionStartsEndsWith; -using FunctionEndsWith = FunctionStartsEndsWith; - - -void registerFunctionsString(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} -} diff --git a/dbms/src/Functions/FunctionsString.h b/dbms/src/Functions/FunctionsString.h deleted file mode 100644 index d8744c47342..00000000000 --- a/dbms/src/Functions/FunctionsString.h +++ /dev/null @@ -1,204 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} - -/** String functions - * - * length, empty, notEmpty, - * concat, substring, lower, upper, reverse - * lengthUTF8, substringUTF8, lowerUTF8, upperUTF8, reverseUTF8 - * - * s -> UInt8: empty, notEmpty - * s -> UInt64: length, lengthUTF8 - * s -> s: lower, upper, lowerUTF8, upperUTF8, reverse, reverseUTF8 - * s, s -> s: concat - * s, c1, c2 -> s: substring, substringUTF8 - * s, c1, c2, s2 -> s: replace, replaceUTF8 - * - * The search functions for strings and regular expressions are located separately. - * URL functions are located separately. - * String encoding functions, converting to other types are located separately. - * - * The functions length, empty, notEmpty, reverse also work with arrays. - */ - - -/// xor or do nothing -template -UInt8 xor_or_identity(const UInt8 c, const int mask) -{ - return c ^ mask; -} - -template <> -inline UInt8 xor_or_identity(const UInt8 c, const int) -{ - return c; -} - -/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array -template -inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) -{ - if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// ѐёђѓєѕіїјљњћќѝўџ - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x10); - } - else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) - { - /// А-П - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) - { - /// а-п - *dst++ = *src++; - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) - { - /// Р-Я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*src++, 0x20); - } - else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) - { - /// р-я - *dst++ = xor_or_identity(*src++, 0x1); - *dst++ = xor_or_identity(*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 -struct LowerUpperUTF8Impl -{ - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, - ColumnString::Offsets & res_offsets); - - static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data); - - static void constant(const std::string & data, std::string & res_data); - - /** 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 void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst); - -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, UInt8 * dst); -}; - - -template -class FunctionStringToString : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool isInjective(const Block &) override - { - return is_injective; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isStringOrFixedString(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return arguments[0]; - } - - bool useDefaultImplementationForConstants() const override { return true; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnString::create(); - Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); - block.getByPosition(result).column = std::move(col_res); - } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) - { - auto col_res = ColumnFixedString::create(col->getN()); - Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); - block.getByPosition(result).column = std::move(col_res); - } - else - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -struct NameLowerUTF8 -{ - static constexpr auto name = "lowerUTF8"; -}; -struct NameUpperUTF8 -{ - static constexpr auto name = "upperUTF8"; -}; - - -using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; -using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; - -} diff --git a/dbms/src/Functions/LowerUpperImpl.h b/dbms/src/Functions/LowerUpperImpl.h new file mode 100644 index 00000000000..50806372188 --- /dev/null +++ b/dbms/src/Functions/LowerUpperImpl.h @@ -0,0 +1,67 @@ +#include + + +namespace DB +{ + +template +struct LowerUpperImpl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) + { + res_data.resize(data.size()); + array(data.data(), data.data() + data.size(), res_data.data()); + } + +private: + static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) + { + const auto flip_case_mask = 'A' ^ 'a'; + +#if __SSE2__ + const auto bytes_sse = sizeof(__m128i); + const auto src_end_sse = src_end - (src_end - src) % bytes_sse; + + 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); + + for (; src < src_end_sse; src += bytes_sse, dst += bytes_sse) + { + /// load 16 sequential 8-bit characters + const auto chars = _mm_loadu_si128(reinterpret_cast(src)); + + /// find which 8-bit sequences belong to range [case_lower_bound, case_upper_bound] + 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)); + + /// 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); + } +#endif + + for (; src < src_end; ++src, ++dst) + if (*src >= not_case_lower_bound && *src <= not_case_upper_bound) + *dst = *src ^ flip_case_mask; + else + *dst = *src; + } +}; + +} diff --git a/dbms/src/Functions/LowerUpperUTF8Impl.h b/dbms/src/Functions/LowerUpperUTF8Impl.h new file mode 100644 index 00000000000..0fd0703a128 --- /dev/null +++ b/dbms/src/Functions/LowerUpperUTF8Impl.h @@ -0,0 +1,228 @@ +#include + +#if __SSE2__ +#include +#endif + + +namespace DB +{ + +namespace +{ + /// xor or do nothing + template + UInt8 xor_or_identity(const UInt8 c, const int mask) + { + return c ^ mask; + } + + template <> + inline UInt8 xor_or_identity(const UInt8 c, const int) + { + return c; + } + + /// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array + template + inline void UTF8CyrillicToCase(const UInt8 *& src, UInt8 *& dst) + { + if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// ѐёђѓєѕіїјљњћќѝўџ + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x10); + } + else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu)) + { + /// А-П + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu)) + { + /// а-п + *dst++ = *src++; + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu)) + { + /// Р-Я + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*src++, 0x20); + } + else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu)) + { + /// р-я + *dst++ = xor_or_identity(*src++, 0x1); + *dst++ = xor_or_identity(*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 +struct LowerUpperUTF8Impl +{ + static void vector( + const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + res_offsets.assign(offsets); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data) + { + res_data.resize(data.size()); + array(data.data(), data.data() + data.size(), res_data.data()); + } + + static void constant(const std::string & data, std::string & res_data) + { + res_data.resize(data.size()); + array(reinterpret_cast(data.data()), + reinterpret_cast(data.data() + data.size()), + reinterpret_cast(res_data.data())); + } + + /** 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 void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) + { + 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 + { + static const Poco::UTF8Encoding utf8; + + if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src)) + { + src += chars; + dst += chars; + } + else + { + ++src; + ++dst; + } + } + } + +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, UInt8 * dst) + { +#if __SSE2__ + const auto bytes_sse = sizeof(__m128i); + auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; + + /// 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(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 + const auto expected_end = src + bytes_sse; + + while (src < expected_end) + toCase(src, src_end, dst); + + /// adjust src_end_sse by pushing it forward or backward + const auto diff = src - expected_end; + if (diff != 0) + { + if (src_end_sse + diff < src_end) + src_end_sse += diff; + else + src_end_sse -= bytes_sse - diff; + } + } + } +#endif + /// handle remaining symbols + while (src < src_end) + toCase(src, src_end, dst); + } +}; + +} diff --git a/dbms/src/Functions/appendTrailingCharIfAbsent.cpp b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp new file mode 100644 index 00000000000..4174f2259c9 --- /dev/null +++ b/dbms/src/Functions/appendTrailingCharIfAbsent.cpp @@ -0,0 +1,115 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + + +class FunctionAppendTrailingCharIfAbsent : public IFunction +{ +public: + static constexpr auto name = "appendTrailingCharIfAbsent"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + +private: + size_t getNumberOfArguments() const override + { + return 2; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!isString(arguments[1])) + throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto & column = block.getByPosition(arguments[0]).column; + const auto & column_char = block.getByPosition(arguments[1]).column; + + if (!checkColumnConst(column_char.get())) + throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN}; + + String trailing_char_str = static_cast(*column_char).getValue(); + + if (trailing_char_str.size() != 1) + throw Exception{"Second argument of function " + getName() + " must be a one-character string", ErrorCodes::BAD_ARGUMENTS}; + + if (const auto col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + + const auto & src_data = col->getChars(); + const auto & src_offsets = col->getOffsets(); + + auto & dst_data = col_res->getChars(); + auto & dst_offsets = col_res->getOffsets(); + + const auto size = src_offsets.size(); + dst_data.resize(src_data.size() + size); + dst_offsets.resize(size); + + ColumnString::Offset src_offset{}; + ColumnString::Offset dst_offset{}; + + for (const auto i : ext::range(0, size)) + { + const auto src_length = src_offsets[i] - src_offset; + memcpySmallAllowReadWriteOverflow15(&dst_data[dst_offset], &src_data[src_offset], src_length); + src_offset = src_offsets[i]; + dst_offset += src_length; + + if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) + { + dst_data[dst_offset - 1] = trailing_char_str.front(); + dst_data[dst_offset] = 0; + ++dst_offset; + } + + dst_offsets[i] = dst_offset; + } + + dst_data.resize_assume_reserved(dst_offset); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN}; + } +}; + +void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/concat.cpp b/dbms/src/Functions/concat.cpp new file mode 100644 index 00000000000..3945022a512 --- /dev/null +++ b/dbms/src/Functions/concat.cpp @@ -0,0 +1,151 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +using namespace GatherUtils; + + +template +class ConcatImpl : public IFunction +{ +public: + static constexpr auto name = Name::name; + ConcatImpl(const Context & context) : context(context) {} + static FunctionPtr create(const Context & context) + { + return std::make_shared(context); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override + { + return true; + } + + size_t getNumberOfArguments() const override + { + return 0; + } + + bool isInjective(const Block &) override + { + return is_injective; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!is_injective && !arguments.empty() && isArray(arguments[0])) + return FunctionArrayConcat(context).getReturnTypeImpl(arguments); + + if (arguments.size() < 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be at least 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (const auto arg_idx : ext::range(0, arguments.size())) + { + const auto arg = arguments[arg_idx].get(); + if (!isStringOrFixedString(arg)) + throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + if (!is_injective && !arguments.empty() && isArray(block.getByPosition(arguments[0]).type)) + return FunctionArrayConcat(context).executeImpl(block, arguments, result, input_rows_count); + + if (arguments.size() == 2) + executeBinary(block, arguments, result, input_rows_count); + else + executeNAry(block, arguments, result, input_rows_count); + } + +private: + const Context & context; + + void executeBinary(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); + const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); + + const ColumnString * c0_string = checkAndGetColumn(c0); + const ColumnString * c1_string = checkAndGetColumn(c1); + const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); + const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); + + auto c_res = ColumnString::create(); + + if (c0_string && c1_string) + concat(StringSource(*c0_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); + else if (c0_string && c1_const_string) + concat(StringSource(*c0_string), ConstSource(*c1_const_string), StringSink(*c_res, c0->size())); + else if (c0_const_string && c1_string) + concat(ConstSource(*c0_const_string), StringSource(*c1_string), StringSink(*c_res, c0->size())); + else + { + /// Fallback: use generic implementation for not very important cases. + executeNAry(block, arguments, result, input_rows_count); + return; + } + + block.getByPosition(result).column = std::move(c_res); + } + + void executeNAry(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count) + { + size_t num_sources = arguments.size(); + StringSources sources(num_sources); + + for (size_t i = 0; i < num_sources; ++i) + sources[i] = createDynamicStringSource(*block.getByPosition(arguments[i]).column); + + auto c_res = ColumnString::create(); + concat(sources, StringSink(*c_res, input_rows_count)); + block.getByPosition(result).column = std::move(c_res); + } +}; + +struct NameConcat +{ + static constexpr auto name = "concat"; +}; +struct NameConcatAssumeInjective +{ + static constexpr auto name = "concatAssumeInjective"; +}; + +using FunctionConcat = ConcatImpl; +using FunctionConcatAssumeInjective = ConcatImpl; + +void registerFunctionsConcat(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/empty.cpp b/dbms/src/Functions/empty.cpp new file mode 100644 index 00000000000..dfc13253b48 --- /dev/null +++ b/dbms/src/Functions/empty.cpp @@ -0,0 +1,22 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameEmpty +{ + static constexpr auto name = "empty"; +}; +using FunctionEmpty = FunctionStringOrArrayToT, NameEmpty, UInt8>; + +void registerFunctionEmpty(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/dbms/src/Functions/endsWith.cpp b/dbms/src/Functions/endsWith.cpp new file mode 100644 index 00000000000..40ea7902465 --- /dev/null +++ b/dbms/src/Functions/endsWith.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionEndsWith = FunctionStartsEndsWith; + +void registerFunctionEndsWith(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} + diff --git a/dbms/src/Functions/length.cpp b/dbms/src/Functions/length.cpp new file mode 100644 index 00000000000..653e3166ba9 --- /dev/null +++ b/dbms/src/Functions/length.cpp @@ -0,0 +1,53 @@ +#include +#include +#include + + +namespace DB +{ + + +/** Calculates the length of a string in bytes. + */ +struct LengthImpl +{ + static constexpr auto is_fixed_to_constant = true; + + static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + for (size_t i = 0; i < size; ++i) + res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); + } + + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res) + { + res = n; + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) + { + } + + static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + for (size_t i = 0; i < size; ++i) + res[i] = i == 0 ? (offsets[i]) : (offsets[i] - offsets[i - 1]); + } +}; + + +struct NameLength +{ + static constexpr auto name = "length"; +}; + +using FunctionLength = FunctionStringOrArrayToT; + +void registerFunctionLength(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lengthUTF8.cpp b/dbms/src/Functions/lengthUTF8.cpp new file mode 100644 index 00000000000..734f62afb40 --- /dev/null +++ b/dbms/src/Functions/lengthUTF8.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** If the string is UTF-8 encoded text, it returns the length of the text in code points. + * (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization) + * (not in characters: the length of the text "" can be either 1 or 2, depending on the normalization) + * Otherwise, the behavior is undefined. + */ +struct LengthUTF8Impl +{ + static constexpr auto is_fixed_to_constant = false; + + static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + { + size_t size = offsets.size(); + + ColumnString::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + res[i] = UTF8::countCodePoints(&data[prev_offset], offsets[i] - prev_offset - 1); + prev_offset = offsets[i]; + } + } + + static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/) + { + } + + static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + { + size_t size = data.size() / n; + + for (size_t i = 0; i < size; ++i) + { + res[i] = UTF8::countCodePoints(&data[i * n], n); + } + } + + static void array(const ColumnString::Offsets &, PaddedPODArray &) + { + throw Exception("Cannot apply function lengthUTF8 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } +}; + +struct NameLengthUTF8 +{ + static constexpr auto name = "lengthUTF8"; +}; +using FunctionLengthUTF8 = FunctionStringOrArrayToT; + +void registerFunctionLengthUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lower.cpp b/dbms/src/Functions/lower.cpp new file mode 100644 index 00000000000..38379fa10cd --- /dev/null +++ b/dbms/src/Functions/lower.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameLower +{ + static constexpr auto name = "lower"; +}; +using FunctionLower = FunctionStringToString, NameLower>; + +void registerFunctionLower(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/lowerUTF8.cpp b/dbms/src/Functions/lowerUTF8.cpp new file mode 100644 index 00000000000..fed031b524c --- /dev/null +++ b/dbms/src/Functions/lowerUTF8.cpp @@ -0,0 +1,23 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct NameLowerUTF8 +{ + static constexpr auto name = "lowerUTF8"; +}; + +using FunctionLowerUTF8 = FunctionStringToString>, NameLowerUTF8>; + +void registerFunctionLowerUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/notEmpty.cpp b/dbms/src/Functions/notEmpty.cpp new file mode 100644 index 00000000000..d8057501848 --- /dev/null +++ b/dbms/src/Functions/notEmpty.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameNotEmpty +{ + static constexpr auto name = "notEmpty"; +}; +using FunctionNotEmpty = FunctionStringOrArrayToT, NameNotEmpty, UInt8>; + +void registerFunctionNotEmpty(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp new file mode 100644 index 00000000000..9adf3a52824 --- /dev/null +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -0,0 +1,44 @@ +#include + +namespace DB +{ + +void registerFunctionEmpty(FunctionFactory &); +void registerFunctionNotEmpty(FunctionFactory &); +void registerFunctionLength(FunctionFactory &); +void registerFunctionLengthUTF8(FunctionFactory &); +void registerFunctionLower(FunctionFactory &); +void registerFunctionUpper(FunctionFactory &); +void registerFunctionLowerUTF8(FunctionFactory &); +void registerFunctionUpperUTF8(FunctionFactory &); +void registerFunctionReverse(FunctionFactory &); +void registerFunctionReverseUTF8(FunctionFactory &); +void registerFunctionsConcat(FunctionFactory &); +void registerFunctionSubstring(FunctionFactory &); +void registerFunctionSubstringUTF8(FunctionFactory &); +void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); +void registerFunctionStartsWith(FunctionFactory &); +void registerFunctionEndsWith(FunctionFactory &); + +void registerFunctionsString(FunctionFactory & factory) +{ + registerFunctionEmpty(factory); + registerFunctionNotEmpty(factory); + registerFunctionLength(factory); + registerFunctionLengthUTF8(factory); + registerFunctionLower(factory); + registerFunctionUpper(factory); + registerFunctionLowerUTF8(factory); + registerFunctionUpperUTF8(factory); + registerFunctionReverse(factory); + registerFunctionReverseUTF8(factory); + registerFunctionsConcat(factory); + registerFunctionSubstring(factory); + registerFunctionSubstringUTF8(factory); + registerFunctionAppendTrailingCharIfAbsent(factory); + registerFunctionStartsWith(factory); + registerFunctionEndsWith(factory); +} + +} + diff --git a/dbms/src/Functions/reverse.cpp b/dbms/src/Functions/reverse.cpp new file mode 100644 index 00000000000..8f7f8367c2d --- /dev/null +++ b/dbms/src/Functions/reverse.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** Reverse the string as a sequence of bytes. + */ +struct ReverseImpl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & 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 vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & 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]; + } +}; + + +/// Also works with arrays. +class FunctionReverse : public IFunction +{ +public: + static constexpr auto name = "reverse"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool isInjective(const Block &) override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isStringOrFixedString(arguments[0]) + && !isArray(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return arguments[0]; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr column = block.getByPosition(arguments[0]).column; + if (const ColumnString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + { + auto col_res = ColumnFixedString::create(col->getN()); + ReverseImpl::vector_fixed(col->getChars(), col->getN(), col_res->getChars()); + block.getByPosition(result).column = std::move(col_res); + } + else if (checkColumn(column.get())) + { + FunctionArrayReverse().execute(block, arguments, result, input_rows_count); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +void registerFunctionReverse(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/reverseUTF8.cpp b/dbms/src/Functions/reverseUTF8.cpp new file mode 100644 index 00000000000..756a959141c --- /dev/null +++ b/dbms/src/Functions/reverseUTF8.cpp @@ -0,0 +1,81 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +/** Reverse the sequence of code points in a UTF-8 encoded string. + * The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols. + * If the string is not encoded in UTF-8, then the behavior is undefined. + */ +struct ReverseUTF8Impl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars_t & 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) + { + ColumnString::Offset j = prev_offset; + while (j < offsets[i] - 1) + { + if (data[j] < 0xBF) + { + res_data[offsets[i] + prev_offset - 2 - j] = data[j]; + j += 1; + } + else if (data[j] < 0xE0) + { + memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 1], &data[j], 2); + j += 2; + } + else if (data[j] < 0xF0) + { + memcpy(&res_data[offsets[i] + prev_offset - 2 - j - 2], &data[j], 3); + j += 3; + } + else + { + res_data[offsets[i] + prev_offset - 2 - j] = data[j]; + j += 1; + } + } + + res_data[offsets[i] - 1] = 0; + prev_offset = offsets[i]; + } + } + + static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) + { + throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +struct NameReverseUTF8 +{ + static constexpr auto name = "reverseUTF8"; +}; +using FunctionReverseUTF8 = FunctionStringToString; + +void registerFunctionReverseUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/startsWith.cpp b/dbms/src/Functions/startsWith.cpp new file mode 100644 index 00000000000..8e9a06cd17c --- /dev/null +++ b/dbms/src/Functions/startsWith.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionStartsWith = FunctionStartsEndsWith; + +void registerFunctionStartsWith(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/substring.cpp b/dbms/src/Functions/substring.cpp new file mode 100644 index 00000000000..c8c1ed2d9d4 --- /dev/null +++ b/dbms/src/Functions/substring.cpp @@ -0,0 +1,168 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; + extern const int ZERO_ARRAY_OR_TUPLE_INDEX; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +class FunctionSubstring : public IFunction +{ +public: + static constexpr auto name = "substring"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + size_t number_of_arguments = arguments.size(); + + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(number_of_arguments) + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isStringOrFixedString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isNumber(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + + " of second argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (number_of_arguments == 3 && !isNumber(arguments[2])) + throw Exception("Illegal type " + arguments[2]->getName() + + " of second argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + template + void executeForSource(const ColumnPtr & column_start, const ColumnPtr & column_length, + const ColumnConst * column_start_const, const ColumnConst * column_length_const, + Int64 start_value, Int64 length_value, Block & block, size_t result, Source && source, + size_t input_rows_count) + { + auto col_res = ColumnString::create(); + + if (!column_length) + { + if (column_start_const) + { + if (start_value > 0) + sliceFromLeftConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), start_value - 1); + else if (start_value < 0) + sliceFromRightConstantOffsetUnbounded(source, StringSink(*col_res, input_rows_count), -start_value); + else + throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + } + else + sliceDynamicOffsetUnbounded(source, StringSink(*col_res, input_rows_count), *column_start); + } + else + { + if (column_start_const && column_length_const) + { + if (start_value > 0) + sliceFromLeftConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), start_value - 1, length_value); + else if (start_value < 0) + sliceFromRightConstantOffsetBounded(source, StringSink(*col_res, input_rows_count), -start_value, length_value); + else + throw Exception("Indices in strings are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + } + else + sliceDynamicOffsetBounded(source, StringSink(*col_res, input_rows_count), *column_start, *column_length); + } + + block.getByPosition(result).column = std::move(col_res); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + size_t number_of_arguments = arguments.size(); + + ColumnPtr column_string = block.getByPosition(arguments[0]).column; + ColumnPtr column_start = block.getByPosition(arguments[1]).column; + ColumnPtr column_length; + + if (number_of_arguments == 3) + column_length = block.getByPosition(arguments[2]).column; + + const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); + const ColumnConst * column_length_const = nullptr; + + if (number_of_arguments == 3) + column_length_const = checkAndGetColumn(column_length.get()); + + Int64 start_value = 0; + Int64 length_value = 0; + + if (column_start_const) + { + start_value = column_start_const->getInt(0); + } + if (column_length_const) + { + length_value = column_length_const->getInt(0); + if (length_value < 0) + throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, StringSource(*col), input_rows_count); + else if (const ColumnFixedString * col = checkAndGetColumn(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, FixedStringSource(*col), input_rows_count); + else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, ConstSource(*col), input_rows_count); + else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) + executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, + length_value, block, result, ConstSource(*col), input_rows_count); + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +void registerFunctionSubstring(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/substringUTF8.cpp b/dbms/src/Functions/substringUTF8.cpp new file mode 100644 index 00000000000..aa28a541923 --- /dev/null +++ b/dbms/src/Functions/substringUTF8.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; +} + + +/** If the string is encoded in UTF-8, then it selects a substring of code points in it. + * Otherwise, the behavior is undefined. + */ +struct SubstringUTF8Impl +{ + static void vector(const ColumnString::Chars_t & data, + const ColumnString::Offsets & offsets, + size_t start, + size_t length, + ColumnString::Chars_t & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.reserve(data.size()); + size_t size = offsets.size(); + res_offsets.resize(size); + + ColumnString::Offset prev_offset = 0; + ColumnString::Offset res_offset = 0; + for (size_t i = 0; i < size; ++i) + { + ColumnString::Offset j = prev_offset; + ColumnString::Offset pos = 1; + ColumnString::Offset bytes_start = 0; + ColumnString::Offset bytes_length = 0; + while (j < offsets[i] - 1) + { + if (pos == start) + bytes_start = j - prev_offset + 1; + + if (data[j] < 0xBF) + j += 1; + else if (data[j] < 0xE0) + j += 2; + else if (data[j] < 0xF0) + j += 3; + else + j += 1; + + if (pos >= start && pos < start + length) + bytes_length = j - prev_offset + 1 - bytes_start; + else if (pos >= start + length) + break; + + ++pos; + } + + if (bytes_start == 0) + { + res_data.resize(res_data.size() + 1); + res_data[res_offset] = 0; + ++res_offset; + } + else + { + size_t bytes_to_copy = std::min(offsets[i] - prev_offset - bytes_start, bytes_length); + res_data.resize(res_data.size() + bytes_to_copy + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], &data[prev_offset + bytes_start - 1], bytes_to_copy); + res_offset += bytes_to_copy + 1; + res_data[res_offset - 1] = 0; + } + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + } +}; + + +class FunctionSubstringUTF8 : public IFunction +{ +public: + static constexpr auto name = "substringUTF8"; + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 3; + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception( + "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isNumber(arguments[1]) || !isNumber(arguments[2])) + throw Exception("Illegal type " + (isNumber(arguments[1]) ? arguments[2]->getName() : arguments[1]->getName()) + + " of argument of function " + + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnPtr column_start = block.getByPosition(arguments[1]).column; + const ColumnPtr column_length = block.getByPosition(arguments[2]).column; + + if (!column_start->isColumnConst() || !column_length->isColumnConst()) + throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); + + Field start_field = (*block.getByPosition(arguments[1]).column)[0]; + Field length_field = (*block.getByPosition(arguments[2]).column)[0]; + + if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64) + throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type."); + + UInt64 start = start_field.get(); + UInt64 length = length_field.get(); + + if (start == 0) + throw Exception("Second argument of function substring must be greater than 0.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + /// Otherwise may lead to overflow and pass bounds check inside inner loop. + if (start >= 0x8000000000000000ULL || length >= 0x8000000000000000ULL) + throw Exception("Too large values of 2nd or 3rd argument provided for function substring.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (const ColumnString * col = checkAndGetColumn(column_string.get())) + { + auto col_res = ColumnString::create(); + SubstringUTF8Impl::vector(col->getChars(), col->getOffsets(), start, length, col_res->getChars(), col_res->getOffsets()); + block.getByPosition(result).column = std::move(col_res); + } + else + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +void registerFunctionSubstringUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/upper.cpp b/dbms/src/Functions/upper.cpp new file mode 100644 index 00000000000..9a884ccedd8 --- /dev/null +++ b/dbms/src/Functions/upper.cpp @@ -0,0 +1,21 @@ +#include +#include +#include +#include + + +namespace DB +{ + +struct NameUpper +{ + static constexpr auto name = "upper"; +}; +using FunctionUpper = FunctionStringToString, NameUpper>; + +void registerFunctionUpper(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/upperUTF8.cpp b/dbms/src/Functions/upperUTF8.cpp new file mode 100644 index 00000000000..048eb7b60c9 --- /dev/null +++ b/dbms/src/Functions/upperUTF8.cpp @@ -0,0 +1,23 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct NameUpperUTF8 +{ + static constexpr auto name = "upperUTF8"; +}; + +using FunctionUpperUTF8 = FunctionStringToString>, NameUpperUTF8>; + +void registerFunctionUpperUTF8(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +}