ClickHouse/src/Functions/FunctionsStringArray.h

895 lines
28 KiB
C++
Raw Normal View History

2012-09-17 04:34:19 +00:00
#pragma once
2021-10-12 21:23:40 +00:00
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFixedString.h>
2021-10-19 17:54:27 +00:00
#include <Columns/ColumnNullable.h>
2021-10-12 21:23:40 +00:00
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
2021-10-12 21:23:40 +00:00
#include <DataTypes/DataTypeNullable.h>
2019-02-10 17:40:52 +00:00
#include <DataTypes/DataTypeString.h>
2021-10-12 21:23:40 +00:00
#include <Functions/FunctionHelpers.h>
2021-10-22 20:19:32 +00:00
#include <Functions/FunctionsConversion.h>
2021-05-17 07:30:42 +00:00
#include <Functions/IFunction.h>
#include <Functions/Regexps.h>
#include <IO/WriteHelpers.h>
2021-10-12 21:23:40 +00:00
#include <Interpreters/Context_fwd.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/assert_cast.h>
#include <Common/typeid_cast.h>
2012-09-17 04:34:19 +00:00
namespace DB
{
2017-06-13 02:06:53 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
2022-01-30 01:10:27 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2017-06-13 02:06:53 +00:00
}
2017-05-27 15:45:25 +00:00
/** Functions that split strings into an array of strings or vice versa.
2012-09-17 04:34:19 +00:00
*
* splitByChar(sep, s)
* splitByString(sep, s)
* splitByRegexp(regexp, s)
*
* splitByWhitespace(s) - split the string by whitespace characters
* splitByNonAlpha(s) - split the string by whitespace and punctuation characters
*
2017-05-27 15:45:25 +00:00
* extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp.
* - first subpattern, if regexp has subpattern;
* - zero subpattern (the match part, otherwise);
* - otherwise, an empty array
*
* arrayStringConcat(arr)
* arrayStringConcat(arr, delimiter)
2017-05-27 15:45:25 +00:00
* - join an array of strings into one string via a separator.
*
2017-05-27 15:45:25 +00:00
* alphaTokens(s) - select from the string subsequence `[a-zA-Z]+`.
*
2017-05-27 15:45:25 +00:00
* URL functions are located separately.
2012-09-17 04:34:19 +00:00
*/
using Pos = const char *;
2012-09-17 04:34:19 +00:00
2017-05-27 15:45:25 +00:00
/// Substring generators. All of them have a common interface.
class AlphaTokensImpl
2012-09-17 04:34:19 +00:00
{
private:
Pos pos;
Pos end;
public:
2017-05-27 15:45:25 +00:00
/// Get the name of the function.
static constexpr auto name = "alphaTokens";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
2017-05-27 15:45:25 +00:00
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
2017-05-27 15:45:25 +00:00
/// Initialize by the function arguments.
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
2017-05-27 15:45:25 +00:00
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
2017-05-27 15:45:25 +00:00
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
2017-05-27 15:45:25 +00:00
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && !isAlphaASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
while (pos < end && isAlphaASCII(*pos))
++pos;
token_end = pos;
return true;
}
};
2012-09-17 04:34:19 +00:00
class SplitByNonAlphaImpl
{
private:
Pos pos;
Pos end;
public:
/// Get the name of the function.
static constexpr auto name = "splitByNonAlpha";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// Initialize by the function arguments.
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && (isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
if (pos == end)
return false;
token_begin = pos;
while (pos < end && !(isWhitespaceASCII(*pos) || isPunctuationASCII(*pos)))
++pos;
token_end = pos;
return true;
}
};
class SplitByWhitespaceImpl
{
private:
Pos pos;
Pos end;
public:
/// Get the name of the function.
static constexpr auto name = "splitByWhitespace";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 1; }
/// Check the type of the function's arguments.
static void checkArguments(const DataTypes & arguments)
{
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// Initialize by the function arguments.
void init(const ColumnsWithTypeAndName & /*arguments*/) {}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
/// Skip garbage
while (pos < end && isWhitespaceASCII(*pos))
++pos;
if (pos == end)
return false;
token_begin = pos;
while (pos < end && !isWhitespaceASCII(*pos))
++pos;
token_end = pos;
return true;
}
};
2012-09-17 04:34:19 +00:00
class SplitByCharImpl
2012-09-17 04:34:19 +00:00
{
private:
Pos pos;
Pos end;
char sep;
2022-02-05 11:30:40 +00:00
std::optional<UInt64> max_split;
UInt64 curr_split = 0;
2012-09-17 04:34:19 +00:00
public:
static constexpr auto name = "splitByChar";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return true; }
static size_t getNumberOfArguments() { return 0; }
static void checkArguments(const DataTypes & arguments)
{
2022-01-30 01:10:27 +00:00
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 {}.",
arguments.size());
if (!isString(arguments[0]))
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
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);
2022-01-29 15:42:04 +00:00
2022-01-30 01:10:27 +00:00
if (arguments.size() == 3 && !isNativeInteger(arguments[2]))
2022-01-29 15:42:04 +00:00
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)
{
2020-10-18 19:00:13 +00:00
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
2020-10-18 19:00:13 +00:00
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
String sep_str = col->getValue<String>();
if (sep_str.size() != 1)
throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.", ErrorCodes::BAD_ARGUMENTS);
sep = sep_str[0];
2022-01-29 15:42:04 +00:00
if (arguments.size() > 2)
{
2022-02-05 11:30:40 +00:00
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]))))
2022-01-29 15:42:04 +00:00
{
2022-01-30 05:23:11 +00:00
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of third argument of function {}",
arguments[2].column->getName(),
getName());
2022-01-29 15:42:04 +00:00
}
}
}
template <typename DataType>
2022-02-05 11:30:40 +00:00
std::optional<UInt64> getMaxSplit(const ColumnWithTypeAndName & argument)
2022-01-29 15:42:04 +00:00
{
const auto * col = checkAndGetColumnConst<ColumnVector<DataType>>(argument.column.get());
if (!col)
return std::nullopt;
2022-02-05 11:30:40 +00:00
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;
}
2017-05-27 15:45:25 +00:00
/// Returns the position of the argument, that is the column of strings
size_t getStringsArgumentPosition()
{
return 1;
}
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos)
return false;
token_begin = pos;
2022-02-05 11:30:40 +00:00
if (unlikely(max_split && curr_split >= *max_split))
2022-01-29 15:42:04 +00:00
{
token_end = end;
pos = nullptr;
return true;
}
2022-02-05 11:30:40 +00:00
pos = reinterpret_cast<Pos>(memchr(pos, sep, end - pos));
if (pos)
{
token_end = pos;
++pos;
2022-01-29 15:42:04 +00:00
++curr_split;
}
else
token_end = end;
return true;
}
};
class SplitByStringImpl
{
private:
Pos pos;
Pos end;
String sep;
public:
static constexpr auto name = "splitByString";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 2; }
static void checkArguments(const DataTypes & arguments)
{
SplitByCharImpl::checkArguments(arguments);
}
void init(const ColumnsWithTypeAndName & arguments)
{
2020-10-18 19:00:13 +00:00
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
2020-10-18 19:00:13 +00:00
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
sep = col->getValue<String>();
}
2017-05-27 15:45:25 +00:00
/// Returns the position of the argument that is the column of strings
size_t getStringsArgumentPosition()
{
return 1;
}
2017-05-27 15:45:25 +00:00
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
2017-05-27 15:45:25 +00:00
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (sep.empty())
{
if (pos == end)
return false;
token_begin = pos;
pos += 1;
token_end = pos;
}
else
{
if (!pos)
return false;
token_begin = pos;
pos = reinterpret_cast<Pos>(memmem(pos, end - pos, sep.data(), sep.size()));
if (pos)
{
token_end = pos;
pos += sep.size();
}
else
token_end = end;
}
return true;
}
};
2021-05-13 02:37:09 +00:00
class SplitByRegexpImpl
{
private:
Regexps::Pool::Pointer re;
OptimizedRegularExpression::MatchVec matches;
Pos pos;
Pos end;
public:
static constexpr auto name = "splitByRegexp";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
2021-05-13 02:37:09 +00:00
static size_t getNumberOfArguments() { return 2; }
/// Check the type of function arguments.
static void checkArguments(const DataTypes & arguments)
{
SplitByStringImpl::checkArguments(arguments);
}
/// Initialize by the function arguments.
void init(const ColumnsWithTypeAndName & arguments)
{
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get());
if (!col)
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
if (!col->getValue<String>().empty())
re = Regexps::get<false, false>(col->getValue<String>());
2021-05-13 02:37:09 +00:00
}
/// Returns the position of the argument that is the column of strings
size_t getStringsArgumentPosition()
{
return 1;
}
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!re)
2021-05-13 02:37:09 +00:00
{
if (pos == end)
return false;
token_begin = pos;
pos += 1;
token_end = pos;
2021-05-13 02:37:09 +00:00
}
else
{
if (!pos || pos > end)
return false;
token_begin = pos;
if (!re->match(pos, end - pos, matches) || !matches[0].length)
{
token_end = end;
pos = end + 1;
}
else
{
token_end = pos + matches[0].offset;
pos = token_end + matches[0].length;
}
2021-05-13 02:37:09 +00:00
}
return true;
}
};
class ExtractAllImpl
{
private:
Regexps::Pool::Pointer re;
OptimizedRegularExpression::MatchVec matches;
size_t capture;
Pos pos;
Pos end;
public:
static constexpr auto name = "extractAll";
static String getName() { return name; }
2022-01-30 01:10:27 +00:00
static bool isVariadic() { return false; }
static size_t getNumberOfArguments() { return 2; }
2017-05-27 15:45:25 +00:00
/// Check the type of function arguments.
2018-11-24 01:48:06 +00:00
static void checkArguments(const DataTypes & arguments)
{
SplitByStringImpl::checkArguments(arguments);
}
2017-05-27 15:45:25 +00:00
/// Initialize by the function arguments.
void init(const ColumnsWithTypeAndName & arguments)
{
2020-10-18 19:00:13 +00:00
const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col)
2020-10-18 19:00:13 +00:00
throw Exception("Illegal column " + arguments[1].column->getName()
+ " of first argument of function " + getName() + ". Must be constant string.",
ErrorCodes::ILLEGAL_COLUMN);
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
re = Regexps::get<false, false>(col->getValue<String>());
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
matches.resize(capture + 1);
}
2017-05-27 15:45:25 +00:00
/// Returns the position of the argument that is the column of strings
size_t getStringsArgumentPosition()
{
return 0;
}
2017-05-27 15:45:25 +00:00
/// Called for each next string.
void set(Pos pos_, Pos end_)
{
pos = pos_;
end = end_;
}
2017-05-27 15:45:25 +00:00
/// Get the next token, if any, or return false.
bool get(Pos & token_begin, Pos & token_end)
{
if (!pos || pos > end)
return false;
if (!re->match(pos, end - pos, matches) || !matches[0].length)
return false;
2018-12-27 01:51:22 +00:00
if (matches[capture].offset == std::string::npos)
{
/// Empty match.
token_begin = pos;
token_end = pos;
}
else
{
token_begin = pos + matches[capture].offset;
token_end = token_begin + matches[capture].length;
}
pos += matches[0].offset + matches[0].length;
return true;
}
};
2017-05-27 15:45:25 +00:00
/// A function that takes a string, and returns an array of substrings created by some generator.
template <typename Generator>
class FunctionTokens : public IFunction
{
public:
static constexpr auto name = Generator::name;
2021-06-01 12:20:52 +00:00
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionTokens>(); }
String getName() const override
{
return name;
}
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
2022-01-30 01:10:27 +00:00
bool isVariadic() const override { return Generator::isVariadic(); }
size_t getNumberOfArguments() const override { return Generator::getNumberOfArguments(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
Generator::checkArguments(arguments);
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
Generator generator;
2020-10-18 19:00:13 +00:00
generator.init(arguments);
const auto & array_argument = arguments[generator.getStringsArgumentPosition()];
2020-10-18 19:00:13 +00:00
const ColumnString * col_str = checkAndGetColumn<ColumnString>(array_argument.column.get());
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
const ColumnConst * col_const_str =
2020-10-18 19:00:13 +00:00
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();
if (col_str)
{
const ColumnString::Chars & src_chars = col_str->getChars();
const ColumnString::Offsets & src_offsets = col_str->getOffsets();
res_offsets.reserve(src_offsets.size());
2017-05-27 15:45:25 +00:00
res_strings_offsets.reserve(src_offsets.size() * 5); /// Constant 5 - at random.
res_strings_chars.reserve(src_chars.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
size_t size = src_offsets.size();
ColumnString::Offset current_src_offset = 0;
ColumnArray::Offset current_dst_offset = 0;
ColumnString::Offset current_dst_strings_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;
generator.set(pos, end);
size_t j = 0;
while (generator.get(token_begin, token_end))
{
size_t token_size = token_end - token_begin;
res_strings_chars.resize(res_strings_chars.size() + token_size + 1);
memcpySmallAllowReadWriteOverflow15(&res_strings_chars[current_dst_strings_offset], token_begin, token_size);
res_strings_chars[current_dst_strings_offset + token_size] = 0;
current_dst_strings_offset += token_size + 1;
res_strings_offsets.push_back(current_dst_strings_offset);
++j;
}
current_dst_offset += j;
res_offsets.push_back(current_dst_offset);
}
2020-10-18 19:00:13 +00:00
return col_res;
}
else if (col_const_str)
{
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
String src = col_const_str->getValue<String>();
Array dst;
generator.set(src.data(), src.data() + src.size());
Pos token_begin = nullptr;
Pos token_end = nullptr;
while (generator.get(token_begin, token_end))
dst.push_back(String(token_begin, token_end - token_begin));
2020-10-18 19:00:13 +00:00
return result_type->createColumnConst(col_const_str->size(), dst);
}
else
2020-10-18 19:00:13 +00:00
throw Exception("Illegal columns " + array_argument.column->getName()
+ ", " + array_argument.column->getName()
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
2012-09-17 04:34:19 +00:00
};
2021-10-22 20:19:32 +00:00
/// Joins an array of type serializable to string into one string via a separator.
class FunctionArrayStringConcat : public IFunction
{
private:
2021-10-19 17:54:27 +00:00
static void executeInternal(
const ColumnString::Chars & src_chars,
const ColumnString::Offsets & src_string_offsets,
const ColumnArray::Offsets & src_array_offsets,
2021-10-19 17:54:27 +00:00
const char * delimiter,
const size_t delimiter_size,
ColumnString::Chars & dst_chars,
2021-10-19 17:54:27 +00:00
ColumnString::Offsets & dst_string_offsets,
const char8_t * null_map)
{
size_t size = src_array_offsets.size();
if (!size)
return;
2017-05-27 15:45:25 +00:00
/// With a small margin - as if the separator goes after the last string of the array.
dst_chars.resize(
src_chars.size()
2017-05-27 15:45:25 +00:00
+ delimiter_size * src_string_offsets.size() /// Separators after each string...
+ src_array_offsets.size() /// Zero byte after each joined string
- src_string_offsets.size()); /// The former zero byte after each string of the array
2017-05-27 15:45:25 +00:00
/// There will be as many strings as there were arrays.
dst_string_offsets.resize(src_array_offsets.size());
ColumnArray::Offset current_src_array_offset = 0;
ColumnString::Offset current_dst_string_offset = 0;
2017-05-27 15:45:25 +00:00
/// Loop through the array of strings.
for (size_t i = 0; i < size; ++i)
{
2021-10-19 17:54:27 +00:00
bool first_non_null = true;
2017-05-27 15:45:25 +00:00
/// 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)
{
2021-10-19 17:54:27 +00:00
if (unlikely(null_map && null_map[current_src_array_offset]))
continue;
if (!first_non_null)
{
memcpy(&dst_chars[current_dst_string_offset], delimiter, delimiter_size);
current_dst_string_offset += delimiter_size;
}
first_non_null = false;
const auto current_src_string_offset = current_src_array_offset ? src_string_offsets[current_src_array_offset - 1] : 0;
size_t bytes_to_copy = src_string_offsets[current_src_array_offset] - current_src_string_offset - 1;
memcpySmallAllowReadWriteOverflow15(
&dst_chars[current_dst_string_offset], &src_chars[current_src_string_offset], bytes_to_copy);
current_dst_string_offset += bytes_to_copy;
}
dst_chars[current_dst_string_offset] = 0;
++current_dst_string_offset;
dst_string_offsets[i] = current_dst_string_offset;
}
dst_chars.resize(dst_string_offsets.back());
}
2021-10-19 17:54:27 +00:00
static void executeInternal(
const ColumnString & col_string,
const ColumnArray & col_arr,
const String & delimiter,
ColumnString & col_res,
const char8_t * null_map = nullptr)
{
executeInternal(
col_string.getChars(),
col_string.getOffsets(),
col_arr.getOffsets(),
delimiter.data(),
delimiter.size(),
col_res.getChars(),
col_res.getOffsets(),
null_map);
}
2021-10-22 20:19:32 +00:00
static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type)
{
if (isString(nested_type))
{
return col_arr.getDataPtr();
}
else if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData());
col_nullable && isString(col_nullable->getNestedColumn().getDataType()))
{
return col_nullable->getNestedColumnPtr();
}
else
{
ColumnsWithTypeAndName cols;
cols.emplace_back(col_arr.getDataPtr(), nested_type, "tmp");
2021-12-15 17:40:36 +00:00
return ConvertImplGenericToString<ColumnString>::execute(cols, std::make_shared<DataTypeString>(), col_arr.size());
2021-10-22 20:19:32 +00:00
}
}
public:
static constexpr auto name = "arrayStringConcat";
2021-06-01 12:20:52 +00:00
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayStringConcat>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
2021-10-12 21:23:40 +00:00
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
String delimiter;
if (arguments.size() == 2)
{
2020-10-18 19:00:13 +00:00
const ColumnConst * col_delim = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
if (!col_delim)
throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_COLUMN);
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
delimiter = col_delim->getValue<String>();
}
2021-10-22 20:19:32 +00:00
const auto & nested_type = assert_cast<const DataTypeArray &>(*arguments[0].type).getNestedType();
if (const ColumnConst * col_const_arr = checkAndGetColumnConst<ColumnArray>(arguments[0].column.get());
col_const_arr && isString(nested_type))
{
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
Array src_arr = col_const_arr->getValue<Array>();
String dst_str;
2021-10-12 21:23:40 +00:00
bool first_non_null = true;
for (size_t i = 0, size = src_arr.size(); i < size; ++i)
{
2021-10-12 21:23:40 +00:00
if (src_arr[i].isNull())
continue;
if (!first_non_null)
dst_str += delimiter;
2021-10-12 21:23:40 +00:00
first_non_null = false;
dst_str += src_arr[i].get<const String &>();
}
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
2020-10-18 19:00:13 +00:00
return result_type->createColumnConst(col_const_arr->size(), dst_str);
}
2021-10-22 20:19:32 +00:00
ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst();
const ColumnArray & col_arr = assert_cast<const ColumnArray &>(*src_column.get());
ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type);
const ColumnString & col_string = assert_cast<const ColumnString &>(*str_subcolumn.get());
auto col_res = ColumnString::create();
if (const ColumnNullable * col_nullable = checkAndGetColumn<ColumnNullable>(col_arr.getData()))
executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data());
else
2021-10-22 20:19:32 +00:00
executeInternal(col_string, col_arr, delimiter, *col_res);
return col_res;
}
};
2017-06-13 02:06:53 +00:00
using FunctionAlphaTokens = FunctionTokens<AlphaTokensImpl>;
using FunctionSplitByNonAlpha = FunctionTokens<SplitByNonAlphaImpl>;
using FunctionSplitByWhitespace = FunctionTokens<SplitByWhitespaceImpl>;
2017-06-13 02:06:53 +00:00
using FunctionSplitByChar = FunctionTokens<SplitByCharImpl>;
using FunctionSplitByString = FunctionTokens<SplitByStringImpl>;
2021-05-13 02:37:09 +00:00
using FunctionSplitByRegexp = FunctionTokens<SplitByRegexpImpl>;
2017-06-13 02:06:53 +00:00
using FunctionExtractAll = FunctionTokens<ExtractAllImpl>;
2012-09-17 04:34:19 +00:00
}