2020-03-06 02:12:18 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Storages/ColumnsDescription.h>
|
|
|
|
#include <Storages/StorageGenerateRandom.h>
|
|
|
|
#include <Storages/StorageFactory.h>
|
2022-06-23 20:04:06 +00:00
|
|
|
#include <Storages/checkAndGetLiteralArgument.h>
|
2020-03-06 02:12:18 +00:00
|
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2020-03-06 02:12:18 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
|
2023-01-30 14:23:22 +00:00
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
|
|
|
#include <Columns/ColumnMap.h>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
#include <Columns/ColumnVector.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2020-03-06 02:12:18 +00:00
|
|
|
#include <DataTypes/DataTypeDateTime64.h>
|
|
|
|
#include <DataTypes/DataTypeDecimalBase.h>
|
2023-01-30 14:23:22 +00:00
|
|
|
#include <DataTypes/DataTypeEnum.h>
|
2020-03-07 22:13:01 +00:00
|
|
|
#include <DataTypes/DataTypeFixedString.h>
|
2023-01-26 14:35:20 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
2023-01-30 14:23:22 +00:00
|
|
|
#include <DataTypes/DataTypeMap.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2023-04-05 16:23:06 +00:00
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2020-04-12 23:08:32 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <Common/randomSeed.h>
|
2023-06-04 02:44:51 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/unaligned.h>
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
|
|
|
|
#include <pcg_random.hpp>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NOT_IMPLEMENTED;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2020-06-28 14:33:02 +00:00
|
|
|
extern const int TOO_LARGE_ARRAY_SIZE;
|
|
|
|
extern const int TOO_LARGE_STRING_SIZE;
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
2023-03-14 13:44:05 +00:00
|
|
|
void fillBufferWithRandomData(char * __restrict data, size_t limit, size_t size_of_type, pcg64 & rng, [[maybe_unused]] bool flip_bytes = false)
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
2023-03-14 13:44:05 +00:00
|
|
|
size_t size = limit * size_of_type;
|
2020-03-06 02:12:18 +00:00
|
|
|
char * __restrict end = data + size;
|
|
|
|
while (data < end)
|
|
|
|
{
|
|
|
|
/// The loop can be further optimized.
|
|
|
|
UInt64 number = rng();
|
2023-03-14 13:44:05 +00:00
|
|
|
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
|
2023-04-21 10:38:45 +00:00
|
|
|
unalignedStoreLittleEndian<UInt64>(data, number);
|
2023-03-14 13:44:05 +00:00
|
|
|
#else
|
2020-03-06 02:12:18 +00:00
|
|
|
unalignedStore<UInt64>(data, number);
|
2023-03-14 13:44:05 +00:00
|
|
|
#endif
|
2020-03-07 19:32:04 +00:00
|
|
|
data += sizeof(UInt64); /// We assume that data has at least 7-byte padding (see PaddedPODArray)
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
2023-03-14 13:44:05 +00:00
|
|
|
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
|
|
|
|
if (flip_bytes)
|
|
|
|
{
|
|
|
|
data = end - size;
|
|
|
|
while (data < end)
|
|
|
|
{
|
|
|
|
char * rev_end = data + size_of_type;
|
|
|
|
std::reverse(data, rev_end);
|
|
|
|
data += size_of_type;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
#endif
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2023-06-04 02:44:51 +00:00
|
|
|
size_t estimateValueSize(
|
|
|
|
const DataTypePtr type,
|
|
|
|
UInt64 max_array_length,
|
|
|
|
UInt64 max_string_length)
|
|
|
|
{
|
|
|
|
if (type->haveMaximumSizeOfValue())
|
|
|
|
return type->getMaximumSizeOfValueInMemory();
|
|
|
|
|
|
|
|
TypeIndex idx = type->getTypeId();
|
|
|
|
|
|
|
|
switch (idx)
|
|
|
|
{
|
|
|
|
case TypeIndex::String:
|
|
|
|
{
|
|
|
|
return max_string_length + sizeof(size_t) + 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// The logic in this function should reflect the logic of fillColumnWithRandomData.
|
|
|
|
case TypeIndex::Array:
|
|
|
|
{
|
|
|
|
auto nested_type = typeid_cast<const DataTypeArray &>(*type).getNestedType();
|
|
|
|
return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length);
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Map:
|
|
|
|
{
|
|
|
|
const DataTypePtr & nested_type = typeid_cast<const DataTypeMap &>(*type).getNestedType();
|
|
|
|
return sizeof(size_t) + estimateValueSize(nested_type, max_array_length / 2, max_string_length);
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Tuple:
|
|
|
|
{
|
|
|
|
auto elements = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
|
|
|
|
const size_t tuple_size = elements.size();
|
|
|
|
size_t res = 0;
|
|
|
|
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
res += estimateValueSize(elements[i], max_array_length, max_string_length);
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Nullable:
|
|
|
|
{
|
|
|
|
auto nested_type = typeid_cast<const DataTypeNullable &>(*type).getNestedType();
|
|
|
|
return 1 + estimateValueSize(nested_type, max_array_length, max_string_length);
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::LowCardinality:
|
|
|
|
{
|
|
|
|
auto nested_type = typeid_cast<const DataTypeLowCardinality &>(*type).getDictionaryType();
|
|
|
|
return sizeof(size_t) + estimateValueSize(nested_type, max_array_length, max_string_length);
|
|
|
|
}
|
|
|
|
|
|
|
|
default:
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The 'GenerateRandom' is not implemented for type {}", type->getName());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-03-06 02:12:18 +00:00
|
|
|
ColumnPtr fillColumnWithRandomData(
|
2020-04-12 23:08:32 +00:00
|
|
|
const DataTypePtr type,
|
|
|
|
UInt64 limit,
|
|
|
|
UInt64 max_array_length,
|
|
|
|
UInt64 max_string_length,
|
|
|
|
pcg64 & rng,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context)
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
|
|
|
TypeIndex idx = type->getTypeId();
|
|
|
|
|
|
|
|
switch (idx)
|
|
|
|
{
|
|
|
|
case TypeIndex::String:
|
|
|
|
{
|
2020-03-07 19:32:04 +00:00
|
|
|
/// Mostly the same as the implementation of randomPrintableASCII function.
|
2020-03-06 02:34:35 +00:00
|
|
|
|
2020-03-07 19:32:04 +00:00
|
|
|
auto column = ColumnString::create();
|
|
|
|
ColumnString::Chars & data_to = column->getChars();
|
|
|
|
ColumnString::Offsets & offsets_to = column->getOffsets();
|
|
|
|
offsets_to.resize(limit);
|
2020-03-06 02:34:35 +00:00
|
|
|
|
2020-03-07 19:32:04 +00:00
|
|
|
IColumn::Offset offset = 0;
|
|
|
|
for (size_t row_num = 0; row_num < limit; ++row_num)
|
2020-03-06 02:34:35 +00:00
|
|
|
{
|
2020-03-07 19:32:04 +00:00
|
|
|
size_t length = rng() % (max_string_length + 1); /// Slow
|
|
|
|
|
|
|
|
IColumn::Offset next_offset = offset + length + 1;
|
|
|
|
data_to.resize(next_offset);
|
|
|
|
offsets_to[row_num] = next_offset;
|
|
|
|
|
|
|
|
auto * data_to_ptr = data_to.data(); /// avoid assert on array indexing after end
|
|
|
|
for (size_t pos = offset, end = offset + length; pos < end; pos += 4) /// We have padding in column buffers that we can overwrite.
|
|
|
|
{
|
|
|
|
UInt64 rand = rng();
|
|
|
|
|
|
|
|
UInt16 rand1 = rand;
|
|
|
|
UInt16 rand2 = rand >> 16;
|
|
|
|
UInt16 rand3 = rand >> 32;
|
|
|
|
UInt16 rand4 = rand >> 48;
|
|
|
|
|
|
|
|
/// Printable characters are from range [32; 126].
|
|
|
|
/// https://lemire.me/blog/2016/06/27/a-fast-alternative-to-the-modulo-reduction/
|
2020-03-06 02:34:35 +00:00
|
|
|
|
2020-03-07 19:32:04 +00:00
|
|
|
data_to_ptr[pos + 0] = 32 + ((rand1 * 95) >> 16);
|
|
|
|
data_to_ptr[pos + 1] = 32 + ((rand2 * 95) >> 16);
|
|
|
|
data_to_ptr[pos + 2] = 32 + ((rand3 * 95) >> 16);
|
|
|
|
data_to_ptr[pos + 3] = 32 + ((rand4 * 95) >> 16);
|
|
|
|
|
|
|
|
/// NOTE gcc failed to vectorize this code (aliasing of char?)
|
|
|
|
/// TODO Implement SIMD optimizations from Danila Kutenin.
|
|
|
|
}
|
|
|
|
|
|
|
|
data_to[offset + length] = 0;
|
|
|
|
|
|
|
|
offset = next_offset;
|
|
|
|
}
|
|
|
|
|
|
|
|
return column;
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Enum8:
|
|
|
|
{
|
|
|
|
auto column = ColumnVector<Int8>::create();
|
|
|
|
auto values = typeid_cast<const DataTypeEnum<Int8> *>(type.get())->getValues();
|
|
|
|
auto & data = column->getData();
|
|
|
|
data.resize(limit);
|
|
|
|
|
|
|
|
UInt8 size = values.size();
|
|
|
|
UInt8 off;
|
|
|
|
for (UInt64 i = 0; i < limit; ++i)
|
|
|
|
{
|
|
|
|
off = static_cast<UInt8>(rng()) % size;
|
|
|
|
data[i] = values[off].second;
|
|
|
|
}
|
|
|
|
|
|
|
|
return column;
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Enum16:
|
|
|
|
{
|
|
|
|
auto column = ColumnVector<Int16>::create();
|
|
|
|
auto values = typeid_cast<const DataTypeEnum<Int16> *>(type.get())->getValues();
|
|
|
|
auto & data = column->getData();
|
|
|
|
data.resize(limit);
|
|
|
|
|
|
|
|
UInt16 size = values.size();
|
|
|
|
UInt8 off;
|
|
|
|
for (UInt64 i = 0; i < limit; ++i)
|
|
|
|
{
|
|
|
|
off = static_cast<UInt16>(rng()) % size;
|
|
|
|
data[i] = values[off].second;
|
|
|
|
}
|
|
|
|
|
|
|
|
return column;
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Array:
|
|
|
|
{
|
2023-01-26 14:35:20 +00:00
|
|
|
auto nested_type = typeid_cast<const DataTypeArray &>(*type).getNestedType();
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
auto offsets_column = ColumnVector<ColumnArray::Offset>::create();
|
|
|
|
auto & offsets = offsets_column->getData();
|
|
|
|
|
|
|
|
UInt64 offset = 0;
|
|
|
|
offsets.resize(limit);
|
|
|
|
for (UInt64 i = 0; i < limit; ++i)
|
|
|
|
{
|
2020-03-07 19:32:04 +00:00
|
|
|
offset += static_cast<UInt64>(rng()) % (max_array_length + 1);
|
2020-03-06 02:12:18 +00:00
|
|
|
offsets[i] = offset;
|
|
|
|
}
|
|
|
|
|
2023-06-04 02:44:51 +00:00
|
|
|
/// This division by two makes the size growth subexponential on depth.
|
|
|
|
auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length / 2, max_string_length, rng, context);
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2022-03-02 17:22:12 +00:00
|
|
|
return ColumnArray::create(data_column, std::move(offsets_column));
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
2023-01-30 14:23:22 +00:00
|
|
|
case TypeIndex::Map:
|
|
|
|
{
|
|
|
|
const DataTypePtr & nested_type = typeid_cast<const DataTypeMap &>(*type).getNestedType();
|
2023-06-04 02:44:51 +00:00
|
|
|
auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length / 2, max_string_length, rng, context);
|
2023-01-30 14:23:22 +00:00
|
|
|
return ColumnMap::create(nested_column);
|
|
|
|
}
|
|
|
|
|
2020-03-06 02:12:18 +00:00
|
|
|
case TypeIndex::Tuple:
|
|
|
|
{
|
|
|
|
auto elements = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
|
|
|
|
const size_t tuple_size = elements.size();
|
|
|
|
Columns tuple_columns(tuple_size);
|
|
|
|
|
|
|
|
for (size_t i = 0; i < tuple_size; ++i)
|
|
|
|
tuple_columns[i] = fillColumnWithRandomData(elements[i], limit, max_array_length, max_string_length, rng, context);
|
|
|
|
|
|
|
|
return ColumnTuple::create(std::move(tuple_columns));
|
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::Nullable:
|
|
|
|
{
|
2023-01-26 14:35:20 +00:00
|
|
|
auto nested_type = typeid_cast<const DataTypeNullable &>(*type).getNestedType();
|
2020-03-06 02:12:18 +00:00
|
|
|
auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length, max_string_length, rng, context);
|
|
|
|
|
|
|
|
auto null_map_column = ColumnUInt8::create();
|
|
|
|
auto & null_map = null_map_column->getData();
|
|
|
|
null_map.resize(limit);
|
|
|
|
for (UInt64 i = 0; i < limit; ++i)
|
|
|
|
null_map[i] = rng() % 16 == 0; /// No real motivation for this.
|
|
|
|
|
2022-03-02 17:22:12 +00:00
|
|
|
return ColumnNullable::create(nested_column, std::move(null_map_column));
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
case TypeIndex::UInt8:
|
|
|
|
{
|
|
|
|
auto column = ColumnUInt8::create();
|
2023-04-05 16:23:06 +00:00
|
|
|
auto & data = column->getData();
|
|
|
|
data.resize(limit);
|
|
|
|
if (isBool(type))
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < limit; ++i)
|
|
|
|
data[i] = rng() % 2;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(data.data()), limit, sizeof(UInt8), rng);
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::UInt16: [[fallthrough]];
|
|
|
|
case TypeIndex::Date:
|
|
|
|
{
|
|
|
|
auto column = ColumnUInt16::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UInt16), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2021-12-13 07:52:12 +00:00
|
|
|
case TypeIndex::Date32:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt32::create();
|
|
|
|
column->getData().resize(limit);
|
2022-07-25 17:06:11 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < limit; ++i)
|
|
|
|
column->getData()[i] = (rng() % static_cast<UInt64>(DATE_LUT_SIZE)) - DAYNUM_OFFSET_EPOCH;
|
|
|
|
|
2021-12-13 07:52:12 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
case TypeIndex::UInt32: [[fallthrough]];
|
|
|
|
case TypeIndex::DateTime:
|
|
|
|
{
|
|
|
|
auto column = ColumnUInt32::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UInt32), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::UInt64:
|
|
|
|
{
|
|
|
|
auto column = ColumnUInt64::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UInt64), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2021-05-04 17:41:46 +00:00
|
|
|
case TypeIndex::UInt128:
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
|
|
|
auto column = ColumnUInt128::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UInt128), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2021-05-04 17:41:46 +00:00
|
|
|
case TypeIndex::UInt256:
|
|
|
|
{
|
|
|
|
auto column = ColumnUInt256::create();
|
|
|
|
column->getData().resize(limit);
|
2023-09-13 11:55:02 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UInt256), rng, true);
|
2021-05-04 17:41:46 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::UUID:
|
|
|
|
{
|
|
|
|
auto column = ColumnUUID::create();
|
|
|
|
column->getData().resize(limit);
|
|
|
|
/// NOTE This is slightly incorrect as random UUIDs should have fixed version 4.
|
2023-08-16 20:12:31 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(UUID), rng, true);
|
2021-05-04 17:41:46 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
case TypeIndex::Int8:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt8::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int8), rng);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Int16:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt16::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int16), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Int32:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt32::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int32), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Int64:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt64::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int64), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2021-05-04 17:41:46 +00:00
|
|
|
case TypeIndex::Int128:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt128::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int128), rng, true);
|
2021-05-04 17:41:46 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Int256:
|
|
|
|
{
|
|
|
|
auto column = ColumnInt256::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Int256), rng, true);
|
2021-05-04 17:41:46 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
case TypeIndex::Float32:
|
|
|
|
{
|
|
|
|
auto column = ColumnFloat32::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Float32), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Float64:
|
|
|
|
{
|
|
|
|
auto column = ColumnFloat64::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(Float64), rng, true);
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Decimal32:
|
|
|
|
{
|
2023-04-05 16:23:06 +00:00
|
|
|
const auto & decimal_type = assert_cast<const DataTypeDecimal<Decimal32> &>(*type);
|
|
|
|
auto column = decimal_type.createColumn();
|
2020-03-06 02:12:18 +00:00
|
|
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal32> &>(*column);
|
2023-04-05 16:23:06 +00:00
|
|
|
auto & data = column_concrete.getData();
|
|
|
|
data.resize(limit);
|
|
|
|
/// Generate numbers from range [-10^P + 1, 10^P - 1]
|
|
|
|
Int32 range = common::exp10_i32(decimal_type.getPrecision());
|
|
|
|
for (size_t i = 0; i != limit; ++i)
|
|
|
|
data[i] = static_cast<Int32>(rng()) % range;
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2023-04-05 16:23:06 +00:00
|
|
|
case TypeIndex::Decimal64:
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
2023-04-05 16:23:06 +00:00
|
|
|
const auto & decimal_type = assert_cast<const DataTypeDecimal<Decimal64> &>(*type);
|
2020-03-06 02:12:18 +00:00
|
|
|
auto column = type->createColumn();
|
|
|
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal64> &>(*column);
|
2023-04-05 16:23:06 +00:00
|
|
|
auto & data = column_concrete.getData();
|
|
|
|
data.resize(limit);
|
|
|
|
/// Generate numbers from range [-10^P + 1, 10^P - 1]
|
|
|
|
Int64 range = common::exp10_i64(decimal_type.getPrecision());
|
|
|
|
for (size_t i = 0; i != limit; ++i)
|
|
|
|
data[i] = static_cast<Int64>(rng()) % range;
|
|
|
|
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::Decimal128:
|
|
|
|
{
|
2023-04-05 16:23:06 +00:00
|
|
|
const auto & decimal_type = assert_cast<const DataTypeDecimal<Decimal128> &>(*type);
|
2020-03-06 02:12:18 +00:00
|
|
|
auto column = type->createColumn();
|
|
|
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal128> &>(*column);
|
2023-04-05 16:23:06 +00:00
|
|
|
auto & data = column_concrete.getData();
|
|
|
|
data.resize(limit);
|
|
|
|
/// Generate numbers from range [-10^P + 1, 10^P - 1]
|
|
|
|
Int128 range = common::exp10_i128(decimal_type.getPrecision());
|
|
|
|
for (size_t i = 0; i != limit; ++i)
|
|
|
|
data[i] = Int128({rng(), rng()}) % range;
|
2020-03-06 02:12:18 +00:00
|
|
|
return column;
|
|
|
|
}
|
2021-05-04 17:41:46 +00:00
|
|
|
case TypeIndex::Decimal256:
|
|
|
|
{
|
2023-04-05 16:23:06 +00:00
|
|
|
const auto & decimal_type = assert_cast<const DataTypeDecimal<Decimal256> &>(*type);
|
2021-05-04 17:41:46 +00:00
|
|
|
auto column = type->createColumn();
|
|
|
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal256> &>(*column);
|
2023-04-05 16:23:06 +00:00
|
|
|
auto & data = column_concrete.getData();
|
|
|
|
data.resize(limit);
|
|
|
|
/// Generate numbers from range [-10^P + 1, 10^P - 1]
|
|
|
|
Int256 range = common::exp10_i256(decimal_type.getPrecision());
|
|
|
|
for (size_t i = 0; i != limit; ++i)
|
|
|
|
data[i] = Int256({rng(), rng(), rng(), rng()}) % range;
|
2021-05-04 17:41:46 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-07 22:13:01 +00:00
|
|
|
case TypeIndex::FixedString:
|
|
|
|
{
|
|
|
|
size_t n = typeid_cast<const DataTypeFixedString &>(*type).getN();
|
|
|
|
auto column = ColumnFixedString::create(n);
|
|
|
|
column->getChars().resize(limit * n);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getChars().data()), limit, n, rng);
|
2020-03-07 22:13:01 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-06 02:34:35 +00:00
|
|
|
case TypeIndex::DateTime64:
|
|
|
|
{
|
|
|
|
auto column = type->createColumn();
|
2020-11-14 06:28:52 +00:00
|
|
|
auto & column_concrete = typeid_cast<ColumnDecimal<DateTime64> &>(*column);
|
2020-03-06 02:34:35 +00:00
|
|
|
column_concrete.getData().resize(limit);
|
|
|
|
|
|
|
|
UInt64 range = (1ULL << 32) * intExp10(typeid_cast<const DataTypeDateTime64 &>(*type).getScale());
|
|
|
|
|
|
|
|
for (size_t i = 0; i < limit; ++i)
|
|
|
|
column_concrete.getData()[i] = rng() % range; /// Slow
|
|
|
|
|
|
|
|
return column;
|
|
|
|
}
|
2023-01-26 14:35:20 +00:00
|
|
|
case TypeIndex::LowCardinality:
|
|
|
|
{
|
|
|
|
/// We are generating the values using the same random distribution as for full columns
|
|
|
|
/// so it's not in fact "low cardinality",
|
|
|
|
/// but it's ok for testing purposes, because the LowCardinality data type supports high cardinality data as well.
|
|
|
|
|
|
|
|
auto nested_type = typeid_cast<const DataTypeLowCardinality &>(*type).getDictionaryType();
|
|
|
|
auto nested_column = fillColumnWithRandomData(nested_type, limit, max_array_length, max_string_length, rng, context);
|
|
|
|
|
|
|
|
auto column = type->createColumn();
|
|
|
|
typeid_cast<ColumnLowCardinality &>(*column).insertRangeFromFullColumn(*nested_column, 0, limit);
|
|
|
|
|
|
|
|
return column;
|
|
|
|
}
|
2023-01-30 13:18:25 +00:00
|
|
|
case TypeIndex::IPv4:
|
|
|
|
{
|
|
|
|
auto column = ColumnIPv4::create();
|
|
|
|
column->getData().resize(limit);
|
2023-09-13 11:55:02 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(IPv4), rng, true);
|
2023-01-30 13:18:25 +00:00
|
|
|
return column;
|
|
|
|
}
|
|
|
|
case TypeIndex::IPv6:
|
|
|
|
{
|
|
|
|
auto column = ColumnIPv6::create();
|
|
|
|
column->getData().resize(limit);
|
2023-03-14 13:44:05 +00:00
|
|
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit, sizeof(IPv6), rng);
|
2023-01-30 13:18:25 +00:00
|
|
|
return column;
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
default:
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "The 'GenerateRandom' is not implemented for type {}", type->getName());
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-20 19:49:31 +00:00
|
|
|
class GenerateSource : public ISource
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-04-10 23:33:54 +00:00
|
|
|
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_)
|
2023-10-31 13:56:01 +00:00
|
|
|
: ISource(Nested::flattenNested(prepareBlockToFill(block_header_)))
|
2020-05-07 14:14:56 +00:00
|
|
|
, block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
|
|
|
, block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {}
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
String getName() const override { return "GenerateRandom"; }
|
|
|
|
|
|
|
|
protected:
|
|
|
|
Chunk generate() override
|
|
|
|
{
|
|
|
|
Columns columns;
|
2020-05-07 14:14:56 +00:00
|
|
|
columns.reserve(block_to_fill.columns());
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2020-05-07 14:14:56 +00:00
|
|
|
for (const auto & elem : block_to_fill)
|
|
|
|
columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context));
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2023-10-31 13:56:01 +00:00
|
|
|
columns = Nested::flattenNested(block_to_fill.cloneWithColumns(columns)).getColumns();
|
2020-05-07 14:14:56 +00:00
|
|
|
return {std::move(columns), block_size};
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
UInt64 block_size;
|
|
|
|
UInt64 max_array_length;
|
|
|
|
UInt64 max_string_length;
|
2020-05-07 14:14:56 +00:00
|
|
|
Block block_to_fill;
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2020-03-07 22:07:09 +00:00
|
|
|
pcg64 rng;
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context;
|
2020-05-07 14:14:56 +00:00
|
|
|
|
|
|
|
static Block & prepareBlockToFill(Block & block)
|
|
|
|
{
|
|
|
|
/// To support Nested types, we will collect them to single Array of Tuple.
|
|
|
|
auto names_and_types = Nested::collect(block.getNamesAndTypesList());
|
|
|
|
block.clear();
|
|
|
|
|
|
|
|
for (auto & column : names_and_types)
|
|
|
|
block.insert(ColumnWithTypeAndName(column.type, column.name));
|
|
|
|
|
|
|
|
return block;
|
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-04-23 12:18:23 +00:00
|
|
|
StorageGenerateRandom::StorageGenerateRandom(
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const String & comment,
|
|
|
|
UInt64 max_array_length_,
|
|
|
|
UInt64 max_string_length_,
|
2023-03-14 13:44:05 +00:00
|
|
|
const std::optional<UInt64> & random_seed_)
|
2020-03-06 02:12:18 +00:00
|
|
|
: IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
|
|
|
{
|
2020-06-28 14:33:02 +00:00
|
|
|
static constexpr size_t MAX_ARRAY_SIZE = 1 << 30;
|
|
|
|
static constexpr size_t MAX_STRING_SIZE = 1 << 30;
|
|
|
|
|
|
|
|
if (max_array_length > MAX_ARRAY_SIZE)
|
|
|
|
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size in GenerateRandom: {}, maximum: {}",
|
|
|
|
max_array_length, MAX_ARRAY_SIZE);
|
|
|
|
if (max_string_length > MAX_STRING_SIZE)
|
|
|
|
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size in GenerateRandom: {}, maximum: {}",
|
|
|
|
max_string_length, MAX_STRING_SIZE);
|
|
|
|
|
2020-03-07 22:07:09 +00:00
|
|
|
random_seed = random_seed_ ? sipHash64(*random_seed_) : randomSeed();
|
2020-06-19 15:39:41 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
2021-04-23 12:18:23 +00:00
|
|
|
storage_metadata.setComment(comment);
|
2020-06-19 15:39:41 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void registerStorageGenerateRandom(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("GenerateRandom", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
|
|
|
if (engine_args.size() > 3)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Storage GenerateRandom requires at most three arguments: "
|
|
|
|
"random_seed, max_string_length, max_array_length.");
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2020-03-07 20:35:55 +00:00
|
|
|
std::optional<UInt64> random_seed;
|
|
|
|
UInt64 max_string_length = 10;
|
|
|
|
UInt64 max_array_length = 10;
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2020-03-09 01:03:43 +00:00
|
|
|
if (!engine_args.empty())
|
2020-03-07 21:09:45 +00:00
|
|
|
{
|
2022-06-23 20:04:06 +00:00
|
|
|
const auto & ast_literal = engine_args[0]->as<const ASTLiteral &>();
|
|
|
|
if (!ast_literal.value.isNull())
|
|
|
|
random_seed = checkAndGetLiteralArgument<UInt64>(ast_literal, "random_seed");
|
2020-03-07 21:09:45 +00:00
|
|
|
}
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
if (engine_args.size() >= 2)
|
2022-06-23 20:04:06 +00:00
|
|
|
max_string_length = checkAndGetLiteralArgument<UInt64>(engine_args[1], "max_string_length");
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
if (engine_args.size() == 3)
|
2022-06-23 20:04:06 +00:00
|
|
|
max_array_length = checkAndGetLiteralArgument<UInt64>(engine_args[2], "max_array_length");
|
2020-03-06 02:12:18 +00:00
|
|
|
|
2022-04-19 20:47:29 +00:00
|
|
|
return std::make_shared<StorageGenerateRandom>(args.table_id, args.columns, args.comment, max_array_length, max_string_length, random_seed);
|
2020-03-06 02:12:18 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageGenerateRandom::read(
|
2020-03-06 02:12:18 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context,
|
2020-03-06 02:12:18 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams)
|
2020-03-06 02:12:18 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
Pipes pipes;
|
|
|
|
pipes.reserve(num_streams);
|
|
|
|
|
2021-07-09 03:15:41 +00:00
|
|
|
const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns();
|
2020-03-06 02:12:18 +00:00
|
|
|
Block block_header;
|
|
|
|
for (const auto & name : column_names)
|
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
const auto & name_type = our_columns.get(name);
|
2020-03-06 02:12:18 +00:00
|
|
|
MutableColumnPtr column = name_type.type->createColumn();
|
|
|
|
block_header.insert({std::move(column), name_type.type, name_type.name});
|
|
|
|
}
|
|
|
|
|
2023-06-04 02:44:51 +00:00
|
|
|
/// Correction of block size for wide tables.
|
|
|
|
size_t preferred_block_size_bytes = context->getSettingsRef().preferred_block_size_bytes;
|
|
|
|
if (preferred_block_size_bytes)
|
|
|
|
{
|
|
|
|
size_t estimated_row_size_bytes = estimateValueSize(std::make_shared<DataTypeTuple>(block_header.getDataTypes()), max_array_length, max_string_length);
|
|
|
|
|
|
|
|
size_t estimated_block_size_bytes = 0;
|
|
|
|
if (common::mulOverflow(max_block_size, estimated_row_size_bytes, estimated_block_size_bytes))
|
|
|
|
throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large estimated block size in GenerateRandom table: its estimation leads to 64bit overflow");
|
|
|
|
chassert(estimated_block_size_bytes != 0);
|
|
|
|
|
|
|
|
if (estimated_block_size_bytes > preferred_block_size_bytes)
|
|
|
|
{
|
|
|
|
max_block_size = static_cast<size_t>(max_block_size * (static_cast<double>(preferred_block_size_bytes) / estimated_block_size_bytes));
|
|
|
|
if (max_block_size == 0)
|
|
|
|
max_block_size = 1;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-03-06 02:12:18 +00:00
|
|
|
/// Will create more seed values for each source from initial seed.
|
2020-03-07 22:07:09 +00:00
|
|
|
pcg64 generate(random_seed);
|
2020-03-06 02:12:18 +00:00
|
|
|
|
|
|
|
for (UInt64 i = 0; i < num_streams; ++i)
|
|
|
|
pipes.emplace_back(std::make_shared<GenerateSource>(max_block_size, max_array_length, max_string_length, generate(), block_header, context));
|
|
|
|
|
2020-08-06 12:24:05 +00:00
|
|
|
return Pipe::unitePipes(std::move(pipes));
|
2020-03-06 02:12:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|