diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 18426e35353..e2360f34c87 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -312,6 +312,65 @@ struct MatchImpl }; +/** Применяет регексп и достаёт: + * - первый subpattern, если в regexp-е *есть subpattern; + * - нулевой subpattern (сматчившуюся часть, иначе); + * - если не сматчилось - пустую строку. + */ +struct ExtractImpl +{ + static void vector(const std::vector & data, const ColumnArray::Offsets_t & offsets, + const std::string & pattern, + std::vector & 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(&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 vdata(data.begin(), data.end()); + ColumnArray::Offsets_t offsets(1, data.size()); + std::vector 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 class FunctionsStringSearch : public IFunction { @@ -378,16 +437,82 @@ public: }; +template +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(&*arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!dynamic_cast(&*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(&*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(&*column)) + { + ColumnString * col_res = new ColumnString; + block.getByPosition(result).column = col_res; + + ColumnUInt8::Container_t & vec_res = dynamic_cast(col_res->getData()).getData(); + ColumnString::Offsets_t & offsets_res = col_res->getOffsets(); + Impl::vector(dynamic_cast(col->getData()).getData(), col->getOffsets(), col_needle->getData(), vec_res, offsets_res); + } + else if (const ColumnConstString * col = dynamic_cast(&*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 FunctionPosition; typedef FunctionsStringSearch FunctionPositionUTF8; typedef FunctionsStringSearch, NameMatch> FunctionMatch; -typedef FunctionsStringSearch, NameLike> FunctionLike; +typedef FunctionsStringSearch, NameLike> FunctionLike; typedef FunctionsStringSearch, NameNotLike> FunctionNotLike; +typedef FunctionsStringSearchToString FunctionExtract; } diff --git a/dbms/src/Functions/FunctionFactory.cpp b/dbms/src/Functions/FunctionFactory.cpp index cb8d6d60ce1..0f9c49cbdd0 100644 --- a/dbms/src/Functions/FunctionFactory.cpp +++ b/dbms/src/Functions/FunctionFactory.cpp @@ -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;