ClickHouse/dbms/include/DB/Functions/FunctionsString.h

201 lines
6.7 KiB
C++
Raw Normal View History

#pragma once
2011-10-03 04:06:34 +00:00
2011-10-03 05:29:11 +00:00
#include <Poco/UTF8Encoding.h>
#include <Poco/Unicode.h>
2011-10-03 04:06:34 +00:00
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/DataTypes/DataTypeString.h>
2011-10-03 04:06:34 +00:00
#include <DB/Functions/IFunction.h>
2011-10-03 04:06:34 +00:00
namespace DB
{
/** Функции работы со строками:
*
* length, empty, notEmpty,
2016-06-23 22:28:15 +00:00
* concat, substring, lower, upper, reverse
* lengthUTF8, substringUTF8, lowerUTF8, upperUTF8, reverseUTF8
2011-10-03 04:06:34 +00:00
*
* s -> UInt8: empty, notEmpty
2011-10-03 04:06:34 +00:00
* s -> UInt64: length, lengthUTF8
2016-06-23 22:28:15 +00:00
* s -> s: lower, upper, lowerUTF8, upperUTF8, reverse, reverseUTF8
2011-10-03 04:06:34 +00:00
* s, s -> s: concat
2011-10-10 10:05:39 +00:00
* s, c1, c2 -> s: substring, substringUTF8
2011-10-03 04:06:34 +00:00
* s, c1, c2, s2 -> s: replace, replaceUTF8
*
2011-10-03 04:06:34 +00:00
* Функции поиска строк и регулярных выражений расположены отдельно.
* Функции работы с URL расположены отдельно.
* Функции кодирования строк, конвертации в другие типы расположены отдельно.
2012-09-03 04:45:46 +00:00
*
* Функции length, empty, notEmpty, reverse также работают с массивами.
2011-10-03 04:06:34 +00:00
*/
/// xor or do nothing
template <bool>
UInt8 xor_or_identity(const UInt8 c, const int mask)
2011-10-03 04:06:34 +00:00
{
return c ^ mask;
2011-10-03 04:06:34 +00:00
};
template <>
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
2011-10-03 05:29:11 +00:00
{
return c;
}
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
template <bool to_lower>
inline void UTF8CyrillicToCase(const UInt8 *& src, const UInt8 * const src_end, UInt8 *& dst)
{
if (src[0] == 0xD0u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// ЀЁЂЃЄЅІЇЈЉЊЋЌЍЎЏ
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD1u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// ѐёђѓєѕіїјљњћќѝўџ
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x10);
}
else if (src[0] == 0xD0u && (src[1] >= 0x90u && src[1] <= 0x9Fu))
{
/// А
*dst++ = *src++;
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xB0u && src[1] <= 0xBFu))
{
/// а-п
*dst++ = *src++;
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD0u && (src[1] >= 0xA0u && src[1] <= 0xAFu))
{
/// Р
*dst++ = xor_or_identity<to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<to_lower>(*src++, 0x20);
}
else if (src[0] == 0xD1u && (src[1] >= 0x80u && src[1] <= 0x8Fu))
{
/// р
*dst++ = xor_or_identity<!to_lower>(*src++, 0x1);
*dst++ = xor_or_identity<!to_lower>(*src++, 0x20);
}
}
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
* Замечание: предполагается, что после перевода символа в другой регистр,
* длина его мультибайтовой последовательности в UTF-8 не меняется.
* Иначе - поведение не определено.
*/
template <char not_case_lower_bound,
char not_case_upper_bound,
int to_case(int),
void cyrillic_to_case(const UInt8 *&, const UInt8 *, UInt8 *&)>
struct LowerUpperUTF8Impl
{
static void vector(const ColumnString::Chars_t & data,
const ColumnString::Offsets_t & offsets,
ColumnString::Chars_t & res_data,
ColumnString::Offsets_t & 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 * const 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);
2011-10-10 12:26:26 +00:00
};
template <typename Impl, typename Name, bool is_injective = false>
class FunctionStringToString : public IFunction
2011-10-03 04:06:34 +00:00
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context & context)
2011-10-03 04:06:34 +00:00
{
return std::make_shared<FunctionStringToString>();
2011-10-03 04:06:34 +00:00
}
String getName() const override
2011-10-03 04:06:34 +00:00
{
return name;
2011-10-03 04:06:34 +00:00
}
size_t getNumberOfArguments() const override
2011-10-03 04:06:34 +00:00
{
return 1;
2011-10-03 04:06:34 +00:00
}
bool isInjective(const Block &) override
2011-10-03 05:29:11 +00:00
{
return is_injective;
2011-10-03 05:29:11 +00:00
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
2011-10-03 05:29:11 +00:00
{
if (!typeid_cast<const DataTypeString *>(&*arguments[0]) && !typeid_cast<const DataTypeFixedString *>(&*arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
2011-10-03 05:29:11 +00:00
return arguments[0]->clone();
2011-10-03 05:29:11 +00:00
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
2011-10-03 05:29:11 +00:00
{
const ColumnPtr column = block.safeGetByPosition(arguments[0]).column;
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
2011-10-03 05:29:11 +00:00
{
std::shared_ptr<ColumnString> col_res = std::make_shared<ColumnString>();
block.safeGetByPosition(result).column = col_res;
Impl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
2011-10-03 05:29:11 +00:00
}
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(&*column))
2011-10-03 05:29:11 +00:00
{
auto col_res = std::make_shared<ColumnFixedString>(col->getN());
block.safeGetByPosition(result).column = col_res;
Impl::vector_fixed(col->getChars(), col->getN(), col_res->getChars());
2011-10-03 05:29:11 +00:00
}
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
2011-10-03 05:29:11 +00:00
{
String res;
Impl::constant(col->getData(), res);
auto col_res = std::make_shared<ColumnConstString>(col->size(), res);
block.safeGetByPosition(result).column = col_res;
2011-10-03 05:29:11 +00:00
}
else
throw Exception(
"Illegal column " + block.safeGetByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
2011-10-03 05:29:11 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
};
struct NameLowerUTF8
{
static constexpr auto name = "lowerUTF8";
};
struct NameUpperUTF8
2011-10-10 06:10:49 +00:00
{
static constexpr auto name = "upperUTF8";
2011-10-10 10:05:39 +00:00
};
typedef FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>
FunctionLowerUTF8;
typedef FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>
FunctionUpperUTF8;
2011-10-03 04:06:34 +00:00
}