2017-03-10 17:53:32 +00:00
|
|
|
|
#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>
|
2017-03-10 17:53:32 +00:00
|
|
|
|
#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>
|
2015-05-28 15:34:37 +00:00
|
|
|
|
|
2011-10-03 04:06:34 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
/** Функции работы со строками:
|
|
|
|
|
*
|
2012-09-23 00:04:17 +00:00
|
|
|
|
* 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
|
|
|
|
*
|
2017-04-01 07:20:54 +00:00
|
|
|
|
* 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
|
2014-06-26 00:58:14 +00:00
|
|
|
|
*
|
2011-10-03 04:06:34 +00:00
|
|
|
|
* Функции поиска строк и регулярных выражений расположены отдельно.
|
|
|
|
|
* Функции работы с URL расположены отдельно.
|
|
|
|
|
* Функции кодирования строк, конвертации в другие типы расположены отдельно.
|
2012-09-03 04:45:46 +00:00
|
|
|
|
*
|
2015-10-19 21:07:24 +00:00
|
|
|
|
* Функции length, empty, notEmpty, reverse также работают с массивами.
|
2011-10-03 04:06:34 +00:00
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
|
2017-03-10 17:53:32 +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
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
return c ^ mask;
|
2011-10-03 04:06:34 +00:00
|
|
|
|
};
|
2017-03-10 17:53:32 +00:00
|
|
|
|
template <>
|
|
|
|
|
inline UInt8 xor_or_identity<false>(const UInt8 c, const int)
|
2011-10-03 05:29:11 +00:00
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
return c;
|
2017-03-10 17:53:32 +00:00
|
|
|
|
}
|
2015-06-04 12:16:42 +00:00
|
|
|
|
|
|
|
|
|
/// It is caller's responsibility to ensure the presence of a valid cyrillic sequence in array
|
|
|
|
|
template <bool to_lower>
|
2017-03-10 17:53:32 +00:00
|
|
|
|
inline void UTF8CyrillicToCase(const UInt8 *& src, const UInt8 * const src_end, UInt8 *& dst)
|
2015-06-04 12:16:42 +00:00
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
2017-03-10 17:53:32 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-06-04 12:16:42 +00:00
|
|
|
|
|
2015-06-10 12:47:27 +00:00
|
|
|
|
/** Если строка содержит текст в кодировке UTF-8 - перевести его в нижний (верхний) регистр.
|
|
|
|
|
* Замечание: предполагается, что после перевода символа в другой регистр,
|
|
|
|
|
* длина его мультибайтовой последовательности в UTF-8 не меняется.
|
|
|
|
|
* Иначе - поведение не определено.
|
|
|
|
|
*/
|
2017-03-10 17:53:32 +00:00
|
|
|
|
template <char not_case_lower_bound,
|
2017-04-01 07:20:54 +00:00
|
|
|
|
char not_case_upper_bound,
|
|
|
|
|
int to_case(int),
|
|
|
|
|
void cyrillic_to_case(const UInt8 *&, const UInt8 *, UInt8 *&)>
|
2015-06-10 12:47:27 +00:00
|
|
|
|
struct LowerUpperUTF8Impl
|
2015-05-28 12:32:43 +00:00
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static void vector(const ColumnString::Chars_t & data,
|
|
|
|
|
const ColumnString::Offsets_t & offsets,
|
|
|
|
|
ColumnString::Chars_t & res_data,
|
|
|
|
|
ColumnString::Offsets_t & res_offsets);
|
2015-05-28 12:32:43 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data);
|
2015-05-28 12:32:43 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static void constant(const std::string & data, std::string & res_data);
|
2015-05-28 12:32:43 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/** 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);
|
2015-09-24 12:58:18 +00:00
|
|
|
|
|
2015-05-28 12:32:43 +00:00
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static constexpr auto ascii_upper_bound = '\x7f';
|
|
|
|
|
static constexpr auto flip_case_mask = 'A' ^ 'a';
|
2015-09-24 12:58:18 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst);
|
2011-10-10 12:26:26 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2017-03-10 17:53:32 +00:00
|
|
|
|
template <typename Impl, typename Name, bool is_injective = false>
|
|
|
|
|
class FunctionStringToString : public IFunction
|
2011-10-03 04:06:34 +00:00
|
|
|
|
{
|
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static constexpr auto name = Name::name;
|
|
|
|
|
static FunctionPtr create(const Context & context)
|
|
|
|
|
{
|
|
|
|
|
return std::make_shared<FunctionStringToString>();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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 (!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);
|
|
|
|
|
|
|
|
|
|
return arguments[0]->clone();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
|
|
|
|
|
{
|
|
|
|
|
const ColumnPtr column = block.safeGetByPosition(arguments[0]).column;
|
|
|
|
|
if (const ColumnString * col = typeid_cast<const ColumnString *>(&*column))
|
|
|
|
|
{
|
|
|
|
|
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());
|
|
|
|
|
}
|
|
|
|
|
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(&*column))
|
|
|
|
|
{
|
|
|
|
|
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());
|
|
|
|
|
}
|
|
|
|
|
else if (const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*column))
|
|
|
|
|
{
|
|
|
|
|
String res;
|
|
|
|
|
Impl::constant(col->getData(), res);
|
|
|
|
|
auto col_res = std::make_shared<ColumnConstString>(col->size(), res);
|
|
|
|
|
block.safeGetByPosition(result).column = col_res;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw Exception(
|
|
|
|
|
"Illegal column " + block.safeGetByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
}
|
2011-10-03 05:29:11 +00:00
|
|
|
|
};
|
|
|
|
|
|
2017-03-10 17:53:32 +00:00
|
|
|
|
struct NameLowerUTF8
|
2015-10-19 21:07:24 +00:00
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static constexpr auto name = "lowerUTF8";
|
2015-10-19 21:07:24 +00:00
|
|
|
|
};
|
2017-03-10 17:53:32 +00:00
|
|
|
|
struct NameUpperUTF8
|
2011-10-10 06:10:49 +00:00
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
static constexpr auto name = "upperUTF8";
|
2011-10-10 10:05:39 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
2017-03-10 17:53:32 +00:00
|
|
|
|
typedef FunctionStringToString<LowerUpperUTF8Impl<'A', 'Z', Poco::Unicode::toLower, UTF8CyrillicToCase<true>>, NameLowerUTF8>
|
2017-04-01 07:20:54 +00:00
|
|
|
|
FunctionLowerUTF8;
|
2017-03-10 17:53:32 +00:00
|
|
|
|
typedef FunctionStringToString<LowerUpperUTF8Impl<'a', 'z', Poco::Unicode::toUpper, UTF8CyrillicToCase<false>>, NameUpperUTF8>
|
2017-04-01 07:20:54 +00:00
|
|
|
|
FunctionUpperUTF8;
|
2011-10-03 04:06:34 +00:00
|
|
|
|
}
|