#include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE; } namespace { class FunctionRegexpExtract : public IFunction { public: static constexpr auto name = "regexpExtract"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 2 && arguments.size() != 3) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}", getName(), arguments.size()); if (!isString(arguments[0])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}", arguments[0]->getName(), getName()); if (!isString(arguments[1])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}", arguments[1]->getName(), getName()); if (arguments.size() > 2 && !isInteger(arguments[2])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of third argument of function {}", arguments[2]->getName(), getName()); return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override { const ColumnPtr column = arguments[0].column; const ColumnPtr column_pattern = arguments[1].column; const ColumnPtr column_index = arguments.size() > 2 ? arguments[2].column : nullptr; /// Check if the second argument is const column const ColumnConst * col_pattern = typeid_cast(&*column_pattern); if (!col_pattern) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function {} must be constant string", getName()); /// Check if the first argument is string column(const or not) const ColumnConst * col_const = typeid_cast(&*column); const ColumnString * col = nullptr; if (col_const) col = typeid_cast(&col_const->getDataColumn()); else col = typeid_cast(&*column); if (!col) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); auto col_res = ColumnString::create(); ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); if (col_const) constantVector(col_const->getValue(), col_pattern->getValue(), column_index, vec_res, offsets_res); else if (!column_index || isColumnConst(*column_index)) { const auto * col_const_index = typeid_cast(&*column_index); ssize_t index = !col_const_index ? 1 : col_const_index->getInt(0); vectorConstant(col->getChars(), col->getOffsets(), col_pattern->getValue(), index, vec_res, offsets_res); } else vectorVector(col->getChars(), col->getOffsets(), col_pattern->getValue(), column_index, vec_res, offsets_res); return col_res; } private: static void vectorConstant( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, ssize_t index, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { const Regexps::Regexp regexp = Regexps::createRegexp(pattern); unsigned capture = regexp.getNumberOfSubpatterns(); if (index < 0 || index >= capture + 1) throw Exception( ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE, "Index value {} is out of range, should be in [0, {})", index, capture + 1); OptimizedRegularExpression::MatchVec matches; matches.reserve(index + 1); res_data.reserve(data.size() / 5); res_offsets.resize(offsets.size()); 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]; unsigned count = regexp.match( reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1, matches, static_cast(index + 1)); if (count == index + 1 && matches[index].offset != std::string::npos) { const auto & match = matches[index]; res_data.resize(res_offset + match.length + 1); memcpySmallAllowReadWriteOverflow15(&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 vectorVector( const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, const ColumnPtr & column_index, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.reserve(data.size() / 5); res_offsets.resize(offsets.size()); const Regexps::Regexp regexp = Regexps::createRegexp(pattern); unsigned capture = regexp.getNumberOfSubpatterns(); 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]; ssize_t index = column_index->getInt(i); if (index < 0 || index >= capture + 1) throw Exception( ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE, "Index value {} is out of range, should be in [0, {})", index, capture + 1); unsigned count = regexp.match( reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1, matches, static_cast(index + 1)); if (count == index + 1 && matches[index].offset != std::string::npos) { const auto & match = matches[index]; res_data.resize(res_offset + match.length + 1); memcpySmallAllowReadWriteOverflow15(&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 constantVector( const std::string & str, const std::string & pattern, const ColumnPtr & column_index, ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { size_t rows = column_index->size(); res_data.reserve(str.size() / 5); res_offsets.resize(rows); const Regexps::Regexp regexp = Regexps::createRegexp(pattern); unsigned capture = regexp.getNumberOfSubpatterns(); OptimizedRegularExpression::MatchVec matches; matches.reserve(capture + 1); unsigned count = regexp.match(str.data(), str.size(), matches, static_cast(capture + 1)); bool found = count == capture + 1; size_t res_offset = 0; for (size_t i = 0; i < rows; ++i) { ssize_t index = column_index->getInt(i); if (index < 0 || index >= capture + 1) throw Exception( ErrorCodes::INDEX_OF_POSITIONAL_ARGUMENT_IS_OUT_OF_RANGE, "Index value {} is out of range, should be in [0, {})", index, capture + 1); if (found && matches[index].offset != std::string::npos) { const auto & match = matches[index]; res_data.resize(res_offset + match.length + 1); memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], str.data() + 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; } } }; } REGISTER_FUNCTION(RegexpExtract) { factory.registerFunction(); /// For Spark compatibility. factory.registerAlias("REGEXP_EXTRACT", "regexpExtract", FunctionFactory::CaseInsensitive); } }