未验证 提交 2cbd1263 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #4053 from danlark1/master

MultiVolnitsky added with tests and some benchmark, many multiFunctions are added to support multistring search
此差异已折叠。
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Core/Field.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <IO/WriteHelpers.h>
#include <common/StringRef.h>
namespace DB
{
/** Search and replace functions in strings:
*
* position(haystack, needle) - the normal search for a substring in a string, returns the position (in bytes) of the found substring starting with 1, or 0 if no substring is found.
......@@ -35,12 +38,28 @@ namespace DB
*
* replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence.
* replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences.
*
* multiPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurences (positions) of all the const patterns inside haystack
* multiPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*
* multiSearch(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
* multiSearchUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* multiSearchCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* firstMatch(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
* firstMatchUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
* firstMatchCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
* firstMatchCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
*/
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
template <typename Impl, typename Name>
......@@ -48,20 +67,11 @@ class FunctionsStringSearch : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionsStringSearch>();
}
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearch>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override
{
return 2;
}
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
......@@ -90,7 +100,8 @@ public:
{
ResultType res{};
Impl::constant_constant(col_haystack_const->getValue<String>(), col_needle_const->getValue<String>(), res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
block.getByPosition(result).column
= block.getByPosition(result).type->createColumnConst(col_haystack_const->size(), toField(res));
return;
}
......@@ -103,20 +114,22 @@ public:
const ColumnString * col_needle_vector = checkAndGetColumn<ColumnString>(&*column_needle);
if (col_haystack_vector && col_needle_vector)
Impl::vector_vector(col_haystack_vector->getChars(),
Impl::vector_vector(
col_haystack_vector->getChars(),
col_haystack_vector->getOffsets(),
col_needle_vector->getChars(),
col_needle_vector->getOffsets(),
vec_res);
else if (col_haystack_vector && col_needle_const)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
Impl::vector_constant(
col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), col_needle_const->getValue<String>(), vec_res);
else if (col_haystack_const && col_needle_vector)
Impl::constant_vector(col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
Impl::constant_vector(
col_haystack_const->getValue<String>(), col_needle_vector->getChars(), col_needle_vector->getOffsets(), vec_res);
else
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
+ block.getByPosition(arguments[1]).column->getName()
+ " of arguments of function "
+ getName(),
throw Exception(
"Illegal columns " + block.getByPosition(arguments[0]).column->getName() + " and "
+ block.getByPosition(arguments[1]).column->getName() + " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
......@@ -129,20 +142,11 @@ class FunctionsStringSearchToString : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionsStringSearchToString>();
}
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
size_t getNumberOfArguments() const override
{
return 2;
}
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
......@@ -186,4 +190,156 @@ public:
}
};
template <typename Impl, typename Name>
class FunctionsMultiStringPosition : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringPosition>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() + 1 >= std::numeric_limits<UInt8>::max())
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(arguments.size())
+ ", should be at most 255.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
std::vector<StringRef> refs;
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto col_offsets = ColumnArray::ColumnOffsets::create(column_haystack_size);
auto & vec_res = col_res->getData();
auto & offsets_res = col_offsets->getData();
vec_res.resize(column_haystack_size * refs.size());
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
size_t refs_size = refs.size();
size_t accum = refs_size;
for (size_t i = 0; i < column_haystack_size; ++i, accum += refs_size)
offsets_res[i] = accum;
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
}
};
template <typename Impl, typename Name>
class FunctionsMultiStringSearch : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsMultiStringSearch>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() + 1 >= std::numeric_limits<UInt8>::max())
throw Exception(
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(arguments.size())
+ ", should be at most 255.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[1].get());
if (!array_type || !checkAndGetDataType<DataTypeString>(array_type->getNestedType().get()))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<typename Impl::ResultType>>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
using ResultType = typename Impl::ResultType;
const ColumnPtr & column_haystack = block.getByPosition(arguments[0]).column;
const ColumnString * col_haystack_vector = checkAndGetColumn<ColumnString>(&*column_haystack);
const ColumnPtr & arr_ptr = block.getByPosition(arguments[1]).column;
const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arr_ptr.get());
if (!col_const_arr)
throw Exception(
"Illegal column " + block.getByPosition(arguments[1]).column->getName() + ". The array is not const",
ErrorCodes::ILLEGAL_COLUMN);
Array src_arr = col_const_arr->getValue<Array>();
std::vector<StringRef> refs;
refs.reserve(src_arr.size());
for (const auto & el : src_arr)
refs.emplace_back(el.get<String>());
const size_t column_haystack_size = column_haystack->size();
auto col_res = ColumnVector<ResultType>::create();
auto & vec_res = col_res->getData();
vec_res.resize(column_haystack_size);
if (col_haystack_vector)
Impl::vector_constant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res);
else
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = std::move(col_res);
}
};
}
<test>
<name>Constant column string search</name>
<tags>
<tag>search</tag>
</tags>
<preconditions>
<table_exists>hits_100m_single</table_exists>
</preconditions>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>50</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<query><![CDATA[select count(position(URL, 'yandex')), count(position(URL, 'google')) FROM hits_100m_single]]></query>
<query><![CDATA[select count(multiPosition(URL, ['yandex', 'google'])) FROM hits_100m_single]]></query>
<query><![CDATA[select count(match(URL, 'yandex|google')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex')), sum(match(URL, 'google')), sum(match(URL, 'yahoo')), sum(match(URL, 'pikabu')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(multiSearch(URL, ['yandex', 'google', 'yahoo', 'pikabu'])) from hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex|google|yahoo|pikabu')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex')), sum(match(URL, 'google')), sum(match(URL, 'http')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(multiSearch(URL, ['yandex', 'google', 'http'])) from hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex|google|http')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex')), sum(match(URL, 'google')), sum(match(URL, 'facebook')), sum(match(URL, 'wikipedia')), sum(match(URL, 'reddit')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(multiSearch(URL, ['yandex', 'google', 'facebook', 'wikipedia', 'reddit'])) from hits_100m_single]]></query>
<query><![CDATA[select sum(match(URL, 'yandex|google|facebook|wikipedia|reddit')) FROM hits_100m_single]]></query>
<query><![CDATA[select sum(firstMatch(URL, ['yandex', 'google', 'http', 'facebook', 'google'])) from hits_100m_single]]></query>
<main_metric>
<min_time/>
</main_metric>
</test>
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册