Refactorings and cleanups (semantics did not change)

This commit is contained in:
Robert Schulze 2023-09-11 16:03:03 +00:00
parent 711876dfa8
commit 68cdfbcc6c
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 121 additions and 242 deletions

View File

@ -105,7 +105,7 @@ void validateArgumentType(const IFunction & func, const DataTypes & arguments,
const auto & argument = arguments[argument_index];
if (!validator_func(*argument))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of {} argument of function {} expected {}",
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of {} argument of function {}, expected {}",
argument->getName(), std::to_string(argument_index), func.getName(), expected_type_description);
}

View File

@ -9,19 +9,17 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const DataTypes & arguments) const
DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
if (arguments.size() != 1 && arguments.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2.",
getName(), arguments.size());
FunctionArgumentDescriptors mandatory_args{
{"arr", &isArray<IDataType>, nullptr, "Array"},
};
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be an array.", getName());
FunctionArgumentDescriptors optional_args{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
};
if (arguments.size() == 2 && !isString(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Second argument for function {} must be constant string.", getName());
validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
return std::make_shared<DataTypeString>();
}

View File

@ -66,7 +66,6 @@ private:
Pos end;
public:
/// Get the name of the function.
static constexpr auto name = "alphaTokens";
static String getName() { return name; }
@ -74,18 +73,22 @@ public:
static size_t getNumberOfArguments() { return 0; }
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (arguments.empty() || arguments.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName());
FunctionArgumentDescriptors mandatory_args{
{"s", &isString<IDataType>, nullptr, "String"},
};
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
/// Initialize by the function arguments.
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional(1);
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Called for each next string.
@ -95,18 +98,6 @@ public:
end = end_;
}
/// Returns the position of the argument, that is the column of strings
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 1;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
@ -142,18 +133,14 @@ public:
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (arguments.empty() || arguments.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName());
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
SplitByAlphaImpl::checkArguments(func, arguments);
}
/// Initialize by the function arguments.
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional(1);
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Called for each next string.
@ -163,18 +150,6 @@ public:
end = end_;
}
/// Returns the position of the argument, that is the column of strings
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 1;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
@ -203,25 +178,20 @@ private:
Pos end;
public:
/// Get the name of the function.
static constexpr auto name = "splitByWhitespace";
static String getName() { return name; }
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (arguments.empty() || arguments.size() > 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName());
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
return SplitByNonAlphaImpl::checkArguments(func, arguments);
}
/// Initialize by the function arguments.
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional(1);
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Called for each next string.
@ -231,18 +201,6 @@ public:
end = end_;
}
/// Returns the position of the argument, that is the column of strings
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 1;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
@ -269,7 +227,7 @@ class SplitByCharImpl
private:
Pos pos;
Pos end;
char sep;
char separator;
public:
static constexpr auto name = "splitByChar";
@ -277,23 +235,23 @@ public:
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (arguments.size() < 2 || arguments.size() > 3)
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Function '{}' needs at least 2 arguments, at most 3 arguments; passed {}.",
name, arguments.size());
FunctionArgumentDescriptors mandatory_args{
{"separator", &isString<IDataType>, isColumnConst, "const String"},
{"s", &isString<IDataType>, nullptr, "String"}
};
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
FunctionArgumentDescriptors optional_args{
{"max_substrings", &isNativeInteger<IDataType>, isColumnConst, "const Number"},
};
if (!isString(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}. "
"Must be String.", arguments[1]->getName(), getName());
validateFunctionArgumentTypes(func, arguments, mandatory_args, optional_args);
}
static constexpr auto strings_argument_position = 1uz;
static constexpr auto max_substrings_argument_position = std::make_optional(2);
void init(const ColumnsWithTypeAndName & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
@ -307,19 +265,7 @@ public:
if (sep_str.size() != 1)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal separator for function {}. Must be exactly one byte.", getName());
sep = sep_str[0];
}
/// Returns the position of the argument, that is the column of strings
static size_t getStringsArgumentPosition()
{
return 1;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 2;
separator = sep_str[0];
}
void set(Pos pos_, Pos end_)
@ -334,7 +280,7 @@ public:
return false;
token_begin = pos;
pos = reinterpret_cast<Pos>(memchr(pos, sep, end - pos));
pos = reinterpret_cast<Pos>(memchr(pos, separator, end - pos));
if (pos)
{
@ -355,7 +301,7 @@ private:
Pos pos;
Pos end;
String sep;
String separator;
public:
static constexpr auto name = "splitByString";
@ -363,11 +309,14 @@ public:
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByCharImpl::checkArguments(arguments);
SplitByCharImpl::checkArguments(func, arguments);
}
static constexpr auto strings_argument_position = 1uz;
static constexpr auto max_substrings_argument_position = std::make_optional(2);
void init(const ColumnsWithTypeAndName & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
@ -376,19 +325,7 @@ public:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[0].column->getName(), getName());
sep = col->getValue<String>();
}
/// Returns the position of the argument that is the column of strings
static size_t getStringsArgumentPosition()
{
return 1;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 2;
separator = col->getValue<String>();
}
/// Called for each next string.
@ -401,7 +338,7 @@ public:
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (sep.empty())
if (separator.empty())
{
if (pos == end)
return false;
@ -417,12 +354,12 @@ public:
token_begin = pos;
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, sep.data(), sep.size()));
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, separator.data(), separator.size()));
if (pos)
{
token_end = pos;
pos += sep.size();
pos += separator.size();
}
else
token_end = end;
@ -448,13 +385,14 @@ public:
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
/// Check the type of function arguments.
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByStringImpl::checkArguments(arguments);
SplitByStringImpl::checkArguments(func, arguments);
}
/// Initialize by the function arguments.
static constexpr auto strings_argument_position = 1uz;
static constexpr auto max_substrings_argument_position = std::make_optional(2);
void init(const ColumnsWithTypeAndName & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
@ -467,18 +405,6 @@ public:
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
}
/// Returns the position of the argument that is the column of strings
static size_t getStringsArgumentPosition()
{
return 1;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return 2;
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
@ -536,13 +462,19 @@ public:
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 2; }
/// Check the type of function arguments.
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
SplitByStringImpl::checkArguments(arguments);
FunctionArgumentDescriptors mandatory_args{
{"haystack", &isString<IDataType>, nullptr, "String"},
{"pattern", &isString<IDataType>, isColumnConst, "const String"}
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
/// Initialize by the function arguments.
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional<size_t>();
void init(const ColumnsWithTypeAndName & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
@ -557,18 +489,6 @@ public:
matches.resize(capture + 1);
}
/// Returns the position of the argument that is the column of strings
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return std::nullopt;
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
@ -611,10 +531,7 @@ public:
static constexpr auto name = Generator::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTokens>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
@ -622,18 +539,9 @@ public:
size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
Generator::checkArguments(arguments);
const auto max_substrings_pos = Generator::getMaxSubstringsArgumentPosition();
if (max_substrings_pos && *max_substrings_pos < arguments.size() && !isNativeInteger(arguments[*max_substrings_pos]))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"{}-th argument for function '{}' must be integer, got '{}' instead",
*max_substrings_pos + 1,
getName(),
arguments[*max_substrings_pos]->getName());
Generator::checkArguments(*this, arguments);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
@ -642,22 +550,24 @@ public:
{
Generator generator;
generator.init(arguments);
const auto & array_argument = arguments[generator.getStringsArgumentPosition()];
const auto & array_argument = arguments[generator.strings_argument_position];
/// Whether we need to limit max tokens returned by Generator::get
/// If max_substrings is std::nullopt, no limit is applied.
auto max_substrings = getMaxSubstrings(arguments);
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
const ColumnConst * col_const_str =
checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
const ColumnConst * col_str_const = checkAndGetColumnConstStringOrFixedString(array_argument.column.get());
auto col_res = ColumnArray::create(ColumnString::create());
ColumnString & res_strings = typeid_cast<ColumnString &>(col_res->getData());
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
ColumnString::Chars & res_strings_chars = res_strings.getChars();
ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets();
ColumnArray::Offsets & res_offsets = col_res->getOffsets();
if (col_str)
{
const ColumnString::Chars & src_chars = col_str->getChars();
@ -701,9 +611,9 @@ public:
return col_res;
}
else if (col_const_str)
else if (col_str_const)
{
String src = col_const_str->getValue<String>();
String src = col_str_const->getValue<String>();
Array dst;
generator.set(src.data(), src.data() + src.size());
@ -713,7 +623,7 @@ public:
while (generator.get(token_begin, token_end) && !(max_substrings && dst.size() >= *max_substrings))
dst.push_back(String(token_begin, token_end - token_begin));
return result_type->createColumnConst(col_const_str->size(), dst);
return result_type->createColumnConst(col_str_const->size(), dst);
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal columns {}, {} of arguments of function {}",
@ -734,7 +644,7 @@ private:
std::optional<size_t> getMaxSubstrings(const ColumnsWithTypeAndName & arguments) const
{
const auto pos = Generator::getMaxSubstringsArgumentPosition();
const auto pos = Generator::max_substrings_argument_position;
if (!pos)
return std::nullopt;
@ -758,7 +668,7 @@ private:
if (max_substrings && *max_substrings <= 0)
return std::nullopt;
return *max_substrings;
return max_substrings;
}
};
@ -803,7 +713,7 @@ private:
/// Loop through the rows within the array. /// NOTE You can do everything in one copy, if the separator has a size of 1.
for (auto next_src_array_offset = src_array_offsets[i]; current_src_array_offset < next_src_array_offset; ++current_src_array_offset)
{
if (unlikely(null_map && null_map[current_src_array_offset]))
if (null_map && null_map[current_src_array_offset]) [[unlikely]]
continue;
if (!first_non_null)
@ -881,7 +791,7 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{

View File

@ -23,27 +23,20 @@ public:
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
FunctionArgumentDescriptors mandatory_args{
{"URL", &isString<IDataType>, nullptr, "String"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional<size_t>();
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return std::nullopt;
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{

View File

@ -22,27 +22,20 @@ public:
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
FunctionArgumentDescriptors mandatory_args{
{"URL", &isString<IDataType>, nullptr, "String"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional<size_t>();
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return std::nullopt;
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{

View File

@ -22,25 +22,17 @@ public:
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
}
/// Returns the position of the argument that is the column of rows
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return std::nullopt;
FunctionArgumentDescriptors mandatory_args{
{"URL", &isString<IDataType>, nullptr, "String"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional<size_t>();
void init(const ColumnsWithTypeAndName & /*arguments*/) {}

View File

@ -22,26 +22,19 @@ public:
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
static void checkArguments(const DataTypes & arguments)
static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments)
{
if (!isString(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. "
"Must be String.", arguments[0]->getName(), getName());
FunctionArgumentDescriptors mandatory_args{
{"URL", &isString<IDataType>, nullptr, "String"},
};
validateFunctionArgumentTypes(func, arguments, mandatory_args);
}
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Returns the position of the argument that is the column of rows
static size_t getStringsArgumentPosition()
{
return 0;
}
/// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function.
static std::optional<size_t> getMaxSubstringsArgumentPosition()
{
return std::nullopt;
}
static constexpr auto strings_argument_position = 0uz;
static constexpr auto max_substrings_argument_position = std::make_optional<size_t>();
/// Called for each next string.
void set(Pos pos_, Pos end_)