mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Fix and optimize countMatches()/countMatchesCaseInsensitive()
- Update after IFunction interfaces changes - move type checks into FunctionCountMatches::getReturnTypeImpl() - Use StringRef over String - Separate out logic for counting sub matches into separate helper - Do not copy other regular expression matches, only the first - Add some comments - Set is_no_capture, to avoid check for number of subpatterns - Add countMatchesCaseInsensitive() - Reguster functions in case-sensitive manner, since this is not SQL standard
This commit is contained in:
parent
0004fd8ab9
commit
737357418f
@ -1,13 +1,29 @@
|
||||
#include "FunctionFactory.h"
|
||||
#include "countMatchesImpl.h"
|
||||
#include "countMatches.h"
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
struct FunctionCountMatchesCaseSensitive
|
||||
{
|
||||
static constexpr auto name = "countMatches";
|
||||
static constexpr bool case_insensitive = false;
|
||||
};
|
||||
struct FunctionCountMatchesCaseInsensitive
|
||||
{
|
||||
static constexpr auto name = "countMatchesCaseInsensitive";
|
||||
static constexpr bool case_insensitive = true;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionCountMatches(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionCountMatches>(FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseSensitive>>(FunctionFactory::CaseSensitive);
|
||||
factory.registerFunction<FunctionCountMatches<FunctionCountMatchesCaseInsensitive>>(FunctionFactory::CaseSensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
125
src/Functions/countMatches.h
Normal file
125
src/Functions/countMatches.h
Normal file
@ -0,0 +1,125 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/Regexps.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
using Pos = const char *;
|
||||
|
||||
template <class CountMatchesBase>
|
||||
class FunctionCountMatches : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = CountMatchesBase::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCountMatches<CountMatchesBase>>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (!isStringOrFixedString(arguments[1].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of second argument (pattern) of function {}. Must be String/FixedString.",
|
||||
arguments[1].type->getName(), getName());
|
||||
if (!isStringOrFixedString(arguments[0].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument (haystack) of function {}. Must be String/FixedString.",
|
||||
arguments[0].type->getName(), getName());
|
||||
const auto * column = arguments[1].column.get();
|
||||
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"The second argument of function {} should be a constant string with the pattern",
|
||||
getName());
|
||||
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
|
||||
{
|
||||
const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
|
||||
Regexps::Pool::Pointer re = Regexps::get<false /* like */, true /* is_no_capture */, CountMatchesBase::case_insensitive>(column_pattern->getValue<String>());
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
|
||||
const IColumn * column_haystack = arguments[0].column.get();
|
||||
|
||||
if (const ColumnString * col_str = checkAndGetColumn<ColumnString>(column_haystack))
|
||||
{
|
||||
auto result_column = ColumnUInt64::create();
|
||||
|
||||
const ColumnString::Chars & src_chars = col_str->getChars();
|
||||
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
|
||||
|
||||
ColumnUInt64::Container & vec_res = result_column->getData();
|
||||
vec_res.resize(input_rows_count);
|
||||
|
||||
size_t size = src_offsets.size();
|
||||
ColumnString::Offset current_src_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
|
||||
current_src_offset = src_offsets[i];
|
||||
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
|
||||
|
||||
StringRef str(pos, end - pos);
|
||||
vec_res[i] = countMatches(str, re, matches);
|
||||
}
|
||||
|
||||
return result_column;
|
||||
}
|
||||
else if (const ColumnConst * col_const_str = checkAndGetColumnConstStringOrFixedString(column_haystack))
|
||||
{
|
||||
StringRef str = col_const_str->getDataColumn().getDataAt(0);
|
||||
uint64_t matches_count = countMatches(str, re, matches);
|
||||
return result_type->createColumnConst(input_rows_count, matches_count);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()");
|
||||
}
|
||||
|
||||
static uint64_t countMatches(StringRef src, Regexps::Pool::Pointer & re, OptimizedRegularExpression::MatchVec & matches)
|
||||
{
|
||||
/// Only one match is required, no need to copy more.
|
||||
static const unsigned matches_limit = 1;
|
||||
|
||||
Pos pos = reinterpret_cast<Pos>(src.data);
|
||||
Pos end = reinterpret_cast<Pos>(src.data + src.size);
|
||||
|
||||
uint64_t match_count = 0;
|
||||
while (true)
|
||||
{
|
||||
if (pos >= end)
|
||||
break;
|
||||
if (!re->match(pos, end - pos, matches, matches_limit))
|
||||
break;
|
||||
/// Progress should be made, but with empty match the progress will not be done.
|
||||
/// Also note that simply check is pattern empty is not enough,
|
||||
/// since for example "'[f]{0}'" will match zero bytes:
|
||||
if (!matches[0].length)
|
||||
break;
|
||||
pos += matches[0].offset + matches[0].length;
|
||||
match_count++;
|
||||
}
|
||||
|
||||
return match_count;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -1,132 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/IFunctionImpl.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/Regexps.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
using Pos = const char *;
|
||||
|
||||
class FunctionCountMatches : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "countMatches";
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCountMatches>(); }
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
|
||||
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get());
|
||||
|
||||
if (!col)
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||
+ " of first argument of function " + getName() + ". Must be constant string.",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
Regexps::Pool::Pointer re = Regexps::get<false, false>(col->getValue<String>());
|
||||
size_t capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
|
||||
OptimizedRegularExpression::MatchVec matches;
|
||||
matches.resize(capture + 1);
|
||||
|
||||
size_t array_argument_position = arguments[0];
|
||||
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(block.getByPosition(array_argument_position).column.get());
|
||||
const ColumnConst * col_const_str =
|
||||
checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get());
|
||||
|
||||
auto col_res = ColumnUInt64::create();
|
||||
ColumnUInt64::Container & vec_res = col_res->getData();
|
||||
|
||||
if (col_str)
|
||||
{
|
||||
const ColumnString::Chars & src_chars = col_str->getChars();
|
||||
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
|
||||
|
||||
vec_res.resize(src_offsets.size());
|
||||
|
||||
size_t size = src_offsets.size();
|
||||
ColumnString::Offset current_src_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Pos pos = reinterpret_cast<Pos>(&src_chars[current_src_offset]);
|
||||
current_src_offset = src_offsets[i];
|
||||
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
|
||||
|
||||
uint64_t match_count = 0;
|
||||
while (true)
|
||||
{
|
||||
if (!pos || pos > end)
|
||||
break;
|
||||
if (!re->match(pos, end - pos, matches) || !matches[0].length)
|
||||
break;
|
||||
pos += matches[0].offset + matches[0].length;
|
||||
match_count++;
|
||||
}
|
||||
vec_res[i] = match_count;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = std::move(col_res);
|
||||
}
|
||||
else if (col_const_str)
|
||||
{
|
||||
String src = col_const_str->getValue<String>();
|
||||
|
||||
Pos pos = reinterpret_cast<Pos>(src.data());
|
||||
Pos end = reinterpret_cast<Pos>(src.data() + src.size());
|
||||
|
||||
uint64_t match_count = 0;
|
||||
while (true)
|
||||
{
|
||||
if (!pos || pos > end)
|
||||
break;
|
||||
if (!re->match(pos, end - pos, matches) || !matches[0].length)
|
||||
break;
|
||||
pos += matches[0].offset + matches[0].length;
|
||||
match_count++;
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = DataTypeUInt64().createColumnConst(col_const_str->size(), match_count);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName()
|
||||
+ ", " + block.getByPosition(array_argument_position).column->getName()
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -208,6 +208,7 @@ SRCS(
|
||||
cos.cpp
|
||||
cosh.cpp
|
||||
countDigits.cpp
|
||||
countMatches.cpp
|
||||
countSubstrings.cpp
|
||||
countSubstringsCaseInsensitive.cpp
|
||||
countSubstringsCaseInsensitiveUTF8.cpp
|
||||
|
Loading…
Reference in New Issue
Block a user