ClickHouse/src/Functions/repeat.cpp

261 lines
9.1 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;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
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)
{
static constexpr UInt64 max_repeat_times = 1000000;
if (repeat_time > max_repeat_times)
throw Exception("Too many times to repeat (" + std::to_string(repeat_time) + "), maximum is: " + std::to_string(max_repeat_times),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
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)
throw Exception("Too large string size (" + std::to_string(size) + ") in function repeat, maximum is: " + std::to_string(max_string_size),
ErrorCodes::TOO_LARGE_STRING_SIZE);
}
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,
2019-09-22 23:30:13 +00:00
UInt64 repeat_time)
2019-09-19 16:50:21 +00:00
{
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
{
2020-08-07 19:45:29 +00:00
size_t repeated_size = (offsets[i] - offsets[i - 1] - 1) * col_num[i] + 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);
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)
{
2019-09-22 23:49:15 +00:00
T repeat_time = col_num[i];
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(
const StringRef & copy_str,
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());
2019-09-22 23:30: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
{
2020-08-07 19:45:29 +00:00
size_t repeated_size = str_size * col_num[i] + 1;
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)
{
2019-09-22 23:49:15 +00:00
T repeat_time = col_num[i];
checkRepeatTime(repeat_time);
2019-09-22 23:30:13 +00:00
process(
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)
{
return castTypeToEither<DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64>(type, std::forward<F>(f));
}
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
bool isSuitableForShortCircuitArgumentsExecution(ColumnsWithTypeAndName & /*arguments*/) const override { return true; }
2019-09-19 16:50:21 +00:00
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isString(arguments[0]))
throw Exception(
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (!isUnsignedInteger(arguments[1]))
throw Exception(
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0];
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
2019-09-19 16:50:21 +00:00
{
2020-10-19 15:27:41 +00:00
const auto & strcolumn = arguments[0].column;
const auto & numcolumn = arguments[1].column;
ColumnPtr res;
2019-09-19 16:50:21 +00:00
if (const ColumnString * col = checkAndGetColumn<ColumnString>(strcolumn.get()))
2019-09-19 16:46:41 +00:00
{
2019-09-19 16:50:21 +00:00
if (const ColumnConst * scale_column_num = checkAndGetColumn<ColumnConst>(numcolumn.get()))
2019-09-19 16:46:41 +00:00
{
2019-09-22 23:30:13 +00:00
UInt64 repeat_time = scale_column_num->getValue<UInt64>();
2019-09-19 16:50:21 +00:00
auto col_res = ColumnString::create();
2019-09-22 23:30:13 +00:00
RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time);
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> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get());
auto col_res = ColumnString::create();
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->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
}
2019-09-22 23:30:13 +00:00
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(strcolumn.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.
StringRef copy_str = col_const->getDataColumn().getDataAt(0);
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> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get());
2019-09-22 23:02:03 +00:00
auto col_res = ColumnString::create();
2019-09-22 23:30:13 +00:00
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->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(
2020-10-19 15:27:41 +00:00
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
2019-09-22 23:30:13 +00:00
ErrorCodes::ILLEGAL_COLUMN);
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
2019-09-19 16:50:21 +00:00
void registerFunctionRepeat(FunctionFactory & factory)
{
2021-01-25 13:20:01 +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
}