mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #9922 from ClickHouse/every_function_in_own_file_12
Every function in its own file, part 12
This commit is contained in:
commit
621bcaf2eb
129
dbms/src/Functions/FunctionsMultiStringPosition.h
Normal file
129
dbms/src/Functions/FunctionsMultiStringPosition.h
Normal file
@ -0,0 +1,129 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Columns/ColumnArray.h>
|
||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <common/StringRef.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
/** multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack
|
||||||
|
* multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
*
|
||||||
|
* multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found
|
||||||
|
* multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
*
|
||||||
|
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
|
||||||
|
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
|
||||||
|
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
|
||||||
|
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstIndexCaseInsensitiveUTF8(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>
|
||||||
|
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; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
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>();
|
||||||
|
|
||||||
|
if (src_arr.size() > std::numeric_limits<UInt8>::max())
|
||||||
|
throw Exception(
|
||||||
|
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
|
||||||
|
+ ", should be at most 255",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
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::vectorConstant(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));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
127
dbms/src/Functions/FunctionsMultiStringSearch.h
Normal file
127
dbms/src/Functions/FunctionsMultiStringSearch.h
Normal file
@ -0,0 +1,127 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Columns/ColumnArray.h>
|
||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <common/StringRef.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
/**
|
||||||
|
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
|
||||||
|
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
|
||||||
|
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
|
||||||
|
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
||||||
|
* multiSearchFirstIndexCaseInsensitiveUTF8(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;
|
||||||
|
extern const int FUNCTION_NOT_ALLOWED;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number.
|
||||||
|
/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions
|
||||||
|
template <typename Impl, typename Name, size_t LimitArgs = std::numeric_limits<UInt8>::max()>
|
||||||
|
class FunctionsMultiStringSearch : public IFunction
|
||||||
|
{
|
||||||
|
static_assert(LimitArgs > 0);
|
||||||
|
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static FunctionPtr create(const Context & context)
|
||||||
|
{
|
||||||
|
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
|
||||||
|
throw Exception(
|
||||||
|
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
|
||||||
|
|
||||||
|
return std::make_shared<FunctionsMultiStringSearch>();
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
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 Impl::getReturnType();
|
||||||
|
}
|
||||||
|
|
||||||
|
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>();
|
||||||
|
|
||||||
|
if (src_arr.size() > LimitArgs)
|
||||||
|
throw Exception(
|
||||||
|
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
|
||||||
|
+ ", should be at most " + std::to_string(LimitArgs),
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
std::vector<StringRef> refs;
|
||||||
|
refs.reserve(src_arr.size());
|
||||||
|
|
||||||
|
for (const auto & el : src_arr)
|
||||||
|
refs.emplace_back(el.get<String>());
|
||||||
|
|
||||||
|
auto col_res = ColumnVector<ResultType>::create();
|
||||||
|
auto col_offsets = ColumnArray::ColumnOffsets::create();
|
||||||
|
|
||||||
|
auto & vec_res = col_res->getData();
|
||||||
|
auto & offsets_res = col_offsets->getData();
|
||||||
|
|
||||||
|
/// The blame for resizing output is for the callee.
|
||||||
|
if (col_haystack_vector)
|
||||||
|
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
|
||||||
|
else
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
if constexpr (Impl::is_column_array)
|
||||||
|
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
||||||
|
else
|
||||||
|
block.getByPosition(result).column = std::move(col_res);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -1,5 +1,7 @@
|
|||||||
#include "FunctionsStringRegex.h"
|
#include "FunctionsStringRegex.h"
|
||||||
#include "FunctionsStringSearch.h"
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionsStringSearchToString.h"
|
||||||
#include <Columns/ColumnFixedString.h>
|
#include <Columns/ColumnFixedString.h>
|
||||||
#include <DataTypes/DataTypeFixedString.h>
|
#include <DataTypes/DataTypeFixedString.h>
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
|
@ -1,707 +0,0 @@
|
|||||||
#include "FunctionsStringSearch.h"
|
|
||||||
|
|
||||||
#include <algorithm>
|
|
||||||
#include <memory>
|
|
||||||
#include <string>
|
|
||||||
#include <vector>
|
|
||||||
#include <Columns/ColumnFixedString.h>
|
|
||||||
#include <DataTypes/DataTypeFixedString.h>
|
|
||||||
#include <Functions/FunctionFactory.h>
|
|
||||||
#include <Functions/Regexps.h>
|
|
||||||
#include <IO/WriteHelpers.h>
|
|
||||||
#include <Poco/UTF8String.h>
|
|
||||||
#include <Common/Volnitsky.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int ILLEGAL_COLUMN;
|
|
||||||
}
|
|
||||||
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
|
|
||||||
*/
|
|
||||||
struct PositionCaseSensitiveASCII
|
|
||||||
{
|
|
||||||
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
|
|
||||||
using SearcherInBigHaystack = Volnitsky;
|
|
||||||
|
|
||||||
/// For search many substrings in one string
|
|
||||||
using MultiSearcherInBigHaystack = MultiVolnitsky;
|
|
||||||
|
|
||||||
/// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization.
|
|
||||||
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
|
||||||
|
|
||||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
|
||||||
{
|
|
||||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
|
||||||
}
|
|
||||||
|
|
||||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
|
||||||
{
|
|
||||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
|
||||||
{
|
|
||||||
return MultiSearcherInBigHaystack(needles);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
|
|
||||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
|
||||||
|
|
||||||
/// Convert string to lowercase. Only for case-insensitive search.
|
|
||||||
/// Implementation is permitted to be inefficient because it is called for single string.
|
|
||||||
static void toLowerIfNeed(std::string &) { }
|
|
||||||
};
|
|
||||||
|
|
||||||
struct PositionCaseInsensitiveASCII
|
|
||||||
{
|
|
||||||
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
|
|
||||||
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
|
|
||||||
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
|
|
||||||
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
|
|
||||||
|
|
||||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
|
|
||||||
{
|
|
||||||
return SearcherInBigHaystack(needle_data, needle_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
|
||||||
{
|
|
||||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
|
||||||
{
|
|
||||||
return MultiSearcherInBigHaystack(needles);
|
|
||||||
}
|
|
||||||
|
|
||||||
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
|
||||||
|
|
||||||
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
|
|
||||||
};
|
|
||||||
|
|
||||||
struct PositionCaseSensitiveUTF8
|
|
||||||
{
|
|
||||||
using SearcherInBigHaystack = VolnitskyUTF8;
|
|
||||||
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
|
|
||||||
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
|
||||||
|
|
||||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
|
||||||
{
|
|
||||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
|
||||||
}
|
|
||||||
|
|
||||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
|
||||||
{
|
|
||||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
|
||||||
{
|
|
||||||
return MultiSearcherInBigHaystack(needles);
|
|
||||||
}
|
|
||||||
|
|
||||||
static size_t countChars(const char * begin, const char * end)
|
|
||||||
{
|
|
||||||
size_t res = 0;
|
|
||||||
for (auto it = begin; it != end; ++it)
|
|
||||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
|
||||||
++res;
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
static void toLowerIfNeed(std::string &) { }
|
|
||||||
};
|
|
||||||
|
|
||||||
struct PositionCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
|
|
||||||
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
|
|
||||||
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
|
|
||||||
|
|
||||||
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
|
||||||
{
|
|
||||||
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
|
||||||
}
|
|
||||||
|
|
||||||
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
|
||||||
{
|
|
||||||
return SearcherInSmallHaystack(needle_data, needle_size);
|
|
||||||
}
|
|
||||||
|
|
||||||
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
|
||||||
{
|
|
||||||
return MultiSearcherInBigHaystack(needles);
|
|
||||||
}
|
|
||||||
|
|
||||||
static size_t countChars(const char * begin, const char * end)
|
|
||||||
{
|
|
||||||
size_t res = 0;
|
|
||||||
for (auto it = begin; it != end; ++it)
|
|
||||||
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
|
||||||
++res;
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename Impl>
|
|
||||||
struct PositionImpl
|
|
||||||
{
|
|
||||||
static constexpr bool use_default_implementation_for_constants = false;
|
|
||||||
|
|
||||||
using ResultType = UInt64;
|
|
||||||
|
|
||||||
/// Find one substring in many strings.
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
|
||||||
{
|
|
||||||
const UInt8 * begin = data.data();
|
|
||||||
const UInt8 * pos = begin;
|
|
||||||
const UInt8 * end = pos + data.size();
|
|
||||||
|
|
||||||
/// Current index in the array of strings.
|
|
||||||
size_t i = 0;
|
|
||||||
|
|
||||||
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
|
|
||||||
|
|
||||||
/// We will search for the next occurrence in all strings at once.
|
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
|
||||||
{
|
|
||||||
/// Determine which index it refers to.
|
|
||||||
while (begin + offsets[i] <= pos)
|
|
||||||
{
|
|
||||||
res[i] = 0;
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// We check that the entry does not pass through the boundaries of strings.
|
|
||||||
if (pos + needle.size() < begin + offsets[i])
|
|
||||||
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
|
||||||
else
|
|
||||||
res[i] = 0;
|
|
||||||
|
|
||||||
pos = begin + offsets[i];
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (i < res.size())
|
|
||||||
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Search for substring in string.
|
|
||||||
static void constantConstant(std::string data, std::string needle, UInt64 & res)
|
|
||||||
{
|
|
||||||
Impl::toLowerIfNeed(data);
|
|
||||||
Impl::toLowerIfNeed(needle);
|
|
||||||
|
|
||||||
res = data.find(needle);
|
|
||||||
if (res == std::string::npos)
|
|
||||||
res = 0;
|
|
||||||
else
|
|
||||||
res = 1 + Impl::countChars(data.data(), data.data() + res);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Search each time for a different single substring inside each time different string.
|
|
||||||
static void vectorVector(
|
|
||||||
const ColumnString::Chars & haystack_data,
|
|
||||||
const ColumnString::Offsets & haystack_offsets,
|
|
||||||
const ColumnString::Chars & needle_data,
|
|
||||||
const ColumnString::Offsets & needle_offsets,
|
|
||||||
PaddedPODArray<UInt64> & res)
|
|
||||||
{
|
|
||||||
ColumnString::Offset prev_haystack_offset = 0;
|
|
||||||
ColumnString::Offset prev_needle_offset = 0;
|
|
||||||
|
|
||||||
size_t size = haystack_offsets.size();
|
|
||||||
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
|
||||||
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
|
|
||||||
|
|
||||||
if (0 == needle_size)
|
|
||||||
{
|
|
||||||
/// An empty string is always at the very beginning of `haystack`.
|
|
||||||
res[i] = 1;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/// It is assumed that the StringSearcher is not very difficult to initialize.
|
|
||||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
|
||||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
|
|
||||||
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
|
|
||||||
|
|
||||||
/// searcher returns a pointer to the found substring or to the end of `haystack`.
|
|
||||||
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
|
|
||||||
- &haystack_data[prev_haystack_offset];
|
|
||||||
|
|
||||||
if (pos != haystack_size)
|
|
||||||
{
|
|
||||||
res[i] = 1
|
|
||||||
+ Impl::countChars(
|
|
||||||
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
|
||||||
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
|
|
||||||
}
|
|
||||||
else
|
|
||||||
res[i] = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
prev_haystack_offset = haystack_offsets[i];
|
|
||||||
prev_needle_offset = needle_offsets[i];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Find many substrings in single string.
|
|
||||||
static void constantVector(
|
|
||||||
const String & haystack,
|
|
||||||
const ColumnString::Chars & needle_data,
|
|
||||||
const ColumnString::Offsets & needle_offsets,
|
|
||||||
PaddedPODArray<UInt64> & res)
|
|
||||||
{
|
|
||||||
// NOTE You could use haystack indexing. But this is a rare case.
|
|
||||||
|
|
||||||
ColumnString::Offset prev_needle_offset = 0;
|
|
||||||
|
|
||||||
size_t size = needle_offsets.size();
|
|
||||||
|
|
||||||
for (size_t i = 0; i < size; ++i)
|
|
||||||
{
|
|
||||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
|
||||||
|
|
||||||
if (0 == needle_size)
|
|
||||||
{
|
|
||||||
res[i] = 1;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
|
||||||
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
|
||||||
|
|
||||||
size_t pos = searcher.search(
|
|
||||||
reinterpret_cast<const UInt8 *>(haystack.data()),
|
|
||||||
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
|
|
||||||
- reinterpret_cast<const UInt8 *>(haystack.data());
|
|
||||||
|
|
||||||
if (pos != haystack.size())
|
|
||||||
{
|
|
||||||
res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
res[i] = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
prev_needle_offset = needle_offsets[i];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename... Args>
|
|
||||||
static void vectorFixedConstant(Args &&...)
|
|
||||||
{
|
|
||||||
throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename Impl>
|
|
||||||
struct MultiSearchAllPositionsImpl
|
|
||||||
{
|
|
||||||
using ResultType = UInt64;
|
|
||||||
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & haystack_data,
|
|
||||||
const ColumnString::Offsets & haystack_offsets,
|
|
||||||
const std::vector<StringRef> & needles,
|
|
||||||
PaddedPODArray<UInt64> & res)
|
|
||||||
{
|
|
||||||
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
|
||||||
{
|
|
||||||
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
|
||||||
};
|
|
||||||
|
|
||||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
|
||||||
|
|
||||||
const size_t haystack_string_size = haystack_offsets.size();
|
|
||||||
const size_t needles_size = needles.size();
|
|
||||||
|
|
||||||
/// Something can be uninitialized after the search itself
|
|
||||||
std::fill(res.begin(), res.end(), 0);
|
|
||||||
|
|
||||||
while (searcher.hasMoreToSearch())
|
|
||||||
{
|
|
||||||
size_t prev_offset = 0;
|
|
||||||
for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size)
|
|
||||||
{
|
|
||||||
const auto * haystack = &haystack_data[prev_offset];
|
|
||||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
|
||||||
searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback);
|
|
||||||
prev_offset = haystack_offsets[j];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename Impl>
|
|
||||||
struct MultiSearchImpl
|
|
||||||
{
|
|
||||||
using ResultType = UInt8;
|
|
||||||
static constexpr bool is_using_hyperscan = false;
|
|
||||||
/// Variable for understanding, if we used offsets for the output, most
|
|
||||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
|
||||||
static constexpr bool is_column_array = false;
|
|
||||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
|
||||||
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & haystack_data,
|
|
||||||
const ColumnString::Offsets & haystack_offsets,
|
|
||||||
const std::vector<StringRef> & needles,
|
|
||||||
PaddedPODArray<UInt8> & res,
|
|
||||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
|
||||||
{
|
|
||||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
|
||||||
const size_t haystack_string_size = haystack_offsets.size();
|
|
||||||
res.resize(haystack_string_size);
|
|
||||||
size_t iteration = 0;
|
|
||||||
while (searcher.hasMoreToSearch())
|
|
||||||
{
|
|
||||||
size_t prev_offset = 0;
|
|
||||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
|
||||||
{
|
|
||||||
const auto * haystack = &haystack_data[prev_offset];
|
|
||||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
|
||||||
if (iteration == 0 || !res[j])
|
|
||||||
res[j] = searcher.searchOne(haystack, haystack_end);
|
|
||||||
prev_offset = haystack_offsets[j];
|
|
||||||
}
|
|
||||||
++iteration;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename Impl>
|
|
||||||
struct MultiSearchFirstPositionImpl
|
|
||||||
{
|
|
||||||
using ResultType = UInt64;
|
|
||||||
static constexpr bool is_using_hyperscan = false;
|
|
||||||
/// Variable for understanding, if we used offsets for the output, most
|
|
||||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
|
||||||
static constexpr bool is_column_array = false;
|
|
||||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
|
||||||
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & haystack_data,
|
|
||||||
const ColumnString::Offsets & haystack_offsets,
|
|
||||||
const std::vector<StringRef> & needles,
|
|
||||||
PaddedPODArray<UInt64> & res,
|
|
||||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
|
||||||
{
|
|
||||||
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
|
||||||
{
|
|
||||||
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
|
||||||
};
|
|
||||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
|
||||||
const size_t haystack_string_size = haystack_offsets.size();
|
|
||||||
res.resize(haystack_string_size);
|
|
||||||
size_t iteration = 0;
|
|
||||||
while (searcher.hasMoreToSearch())
|
|
||||||
{
|
|
||||||
size_t prev_offset = 0;
|
|
||||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
|
||||||
{
|
|
||||||
const auto * haystack = &haystack_data[prev_offset];
|
|
||||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
|
||||||
if (iteration == 0 || res[j] == 0)
|
|
||||||
res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
|
||||||
else
|
|
||||||
{
|
|
||||||
UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
|
||||||
if (result != 0)
|
|
||||||
res[j] = std::min(result, res[j]);
|
|
||||||
}
|
|
||||||
prev_offset = haystack_offsets[j];
|
|
||||||
}
|
|
||||||
++iteration;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
template <typename Impl>
|
|
||||||
struct MultiSearchFirstIndexImpl
|
|
||||||
{
|
|
||||||
using ResultType = UInt64;
|
|
||||||
static constexpr bool is_using_hyperscan = false;
|
|
||||||
/// Variable for understanding, if we used offsets for the output, most
|
|
||||||
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
|
||||||
static constexpr bool is_column_array = false;
|
|
||||||
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
|
||||||
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & haystack_data,
|
|
||||||
const ColumnString::Offsets & haystack_offsets,
|
|
||||||
const std::vector<StringRef> & needles,
|
|
||||||
PaddedPODArray<UInt64> & res,
|
|
||||||
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
|
||||||
{
|
|
||||||
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
|
||||||
const size_t haystack_string_size = haystack_offsets.size();
|
|
||||||
res.resize(haystack_string_size);
|
|
||||||
size_t iteration = 0;
|
|
||||||
while (searcher.hasMoreToSearch())
|
|
||||||
{
|
|
||||||
size_t prev_offset = 0;
|
|
||||||
for (size_t j = 0; j < haystack_string_size; ++j)
|
|
||||||
{
|
|
||||||
const auto * haystack = &haystack_data[prev_offset];
|
|
||||||
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
|
||||||
/// hasMoreToSearch traverse needles in increasing order
|
|
||||||
if (iteration == 0 || res[j] == 0)
|
|
||||||
res[j] = searcher.searchOneFirstIndex(haystack, haystack_end);
|
|
||||||
prev_offset = haystack_offsets[j];
|
|
||||||
}
|
|
||||||
++iteration;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
|
|
||||||
*/
|
|
||||||
template <typename TokenSearcher, bool negate_result = false>
|
|
||||||
struct HasTokenImpl
|
|
||||||
{
|
|
||||||
using ResultType = UInt8;
|
|
||||||
|
|
||||||
static constexpr bool use_default_implementation_for_constants = true;
|
|
||||||
|
|
||||||
static void vectorConstant(
|
|
||||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
|
||||||
{
|
|
||||||
if (offsets.empty())
|
|
||||||
return;
|
|
||||||
|
|
||||||
const UInt8 * begin = data.data();
|
|
||||||
const UInt8 * pos = begin;
|
|
||||||
const UInt8 * end = pos + data.size();
|
|
||||||
|
|
||||||
/// The current index in the array of strings.
|
|
||||||
size_t i = 0;
|
|
||||||
|
|
||||||
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
|
|
||||||
|
|
||||||
/// We will search for the next occurrence in all rows at once.
|
|
||||||
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
|
||||||
{
|
|
||||||
/// Let's determine which index it refers to.
|
|
||||||
while (begin + offsets[i] <= pos)
|
|
||||||
{
|
|
||||||
res[i] = negate_result;
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// We check that the entry does not pass through the boundaries of strings.
|
|
||||||
if (pos + pattern.size() < begin + offsets[i])
|
|
||||||
res[i] = !negate_result;
|
|
||||||
else
|
|
||||||
res[i] = negate_result;
|
|
||||||
|
|
||||||
pos = begin + offsets[i];
|
|
||||||
++i;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Tail, in which there can be no substring.
|
|
||||||
if (i < res.size())
|
|
||||||
memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0]));
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename... Args>
|
|
||||||
static void vectorVector(Args &&...)
|
|
||||||
{
|
|
||||||
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Search different needles in single haystack.
|
|
||||||
template <typename... Args>
|
|
||||||
static void constantVector(Args &&...)
|
|
||||||
{
|
|
||||||
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
|
|
||||||
template <typename... Args>
|
|
||||||
static void vectorFixedConstant(Args &&...)
|
|
||||||
{
|
|
||||||
throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
struct NamePosition
|
|
||||||
{
|
|
||||||
static constexpr auto name = "position";
|
|
||||||
};
|
|
||||||
struct NamePositionUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "positionUTF8";
|
|
||||||
};
|
|
||||||
struct NamePositionCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "positionCaseInsensitive";
|
|
||||||
};
|
|
||||||
struct NamePositionCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "positionCaseInsensitiveUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAllPositions
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAllPositions";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAllPositionsUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAllPositionsUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAllPositionsCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAny
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAny";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAnyUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAnyUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAnyCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAnyCaseInsensitive";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchAnyCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstIndex
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstIndex";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstIndexUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstIndexUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstIndexCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstPosition
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstPosition";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstPositionUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstPositionUTF8";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstPositionCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
|
|
||||||
};
|
|
||||||
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
|
|
||||||
{
|
|
||||||
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
|
|
||||||
};
|
|
||||||
|
|
||||||
struct NameHasToken
|
|
||||||
{
|
|
||||||
static constexpr auto name = "hasToken";
|
|
||||||
};
|
|
||||||
|
|
||||||
struct NameHasTokenCaseInsensitive
|
|
||||||
{
|
|
||||||
static constexpr auto name = "hasTokenCaseInsensitive";
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
|
|
||||||
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
|
|
||||||
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
|
|
||||||
using FunctionPositionCaseInsensitiveUTF8
|
|
||||||
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
|
|
||||||
|
|
||||||
using FunctionMultiSearchAllPositions
|
|
||||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
|
|
||||||
using FunctionMultiSearchAllPositionsUTF8
|
|
||||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
|
|
||||||
using FunctionMultiSearchAllPositionsCaseInsensitive
|
|
||||||
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
|
|
||||||
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
|
|
||||||
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
|
|
||||||
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
|
|
||||||
|
|
||||||
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
|
|
||||||
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
|
|
||||||
using FunctionMultiSearchCaseInsensitive
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
|
|
||||||
using FunctionMultiSearchCaseInsensitiveUTF8
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
|
|
||||||
|
|
||||||
using FunctionMultiSearchFirstIndex
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
|
|
||||||
using FunctionMultiSearchFirstIndexUTF8
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
|
|
||||||
using FunctionMultiSearchFirstIndexCaseInsensitive
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
|
|
||||||
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
|
|
||||||
|
|
||||||
using FunctionMultiSearchFirstPosition
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
|
|
||||||
using FunctionMultiSearchFirstPositionUTF8
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
|
|
||||||
using FunctionMultiSearchFirstPositionCaseInsensitive
|
|
||||||
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
|
|
||||||
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
|
|
||||||
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
|
|
||||||
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
|
|
||||||
|
|
||||||
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
|
|
||||||
using FunctionHasTokenCaseInsensitive
|
|
||||||
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
|
|
||||||
|
|
||||||
void registerFunctionsStringSearch(FunctionFactory & factory)
|
|
||||||
{
|
|
||||||
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
|
|
||||||
factory.registerFunction<FunctionPositionUTF8>();
|
|
||||||
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
|
||||||
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
|
||||||
|
|
||||||
factory.registerFunction<FunctionMultiSearchAllPositions>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
|
|
||||||
|
|
||||||
factory.registerFunction<FunctionMultiSearch>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchUTF8>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
|
|
||||||
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstIndex>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
|
|
||||||
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstPosition>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
|
|
||||||
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
|
|
||||||
|
|
||||||
factory.registerFunction<FunctionHasToken>();
|
|
||||||
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
|
|
||||||
|
|
||||||
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
|
|
||||||
}
|
|
||||||
}
|
|
@ -1,6 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <Columns/ColumnArray.h>
|
|
||||||
#include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
#include <Columns/ColumnString.h>
|
#include <Columns/ColumnString.h>
|
||||||
#include <Columns/ColumnFixedString.h>
|
#include <Columns/ColumnFixedString.h>
|
||||||
@ -10,9 +9,7 @@
|
|||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <Functions/FunctionHelpers.h>
|
#include <Functions/FunctionHelpers.h>
|
||||||
#include <Functions/IFunctionImpl.h>
|
#include <Functions/IFunctionImpl.h>
|
||||||
#include <IO/WriteHelpers.h>
|
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <common/StringRef.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -37,40 +34,12 @@ namespace DB
|
|||||||
* - the zero subpattern (the match part, otherwise);
|
* - the zero subpattern (the match part, otherwise);
|
||||||
* - if not match - an empty string.
|
* - if not match - an empty string.
|
||||||
* extract(haystack, pattern)
|
* extract(haystack, pattern)
|
||||||
*
|
|
||||||
* replaceOne(haystack, pattern, replacement) - replacing the pattern with the specified rules, only the first occurrence.
|
|
||||||
* replaceAll(haystack, pattern, replacement) - replacing the pattern with the specified rules, all occurrences.
|
|
||||||
*
|
|
||||||
* 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.
|
|
||||||
*
|
|
||||||
* multiSearchAllPositions(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack
|
|
||||||
* multiSearchAllPositionsUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchAllPositionsCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
*
|
|
||||||
* multiSearchFirstPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first position of the haystack matched by strings or zero if nothing was found
|
|
||||||
* multiSearchFirstPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchFirstPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
*
|
|
||||||
* multiSearchAny(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find any of the const patterns inside haystack and return 0 or 1
|
|
||||||
* multiSearchAnyUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchAnyCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchAnyCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
|
|
||||||
* multiSearchFirstIndex(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- returns the first index of the matched string or zero if nothing was found
|
|
||||||
* multiSearchFirstIndexUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchFirstIndexCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
* multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n])
|
|
||||||
*/
|
*/
|
||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
extern const int ILLEGAL_COLUMN;
|
extern const int ILLEGAL_COLUMN;
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
||||||
extern const int FUNCTION_NOT_ALLOWED;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename Impl, typename Name>
|
template <typename Impl, typename Name>
|
||||||
@ -163,224 +132,4 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
template <typename Impl, typename Name>
|
|
||||||
class FunctionsStringSearchToString : public IFunction
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr auto name = Name::name;
|
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); }
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
||||||
{
|
|
||||||
if (!isString(arguments[0]))
|
|
||||||
throw Exception(
|
|
||||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
|
|
||||||
if (!isString(arguments[1]))
|
|
||||||
throw Exception(
|
|
||||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
||||||
|
|
||||||
return std::make_shared<DataTypeString>();
|
|
||||||
}
|
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
|
||||||
{
|
|
||||||
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
|
||||||
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
|
|
||||||
|
|
||||||
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
|
|
||||||
if (!col_needle)
|
|
||||||
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
|
||||||
{
|
|
||||||
auto col_res = ColumnString::create();
|
|
||||||
|
|
||||||
ColumnString::Chars & vec_res = col_res->getChars();
|
|
||||||
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
|
||||||
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
|
|
||||||
|
|
||||||
block.getByPosition(result).column = std::move(col_res);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception(
|
|
||||||
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
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; }
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
||||||
{
|
|
||||||
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>();
|
|
||||||
|
|
||||||
if (src_arr.size() > std::numeric_limits<UInt8>::max())
|
|
||||||
throw Exception(
|
|
||||||
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
|
|
||||||
+ ", should be at most 255",
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
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::vectorConstant(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));
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
/// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number.
|
|
||||||
/// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions
|
|
||||||
template <typename Impl, typename Name, size_t LimitArgs = std::numeric_limits<UInt8>::max()>
|
|
||||||
class FunctionsMultiStringSearch : public IFunction
|
|
||||||
{
|
|
||||||
static_assert(LimitArgs > 0);
|
|
||||||
|
|
||||||
public:
|
|
||||||
static constexpr auto name = Name::name;
|
|
||||||
static FunctionPtr create(const Context & context)
|
|
||||||
{
|
|
||||||
if (Impl::is_using_hyperscan && !context.getSettingsRef().allow_hyperscan)
|
|
||||||
throw Exception(
|
|
||||||
"Hyperscan functions are disabled, because setting 'allow_hyperscan' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED);
|
|
||||||
|
|
||||||
return std::make_shared<FunctionsMultiStringSearch>();
|
|
||||||
}
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
|
||||||
{
|
|
||||||
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 Impl::getReturnType();
|
|
||||||
}
|
|
||||||
|
|
||||||
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>();
|
|
||||||
|
|
||||||
if (src_arr.size() > LimitArgs)
|
|
||||||
throw Exception(
|
|
||||||
"Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(src_arr.size())
|
|
||||||
+ ", should be at most " + std::to_string(LimitArgs),
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
std::vector<StringRef> refs;
|
|
||||||
refs.reserve(src_arr.size());
|
|
||||||
|
|
||||||
for (const auto & el : src_arr)
|
|
||||||
refs.emplace_back(el.get<String>());
|
|
||||||
|
|
||||||
auto col_res = ColumnVector<ResultType>::create();
|
|
||||||
auto col_offsets = ColumnArray::ColumnOffsets::create();
|
|
||||||
|
|
||||||
auto & vec_res = col_res->getData();
|
|
||||||
auto & offsets_res = col_offsets->getData();
|
|
||||||
|
|
||||||
/// The blame for resizing output is for the callee.
|
|
||||||
if (col_haystack_vector)
|
|
||||||
Impl::vectorConstant(col_haystack_vector->getChars(), col_haystack_vector->getOffsets(), refs, vec_res, offsets_res);
|
|
||||||
else
|
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
if constexpr (Impl::is_column_array)
|
|
||||||
block.getByPosition(result).column = ColumnArray::create(std::move(col_res), std::move(col_offsets));
|
|
||||||
else
|
|
||||||
block.getByPosition(result).column = std::move(col_res);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
87
dbms/src/Functions/FunctionsStringSearchToString.h
Normal file
87
dbms/src/Functions/FunctionsStringSearchToString.h
Normal file
@ -0,0 +1,87 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Columns/ColumnArray.h>
|
||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
#include <common/StringRef.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
/** Applies regexp re2 and extracts:
|
||||||
|
* - the first subpattern, if the regexp has a subpattern;
|
||||||
|
* - the zero subpattern (the match part, otherwise);
|
||||||
|
* - if not match - an empty string.
|
||||||
|
* extract(haystack, pattern)
|
||||||
|
*/
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Impl, typename Name>
|
||||||
|
class FunctionsStringSearchToString : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = Name::name;
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionsStringSearchToString>(); }
|
||||||
|
|
||||||
|
String getName() const override { return name; }
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
{
|
||||||
|
if (!isString(arguments[0]))
|
||||||
|
throw Exception(
|
||||||
|
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
if (!isString(arguments[1]))
|
||||||
|
throw Exception(
|
||||||
|
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
|
return std::make_shared<DataTypeString>();
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
const ColumnPtr column = block.getByPosition(arguments[0]).column;
|
||||||
|
const ColumnPtr column_needle = block.getByPosition(arguments[1]).column;
|
||||||
|
|
||||||
|
const ColumnConst * col_needle = typeid_cast<const ColumnConst *>(&*column_needle);
|
||||||
|
if (!col_needle)
|
||||||
|
throw Exception("Second argument of function " + getName() + " must be constant string", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
if (const ColumnString * col = checkAndGetColumn<ColumnString>(column.get()))
|
||||||
|
{
|
||||||
|
auto col_res = ColumnString::create();
|
||||||
|
|
||||||
|
ColumnString::Chars & vec_res = col_res->getChars();
|
||||||
|
ColumnString::Offsets & offsets_res = col_res->getOffsets();
|
||||||
|
Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue<String>(), vec_res, offsets_res);
|
||||||
|
|
||||||
|
block.getByPosition(result).column = std::move(col_res);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception(
|
||||||
|
"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
83
dbms/src/Functions/HasTokenImpl.h
Normal file
83
dbms/src/Functions/HasTokenImpl.h
Normal file
@ -0,0 +1,83 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation.
|
||||||
|
*/
|
||||||
|
template <typename TokenSearcher, bool negate_result = false>
|
||||||
|
struct HasTokenImpl
|
||||||
|
{
|
||||||
|
using ResultType = UInt8;
|
||||||
|
|
||||||
|
static constexpr bool use_default_implementation_for_constants = true;
|
||||||
|
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray<UInt8> & res)
|
||||||
|
{
|
||||||
|
if (offsets.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
const UInt8 * begin = data.data();
|
||||||
|
const UInt8 * pos = begin;
|
||||||
|
const UInt8 * end = pos + data.size();
|
||||||
|
|
||||||
|
/// The current index in the array of strings.
|
||||||
|
size_t i = 0;
|
||||||
|
|
||||||
|
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
|
||||||
|
|
||||||
|
/// We will search for the next occurrence in all rows at once.
|
||||||
|
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||||
|
{
|
||||||
|
/// Let's determine which index it refers to.
|
||||||
|
while (begin + offsets[i] <= pos)
|
||||||
|
{
|
||||||
|
res[i] = negate_result;
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// We check that the entry does not pass through the boundaries of strings.
|
||||||
|
if (pos + pattern.size() < begin + offsets[i])
|
||||||
|
res[i] = !negate_result;
|
||||||
|
else
|
||||||
|
res[i] = negate_result;
|
||||||
|
|
||||||
|
pos = begin + offsets[i];
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Tail, in which there can be no substring.
|
||||||
|
if (i < res.size())
|
||||||
|
memset(&res[i], negate_result, (res.size() - i) * sizeof(res[0]));
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename... Args>
|
||||||
|
static void vectorVector(Args &&...)
|
||||||
|
{
|
||||||
|
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Search different needles in single haystack.
|
||||||
|
template <typename... Args>
|
||||||
|
static void constantVector(Args &&...)
|
||||||
|
{
|
||||||
|
throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename... Args>
|
||||||
|
static void vectorFixedConstant(Args &&...)
|
||||||
|
{
|
||||||
|
throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
48
dbms/src/Functions/MultiSearchAllPositionsImpl.h
Normal file
48
dbms/src/Functions/MultiSearchAllPositionsImpl.h
Normal file
@ -0,0 +1,48 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
template <typename Impl>
|
||||||
|
struct MultiSearchAllPositionsImpl
|
||||||
|
{
|
||||||
|
using ResultType = UInt64;
|
||||||
|
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & haystack_data,
|
||||||
|
const ColumnString::Offsets & haystack_offsets,
|
||||||
|
const std::vector<StringRef> & needles,
|
||||||
|
PaddedPODArray<UInt64> & res)
|
||||||
|
{
|
||||||
|
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
||||||
|
{
|
||||||
|
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
||||||
|
};
|
||||||
|
|
||||||
|
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||||
|
|
||||||
|
const size_t haystack_string_size = haystack_offsets.size();
|
||||||
|
const size_t needles_size = needles.size();
|
||||||
|
|
||||||
|
/// Something can be uninitialized after the search itself
|
||||||
|
std::fill(res.begin(), res.end(), 0);
|
||||||
|
|
||||||
|
while (searcher.hasMoreToSearch())
|
||||||
|
{
|
||||||
|
size_t prev_offset = 0;
|
||||||
|
for (size_t j = 0, from = 0; j < haystack_string_size; ++j, from += needles_size)
|
||||||
|
{
|
||||||
|
const auto * haystack = &haystack_data[prev_offset];
|
||||||
|
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||||
|
searcher.searchOneAll(haystack, haystack_end, res.data() + from, res_callback);
|
||||||
|
prev_offset = haystack_offsets[j];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
48
dbms/src/Functions/MultiSearchFirstIndexImpl.h
Normal file
48
dbms/src/Functions/MultiSearchFirstIndexImpl.h
Normal file
@ -0,0 +1,48 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
template <typename Impl>
|
||||||
|
struct MultiSearchFirstIndexImpl
|
||||||
|
{
|
||||||
|
using ResultType = UInt64;
|
||||||
|
static constexpr bool is_using_hyperscan = false;
|
||||||
|
/// Variable for understanding, if we used offsets for the output, most
|
||||||
|
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||||
|
static constexpr bool is_column_array = false;
|
||||||
|
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||||
|
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & haystack_data,
|
||||||
|
const ColumnString::Offsets & haystack_offsets,
|
||||||
|
const std::vector<StringRef> & needles,
|
||||||
|
PaddedPODArray<UInt64> & res,
|
||||||
|
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||||
|
{
|
||||||
|
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||||
|
const size_t haystack_string_size = haystack_offsets.size();
|
||||||
|
res.resize(haystack_string_size);
|
||||||
|
size_t iteration = 0;
|
||||||
|
while (searcher.hasMoreToSearch())
|
||||||
|
{
|
||||||
|
size_t prev_offset = 0;
|
||||||
|
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||||
|
{
|
||||||
|
const auto * haystack = &haystack_data[prev_offset];
|
||||||
|
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||||
|
/// hasMoreToSearch traverse needles in increasing order
|
||||||
|
if (iteration == 0 || res[j] == 0)
|
||||||
|
res[j] = searcher.searchOneFirstIndex(haystack, haystack_end);
|
||||||
|
prev_offset = haystack_offsets[j];
|
||||||
|
}
|
||||||
|
++iteration;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
57
dbms/src/Functions/MultiSearchFirstPositionImpl.h
Normal file
57
dbms/src/Functions/MultiSearchFirstPositionImpl.h
Normal file
@ -0,0 +1,57 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
template <typename Impl>
|
||||||
|
struct MultiSearchFirstPositionImpl
|
||||||
|
{
|
||||||
|
using ResultType = UInt64;
|
||||||
|
static constexpr bool is_using_hyperscan = false;
|
||||||
|
/// Variable for understanding, if we used offsets for the output, most
|
||||||
|
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||||
|
static constexpr bool is_column_array = false;
|
||||||
|
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||||
|
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & haystack_data,
|
||||||
|
const ColumnString::Offsets & haystack_offsets,
|
||||||
|
const std::vector<StringRef> & needles,
|
||||||
|
PaddedPODArray<UInt64> & res,
|
||||||
|
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||||
|
{
|
||||||
|
auto res_callback = [](const UInt8 * start, const UInt8 * end) -> UInt64
|
||||||
|
{
|
||||||
|
return 1 + Impl::countChars(reinterpret_cast<const char *>(start), reinterpret_cast<const char *>(end));
|
||||||
|
};
|
||||||
|
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||||
|
const size_t haystack_string_size = haystack_offsets.size();
|
||||||
|
res.resize(haystack_string_size);
|
||||||
|
size_t iteration = 0;
|
||||||
|
while (searcher.hasMoreToSearch())
|
||||||
|
{
|
||||||
|
size_t prev_offset = 0;
|
||||||
|
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||||
|
{
|
||||||
|
const auto * haystack = &haystack_data[prev_offset];
|
||||||
|
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||||
|
if (iteration == 0 || res[j] == 0)
|
||||||
|
res[j] = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
||||||
|
else
|
||||||
|
{
|
||||||
|
UInt64 result = searcher.searchOneFirstPosition(haystack, haystack_end, res_callback);
|
||||||
|
if (result != 0)
|
||||||
|
res[j] = std::min(result, res[j]);
|
||||||
|
}
|
||||||
|
prev_offset = haystack_offsets[j];
|
||||||
|
}
|
||||||
|
++iteration;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
47
dbms/src/Functions/MultiSearchImpl.h
Normal file
47
dbms/src/Functions/MultiSearchImpl.h
Normal file
@ -0,0 +1,47 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <vector>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
template <typename Impl>
|
||||||
|
struct MultiSearchImpl
|
||||||
|
{
|
||||||
|
using ResultType = UInt8;
|
||||||
|
static constexpr bool is_using_hyperscan = false;
|
||||||
|
/// Variable for understanding, if we used offsets for the output, most
|
||||||
|
/// likely to determine whether the function returns ColumnVector of ColumnArray.
|
||||||
|
static constexpr bool is_column_array = false;
|
||||||
|
static auto getReturnType() { return std::make_shared<DataTypeNumber<ResultType>>(); }
|
||||||
|
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & haystack_data,
|
||||||
|
const ColumnString::Offsets & haystack_offsets,
|
||||||
|
const std::vector<StringRef> & needles,
|
||||||
|
PaddedPODArray<UInt8> & res,
|
||||||
|
[[maybe_unused]] PaddedPODArray<UInt64> & offsets)
|
||||||
|
{
|
||||||
|
auto searcher = Impl::createMultiSearcherInBigHaystack(needles);
|
||||||
|
const size_t haystack_string_size = haystack_offsets.size();
|
||||||
|
res.resize(haystack_string_size);
|
||||||
|
size_t iteration = 0;
|
||||||
|
while (searcher.hasMoreToSearch())
|
||||||
|
{
|
||||||
|
size_t prev_offset = 0;
|
||||||
|
for (size_t j = 0; j < haystack_string_size; ++j)
|
||||||
|
{
|
||||||
|
const auto * haystack = &haystack_data[prev_offset];
|
||||||
|
const auto * haystack_end = haystack + haystack_offsets[j] - prev_offset - 1;
|
||||||
|
if (iteration == 0 || !res[j])
|
||||||
|
res[j] = searcher.searchOne(haystack, haystack_end);
|
||||||
|
prev_offset = haystack_offsets[j];
|
||||||
|
}
|
||||||
|
++iteration;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
307
dbms/src/Functions/PositionImpl.h
Normal file
307
dbms/src/Functions/PositionImpl.h
Normal file
@ -0,0 +1,307 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
|
||||||
|
#include <string>
|
||||||
|
#include <vector>
|
||||||
|
#include <Poco/UTF8String.h>
|
||||||
|
#include <Common/Volnitsky.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Implementation details for functions of 'position' family depending on ASCII/UTF8 and case sensitiveness.
|
||||||
|
*/
|
||||||
|
struct PositionCaseSensitiveASCII
|
||||||
|
{
|
||||||
|
/// For searching single substring inside big-enough contiguous chunk of data. Coluld have slightly expensive initialization.
|
||||||
|
using SearcherInBigHaystack = Volnitsky;
|
||||||
|
|
||||||
|
/// For search many substrings in one string
|
||||||
|
using MultiSearcherInBigHaystack = MultiVolnitsky;
|
||||||
|
|
||||||
|
/// For searching single substring, that is different each time. This object is created for each row of data. It must have cheap initialization.
|
||||||
|
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
||||||
|
|
||||||
|
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||||
|
{
|
||||||
|
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||||
|
}
|
||||||
|
|
||||||
|
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||||
|
{
|
||||||
|
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||||
|
{
|
||||||
|
return MultiSearcherInBigHaystack(needles);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Number of code points between 'begin' and 'end' (this has different behaviour for ASCII and UTF-8).
|
||||||
|
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||||
|
|
||||||
|
/// Convert string to lowercase. Only for case-insensitive search.
|
||||||
|
/// Implementation is permitted to be inefficient because it is called for single string.
|
||||||
|
static void toLowerIfNeed(std::string &) { }
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct PositionCaseInsensitiveASCII
|
||||||
|
{
|
||||||
|
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
|
||||||
|
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
|
||||||
|
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitive;
|
||||||
|
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
|
||||||
|
|
||||||
|
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t /*haystack_size_hint*/)
|
||||||
|
{
|
||||||
|
return SearcherInBigHaystack(needle_data, needle_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||||
|
{
|
||||||
|
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||||
|
{
|
||||||
|
return MultiSearcherInBigHaystack(needles);
|
||||||
|
}
|
||||||
|
|
||||||
|
static size_t countChars(const char * begin, const char * end) { return end - begin; }
|
||||||
|
|
||||||
|
static void toLowerIfNeed(std::string & s) { std::transform(std::begin(s), std::end(s), std::begin(s), tolower); }
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct PositionCaseSensitiveUTF8
|
||||||
|
{
|
||||||
|
using SearcherInBigHaystack = VolnitskyUTF8;
|
||||||
|
using MultiSearcherInBigHaystack = MultiVolnitskyUTF8;
|
||||||
|
using SearcherInSmallHaystack = LibCASCIICaseSensitiveStringSearcher;
|
||||||
|
|
||||||
|
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||||
|
{
|
||||||
|
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||||
|
}
|
||||||
|
|
||||||
|
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||||
|
{
|
||||||
|
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||||
|
{
|
||||||
|
return MultiSearcherInBigHaystack(needles);
|
||||||
|
}
|
||||||
|
|
||||||
|
static size_t countChars(const char * begin, const char * end)
|
||||||
|
{
|
||||||
|
size_t res = 0;
|
||||||
|
for (auto it = begin; it != end; ++it)
|
||||||
|
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
||||||
|
++res;
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
static void toLowerIfNeed(std::string &) { }
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
struct PositionCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
using SearcherInBigHaystack = VolnitskyCaseInsensitiveUTF8;
|
||||||
|
using MultiSearcherInBigHaystack = MultiVolnitskyCaseInsensitiveUTF8;
|
||||||
|
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
|
||||||
|
|
||||||
|
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
|
||||||
|
{
|
||||||
|
return SearcherInBigHaystack(needle_data, needle_size, haystack_size_hint);
|
||||||
|
}
|
||||||
|
|
||||||
|
static SearcherInSmallHaystack createSearcherInSmallHaystack(const char * needle_data, size_t needle_size)
|
||||||
|
{
|
||||||
|
return SearcherInSmallHaystack(needle_data, needle_size);
|
||||||
|
}
|
||||||
|
|
||||||
|
static MultiSearcherInBigHaystack createMultiSearcherInBigHaystack(const std::vector<StringRef> & needles)
|
||||||
|
{
|
||||||
|
return MultiSearcherInBigHaystack(needles);
|
||||||
|
}
|
||||||
|
|
||||||
|
static size_t countChars(const char * begin, const char * end)
|
||||||
|
{
|
||||||
|
size_t res = 0;
|
||||||
|
for (auto it = begin; it != end; ++it)
|
||||||
|
if (!UTF8::isContinuationOctet(static_cast<UInt8>(*it)))
|
||||||
|
++res;
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
static void toLowerIfNeed(std::string & s) { Poco::UTF8::toLowerInPlace(s); }
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
template <typename Impl>
|
||||||
|
struct PositionImpl
|
||||||
|
{
|
||||||
|
static constexpr bool use_default_implementation_for_constants = false;
|
||||||
|
|
||||||
|
using ResultType = UInt64;
|
||||||
|
|
||||||
|
/// Find one substring in many strings.
|
||||||
|
static void vectorConstant(
|
||||||
|
const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray<UInt64> & res)
|
||||||
|
{
|
||||||
|
const UInt8 * begin = data.data();
|
||||||
|
const UInt8 * pos = begin;
|
||||||
|
const UInt8 * end = pos + data.size();
|
||||||
|
|
||||||
|
/// Current index in the array of strings.
|
||||||
|
size_t i = 0;
|
||||||
|
|
||||||
|
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
|
||||||
|
|
||||||
|
/// We will search for the next occurrence in all strings at once.
|
||||||
|
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
|
||||||
|
{
|
||||||
|
/// Determine which index it refers to.
|
||||||
|
while (begin + offsets[i] <= pos)
|
||||||
|
{
|
||||||
|
res[i] = 0;
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// We check that the entry does not pass through the boundaries of strings.
|
||||||
|
if (pos + needle.size() < begin + offsets[i])
|
||||||
|
res[i] = 1 + Impl::countChars(reinterpret_cast<const char *>(begin + offsets[i - 1]), reinterpret_cast<const char *>(pos));
|
||||||
|
else
|
||||||
|
res[i] = 0;
|
||||||
|
|
||||||
|
pos = begin + offsets[i];
|
||||||
|
++i;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (i < res.size())
|
||||||
|
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Search for substring in string.
|
||||||
|
static void constantConstant(std::string data, std::string needle, UInt64 & res)
|
||||||
|
{
|
||||||
|
Impl::toLowerIfNeed(data);
|
||||||
|
Impl::toLowerIfNeed(needle);
|
||||||
|
|
||||||
|
res = data.find(needle);
|
||||||
|
if (res == std::string::npos)
|
||||||
|
res = 0;
|
||||||
|
else
|
||||||
|
res = 1 + Impl::countChars(data.data(), data.data() + res);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Search each time for a different single substring inside each time different string.
|
||||||
|
static void vectorVector(
|
||||||
|
const ColumnString::Chars & haystack_data,
|
||||||
|
const ColumnString::Offsets & haystack_offsets,
|
||||||
|
const ColumnString::Chars & needle_data,
|
||||||
|
const ColumnString::Offsets & needle_offsets,
|
||||||
|
PaddedPODArray<UInt64> & res)
|
||||||
|
{
|
||||||
|
ColumnString::Offset prev_haystack_offset = 0;
|
||||||
|
ColumnString::Offset prev_needle_offset = 0;
|
||||||
|
|
||||||
|
size_t size = haystack_offsets.size();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||||
|
size_t haystack_size = haystack_offsets[i] - prev_haystack_offset - 1;
|
||||||
|
|
||||||
|
if (0 == needle_size)
|
||||||
|
{
|
||||||
|
/// An empty string is always at the very beginning of `haystack`.
|
||||||
|
res[i] = 1;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/// It is assumed that the StringSearcher is not very difficult to initialize.
|
||||||
|
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
||||||
|
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
|
||||||
|
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
|
||||||
|
|
||||||
|
/// searcher returns a pointer to the found substring or to the end of `haystack`.
|
||||||
|
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
|
||||||
|
- &haystack_data[prev_haystack_offset];
|
||||||
|
|
||||||
|
if (pos != haystack_size)
|
||||||
|
{
|
||||||
|
res[i] = 1
|
||||||
|
+ Impl::countChars(
|
||||||
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset]),
|
||||||
|
reinterpret_cast<const char *>(&haystack_data[prev_haystack_offset + pos]));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res[i] = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
prev_haystack_offset = haystack_offsets[i];
|
||||||
|
prev_needle_offset = needle_offsets[i];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Find many substrings in single string.
|
||||||
|
static void constantVector(
|
||||||
|
const String & haystack,
|
||||||
|
const ColumnString::Chars & needle_data,
|
||||||
|
const ColumnString::Offsets & needle_offsets,
|
||||||
|
PaddedPODArray<UInt64> & res)
|
||||||
|
{
|
||||||
|
// NOTE You could use haystack indexing. But this is a rare case.
|
||||||
|
|
||||||
|
ColumnString::Offset prev_needle_offset = 0;
|
||||||
|
|
||||||
|
size_t size = needle_offsets.size();
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
{
|
||||||
|
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||||
|
|
||||||
|
if (0 == needle_size)
|
||||||
|
{
|
||||||
|
res[i] = 1;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(
|
||||||
|
reinterpret_cast<const char *>(&needle_data[prev_needle_offset]), needle_offsets[i] - prev_needle_offset - 1);
|
||||||
|
|
||||||
|
size_t pos = searcher.search(
|
||||||
|
reinterpret_cast<const UInt8 *>(haystack.data()),
|
||||||
|
reinterpret_cast<const UInt8 *>(haystack.data()) + haystack.size())
|
||||||
|
- reinterpret_cast<const UInt8 *>(haystack.data());
|
||||||
|
|
||||||
|
if (pos != haystack.size())
|
||||||
|
{
|
||||||
|
res[i] = 1 + Impl::countChars(haystack.data(), haystack.data() + pos);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
res[i] = 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
prev_needle_offset = needle_offsets[i];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename... Args>
|
||||||
|
static void vectorFixedConstant(Args &&...)
|
||||||
|
{
|
||||||
|
throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -1,5 +1,5 @@
|
|||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionsStringSearch.h>
|
#include <Functions/FunctionsStringSearchToString.h>
|
||||||
#include <common/find_symbols.h>
|
#include <common/find_symbols.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionsStringSearch.h>
|
#include <Functions/FunctionsStringSearchToString.h>
|
||||||
#include <common/find_symbols.h>
|
#include <common/find_symbols.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
22
dbms/src/Functions/hasToken.cpp
Normal file
22
dbms/src/Functions/hasToken.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include "HasTokenImpl.h"
|
||||||
|
#include <Common/Volnitsky.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameHasToken
|
||||||
|
{
|
||||||
|
static constexpr auto name = "hasToken";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionHasToken = FunctionsStringSearch<HasTokenImpl<VolnitskyCaseSensitiveToken, false>, NameHasToken>;
|
||||||
|
|
||||||
|
void registerFunctionHasToken(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionHasToken>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/hasTokenCaseInsensitive.cpp
Normal file
23
dbms/src/Functions/hasTokenCaseInsensitive.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include "HasTokenImpl.h"
|
||||||
|
#include <Common/Volnitsky.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameHasTokenCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "hasTokenCaseInsensitive";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionHasTokenCaseInsensitive
|
||||||
|
= FunctionsStringSearch<HasTokenImpl<VolnitskyCaseInsensitiveToken, false>, NameHasTokenCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionHasTokenCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionHasTokenCaseInsensitive>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchAllPositions.cpp
Normal file
23
dbms/src/Functions/multiSearchAllPositions.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringPosition.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchAllPositionsImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAllPositions
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAllPositions";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchAllPositions
|
||||||
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveASCII>, NameMultiSearchAllPositions>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAllPositions(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchAllPositions>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringPosition.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchAllPositionsImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAllPositionsCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAllPositionsCaseInsensitive";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchAllPositionsCaseInsensitive
|
||||||
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAllPositionsCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitive>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,24 @@
|
|||||||
|
#include "FunctionsMultiStringPosition.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchAllPositionsImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAllPositionsCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAllPositionsCaseInsensitiveUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchAllPositionsCaseInsensitiveUTF8 = FunctionsMultiStringPosition<
|
||||||
|
MultiSearchAllPositionsImpl<PositionCaseInsensitiveUTF8>,
|
||||||
|
NameMultiSearchAllPositionsCaseInsensitiveUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchAllPositionsCaseInsensitiveUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchAllPositionsUTF8.cpp
Normal file
23
dbms/src/Functions/multiSearchAllPositionsUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringPosition.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchAllPositionsImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAllPositionsUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAllPositionsUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchAllPositionsUTF8
|
||||||
|
= FunctionsMultiStringPosition<MultiSearchAllPositionsImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAllPositionsUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchAllPositionsUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
22
dbms/src/Functions/multiSearchAny.cpp
Normal file
22
dbms/src/Functions/multiSearchAny.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAny
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAny";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearch = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveASCII>, NameMultiSearchAny>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAny(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearch>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
22
dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp
Normal file
22
dbms/src/Functions/multiSearchAnyCaseInsensitive.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAnyCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAnyCaseInsensitive";
|
||||||
|
};
|
||||||
|
using FunctionMultiSearchCaseInsensitive
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveASCII>, NameMultiSearchAnyCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchCaseInsensitive>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp
Normal file
23
dbms/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAnyCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAnyCaseInsensitiveUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchCaseInsensitiveUTF8
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchAnyCaseInsensitiveUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchCaseInsensitiveUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/multiSearchAnyUTF8.cpp
Normal file
21
dbms/src/Functions/multiSearchAnyUTF8.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchAnyUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchAnyUTF8";
|
||||||
|
};
|
||||||
|
using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch<MultiSearchImpl<PositionCaseSensitiveUTF8>, NameMultiSearchAnyUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAnyUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchFirstIndex.cpp
Normal file
23
dbms/src/Functions/multiSearchFirstIndex.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstIndexImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstIndex
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstIndex";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstIndex
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstIndex>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstIndex(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstIndex>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp
Normal file
23
dbms/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstIndexImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstIndexCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstIndexCaseInsensitive";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstIndexCaseInsensitive
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstIndexCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitive>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstIndexImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstIndexCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstIndexCaseInsensitiveUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseInsensitiveUTF8>, NameMultiSearchFirstIndexCaseInsensitiveUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstIndexCaseInsensitiveUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchFirstIndexUTF8.cpp
Normal file
23
dbms/src/Functions/multiSearchFirstIndexUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstIndexImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstIndexUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstIndexUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstIndexUTF8
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstIndexImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstIndexUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstIndexUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchFirstPosition.cpp
Normal file
23
dbms/src/Functions/multiSearchFirstPosition.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstPositionImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstPosition
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstPosition";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstPosition
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveASCII>, NameMultiSearchFirstPosition>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstPosition(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstPosition>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstPositionImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstPositionCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstPositionCaseInsensitive";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstPositionCaseInsensitive
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseInsensitiveASCII>, NameMultiSearchFirstPositionCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitive>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,24 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstPositionImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstPositionCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstPositionCaseInsensitiveUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch<
|
||||||
|
MultiSearchFirstPositionImpl<PositionCaseInsensitiveUTF8>,
|
||||||
|
NameMultiSearchFirstPositionCaseInsensitiveUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstPositionCaseInsensitiveUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/Functions/multiSearchFirstPositionUTF8.cpp
Normal file
23
dbms/src/Functions/multiSearchFirstPositionUTF8.cpp
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#include "FunctionsMultiStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "MultiSearchFirstPositionImpl.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NameMultiSearchFirstPositionUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "multiSearchFirstPositionUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionMultiSearchFirstPositionUTF8
|
||||||
|
= FunctionsMultiStringSearch<MultiSearchFirstPositionImpl<PositionCaseSensitiveUTF8>, NameMultiSearchFirstPositionUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionMultiSearchFirstPositionUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/position.cpp
Normal file
21
dbms/src/Functions/position.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NamePosition
|
||||||
|
{
|
||||||
|
static constexpr auto name = "position";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionPosition = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveASCII>, NamePosition>;
|
||||||
|
|
||||||
|
void registerFunctionPosition(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionPosition>(FunctionFactory::CaseInsensitive);
|
||||||
|
factory.registerAlias("locate", NamePosition::name, FunctionFactory::CaseInsensitive);
|
||||||
|
}
|
||||||
|
}
|
20
dbms/src/Functions/positionCaseInsensitive.cpp
Normal file
20
dbms/src/Functions/positionCaseInsensitive.cpp
Normal file
@ -0,0 +1,20 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NamePositionCaseInsensitive
|
||||||
|
{
|
||||||
|
static constexpr auto name = "positionCaseInsensitive";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionPositionCaseInsensitive = FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveASCII>, NamePositionCaseInsensitive>;
|
||||||
|
|
||||||
|
void registerFunctionPositionCaseInsensitive(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionPositionCaseInsensitive>();
|
||||||
|
}
|
||||||
|
}
|
22
dbms/src/Functions/positionCaseInsensitiveUTF8.cpp
Normal file
22
dbms/src/Functions/positionCaseInsensitiveUTF8.cpp
Normal file
@ -0,0 +1,22 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NamePositionCaseInsensitiveUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "positionCaseInsensitiveUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionPositionCaseInsensitiveUTF8
|
||||||
|
= FunctionsStringSearch<PositionImpl<PositionCaseInsensitiveUTF8>, NamePositionCaseInsensitiveUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionPositionCaseInsensitiveUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
21
dbms/src/Functions/positionUTF8.cpp
Normal file
21
dbms/src/Functions/positionUTF8.cpp
Normal file
@ -0,0 +1,21 @@
|
|||||||
|
#include "FunctionsStringSearch.h"
|
||||||
|
#include "FunctionFactory.h"
|
||||||
|
#include "PositionImpl.h"
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct NamePositionUTF8
|
||||||
|
{
|
||||||
|
static constexpr auto name = "positionUTF8";
|
||||||
|
};
|
||||||
|
|
||||||
|
using FunctionPositionUTF8 = FunctionsStringSearch<PositionImpl<PositionCaseSensitiveUTF8>, NamePositionUTF8>;
|
||||||
|
|
||||||
|
void registerFunctionPositionUTF8(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionPositionUTF8>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
66
dbms/src/Functions/registerFunctionsStringSearch.cpp
Normal file
66
dbms/src/Functions/registerFunctionsStringSearch.cpp
Normal file
@ -0,0 +1,66 @@
|
|||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class FunctionFactory;
|
||||||
|
|
||||||
|
void registerFunctionPosition(FunctionFactory &);
|
||||||
|
void registerFunctionPositionUTF8(FunctionFactory &);
|
||||||
|
void registerFunctionPositionCaseInsensitive(FunctionFactory &);
|
||||||
|
void registerFunctionPositionCaseInsensitiveUTF8(FunctionFactory &);
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAny(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAnyUTF8(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAnyCaseInsensitive(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAnyCaseInsensitiveUTF8(FunctionFactory &);
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstIndex(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstIndexUTF8(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstIndexCaseInsensitive(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(FunctionFactory &);
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchFirstPosition(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstPositionUTF8(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstPositionCaseInsensitive(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(FunctionFactory &);
|
||||||
|
|
||||||
|
void registerFunctionMultiSearchAllPositions(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAllPositionsUTF8(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAllPositionsCaseInsensitive(FunctionFactory &);
|
||||||
|
void registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(FunctionFactory &);
|
||||||
|
|
||||||
|
void registerFunctionHasToken(FunctionFactory &);
|
||||||
|
void registerFunctionHasTokenCaseInsensitive(FunctionFactory &);
|
||||||
|
|
||||||
|
|
||||||
|
void registerFunctionsStringSearch(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
registerFunctionPosition(factory);
|
||||||
|
registerFunctionPositionUTF8(factory);
|
||||||
|
registerFunctionPositionCaseInsensitive(factory);
|
||||||
|
registerFunctionPositionCaseInsensitiveUTF8(factory);
|
||||||
|
|
||||||
|
registerFunctionMultiSearchAny(factory);
|
||||||
|
registerFunctionMultiSearchAnyUTF8(factory);
|
||||||
|
registerFunctionMultiSearchAnyCaseInsensitive(factory);
|
||||||
|
registerFunctionMultiSearchAnyCaseInsensitiveUTF8(factory);
|
||||||
|
|
||||||
|
registerFunctionMultiSearchFirstIndex(factory);
|
||||||
|
registerFunctionMultiSearchFirstIndexUTF8(factory);
|
||||||
|
registerFunctionMultiSearchFirstIndexCaseInsensitive(factory);
|
||||||
|
registerFunctionMultiSearchFirstIndexCaseInsensitiveUTF8(factory);
|
||||||
|
|
||||||
|
registerFunctionMultiSearchFirstPosition(factory);
|
||||||
|
registerFunctionMultiSearchFirstPositionUTF8(factory);
|
||||||
|
registerFunctionMultiSearchFirstPositionCaseInsensitive(factory);
|
||||||
|
registerFunctionMultiSearchFirstPositionCaseInsensitiveUTF8(factory);
|
||||||
|
|
||||||
|
registerFunctionMultiSearchAllPositions(factory);
|
||||||
|
registerFunctionMultiSearchAllPositionsUTF8(factory);
|
||||||
|
registerFunctionMultiSearchAllPositionsCaseInsensitive(factory);
|
||||||
|
registerFunctionMultiSearchAllPositionsCaseInsensitiveUTF8(factory);
|
||||||
|
|
||||||
|
registerFunctionHasToken(factory);
|
||||||
|
registerFunctionHasTokenCaseInsensitive(factory);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -1,6 +1,6 @@
|
|||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionsVisitParam.h>
|
#include <Functions/FunctionsVisitParam.h>
|
||||||
#include <Functions/FunctionsStringSearch.h>
|
#include <Functions/FunctionsStringSearchToString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Functions/FunctionsVisitParam.h>
|
#include <Functions/FunctionsVisitParam.h>
|
||||||
#include <Functions/FunctionsStringSearch.h>
|
#include <Functions/FunctionsStringSearchToString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
|
Loading…
Reference in New Issue
Block a user