mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
dbms: development (string splitting functions) [#CONV-2944].
This commit is contained in:
parent
93703e86ec
commit
7efe870cf9
@ -13,11 +13,16 @@ namespace DB
|
||||
|
||||
/** Функции, разделяющие строки на массив строк или наоборот.
|
||||
*
|
||||
* split(sep, s)
|
||||
* splitByChar(sep, s)
|
||||
* splitByString(sep, s)
|
||||
* splitByRegexp(regexp, s)
|
||||
*
|
||||
* extractAll(regexp, s) - выделить из строки подпоследовательности, соответствующие регекспу.
|
||||
*
|
||||
* join(sep, arr)
|
||||
* join(arr)
|
||||
*
|
||||
* alphaTokens(s) - выделить из строки подпоследовательности [a-zA-Z]+.
|
||||
* extractAll(regexp, s) - выделить из строки подпоследовательности, соответствующие регекспу.
|
||||
*
|
||||
* Функции работы с URL расположены отдельно.
|
||||
*/
|
||||
@ -25,38 +30,21 @@ namespace DB
|
||||
|
||||
typedef const char * Pos;
|
||||
|
||||
/// Получить следующий кусок [a-zA-Z]+
|
||||
inline Pos nextAlphaToken(Pos pos, Pos end, Pos & token_begin, Pos & token_end)
|
||||
|
||||
/// Генераторы подстрок. Все они обладают общим интерфейсом.
|
||||
|
||||
class AlphaTokensImpl
|
||||
{
|
||||
/// Пропускаем мусор
|
||||
while (pos < end && !((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
|
||||
++pos;
|
||||
private:
|
||||
Pos pos;
|
||||
Pos end;
|
||||
|
||||
if (pos == end)
|
||||
return NULL;
|
||||
|
||||
token_begin = pos;
|
||||
|
||||
while (pos < end && ((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
|
||||
++pos;
|
||||
|
||||
token_end = pos;
|
||||
|
||||
return pos;
|
||||
}
|
||||
|
||||
|
||||
class FunctionAlphaTokens : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return "alphaTokens";
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
/// Получить имя фукнции.
|
||||
static String getName() { return "alphaTokens"; }
|
||||
|
||||
/// Проверить типы агрументов функции.
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
@ -66,15 +54,195 @@ public:
|
||||
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// Инициализировать по аргументам функции.
|
||||
void init(Block & block, const ColumnNumbers & arguments) {}
|
||||
|
||||
/// Вызывается для каждой следующей строки.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
pos = pos_;
|
||||
end = end_;
|
||||
}
|
||||
|
||||
/// Получить следующий токен, если есть, или вернуть false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
/// Пропускаем мусор
|
||||
while (pos < end && !((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
|
||||
++pos;
|
||||
|
||||
if (pos == end)
|
||||
return false;
|
||||
|
||||
token_begin = pos;
|
||||
|
||||
while (pos < end && ((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
|
||||
++pos;
|
||||
|
||||
token_end = pos;
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class SplitByCharImpl
|
||||
{
|
||||
private:
|
||||
Pos pos;
|
||||
Pos end;
|
||||
|
||||
char sep;
|
||||
|
||||
public:
|
||||
static String getName() { return "splitByChar"; }
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!dynamic_cast<const DataTypeString *>(&*arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!dynamic_cast<const DataTypeString *>(&*arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
void init(Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
const ColumnConstString * col = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
const String & sep_str = col->getData();
|
||||
|
||||
if (sep_str.size() != 1)
|
||||
throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.");
|
||||
|
||||
sep = sep_str[0];
|
||||
}
|
||||
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
pos = pos_;
|
||||
end = end_;
|
||||
}
|
||||
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
if (!pos)
|
||||
return false;
|
||||
|
||||
token_begin = pos;
|
||||
pos = reinterpret_cast<Pos>(memchr(pos, sep, end - pos));
|
||||
|
||||
if (pos)
|
||||
{
|
||||
token_end = pos;
|
||||
++pos;
|
||||
}
|
||||
else
|
||||
token_end = end;
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class SplitByStringImpl
|
||||
{
|
||||
private:
|
||||
Pos pos;
|
||||
Pos end;
|
||||
|
||||
String sep;
|
||||
|
||||
public:
|
||||
static String getName() { return "splitByString"; }
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
SplitByCharImpl::checkArguments(arguments);
|
||||
}
|
||||
|
||||
void init(Block & block, const ColumnNumbers & arguments)
|
||||
{
|
||||
const ColumnConstString * col = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
sep = col->getData();
|
||||
}
|
||||
|
||||
/// Вызывается для каждой следующей строки.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
pos = pos_;
|
||||
end = end_;
|
||||
}
|
||||
|
||||
/// Получить следующий токен, если есть, или вернуть false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
if (!pos)
|
||||
return false;
|
||||
|
||||
token_begin = pos;
|
||||
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, sep.data(), sep.size()));
|
||||
|
||||
if (pos)
|
||||
{
|
||||
token_end = pos;
|
||||
pos += sep.size();
|
||||
}
|
||||
else
|
||||
token_end = end;
|
||||
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Функция, принимающая строку, и возвращающая массив подстрок, создаваемый некоторым генератором.
|
||||
template <typename Generator>
|
||||
class FunctionTokens : public IFunction
|
||||
{
|
||||
private:
|
||||
Generator generator;
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return Generator::getName();
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
generator.checkArguments(arguments);
|
||||
|
||||
return new DataTypeArray(new DataTypeString);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnString * col_str = dynamic_cast<const ColumnString *>(&*block.getByPosition(arguments[0]).column);
|
||||
const ColumnConstString * col_const_str = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments[0]).column);
|
||||
generator.init(block, arguments);
|
||||
|
||||
const ColumnString * col_str = dynamic_cast<const ColumnString *>(&*block.getByPosition(arguments.back()).column);
|
||||
const ColumnConstString * col_const_str = dynamic_cast<const ColumnConstString *>(&*block.getByPosition(arguments.back()).column);
|
||||
|
||||
ColumnArray * col_res = new ColumnArray(new ColumnString);
|
||||
ColumnString & res_strings = dynamic_cast<ColumnString &>(col_res->getData());
|
||||
@ -102,10 +270,12 @@ public:
|
||||
{
|
||||
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
|
||||
current_src_offset = src_offsets[i];
|
||||
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
|
||||
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
|
||||
|
||||
generator.set(pos, end);
|
||||
|
||||
size_t j = 0;
|
||||
while (NULL != (pos = nextAlphaToken(pos, end, token_begin, token_end)))
|
||||
while (generator.get(token_begin, token_end))
|
||||
{
|
||||
size_t token_size = token_end - token_begin;
|
||||
|
||||
@ -129,23 +299,26 @@ public:
|
||||
String src = col_const_str->getData();
|
||||
Array dst;
|
||||
|
||||
Pos pos = src.data();
|
||||
Pos end = src.data() + src.size();
|
||||
generator.set(src.data(), src.data() + src.size());
|
||||
Pos token_begin = NULL;
|
||||
Pos token_end = NULL;
|
||||
|
||||
while (NULL != (pos = nextAlphaToken(pos, end, token_begin, token_end)))
|
||||
while (generator.get(token_begin, token_end))
|
||||
dst.push_back(String(token_begin, token_end - token_begin));
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(col_const_str->size(), dst);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ ", " + block.getByPosition(arguments[1]).column->getName()
|
||||
throw Exception("Illegal columns " + block.getByPosition(arguments.back()).column->getName()
|
||||
+ ", " + block.getByPosition(arguments.back()).column->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
typedef FunctionTokens<AlphaTokensImpl> FunctionAlphaTokens;
|
||||
typedef FunctionTokens<SplitByCharImpl> FunctionSplitByChar;
|
||||
typedef FunctionTokens<SplitByStringImpl> FunctionSplitByString;
|
||||
|
||||
}
|
||||
|
@ -135,6 +135,8 @@ namespace FunctionsLibrary
|
||||
("array", new FunctionArray)
|
||||
|
||||
("alphaTokens", new FunctionAlphaTokens)
|
||||
("splitByChar", new FunctionSplitByChar)
|
||||
("splitByString", new FunctionSplitByString)
|
||||
;
|
||||
|
||||
return res;
|
||||
|
Loading…
Reference in New Issue
Block a user