mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
clickhouse: added function extract [#CONV-6788].
This commit is contained in:
parent
eca698ad89
commit
ea816297c2
@ -312,6 +312,65 @@ struct MatchImpl
|
||||
};
|
||||
|
||||
|
||||
/** Применяет регексп и достаёт:
|
||||
* - первый subpattern, если в regexp-е *есть subpattern;
|
||||
* - нулевой subpattern (сматчившуюся часть, иначе);
|
||||
* - если не сматчилось - пустую строку.
|
||||
*/
|
||||
struct ExtractImpl
|
||||
{
|
||||
static void vector(const std::vector<UInt8> & data, const ColumnArray::Offsets_t & offsets,
|
||||
const std::string & pattern,
|
||||
std::vector<UInt8> & res_data, ColumnArray::Offsets_t & res_offsets)
|
||||
{
|
||||
res_data.reserve(data.size() / 5);
|
||||
res_offsets.resize(offsets.size());
|
||||
|
||||
const OptimizedRegularExpression & regexp = Regexps::get(pattern);
|
||||
|
||||
int capture = regexp.getRE2()->NumberOfCapturingGroups() > 0 ? 1 : 0;
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
matches.reserve(capture + 1);
|
||||
size_t prev_offset = 0;
|
||||
size_t res_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < offsets.size(); ++i)
|
||||
{
|
||||
size_t cur_offset = offsets[i];
|
||||
|
||||
regexp.match(reinterpret_cast<const char *>(&data[prev_offset]), cur_offset - prev_offset - 1, matches, capture + 1);
|
||||
const OptimizedRegularExpression::Match & match = matches[capture];
|
||||
if (match.offset != std::string::npos)
|
||||
{
|
||||
res_data.resize(res_offset + match.length + 1);
|
||||
memcpy(&res_data[res_offset], &data[prev_offset + match.offset], match.length);
|
||||
res_offset += match.length;
|
||||
}
|
||||
else
|
||||
{
|
||||
res_data.resize(res_offset + 1);
|
||||
}
|
||||
|
||||
res_data[res_offset] = 0;
|
||||
++res_offset;
|
||||
res_offsets[i] = res_offset;
|
||||
|
||||
prev_offset = cur_offset;
|
||||
}
|
||||
}
|
||||
|
||||
static void constant(const std::string & data, const std::string & pattern, std::string & res)
|
||||
{
|
||||
std::vector<UInt8> vdata(data.begin(), data.end());
|
||||
ColumnArray::Offsets_t offsets(1, data.size());
|
||||
std::vector<UInt8> res_vdata;
|
||||
ColumnArray::Offsets_t res_offsets;
|
||||
vector(vdata, offsets, pattern, res_vdata, res_offsets);
|
||||
res = std::string(res_vdata.begin(), res_vdata.end() - 1);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
class FunctionsStringSearch : public IFunction
|
||||
{
|
||||
@ -378,16 +437,82 @@ public:
|
||||
};
|
||||
|
||||
|
||||
template <typename Impl, typename Name>
|
||||
class FunctionsStringSearchToString : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return Name::get();
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
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 argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!dynamic_cast<const DataTypeString *>(&*arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
|
||||
|
||||
const ColumnConstString * col_needle = dynamic_cast<const ColumnConstString *>(&*column_needle);
|
||||
if (!col_needle)
|
||||
throw Exception("Second argument of function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (const ColumnString * col = dynamic_cast<const ColumnString *>(&*column))
|
||||
{
|
||||
ColumnString * col_res = new ColumnString;
|
||||
block.getByPosition(result).column = col_res;
|
||||
|
||||
ColumnUInt8::Container_t & vec_res = dynamic_cast<ColumnUInt8 &>(col_res->getData()).getData();
|
||||
ColumnString::Offsets_t & offsets_res = col_res->getOffsets();
|
||||
Impl::vector(dynamic_cast<const ColumnUInt8 &>(col->getData()).getData(), col->getOffsets(), col_needle->getData(), vec_res, offsets_res);
|
||||
}
|
||||
else if (const ColumnConstString * col = dynamic_cast<const ColumnConstString *>(&*column))
|
||||
{
|
||||
std::string res = 0;
|
||||
Impl::constant(col->getData(), col_needle->getData(), res);
|
||||
|
||||
ColumnConstString * col_res = new ColumnConstString(col->size(), res);
|
||||
block.getByPosition(result).column = col_res;
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct NamePosition { static const char * get() { return "position"; } };
|
||||
struct NamePositionUTF8 { static const char * get() { return "positionUTF8"; } };
|
||||
struct NameMatch { static const char * get() { return "match"; } };
|
||||
struct NameLike { static const char * get() { return "like"; } };
|
||||
struct NameNotLike { static const char * get() { return "notLike"; } };
|
||||
struct NameExtract { static const char * get() { return "extract"; } };
|
||||
|
||||
typedef FunctionsStringSearch<PositionImpl, NamePosition> FunctionPosition;
|
||||
typedef FunctionsStringSearch<PositionUTF8Impl, NamePositionUTF8> FunctionPositionUTF8;
|
||||
typedef FunctionsStringSearch<MatchImpl<false>, NameMatch> FunctionMatch;
|
||||
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
|
||||
typedef FunctionsStringSearch<MatchImpl<true>, NameLike> FunctionLike;
|
||||
typedef FunctionsStringSearch<MatchImpl<true, true>, NameNotLike> FunctionNotLike;
|
||||
typedef FunctionsStringSearchToString<ExtractImpl, NameExtract> FunctionExtract;
|
||||
|
||||
}
|
||||
|
@ -125,6 +125,7 @@ FunctionPtr FunctionFactory::get(
|
||||
else if (name == "match") return new FunctionMatch;
|
||||
else if (name == "like") return new FunctionLike;
|
||||
else if (name == "notLike") return new FunctionNotLike;
|
||||
else if (name == "extract") return new FunctionExtract;
|
||||
|
||||
else if (name == "halfMD5") return new FunctionHalfMD5;
|
||||
else if (name == "cityHash64") return new FunctionCityHash64;
|
||||
|
Loading…
Reference in New Issue
Block a user