mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 12:32:04 +00:00
enable limits for functions using FunctionTokens
This commit is contained in:
parent
011ae8675a
commit
fcbc217a7d
@ -32,12 +32,12 @@ namespace ErrorCodes
|
||||
|
||||
/** Functions that split strings into an array of strings or vice versa.
|
||||
*
|
||||
* splitByChar(sep, s)
|
||||
* splitByString(sep, s)
|
||||
* splitByRegexp(regexp, s)
|
||||
* splitByChar(sep, s[, max_split])
|
||||
* splitByString(sep, s[, max_split])
|
||||
* splitByRegexp(regexp, s[, max_split])
|
||||
*
|
||||
* splitByWhitespace(s) - split the string by whitespace characters
|
||||
* splitByNonAlpha(s) - split the string by whitespace and punctuation characters
|
||||
* splitByWhitespace(s[, max_split]) - split the string by whitespace characters
|
||||
* splitByNonAlpha(s[, max_split]) - split the string by whitespace and punctuation characters
|
||||
*
|
||||
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
|
||||
* - first subpattern, if regexp has subpattern;
|
||||
@ -98,6 +98,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Get the next token, if any, or return false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
@ -157,6 +163,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Get the next token, if any, or return false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
@ -216,6 +228,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Get the next token, if any, or return false.
|
||||
bool get(Pos & token_begin, Pos & token_end)
|
||||
{
|
||||
@ -242,10 +260,7 @@ class SplitByCharImpl
|
||||
private:
|
||||
Pos pos;
|
||||
Pos end;
|
||||
|
||||
char sep;
|
||||
std::optional<UInt64> max_split;
|
||||
UInt64 curr_split = 0;
|
||||
|
||||
public:
|
||||
static constexpr auto name = "splitByChar";
|
||||
@ -268,13 +283,6 @@ public:
|
||||
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);
|
||||
|
||||
if (arguments.size() == 3 && !isNativeInteger(arguments[2]))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Third argument for function '{}' must be integer, got '{}' instead",
|
||||
getName(),
|
||||
arguments[2]->getName());
|
||||
}
|
||||
|
||||
void init(const ColumnsWithTypeAndName & arguments)
|
||||
@ -292,39 +300,6 @@ public:
|
||||
throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
sep = sep_str[0];
|
||||
|
||||
if (arguments.size() > 2)
|
||||
{
|
||||
if (!((max_split = getMaxSplit<UInt8>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<Int8>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<UInt16>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<Int16>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<UInt32>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<Int32>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<UInt64>(arguments[2]))
|
||||
|| (max_split = getMaxSplit<Int64>(arguments[2]))))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {} of third argument of function {}",
|
||||
arguments[2].column->getName(),
|
||||
getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <typename DataType>
|
||||
std::optional<UInt64> getMaxSplit(const ColumnWithTypeAndName & argument)
|
||||
{
|
||||
const auto * col = checkAndGetColumnConst<ColumnVector<DataType>>(argument.column.get());
|
||||
if (!col)
|
||||
return std::nullopt;
|
||||
|
||||
auto value = col->template getValue<DataType>();
|
||||
if (value < 0)
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of third argument of function {}", argument.column->getName(), getName());
|
||||
return value;
|
||||
}
|
||||
|
||||
/// Returns the position of the argument, that is the column of strings
|
||||
@ -333,6 +308,12 @@ public:
|
||||
return 1;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
pos = pos_;
|
||||
@ -346,19 +327,12 @@ public:
|
||||
return false;
|
||||
|
||||
token_begin = pos;
|
||||
if (unlikely(max_split && curr_split >= *max_split))
|
||||
{
|
||||
token_end = end;
|
||||
pos = nullptr;
|
||||
return true;
|
||||
}
|
||||
|
||||
pos = reinterpret_cast<Pos>(memchr(pos, sep, end - pos));
|
||||
|
||||
if (pos)
|
||||
{
|
||||
token_end = pos;
|
||||
++pos;
|
||||
++curr_split;
|
||||
}
|
||||
else
|
||||
token_end = end;
|
||||
@ -405,6 +379,12 @@ public:
|
||||
return 1;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
@ -454,12 +434,13 @@ private:
|
||||
|
||||
Pos pos;
|
||||
Pos end;
|
||||
|
||||
public:
|
||||
static constexpr auto name = "splitByRegexp";
|
||||
static String getName() { return name; }
|
||||
|
||||
static bool isVariadic() { return false; }
|
||||
static size_t getNumberOfArguments() { return 2; }
|
||||
static bool isVariadic() { return true; }
|
||||
static size_t getNumberOfArguments() { return 0; }
|
||||
|
||||
/// Check the type of function arguments.
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
@ -479,7 +460,6 @@ public:
|
||||
|
||||
if (!col->getValue<String>().empty())
|
||||
re = std::make_shared<Regexps::Regexp>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
|
||||
|
||||
}
|
||||
|
||||
/// Returns the position of the argument that is the column of strings
|
||||
@ -488,6 +468,12 @@ public:
|
||||
return 1;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return 2;
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
@ -573,6 +559,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
@ -630,6 +622,16 @@ public:
|
||||
{
|
||||
Generator::checkArguments(arguments);
|
||||
|
||||
const auto max_split_pos = Generator::getMaxSplitArgumentPosition();
|
||||
if (max_split_pos)
|
||||
if (arguments.size() > *max_split_pos && !isNativeInteger(arguments[*max_split_pos]))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"{}-th argument for function '{}' must be integer, got '{}' instead",
|
||||
*max_split_pos + 1,
|
||||
getName(),
|
||||
arguments[*max_split_pos]->getName());
|
||||
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
@ -639,6 +641,10 @@ public:
|
||||
generator.init(arguments);
|
||||
const auto & array_argument = arguments[generator.getStringsArgumentPosition()];
|
||||
|
||||
/// Whether we need to limit max tokens returned by Generator::get
|
||||
/// If max_split is std::nullopt, no limit is applied.
|
||||
auto max_split = getMaxSplit(arguments);
|
||||
|
||||
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
|
||||
const ColumnConst * col_const_str =
|
||||
checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
|
||||
@ -672,9 +678,8 @@ public:
|
||||
Pos end = reinterpret_cast<Pos>(&src_chars[current_src_offset]) - 1;
|
||||
|
||||
generator.set(pos, end);
|
||||
|
||||
size_t j = 0;
|
||||
while (generator.get(token_begin, token_end))
|
||||
while (generator.get(token_begin, token_end) && !(max_split && j >= *max_split))
|
||||
{
|
||||
size_t token_size = token_end - token_begin;
|
||||
|
||||
@ -702,7 +707,7 @@ public:
|
||||
Pos token_begin = nullptr;
|
||||
Pos token_end = nullptr;
|
||||
|
||||
while (generator.get(token_begin, token_end))
|
||||
while (generator.get(token_begin, token_end) && !(max_split && dst.size() >= *max_split))
|
||||
dst.push_back(String(token_begin, token_end - token_begin));
|
||||
|
||||
return result_type->createColumnConst(col_const_str->size(), dst);
|
||||
@ -713,6 +718,47 @@ public:
|
||||
+ " of arguments of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename DataType>
|
||||
std::optional<Int64> getMaxSplitImpl(const ColumnWithTypeAndName & argument) const
|
||||
{
|
||||
const auto * col = checkAndGetColumnConst<ColumnVector<DataType>>(argument.column.get());
|
||||
if (!col)
|
||||
return {};
|
||||
|
||||
auto value = col->template getValue<DataType>();
|
||||
return static_cast<Int64>(value);
|
||||
}
|
||||
|
||||
std::optional<size_t> getMaxSplit(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
const auto pos = Generator::getMaxSplitArgumentPosition();
|
||||
if (!pos)
|
||||
return {};
|
||||
|
||||
if (arguments.size() <= *pos)
|
||||
return {};
|
||||
|
||||
std::optional<Int64> max_split;
|
||||
if (!((max_split = getMaxSplitImpl<UInt8>(arguments[2])) || (max_split = getMaxSplitImpl<Int8>(arguments[2]))
|
||||
|| (max_split = getMaxSplitImpl<UInt16>(arguments[2])) || (max_split = getMaxSplitImpl<Int16>(arguments[2]))
|
||||
|| (max_split = getMaxSplitImpl<UInt32>(arguments[2])) || (max_split = getMaxSplitImpl<Int32>(arguments[2]))
|
||||
|| (max_split = getMaxSplitImpl<UInt64>(arguments[2])) || (max_split = getMaxSplitImpl<Int64>(arguments[2]))))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Illegal column {}, which is {}-th argument of function {}",
|
||||
arguments[*pos].column->getName(),
|
||||
*pos + 1,
|
||||
getName());
|
||||
|
||||
/// If max_split is negative or zero, tokenize will be applied as many times as possible, which is equivalent to
|
||||
/// no max_split argument in function
|
||||
if (max_split && *max_split <= 0)
|
||||
return {};
|
||||
|
||||
return *max_split;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -38,6 +38,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
|
@ -37,6 +37,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
|
@ -35,6 +35,13 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
|
||||
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
|
||||
|
||||
/// Called for each next string.
|
||||
|
@ -37,6 +37,12 @@ public:
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function.
|
||||
static std::optional<size_t> getMaxSplitArgumentPosition()
|
||||
{
|
||||
return std::nullopt;
|
||||
}
|
||||
|
||||
/// Called for each next string.
|
||||
void set(Pos pos_, Pos end_)
|
||||
{
|
||||
|
@ -5,3 +5,8 @@
|
||||
['gbye','bug']
|
||||
['']
|
||||
[]
|
||||
['one','two','three','']
|
||||
['one','two','three','']
|
||||
['one','two','three','']
|
||||
['one']
|
||||
['one','two']
|
||||
|
@ -3,3 +3,9 @@ select splitByRegexp('', 'abcde');
|
||||
select splitByRegexp('<[^<>]*>', x) from (select arrayJoin(['<h1>hello<h2>world</h2></h1>', 'gbye<split>bug']) x);
|
||||
select splitByRegexp('ab', '');
|
||||
select splitByRegexp('', '');
|
||||
|
||||
SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC');
|
||||
SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0);
|
||||
SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1);
|
||||
SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1);
|
||||
SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2);
|
||||
|
@ -1,7 +1,10 @@
|
||||
['1','2','3']
|
||||
['1,2,3']
|
||||
['1','2,3']
|
||||
['1']
|
||||
['1','2']
|
||||
['1','2','3']
|
||||
['1','2','3']
|
||||
['expr1','1+1=2']
|
||||
['expr2','2+2=4=1+3']
|
||||
['1','2','3']
|
||||
['1','2','3']
|
||||
['1','2','3']
|
||||
['expr1']
|
||||
['expr2']
|
||||
|
@ -1,10 +1,12 @@
|
||||
select splitByChar(',', '1,2,3');
|
||||
select splitByChar(',', '1,2,3', 0);
|
||||
select splitByChar(',', '1,2,3', 1);
|
||||
select splitByChar(',', '1,2,3', 2);
|
||||
select splitByChar(',', '1,2,3', 3);
|
||||
select splitByChar(',', '1,2,3', 4);
|
||||
select splitByChar(',', '1,2,3', 0);
|
||||
select splitByChar(',', '1,2,3', -1);
|
||||
select splitByChar(',', '1,2,3', -2);
|
||||
SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3');
|
||||
|
||||
select splitByChar(',', '1,2,3', -2); -- { serverError 44 }
|
||||
select splitByChar(',', '1,2,3', ''); -- { serverError 43 }
|
||||
|
||||
SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3')
|
||||
select splitByChar(',', '1,2,3', ''); -- { serverError 43 }
|
||||
|
Loading…
Reference in New Issue
Block a user