ClickHouse/src/Functions/repeat.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

269 lines
9.4 KiB
C++
Raw Normal View History

2019-09-19 16:46:41 +00:00
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
2021-05-17 07:30:42 +00:00
#include <Functions/IFunction.h>
2019-09-19 16:46:41 +00:00
#include <Functions/castTypeToEither.h>
2019-09-22 23:49:15 +00:00
2019-09-19 16:46:41 +00:00
namespace DB
{
2019-09-19 16:50:21 +00:00
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
2019-09-22 23:49:15 +00:00
extern const int TOO_LARGE_STRING_SIZE;
2019-09-19 16:50:21 +00:00
}
2020-09-07 18:00:37 +00:00
namespace
{
2019-09-19 16:50:21 +00:00
struct RepeatImpl
{
2019-09-22 23:49:15 +00:00
/// Safety threshold against DoS.
static inline void checkRepeatTime(UInt64 repeat_time)
{
2023-05-23 12:16:49 +00:00
static constexpr UInt64 max_repeat_times = 1'000'000;
2019-09-22 23:49:15 +00:00
if (repeat_time > max_repeat_times)
2023-05-23 12:16:49 +00:00
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times);
2019-09-22 23:49:15 +00:00
}
2020-08-07 19:45:29 +00:00
static inline void checkStringSize(UInt64 size)
{
static constexpr UInt64 max_string_size = 1 << 30;
if (size > max_string_size)
2023-05-23 12:16:49 +00:00
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", size, max_string_size);
2020-08-07 19:45:29 +00:00
}
2023-03-02 12:08:39 +00:00
template <typename T>
2019-09-22 23:30:13 +00:00
static void vectorStrConstRepeat(
2019-09-19 16:50:21 +00:00
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets,
2023-03-02 12:08:39 +00:00
T repeat_time)
2019-09-19 16:50:21 +00:00
{
2023-03-02 12:08:39 +00:00
repeat_time = repeat_time < 0 ? 0 : repeat_time;
2019-09-22 23:49:15 +00:00
checkRepeatTime(repeat_time);
2019-09-19 16:50:21 +00:00
UInt64 data_size = 0;
res_offsets.assign(offsets);
for (UInt64 i = 0; i < offsets.size(); ++i)
2019-09-19 16:46:41 +00:00
{
2020-08-07 19:45:29 +00:00
/// Note that accessing -1th element is valid for PaddedPODArray.
size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1;
checkStringSize(repeated_size);
data_size += repeated_size;
2019-09-19 16:50:21 +00:00
res_offsets[i] = data_size;
2019-09-19 16:46:41 +00:00
}
2019-09-19 16:50:21 +00:00
res_data.resize(data_size);
for (UInt64 i = 0; i < res_offsets.size(); ++i)
2019-09-19 16:46:41 +00:00
{
2019-09-22 23:30:13 +00:00
process(data.data() + offsets[i - 1], res_data.data() + res_offsets[i - 1], offsets[i] - offsets[i - 1], repeat_time);
2019-09-19 16:46:41 +00:00
}
2019-09-19 16:50:21 +00:00
}
2019-09-19 16:46:41 +00:00
2019-09-19 16:50:21 +00:00
template <typename T>
2019-09-22 23:30:13 +00:00
static void vectorStrVectorRepeat(
2019-09-19 16:50:21 +00:00
const ColumnString::Chars & data,
const ColumnString::Offsets & offsets,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets,
const PaddedPODArray<T> & col_num)
{
UInt64 data_size = 0;
res_offsets.assign(offsets);
for (UInt64 i = 0; i < col_num.size(); ++i)
2019-09-19 16:46:41 +00:00
{
2023-03-02 12:08:39 +00:00
T repeat_time = col_num[i] < 0 ? 0 : col_num[i];
size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * repeat_time + 1;
2020-08-07 19:45:29 +00:00
checkStringSize(repeated_size);
data_size += repeated_size;
2019-09-19 16:50:21 +00:00
res_offsets[i] = data_size;
2019-09-19 16:46:41 +00:00
}
2019-09-19 16:50:21 +00:00
res_data.resize(data_size);
2019-09-22 23:49:15 +00:00
2019-09-19 16:50:21 +00:00
for (UInt64 i = 0; i < col_num.size(); ++i)
{
2023-03-02 12:08:39 +00:00
T repeat_time = col_num[i] < 0 ? 0 : col_num[i];
2019-09-22 23:49:15 +00:00
checkRepeatTime(repeat_time);
process(data.data() + offsets[i - 1], res_data.data() + res_offsets[i - 1], offsets[i] - offsets[i - 1], repeat_time);
2019-09-19 16:50:21 +00:00
}
}
2019-09-19 16:46:41 +00:00
2019-09-19 16:50:21 +00:00
template <typename T>
2019-09-22 23:30:13 +00:00
static void constStrVectorRepeat(
2022-08-17 12:32:13 +00:00
std::string_view copy_str,
2019-09-22 23:30:13 +00:00
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets,
const PaddedPODArray<T> & col_num)
2019-09-19 16:50:21 +00:00
{
UInt64 data_size = 0;
res_offsets.resize(col_num.size());
2022-08-17 12:32:13 +00:00
UInt64 str_size = copy_str.size();
2019-09-19 16:50:21 +00:00
UInt64 col_size = col_num.size();
for (UInt64 i = 0; i < col_size; ++i)
2019-09-19 16:46:41 +00:00
{
2023-03-02 12:08:39 +00:00
T repeat_time = col_num[i] < 0 ? 0 : col_num[i];
size_t repeated_size = str_size * repeat_time + 1;
2020-08-07 19:45:29 +00:00
checkStringSize(repeated_size);
data_size += repeated_size;
2019-09-19 16:50:21 +00:00
res_offsets[i] = data_size;
}
res_data.resize(data_size);
for (UInt64 i = 0; i < col_size; ++i)
{
2023-03-02 12:08:39 +00:00
T repeat_time = col_num[i] < 0 ? 0 : col_num[i];
2019-09-22 23:49:15 +00:00
checkRepeatTime(repeat_time);
2019-09-22 23:30:13 +00:00
process(
2022-08-17 12:32:13 +00:00
reinterpret_cast<UInt8 *>(const_cast<char *>(copy_str.data())),
2019-09-19 16:50:21 +00:00
res_data.data() + res_offsets[i - 1],
str_size + 1,
2019-09-22 23:49:15 +00:00
repeat_time);
2019-09-19 16:46:41 +00:00
}
2019-09-19 16:50:21 +00:00
}
2019-09-19 16:46:41 +00:00
2019-09-19 16:50:21 +00:00
private:
2020-11-12 16:47:59 +00:00
// A very fast repeat implementation, only invoke memcpy for O(log(n)) times.
// as the calling times decreases, more data will be copied for each memcpy, thus
// SIMD optimization will be more efficient.
2019-09-22 23:30:13 +00:00
static void process(const UInt8 * src, UInt8 * dst, UInt64 size, UInt64 repeat_time)
2019-09-19 16:50:21 +00:00
{
2020-11-13 07:21:08 +00:00
if (unlikely(repeat_time <= 0))
2019-09-19 16:46:41 +00:00
{
2020-11-12 16:47:59 +00:00
*dst = 0;
return;
}
size -= 1;
UInt64 k = 0;
UInt64 last_bit = repeat_time & 1;
repeat_time >>= 1;
const UInt8 * dst_hdr = dst;
memcpy(dst, src, size);
dst += size;
while (repeat_time > 0)
{
UInt64 cpy_size = size * (1ULL << k);
2020-11-12 16:47:59 +00:00
memcpy(dst, dst_hdr, cpy_size);
dst += cpy_size;
if (last_bit)
{
memcpy(dst, dst_hdr, cpy_size);
dst += cpy_size;
}
k += 1;
last_bit = repeat_time & 1;
repeat_time >>= 1;
2019-09-19 16:46:41 +00:00
}
2019-09-19 16:50:21 +00:00
*dst = 0;
}
};
2019-09-22 23:30:13 +00:00
class FunctionRepeat : public IFunction
2019-09-19 16:50:21 +00:00
{
template <typename F>
static bool castType(const IDataType * type, F && f)
{
2023-03-02 12:08:39 +00:00
return castTypeToEither<DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64,
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64>(type, std::forward<F>(f));
2019-09-19 16:50:21 +00:00
}
public:
static constexpr auto name = "repeat";
2021-06-01 12:20:52 +00:00
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionRepeat>(); }
2019-09-19 16:50:21 +00:00
String getName() const override { return name; }
2019-09-19 16:46:41 +00:00
2019-09-19 16:50:21 +00:00
size_t getNumberOfArguments() const override { return 2; }
2019-09-19 16:46:41 +00:00
2021-06-22 16:21:23 +00:00
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
2019-09-19 16:50:21 +00:00
{
FunctionArgumentDescriptors args{
{"s", &isString<IDataType>, nullptr, "String"},
{"n", &isInteger<IDataType>, nullptr, "Integer"},
};
validateFunctionArgumentTypes(*this, arguments, args);
return std::make_shared<DataTypeString>();
2019-09-19 16:50:21 +00:00
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
2019-09-19 16:50:21 +00:00
{
const auto & col_str = arguments[0].column;
const auto & col_num = arguments[1].column;
2020-10-19 15:27:41 +00:00
ColumnPtr res;
2019-09-19 16:50:21 +00:00
if (const ColumnString * col = checkAndGetColumn<ColumnString>(col_str.get()))
2019-09-19 16:46:41 +00:00
{
if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get()))
2019-09-19 16:46:41 +00:00
{
2019-09-19 16:50:21 +00:00
auto col_res = ColumnString::create();
2023-03-02 12:08:39 +00:00
castType(arguments[1].type.get(), [&](const auto & type)
{
using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType;
T times = col_num_const->getValue<T>();
RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), times);
2023-03-02 12:08:39 +00:00
return true;
});
2020-10-19 15:27:41 +00:00
return col_res;
2019-09-19 16:46:41 +00:00
}
2020-10-19 15:27:41 +00:00
else if (castType(arguments[1].type.get(), [&](const auto & type)
2019-09-22 23:30:13 +00:00
{
using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType;
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
2019-09-22 23:30:13 +00:00
auto col_res = ColumnString::create();
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData());
2020-10-19 15:27:41 +00:00
res = std::move(col_res);
2019-09-22 23:30:13 +00:00
return true;
}))
2019-09-19 16:50:21 +00:00
{
2020-10-19 15:27:41 +00:00
return res;
2019-09-22 23:30:13 +00:00
}
2019-09-19 16:50:21 +00:00
}
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(col_str.get()))
2019-09-19 16:50:21 +00:00
{
2019-09-22 23:30:13 +00:00
/// Note that const-const case is handled by useDefaultImplementationForConstants.
2022-08-17 12:32:13 +00:00
std::string_view copy_str = col_const->getDataColumn().getDataAt(0).toView();
2019-09-22 23:30:13 +00:00
2020-10-19 15:27:41 +00:00
if (castType(arguments[1].type.get(), [&](const auto & type)
2019-09-22 23:02:03 +00:00
{
using DataType = std::decay_t<decltype(type)>;
2019-09-22 23:30:13 +00:00
using T = typename DataType::FieldType;
const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
2019-09-22 23:02:03 +00:00
auto col_res = ColumnString::create();
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData());
2020-10-19 15:27:41 +00:00
res = std::move(col_res);
2019-09-22 23:30:13 +00:00
return true;
2019-09-22 23:02:03 +00:00
}))
2019-09-19 16:50:21 +00:00
{
2020-10-19 15:27:41 +00:00
return res;
2019-09-22 23:02:03 +00:00
}
2019-09-19 16:50:21 +00:00
}
2019-09-22 23:30:13 +00:00
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}",
arguments[0].column->getName(), getName());
2019-09-19 16:50:21 +00:00
}
};
2019-09-19 16:46:41 +00:00
2020-09-07 18:00:37 +00:00
}
2019-09-19 16:46:41 +00:00
REGISTER_FUNCTION(Repeat)
2019-09-19 16:50:21 +00:00
{
2022-08-27 20:06:03 +00:00
factory.registerFunction<FunctionRepeat>({}, FunctionFactory::CaseInsensitive);
2019-09-19 16:50:21 +00:00
}
2019-09-22 23:30:13 +00:00
2019-09-19 16:46:41 +00:00
}