enable limits for functions using FunctionTokens

This commit is contained in:
taiyang-li 2022-10-25 16:38:36 +08:00
parent 011ae8675a
commit fcbc217a7d
9 changed files with 157 additions and 70 deletions

View File

@ -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;
}
};

View File

@ -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_)
{

View File

@ -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_)
{

View File

@ -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.

View File

@ -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_)
{

View File

@ -5,3 +5,8 @@
['gbye','bug']
['']
[]
['one','two','three','']
['one','two','three','']
['one','two','three','']
['one']
['one','two']

View File

@ -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);

View File

@ -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']

View File

@ -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 }