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

412 lines
12 KiB
C
Raw Normal View History

2012-09-17 04:34:19 +00:00
#pragma once
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/FunctionsStringSearch.h>
2012-09-17 04:34:19 +00:00
namespace DB
{
/** Функции, разделяющие строки на массив строк или наоборот.
*
* splitByChar(sep, s)
* splitByString(sep, s)
* splitByRegexp(regexp, s)
*
* extractAll(s, regexp) - выделить из строки подпоследовательности, соответствующие регекспу.
* - первый subpattern, если в regexp-е есть subpattern;
* - нулевой subpattern (сматчившуюся часть, иначе);
* - инача, пустой массив
*
2012-09-17 04:34:19 +00:00
* join(sep, arr)
* join(arr)
*
2012-09-22 07:30:40 +00:00
* alphaTokens(s) - выделить из строки подпоследовательности [a-zA-Z]+.
*
2012-09-17 04:34:19 +00:00
* Функции работы с URL расположены отдельно.
*/
typedef const char * Pos;
/// Генераторы подстрок. Все они обладают общим интерфейсом.
class AlphaTokensImpl
2012-09-17 04:34:19 +00:00
{
private:
Pos pos;
Pos end;
public:
/// Получить имя фукнции.
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 "
2013-06-21 20:34:19 +00:00
+ toString(arguments.size()) + ", should be 1.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
2012-09-17 04:34:19 +00:00
if (!typeid_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) {}
2012-09-17 04:34:19 +00:00
/// Вызывается для каждой следующей строки.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
2012-09-17 04:34:19 +00:00
/// Возвращает позицию аргумента, являющегося столбцом строк
size_t getStringsArgumentPosition()
{
return 0;
}
/// Получить следующий токен, если есть, или вернуть false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Пропускаем мусор
while (pos < end && !((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
++pos;
2012-09-17 04:34:19 +00:00
if (pos == end)
return false;
2012-09-17 04:34:19 +00:00
token_begin = pos;
while (pos < end && ((*pos >= 'a' && *pos <= 'z') || (*pos >= 'A' && *pos <= 'Z')))
++pos;
token_end = pos;
return true;
}
};
2012-09-17 04:34:19 +00:00
class SplitByCharImpl
2012-09-17 04:34:19 +00:00
{
private:
Pos pos;
Pos end;
char sep;
2012-09-17 04:34:19 +00:00
public:
static String getName() { return "splitByChar"; }
2012-09-17 04:34:19 +00:00
static void checkArguments(const DataTypes & arguments)
2012-09-17 04:34:19 +00:00
{
if (arguments.size() != 2)
2012-09-17 04:34:19 +00:00
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
2013-06-21 20:34:19 +00:00
+ toString(arguments.size()) + ", should be 2.",
2012-09-17 04:34:19 +00:00
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!typeid_cast<const DataTypeString *>(&*arguments[0]))
2012-09-17 04:34:19 +00:00
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!typeid_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 = typeid_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];
}
/// Возвращает позицию аргумента, являющегося столбцом строк
size_t getStringsArgumentPosition()
{
return 1;
}
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 = typeid_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();
}
/// Возвращает позицию аргумента, являющегося столбцом строк
size_t getStringsArgumentPosition()
{
return 1;
}
/// Вызывается для каждой следующей строки.
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;
}
};
class ExtractAllImpl
{
private:
2014-04-08 07:31:51 +00:00
const OptimizedRegularExpression * re = nullptr;
OptimizedRegularExpression::MatchVec matches;
size_t capture;
Pos pos;
Pos end;
public:
/// Получить имя функции.
static String getName() { return "extractAll"; }
/// Проверить типы агрументов функции.
static void checkArguments( const DataTypes & arguments )
{
SplitByStringImpl::checkArguments(arguments);
}
/// Инициализировать по аргументам функции.
void init(Block & block, const ColumnNumbers & arguments)
{
const ColumnConstString * col = typeid_cast<const ColumnConstString *>(&*block.getByPosition(arguments[1]).column);
if (!col)
throw Exception("Illegal column " + col->getName() + " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
re = &Regexps::get(col->getData());
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
matches.resize(capture + 1);
}
/// Возвращает позицию аргумента, являющегося столбцом строк
size_t getStringsArgumentPosition()
{
return 0;
}
/// Вызывается для каждой следующей строки.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Получить следующий токен, если есть, или вернуть false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos || pos > end)
return false;
2013-09-10 16:52:40 +00:00
if (!re->match(pos, end - pos, matches) || !matches[capture].length)
return false;
token_begin = pos + matches[capture].offset;
token_end = token_begin + matches[capture].length;
pos += matches[capture].offset + matches[capture].length;
return true;
}
};
/// Функция, принимающая строку, и возвращающая массив подстрок, создаваемый некоторым генератором.
template <typename Generator>
class FunctionTokens : public IFunction
{
public:
/// Получить имя функции.
String getName() const
{
return Generator::getName();
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const
{
2013-03-26 19:30:23 +00:00
Generator::checkArguments(arguments);
2012-09-17 04:34:19 +00:00
return new DataTypeArray(new DataTypeString);
}
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
{
2013-03-26 19:30:23 +00:00
Generator generator;
generator.init(block, arguments);
size_t arrayArgumentPosition = arguments[generator.getStringsArgumentPosition()];
const ColumnString * col_str = typeid_cast<const ColumnString *>(&*block.getByPosition(arrayArgumentPosition).column);
const ColumnConstString * col_const_str =
typeid_cast<const ColumnConstString *>(&*block.getByPosition(arrayArgumentPosition).column);
2012-09-17 04:34:19 +00:00
ColumnArray * col_res = new ColumnArray(new ColumnString);
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
2012-09-17 04:34:19 +00:00
ColumnArray::Offsets_t & res_offsets = col_res->getOffsets();
ColumnString::Chars_t & res_strings_chars = res_strings.getChars();
2012-09-17 04:34:19 +00:00
ColumnString::Offsets_t & res_strings_offsets = res_strings.getOffsets();
if (col_str)
{
const ColumnString::Chars_t & src_chars = col_str->getChars();
2012-09-17 04:34:19 +00:00
const ColumnString::Offsets_t & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
res_strings_offsets.reserve(src_offsets.size() * 5); /// Константа 5 - наугад.
res_strings_chars.reserve(src_chars.size());
2014-04-08 07:31:51 +00:00
Pos token_begin = nullptr;
Pos token_end = nullptr;
2012-09-17 04:34:19 +00:00
size_t size = src_offsets.size();
ColumnString::Offset_t current_src_offset = 0;
ColumnArray::Offset_t current_dst_offset = 0;
ColumnString::Offset_t current_dst_strings_offset = 0;
for (size_t i = 0; i < size; ++i)
{
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]) - 1;
generator.set(pos, end);
2012-09-17 04:34:19 +00:00
size_t j = 0;
while (generator.get(token_begin, token_end))
2012-09-17 04:34:19 +00:00
{
size_t token_size = token_end - token_begin;
res_strings_chars.resize(res_strings_chars.size() + token_size + 1);
memcpy(&res_strings_chars[current_dst_strings_offset], token_begin, token_size);
2013-10-31 19:24:22 +00:00
res_strings_chars[current_dst_strings_offset + token_size] = 0;
2012-09-17 04:34:19 +00:00
current_dst_strings_offset += token_size + 1;
res_strings_offsets.push_back(current_dst_strings_offset);
++j;
}
current_dst_offset += j;
res_offsets.push_back(current_dst_offset);
}
block.getByPosition(result).column = col_res;
}
else if (col_const_str)
{
String src = col_const_str->getData();
Array dst;
generator.set(src.data(), src.data() + src.size());
2014-04-08 07:31:51 +00:00
Pos token_begin = nullptr;
Pos token_end = nullptr;
2012-09-17 04:34:19 +00:00
while (generator.get(token_begin, token_end))
2012-09-17 04:34:19 +00:00
dst.push_back(String(token_begin, token_end - token_begin));
2013-03-27 10:07:23 +00:00
block.getByPosition(result).column = new ColumnConstArray(col_const_str->size(), dst, new DataTypeArray(new DataTypeString));
2012-09-17 04:34:19 +00:00
}
else
throw Exception("Illegal columns " + block.getByPosition(arrayArgumentPosition).column->getName()
+ ", " + block.getByPosition(arrayArgumentPosition).column->getName()
2012-09-17 04:34:19 +00:00
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
};
typedef FunctionTokens<AlphaTokensImpl> FunctionAlphaTokens;
typedef FunctionTokens<SplitByCharImpl> FunctionSplitByChar;
typedef FunctionTokens<SplitByStringImpl> FunctionSplitByString;
typedef FunctionTokens<ExtractAllImpl> FunctionExtractAll;
2012-09-17 04:34:19 +00:00
}