mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into clang-tidy
This commit is contained in:
commit
1cba0eaceb
@ -1,491 +0,0 @@
|
|||||||
#include <Storages/IStorage.h>
|
|
||||||
#include <Storages/ColumnsDescription.h>
|
|
||||||
#include <Storages/StorageGenerate.h>
|
|
||||||
#include <Storages/StorageFactory.h>
|
|
||||||
#include <DataStreams/OneBlockInputStream.h>
|
|
||||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
|
||||||
#include <Processors/Pipe.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
|
||||||
|
|
||||||
#include <DataTypes/DataTypeTuple.h>
|
|
||||||
#include <DataTypes/DataTypeEnum.h>
|
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
|
||||||
#include <DataTypes/DataTypeDateTime64.h>
|
|
||||||
#include <DataTypes/DataTypeDecimalBase.h>
|
|
||||||
#include <DataTypes/DataTypeArray.h>
|
|
||||||
#include <Columns/ColumnArray.h>
|
|
||||||
#include <Columns/ColumnFixedString.h>
|
|
||||||
#include <Columns/ColumnString.h>
|
|
||||||
#include <Columns/ColumnVector.h>
|
|
||||||
#include <Columns/ColumnNullable.h>
|
|
||||||
#include <Columns/ColumnTuple.h>
|
|
||||||
|
|
||||||
#include <Common/SipHash.h>
|
|
||||||
#include <Common/randomSeed.h>
|
|
||||||
#include <pcg_random.hpp>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int NOT_IMPLEMENTED;
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int BAD_TYPE_OF_FIELD;
|
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
void fillColumnWithRandomData(IColumn & column, const DataTypePtr type, UInt64 limit,
|
|
||||||
UInt64 max_array_length, UInt64 max_string_length, pcg32 & generator, pcg64_fast & generator64)
|
|
||||||
{
|
|
||||||
TypeIndex idx = type->getTypeId();
|
|
||||||
|
|
||||||
switch (idx)
|
|
||||||
{
|
|
||||||
case TypeIndex::Nothing:
|
|
||||||
throw Exception("Random Generator not implemented for type 'Nothing'.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
case TypeIndex::UInt8:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt8> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<UInt8>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::UInt16:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt16> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<UInt16>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::UInt32:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt32> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<UInt32>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::UInt64:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt64> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
UInt64 a = static_cast<UInt64>(generator64());
|
|
||||||
data[i] = static_cast<UInt64>(a);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::UInt128:
|
|
||||||
throw Exception("There is no DataType 'UInt128' support.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
case TypeIndex::Int8:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int8> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<Int8>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Int16:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int16> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<Int16>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Int32:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int32> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<Int32>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Int64:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int64> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<Int64>(generator64());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Int128:
|
|
||||||
throw Exception("There is no DataType 'Int128' support.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
case TypeIndex::Float32:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Float32> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
double d = 1.0;
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
d = std::numeric_limits<float>::max();
|
|
||||||
data[i] = (d / pcg32::max()) * generator();
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Float64:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<Float64> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
double d = 1.0;
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
d = std::numeric_limits<double>::max();
|
|
||||||
data[i] = (d / pcg64::max()) * generator64();
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Date:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt16> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<UInt16>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::DateTime:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt32> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<UInt32>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::DateTime64:
|
|
||||||
{
|
|
||||||
UInt32 scale;
|
|
||||||
if (auto * ptype = typeid_cast<const DataTypeDateTime64 *>(type.get()))
|
|
||||||
scale = ptype->getScale();
|
|
||||||
else
|
|
||||||
throw Exception("Static cast to DataTypeDateTime64 failed ", ErrorCodes::BAD_TYPE_OF_FIELD);
|
|
||||||
auto & data = typeid_cast<ColumnDecimal<Decimal64> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
UInt32 fractional = static_cast<UInt32>(generator()) % intExp10(scale);
|
|
||||||
UInt32 whole = static_cast<UInt32>(generator());
|
|
||||||
DateTime64 dt = DecimalUtils::decimalFromComponents<DateTime64>(whole, fractional, scale);
|
|
||||||
data[i] = dt;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::String:
|
|
||||||
{
|
|
||||||
auto & column_string = typeid_cast<ColumnString &>(column);
|
|
||||||
auto & offsets = column_string.getOffsets();
|
|
||||||
auto & chars = column_string.getChars();
|
|
||||||
|
|
||||||
UInt64 offset = 0;
|
|
||||||
{
|
|
||||||
offsets.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
offset += 1 + static_cast<UInt64>(generator()) % max_string_length;
|
|
||||||
offsets[i] = offset;
|
|
||||||
}
|
|
||||||
chars.resize(offset);
|
|
||||||
for (UInt64 i = 0; i < offset; ++i)
|
|
||||||
{
|
|
||||||
if (offset - i > 5)
|
|
||||||
{
|
|
||||||
UInt32 r = generator();
|
|
||||||
chars[i] = 32 + (r & 0x7F) % 95;
|
|
||||||
chars[i + 1] = 32 + ((r >> 7) & 0x7F) % 95;
|
|
||||||
chars[i + 2] = 32 + ((r >> 14) & 0x7F) % 95;
|
|
||||||
chars[i + 3] = 32 + ((r >> 21) & 0x7F) % 95;
|
|
||||||
chars[i + 4] = 32 + (r >> 28);
|
|
||||||
i += 4;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
UInt32 r = generator();
|
|
||||||
chars[i] = 32 + (r % 95);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// add terminating zero char
|
|
||||||
for (auto & i : offsets)
|
|
||||||
{
|
|
||||||
chars[i - 1] = 0;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::FixedString:
|
|
||||||
{
|
|
||||||
auto & column_string = typeid_cast<ColumnFixedString &>(column);
|
|
||||||
const size_t len = column_string.sizeOfValueIfFixed();
|
|
||||||
auto & chars = column_string.getChars();
|
|
||||||
|
|
||||||
UInt64 num_chars = static_cast<UInt64>(len) * limit;
|
|
||||||
{
|
|
||||||
chars.resize(num_chars);
|
|
||||||
for (UInt64 i = 0; i < num_chars; ++i)
|
|
||||||
{
|
|
||||||
chars[i] = static_cast<UInt8>(generator());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Enum8:
|
|
||||||
{
|
|
||||||
auto values = typeid_cast<const DataTypeEnum<Int8> *>(type.get())->getValues();
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int8> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
|
|
||||||
UInt8 size = values.size();
|
|
||||||
UInt8 off;
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
off = static_cast<UInt8>(generator()) % size;
|
|
||||||
data[i] = values[off].second;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Enum16:
|
|
||||||
{
|
|
||||||
auto values = typeid_cast<const DataTypeEnum<Int16> *>(type.get())->getValues();
|
|
||||||
auto & data = typeid_cast<ColumnVector<Int16> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
|
|
||||||
UInt16 size = values.size();
|
|
||||||
UInt8 off;
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
off = static_cast<UInt16>(generator()) % size;
|
|
||||||
data[i] = values[off].second;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Decimal32:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnDecimal<Decimal32> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
data[i] = static_cast<Int32>(generator());
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Decimal64:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnDecimal<Decimal64> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
UInt64 a = static_cast<UInt64>(generator()) << 32 | static_cast<UInt64>(generator());
|
|
||||||
data[i] = a;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Decimal128:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnDecimal<Decimal128> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
Int128 x = static_cast<Int128>(generator64()) << 64 | static_cast<Int128>(generator64());
|
|
||||||
data[i] = x;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::UUID:
|
|
||||||
{
|
|
||||||
auto & data = typeid_cast<ColumnVector<UInt128> &>(column).getData();
|
|
||||||
data.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
UInt64 a = static_cast<UInt64>(generator64());
|
|
||||||
UInt64 b = static_cast<UInt64>(generator64());
|
|
||||||
auto x = UInt128(a, b);
|
|
||||||
data[i] = x;
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Array:
|
|
||||||
{
|
|
||||||
auto & column_array = typeid_cast<ColumnArray &>(column);
|
|
||||||
auto nested_type = typeid_cast<const DataTypeArray *>(type.get())->getNestedType();
|
|
||||||
|
|
||||||
auto & offsets = column_array.getOffsets();
|
|
||||||
IColumn & data = column_array.getData();
|
|
||||||
|
|
||||||
UInt64 offset = 0;
|
|
||||||
{
|
|
||||||
offsets.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
offset += static_cast<UInt64>(generator()) % max_array_length;
|
|
||||||
offsets[i] = offset;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
fillColumnWithRandomData(data, nested_type, offset, max_array_length, max_string_length, generator, generator64);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Tuple:
|
|
||||||
{
|
|
||||||
auto &column_tuple = typeid_cast<ColumnTuple &>(column);
|
|
||||||
auto elements = typeid_cast<const DataTypeTuple *>(type.get())->getElements();
|
|
||||||
|
|
||||||
for (size_t i = 0; i < column_tuple.tupleSize(); ++i)
|
|
||||||
{
|
|
||||||
fillColumnWithRandomData(column_tuple.getColumn(i), elements[i], limit, max_array_length, max_string_length, generator, generator64);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Set:
|
|
||||||
throw Exception("Type 'Set' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
case TypeIndex::Interval:
|
|
||||||
throw Exception("Type 'Interval' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
case TypeIndex::Nullable:
|
|
||||||
{
|
|
||||||
auto & column_nullable = typeid_cast<ColumnNullable &>(column);
|
|
||||||
auto nested_type = typeid_cast<const DataTypeNullable *>(type.get())->getNestedType();
|
|
||||||
|
|
||||||
auto & null_map = column_nullable.getNullMapData();
|
|
||||||
IColumn & nested_column = column_nullable.getNestedColumn();
|
|
||||||
|
|
||||||
fillColumnWithRandomData(nested_column, nested_type, limit, max_array_length, max_string_length, generator, generator64);
|
|
||||||
|
|
||||||
null_map.resize(limit);
|
|
||||||
for (UInt64 i = 0; i < limit; ++i)
|
|
||||||
{
|
|
||||||
null_map[i] = generator() < 1024; /// No real motivation for this.
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case TypeIndex::Function:
|
|
||||||
throw Exception("Type 'Function' can not be stored in a table.", ErrorCodes::LOGICAL_ERROR);
|
|
||||||
case TypeIndex::AggregateFunction:
|
|
||||||
throw Exception("Random Generator not implemented for type 'AggregateFunction'.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
case TypeIndex::LowCardinality:
|
|
||||||
throw Exception("Random Generator not implemented for type 'LowCardinality'.", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
StorageGenerate::StorageGenerate(const StorageID & table_id_, const ColumnsDescription & columns_,
|
|
||||||
UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_)
|
|
||||||
: IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
|
||||||
{
|
|
||||||
random_seed = random_seed_ ? random_seed_ : randomSeed();
|
|
||||||
setColumns(columns_);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
class GenerateSource : public SourceWithProgress
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_)
|
|
||||||
: SourceWithProgress(block_header_), block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
|
||||||
, block_header(block_header_), r32(random_seed_), r64(random_seed_) {}
|
|
||||||
|
|
||||||
String getName() const override { return "Generate"; }
|
|
||||||
|
|
||||||
protected:
|
|
||||||
Chunk generate() override
|
|
||||||
{
|
|
||||||
auto columns = block_header.cloneEmptyColumns();
|
|
||||||
DataTypes types = block_header.getDataTypes();
|
|
||||||
auto cur_type = types.cbegin();
|
|
||||||
for (auto & col : columns)
|
|
||||||
{
|
|
||||||
fillColumnWithRandomData(col->assumeMutableRef(), *cur_type, block_size, max_array_length, max_string_length, r32, r64);
|
|
||||||
++cur_type;
|
|
||||||
}
|
|
||||||
return {std::move(columns), block_size};
|
|
||||||
}
|
|
||||||
|
|
||||||
private:
|
|
||||||
UInt64 block_size;
|
|
||||||
UInt64 max_array_length;
|
|
||||||
UInt64 max_string_length;
|
|
||||||
Block block_header;
|
|
||||||
|
|
||||||
pcg32 r32;
|
|
||||||
pcg64_fast r64;
|
|
||||||
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
void registerStorageGenerate(StorageFactory & factory)
|
|
||||||
{
|
|
||||||
factory.registerStorage("Generate", [](const StorageFactory::Arguments & args)
|
|
||||||
{
|
|
||||||
ASTs & engine_args = args.engine_args;
|
|
||||||
|
|
||||||
if (engine_args.size() > 3)
|
|
||||||
throw Exception("Storage Generate requires at most three arguments: "\
|
|
||||||
"max_array_length, max_string_length, random_seed.",
|
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
||||||
|
|
||||||
UInt64 max_array_length_ = 10;
|
|
||||||
UInt64 max_string_length_ = 10;
|
|
||||||
UInt64 random_seed_ = 0; // zero for random
|
|
||||||
|
|
||||||
/// Parsing second argument if present
|
|
||||||
if (engine_args.size() >= 1)
|
|
||||||
max_array_length_ = engine_args[0]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
|
||||||
|
|
||||||
if (engine_args.size() >= 2)
|
|
||||||
max_string_length_ = engine_args[1]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
|
||||||
|
|
||||||
if (engine_args.size() == 3)
|
|
||||||
random_seed_ = engine_args[2]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
|
||||||
|
|
||||||
return StorageGenerate::create(args.table_id, args.columns, max_array_length_, max_string_length_, random_seed_);
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
Pipes StorageGenerate::read(
|
|
||||||
const Names & column_names,
|
|
||||||
const SelectQueryInfo & /*query_info*/,
|
|
||||||
const Context & /*context*/,
|
|
||||||
QueryProcessingStage::Enum /*processed_stage*/,
|
|
||||||
size_t max_block_size,
|
|
||||||
unsigned num_streams)
|
|
||||||
{
|
|
||||||
check(column_names, true);
|
|
||||||
|
|
||||||
Pipes pipes;
|
|
||||||
pipes.reserve(num_streams);
|
|
||||||
|
|
||||||
const ColumnsDescription & columns_ = getColumns();
|
|
||||||
Block block_header;
|
|
||||||
for (const auto & name : column_names)
|
|
||||||
{
|
|
||||||
const auto & name_type = columns_.get(name);
|
|
||||||
MutableColumnPtr column = name_type.type->createColumn();
|
|
||||||
block_header.insert({std::move(column), name_type.type, name_type.name});
|
|
||||||
}
|
|
||||||
|
|
||||||
pcg32 generate(random_seed);
|
|
||||||
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));
|
|
||||||
}
|
|
||||||
return pipes;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
437
dbms/src/Storages/StorageGenerateRandom.cpp
Normal file
437
dbms/src/Storages/StorageGenerateRandom.cpp
Normal file
@ -0,0 +1,437 @@
|
|||||||
|
#include <Storages/IStorage.h>
|
||||||
|
#include <Storages/ColumnsDescription.h>
|
||||||
|
#include <Storages/StorageGenerateRandom.h>
|
||||||
|
#include <Storages/StorageFactory.h>
|
||||||
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||||
|
#include <Processors/Pipe.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
|
||||||
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
|
#include <DataTypes/DataTypeEnum.h>
|
||||||
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/DataTypeDateTime64.h>
|
||||||
|
#include <DataTypes/DataTypeDecimalBase.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypeFixedString.h>
|
||||||
|
#include <Columns/ColumnArray.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnVector.h>
|
||||||
|
#include <Columns/ColumnNullable.h>
|
||||||
|
#include <Columns/ColumnTuple.h>
|
||||||
|
|
||||||
|
#include <Common/SipHash.h>
|
||||||
|
#include <Common/randomSeed.h>
|
||||||
|
#include <common/unaligned.h>
|
||||||
|
|
||||||
|
#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;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
namespace
|
||||||
|
{
|
||||||
|
|
||||||
|
void fillBufferWithRandomData(char * __restrict data, size_t size, pcg64 & rng)
|
||||||
|
{
|
||||||
|
char * __restrict end = data + size;
|
||||||
|
while (data < end)
|
||||||
|
{
|
||||||
|
/// The loop can be further optimized.
|
||||||
|
UInt64 number = rng();
|
||||||
|
unalignedStore<UInt64>(data, number);
|
||||||
|
data += sizeof(UInt64); /// We assume that data has at least 7-byte padding (see PaddedPODArray)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
ColumnPtr fillColumnWithRandomData(
|
||||||
|
const DataTypePtr type, UInt64 limit, UInt64 max_array_length, UInt64 max_string_length, pcg64 & rng, const Context & context)
|
||||||
|
{
|
||||||
|
TypeIndex idx = type->getTypeId();
|
||||||
|
|
||||||
|
switch (idx)
|
||||||
|
{
|
||||||
|
case TypeIndex::String:
|
||||||
|
{
|
||||||
|
/// Mostly the same as the implementation of randomPrintableASCII function.
|
||||||
|
|
||||||
|
auto column = ColumnString::create();
|
||||||
|
ColumnString::Chars & data_to = column->getChars();
|
||||||
|
ColumnString::Offsets & offsets_to = column->getOffsets();
|
||||||
|
offsets_to.resize(limit);
|
||||||
|
|
||||||
|
IColumn::Offset offset = 0;
|
||||||
|
for (size_t row_num = 0; row_num < limit; ++row_num)
|
||||||
|
{
|
||||||
|
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/
|
||||||
|
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
|
||||||
|
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:
|
||||||
|
{
|
||||||
|
auto nested_type = typeid_cast<const DataTypeArray *>(type.get())->getNestedType();
|
||||||
|
|
||||||
|
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)
|
||||||
|
{
|
||||||
|
offset += static_cast<UInt64>(rng()) % (max_array_length + 1);
|
||||||
|
offsets[i] = offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto data_column = fillColumnWithRandomData(nested_type, offset, max_array_length, max_string_length, rng, context);
|
||||||
|
|
||||||
|
return ColumnArray::create(std::move(data_column), std::move(offsets_column));
|
||||||
|
}
|
||||||
|
|
||||||
|
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:
|
||||||
|
{
|
||||||
|
auto nested_type = typeid_cast<const DataTypeNullable *>(type.get())->getNestedType();
|
||||||
|
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.
|
||||||
|
|
||||||
|
return ColumnNullable::create(std::move(nested_column), std::move(null_map_column));
|
||||||
|
}
|
||||||
|
|
||||||
|
case TypeIndex::UInt8:
|
||||||
|
{
|
||||||
|
auto column = ColumnUInt8::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt8), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::UInt16: [[fallthrough]];
|
||||||
|
case TypeIndex::Date:
|
||||||
|
{
|
||||||
|
auto column = ColumnUInt16::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt16), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::UInt32: [[fallthrough]];
|
||||||
|
case TypeIndex::DateTime:
|
||||||
|
{
|
||||||
|
auto column = ColumnUInt32::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt32), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::UInt64:
|
||||||
|
{
|
||||||
|
auto column = ColumnUInt64::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt64), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::UInt128: [[fallthrough]];
|
||||||
|
case TypeIndex::UUID:
|
||||||
|
{
|
||||||
|
auto column = ColumnUInt128::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(UInt128), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Int8:
|
||||||
|
{
|
||||||
|
auto column = ColumnInt8::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Int8), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Int16:
|
||||||
|
{
|
||||||
|
auto column = ColumnInt16::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Int16), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Int32:
|
||||||
|
{
|
||||||
|
auto column = ColumnInt32::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Int32), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Int64:
|
||||||
|
{
|
||||||
|
auto column = ColumnInt64::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Int64), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Float32:
|
||||||
|
{
|
||||||
|
auto column = ColumnFloat32::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Float32), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Float64:
|
||||||
|
{
|
||||||
|
auto column = ColumnFloat64::create();
|
||||||
|
column->getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getData().data()), limit * sizeof(Float64), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Decimal32:
|
||||||
|
{
|
||||||
|
auto column = type->createColumn();
|
||||||
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal32> &>(*column);
|
||||||
|
column_concrete.getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column_concrete.getData().data()), limit * sizeof(Decimal32), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Decimal64: /// TODO Decimal may be generated out of range.
|
||||||
|
{
|
||||||
|
auto column = type->createColumn();
|
||||||
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal64> &>(*column);
|
||||||
|
column_concrete.getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column_concrete.getData().data()), limit * sizeof(Decimal64), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::Decimal128:
|
||||||
|
{
|
||||||
|
auto column = type->createColumn();
|
||||||
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal128> &>(*column);
|
||||||
|
column_concrete.getData().resize(limit);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column_concrete.getData().data()), limit * sizeof(Decimal128), rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::FixedString:
|
||||||
|
{
|
||||||
|
size_t n = typeid_cast<const DataTypeFixedString &>(*type).getN();
|
||||||
|
auto column = ColumnFixedString::create(n);
|
||||||
|
column->getChars().resize(limit * n);
|
||||||
|
fillBufferWithRandomData(reinterpret_cast<char *>(column->getChars().data()), limit * n, rng);
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
case TypeIndex::DateTime64:
|
||||||
|
{
|
||||||
|
auto column = type->createColumn();
|
||||||
|
auto & column_concrete = typeid_cast<ColumnDecimal<Decimal64> &>(*column);
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
|
||||||
|
default:
|
||||||
|
throw Exception("The 'GenerateRandom' is not implemented for type " + type->getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
class GenerateSource : public SourceWithProgress
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, const Context & context_)
|
||||||
|
: SourceWithProgress(block_header_), block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
||||||
|
, block_header(block_header_), rng(random_seed_), context(context_) {}
|
||||||
|
|
||||||
|
String getName() const override { return "GenerateRandom"; }
|
||||||
|
|
||||||
|
protected:
|
||||||
|
Chunk generate() override
|
||||||
|
{
|
||||||
|
Columns columns;
|
||||||
|
columns.reserve(block_header.columns());
|
||||||
|
DataTypes types = block_header.getDataTypes();
|
||||||
|
|
||||||
|
for (const auto & type : types)
|
||||||
|
columns.emplace_back(fillColumnWithRandomData(type, block_size, max_array_length, max_string_length, rng, context));
|
||||||
|
|
||||||
|
return {std::move(columns), block_size};
|
||||||
|
}
|
||||||
|
|
||||||
|
private:
|
||||||
|
UInt64 block_size;
|
||||||
|
UInt64 max_array_length;
|
||||||
|
UInt64 max_string_length;
|
||||||
|
Block block_header;
|
||||||
|
|
||||||
|
pcg64 rng;
|
||||||
|
|
||||||
|
const Context & context;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
StorageGenerateRandom::StorageGenerateRandom(const StorageID & table_id_, const ColumnsDescription & columns_,
|
||||||
|
UInt64 max_array_length_, UInt64 max_string_length_, std::optional<UInt64> random_seed_)
|
||||||
|
: IStorage(table_id_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
||||||
|
{
|
||||||
|
random_seed = random_seed_ ? sipHash64(*random_seed_) : randomSeed();
|
||||||
|
setColumns(columns_);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
void registerStorageGenerateRandom(StorageFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerStorage("GenerateRandom", [](const StorageFactory::Arguments & args)
|
||||||
|
{
|
||||||
|
ASTs & engine_args = args.engine_args;
|
||||||
|
|
||||||
|
if (engine_args.size() > 3)
|
||||||
|
throw Exception("Storage GenerateRandom requires at most three arguments: "
|
||||||
|
"random_seed, max_string_length, max_array_length.",
|
||||||
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
|
std::optional<UInt64> random_seed;
|
||||||
|
UInt64 max_string_length = 10;
|
||||||
|
UInt64 max_array_length = 10;
|
||||||
|
|
||||||
|
if (engine_args.size() >= 1)
|
||||||
|
{
|
||||||
|
const Field & value = engine_args[0]->as<const ASTLiteral &>().value;
|
||||||
|
if (!value.isNull())
|
||||||
|
random_seed = value.safeGet<UInt64>();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (engine_args.size() >= 2)
|
||||||
|
max_string_length = engine_args[1]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||||
|
|
||||||
|
if (engine_args.size() == 3)
|
||||||
|
max_array_length = engine_args[2]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||||
|
|
||||||
|
|
||||||
|
return StorageGenerateRandom::create(args.table_id, args.columns, max_array_length, max_string_length, random_seed);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
Pipes StorageGenerateRandom::read(
|
||||||
|
const Names & column_names,
|
||||||
|
const SelectQueryInfo & /*query_info*/,
|
||||||
|
const Context & context,
|
||||||
|
QueryProcessingStage::Enum /*processed_stage*/,
|
||||||
|
size_t max_block_size,
|
||||||
|
unsigned num_streams)
|
||||||
|
{
|
||||||
|
check(column_names, true);
|
||||||
|
|
||||||
|
Pipes pipes;
|
||||||
|
pipes.reserve(num_streams);
|
||||||
|
|
||||||
|
const ColumnsDescription & columns_ = getColumns();
|
||||||
|
Block block_header;
|
||||||
|
for (const auto & name : column_names)
|
||||||
|
{
|
||||||
|
const auto & name_type = columns_.get(name);
|
||||||
|
MutableColumnPtr column = name_type.type->createColumn();
|
||||||
|
block_header.insert({std::move(column), name_type.type, name_type.name});
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Will create more seed values for each source from initial seed.
|
||||||
|
pcg64 generate(random_seed);
|
||||||
|
|
||||||
|
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));
|
||||||
|
|
||||||
|
return pipes;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -1,5 +1,6 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include <optional>
|
||||||
#include <ext/shared_ptr_helper.h>
|
#include <ext/shared_ptr_helper.h>
|
||||||
#include <Storages/IStorage.h>
|
#include <Storages/IStorage.h>
|
||||||
|
|
||||||
@ -8,11 +9,11 @@ namespace DB
|
|||||||
{
|
{
|
||||||
/* Generates random data for given schema.
|
/* Generates random data for given schema.
|
||||||
*/
|
*/
|
||||||
class StorageGenerate : public ext::shared_ptr_helper<StorageGenerate>, public IStorage
|
class StorageGenerateRandom : public ext::shared_ptr_helper<StorageGenerateRandom>, public IStorage
|
||||||
{
|
{
|
||||||
friend struct ext::shared_ptr_helper<StorageGenerate>;
|
friend struct ext::shared_ptr_helper<StorageGenerateRandom>;
|
||||||
public:
|
public:
|
||||||
std::string getName() const override { return "Generate"; }
|
std::string getName() const override { return "GenerateRandom"; }
|
||||||
|
|
||||||
Pipes read(
|
Pipes read(
|
||||||
const Names & column_names,
|
const Names & column_names,
|
||||||
@ -28,8 +29,8 @@ private:
|
|||||||
UInt64 random_seed = 0;
|
UInt64 random_seed = 0;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageGenerate(const StorageID & table_id_, const ColumnsDescription & columns_,
|
StorageGenerateRandom(const StorageID & table_id_, const ColumnsDescription & columns_,
|
||||||
UInt64 max_array_length, UInt64 max_string_length, UInt64 random_seed);
|
UInt64 max_array_length, UInt64 max_string_length, std::optional<UInt64> random_seed);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
@ -29,7 +29,7 @@ void registerStorages()
|
|||||||
registerStorageView(factory);
|
registerStorageView(factory);
|
||||||
registerStorageMaterializedView(factory);
|
registerStorageMaterializedView(factory);
|
||||||
registerStorageLiveView(factory);
|
registerStorageLiveView(factory);
|
||||||
registerStorageGenerate(factory);
|
registerStorageGenerateRandom(factory);
|
||||||
|
|
||||||
#if USE_AWS_S3
|
#if USE_AWS_S3
|
||||||
registerStorageS3(factory);
|
registerStorageS3(factory);
|
||||||
|
@ -23,7 +23,7 @@ void registerStorageJoin(StorageFactory & factory);
|
|||||||
void registerStorageView(StorageFactory & factory);
|
void registerStorageView(StorageFactory & factory);
|
||||||
void registerStorageMaterializedView(StorageFactory & factory);
|
void registerStorageMaterializedView(StorageFactory & factory);
|
||||||
void registerStorageLiveView(StorageFactory & factory);
|
void registerStorageLiveView(StorageFactory & factory);
|
||||||
void registerStorageGenerate(StorageFactory & factory);
|
void registerStorageGenerateRandom(StorageFactory & factory);
|
||||||
|
|
||||||
#if USE_AWS_S3
|
#if USE_AWS_S3
|
||||||
void registerStorageS3(StorageFactory & factory);
|
void registerStorageS3(StorageFactory & factory);
|
||||||
|
@ -2,7 +2,7 @@
|
|||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
|
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
#include <Storages/StorageGenerate.h>
|
#include <Storages/StorageGenerateRandom.h>
|
||||||
|
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
@ -10,7 +10,7 @@
|
|||||||
|
|
||||||
#include <TableFunctions/ITableFunction.h>
|
#include <TableFunctions/ITableFunction.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
#include <TableFunctions/TableFunctionGenerate.h>
|
#include <TableFunctions/TableFunctionGenerateRandom.h>
|
||||||
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
#include <TableFunctions/parseColumnsListForTableFunction.h>
|
||||||
|
|
||||||
#include "registerTableFunctions.h"
|
#include "registerTableFunctions.h"
|
||||||
@ -25,7 +25,7 @@ namespace ErrorCodes
|
|||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr TableFunctionGenerate::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
|
StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
|
||||||
{
|
{
|
||||||
ASTs & args_func = ast_function->children;
|
ASTs & args_func = ast_function->children;
|
||||||
|
|
||||||
@ -36,41 +36,45 @@ StoragePtr TableFunctionGenerate::executeImpl(const ASTPtr & ast_function, const
|
|||||||
|
|
||||||
if (args.size() < 1)
|
if (args.size() < 1)
|
||||||
throw Exception("Table function '" + getName() + "' requires at least one argument: "
|
throw Exception("Table function '" + getName() + "' requires at least one argument: "
|
||||||
" structure(, max_array_length, max_string_length, random_seed).",
|
" structure, [random_seed, max_string_length, max_array_length].",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
if (args.size() > 4)
|
if (args.size() > 4)
|
||||||
throw Exception("Table function '" + getName() + "' requires at most four arguments: "
|
throw Exception("Table function '" + getName() + "' requires at most four arguments: "
|
||||||
" structure, max_array_length, max_string_length, random_seed.",
|
" structure, [random_seed, max_string_length, max_array_length].",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
/// Parsing first argument as table structure and creating a sample block
|
/// Parsing first argument as table structure and creating a sample block
|
||||||
std::string structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
std::string structure = args[0]->as<const ASTLiteral &>().value.safeGet<String>();
|
||||||
|
|
||||||
UInt64 max_array_length = 10;
|
|
||||||
UInt64 max_string_length = 10;
|
UInt64 max_string_length = 10;
|
||||||
UInt64 random_seed = 0; // zero for random
|
UInt64 max_array_length = 10;
|
||||||
|
std::optional<UInt64> random_seed;
|
||||||
|
|
||||||
/// Parsing second argument if present
|
|
||||||
if (args.size() >= 2)
|
if (args.size() >= 2)
|
||||||
max_array_length = args[1]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
{
|
||||||
|
const Field & value = args[1]->as<const ASTLiteral &>().value;
|
||||||
|
if (!value.isNull())
|
||||||
|
random_seed = value.safeGet<UInt64>();
|
||||||
|
}
|
||||||
|
|
||||||
if (args.size() >= 3)
|
if (args.size() >= 3)
|
||||||
max_string_length = args[2]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
max_string_length = args[2]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||||
|
|
||||||
if (args.size() == 4)
|
if (args.size() == 4)
|
||||||
random_seed = args[3]->as<ASTLiteral &>().value.safeGet<UInt64>();
|
max_array_length = args[3]->as<const ASTLiteral &>().value.safeGet<UInt64>();
|
||||||
|
|
||||||
|
|
||||||
ColumnsDescription columns = parseColumnsListFromString(structure, context);
|
ColumnsDescription columns = parseColumnsListFromString(structure, context);
|
||||||
|
|
||||||
auto res = StorageGenerate::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed);
|
auto res = StorageGenerateRandom::create(StorageID(getDatabaseName(), table_name), columns, max_array_length, max_string_length, random_seed);
|
||||||
res->startup();
|
res->startup();
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerTableFunctionGenerate(TableFunctionFactory & factory)
|
void registerTableFunctionGenerate(TableFunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction<TableFunctionGenerate>(TableFunctionFactory::CaseInsensitive);
|
factory.registerFunction<TableFunctionGenerateRandom>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
@ -4,12 +4,14 @@
|
|||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
/* generate(structure, [max_array_length, max_string_length, random_seed]) - creates a temporary storage that generates columns with random data
|
|
||||||
|
/* generateRandom(structure, [max_array_length, max_string_length, random_seed])
|
||||||
|
* - creates a temporary storage that generates columns with random data
|
||||||
*/
|
*/
|
||||||
class TableFunctionGenerate : public ITableFunction
|
class TableFunctionGenerateRandom : public ITableFunction
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = "generate";
|
static constexpr auto name = "generateRandom";
|
||||||
std::string getName() const override { return name; }
|
std::string getName() const override { return name; }
|
||||||
private:
|
private:
|
||||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
@ -6,23 +6,23 @@
|
|||||||
</any_of>
|
</any_of>
|
||||||
</stop_conditions>
|
</stop_conditions>
|
||||||
|
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8') LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8') LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Enum8(\'hello\' = 1, \'world\' = 5)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('f32 Float32, f64 Float64', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('f32 Float32, f64 Float64', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Tuple(Int32, Int64)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Tuple(Int32, Int64)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Int8)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Int8)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(Int32))', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(Int32))', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Tuple(Int32, Array(Int64))', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Tuple(Int32, Array(Int64))', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Nullable(String)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Nullable(String)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(String)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(String)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i UUID', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i UUID', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i Array(Nullable(UUID))', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(UUID))', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i FixedString(4)', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i FixedString(4)', 10, 10, 1) LIMIT 100000);</query>
|
||||||
<query>SELECT COUNT(*) FROM (SELECT * FROM generate('i String', 10, 10, 1) LIMIT 100000);</query>
|
<query>SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i String', 10, 10, 1) LIMIT 100000);</query>
|
||||||
</test>
|
</test>
|
||||||
|
@ -1,103 +1,103 @@
|
|||||||
100
|
100
|
||||||
-
|
-
|
||||||
[] -183162.1041 ('2074-01-15 19:36:42.413','0345f8ad-8936-8cc9-9ff2-394f225fc318')
|
[] -54259.6828 ('2088-03-01 16:26:24.094','d3c2a216-a98c-d56c-7bf7-62de9f264cf4')
|
||||||
[3] -39049.0845 ('2045-07-04 15:01:09.380','f79d47d8-4030-9916-54b9-495a5ccc7202')
|
[88] 34528.4014 ('2031-12-09 00:40:39.898','9ef777c8-de0e-d25e-e16c-5b624f88523c')
|
||||||
[-95,-104] -14324.1685 ('2023-10-02 06:05:53.887','ea94157b-737b-a272-acd5-c7ab9c6f07c3')
|
[-1] 121968.7945 ('2060-02-05 09:18:12.011','7655e515-d2ca-2f06-0950-e4f44f69aca7')
|
||||||
[] -182420.0194 ('2002-01-08 01:42:50.396','ac579c0f-8523-144f-aa4c-c9587cc27144')
|
[-103,75] -135033.4349 ('2038-12-19 20:38:58.695','86b57d15-292d-2517-9acf-47cd053e7a3a')
|
||||||
[] 127157.2100 ('2006-11-23 14:25:39.542','d07c5204-ef95-6804-83df-01dedaf32522')
|
[110] -202668.6900 ('2009-06-18 01:53:29.808','bc630f78-7d58-0c46-dd4b-27fc35625e96')
|
||||||
[28] -198600.4267 ('2060-09-25 02:57:41.504','b13ff007-c245-d737-85b2-1fa003e57127')
|
[-22,2] 168636.9728 ('2074-09-03 09:20:20.936','7624ce27-9bff-4e9d-3f18-6851a97dd0ca')
|
||||||
[] -87232.0739 ('2027-05-12 20:26:59.405','a2f2cbf4-b11b-6976-7b91-14b6964acbe2')
|
[-22,-62] -75192.4989 ('2085-10-11 21:51:12.855','a4c4d0ed-f448-244e-1723-ca1bba816f2b')
|
||||||
[] -110349.8987 ('2042-11-01 10:51:30.039','445a77b5-0a27-3485-8dd8-c7cc35d2692f')
|
[-2,-90] 133592.5064 ('2010-10-28 21:18:04.633','8ba9103b-f90c-b49b-38c1-223ae5f42bf7')
|
||||||
[32,73] 123253.5669 ('2016-06-21 00:23:02.917','c12095e6-b82c-d81c-4629-acd80e02b080')
|
[-94,80] 197330.6359 ('2024-03-30 22:08:45.772','83442013-3677-5097-065d-72dfbe8a3506')
|
||||||
[-109,85] 34320.8302 ('2080-12-31 16:49:32.509','ebbbe70a-0321-ff18-89de-2bc9a9e4c454')
|
[23] 167557.6237 ('2078-07-25 21:54:42.480','be14d98e-5b24-54ee-c959-d24fa9a58fdd')
|
||||||
[68,76] 160458.5593 ('2030-05-23 03:33:29.681','805b0a62-9ada-a47e-2d5e-63cb5923549c')
|
[46,-10,-63] 185107.1979 ('2040-10-07 06:06:53.504','5ed1fe6a-9313-41d7-4bf9-3948e961509f')
|
||||||
[69] -189322.2887 ('2084-06-19 03:31:58.508','429df3a4-ff18-28d5-9ad8-dcdd78e8b1ae')
|
[-107,68] -163781.3045 ('2021-12-21 19:18:58.933','7b634f19-0863-829e-484b-be288aab54a1')
|
||||||
[] 189158.4731 ('1985-12-21 16:36:51.092','d63c5cbb-9418-ce59-000c-056f88157bfa')
|
[-35,-116,73] -203577.5379 ('2093-08-01 20:21:09.407','d371bad4-b098-ffdd-f84c-6a02390c2939')
|
||||||
[-120] 114890.5905 ('2089-02-19 22:30:18.216','2bc4860a-7214-300a-851e-b61011c346ef')
|
[61] 152284.9386 ('2089-12-20 19:21:33.149','9e8426c1-278a-4d9c-4076-364a95b065e3')
|
||||||
[] 54493.1631 ('2062-08-18 04:57:01.348','c00d218a-913f-b657-1ff9-99927741f7ab')
|
[75] 170968.4171 ('2020-07-17 15:45:31.975','47397a81-bda7-8bd9-59f7-d60e2204fe99')
|
||||||
[103] 88891.6006 ('2071-05-23 21:46:45.644','036d2746-f7aa-b5a4-b716-b8e8a5e041da')
|
[-115,93] -173740.5652 ('2098-04-25 22:10:33.327','117e31dd-102e-ee6c-0dbd-0a4203c18ca5')
|
||||||
[28] -41173.9863 ('2044-03-11 10:06:47.659','6bf54ef0-9bad-54d4-5ca3-02d79883b697')
|
[-20,4,21] 63834.8685 ('2000-07-08 18:09:40.271','10b0fa48-55a3-755a-4a44-36315ae04c1c')
|
||||||
[] -64809.0652 ('2010-11-02 23:46:46.150','ae5cafb4-fe3c-71a5-9a76-0314c44180de')
|
[-110,117,91] -160640.1506 ('1998-04-18 10:58:04.479','6dfa3a8e-6e65-543c-5f50-1ff45835aa5a')
|
||||||
[125,74] 28139.7661 ('1989-07-19 22:10:13.477','ee9b8173-4426-8615-97eb-a904266847e5')
|
[62] 63817.7977 ('2043-01-24 02:07:18.972','98b8ef31-4f65-2f8b-1ea7-b1473900099e')
|
||||||
[-101] -29032.2527 ('2052-08-13 08:55:15.045','ab8ee358-ff53-de7e-f012-cd0eed754ff2')
|
[-2] -175477.0173 ('2007-01-16 07:46:14.781','ec92f616-6e1f-003a-54c6-c5f9118d2f1b')
|
||||||
[82] 170334.6151 ('2034-02-23 18:50:46.847','f1402791-1d23-e56d-25f0-5a51a3cb245a')
|
[] 197663.3035 ('2046-06-30 17:04:56.788','fb3244a4-8af2-104f-2a6f-25a7b7b9a112')
|
||||||
[27,56] 168648.5067 ('2004-05-31 19:47:19.448','5019365e-f74d-b31e-aacb-63d8189e3e3e')
|
[-24] -174299.4691 ('2058-02-23 14:50:58.839','d63ee868-fa93-bf8b-0264-8ebbceb13e3b')
|
||||||
[-74] 89577.4738 ('1997-01-15 09:05:57.691','d144325c-24a9-411e-cc10-78b0637f75a7')
|
[95,38] -65083.7371 ('2015-03-10 13:33:16.429','47bd199c-f99e-51ea-84e9-b65cce9d167c')
|
||||||
[100] 178532.5772 ('2055-04-22 16:20:11.498','41439c27-fba1-1b66-13a7-cf79fded4d9a')
|
[91,110,72] 130908.9643 ('2036-03-16 15:17:53.679','0dd4ca31-1e09-d7e0-f3df-60cad3cfa805')
|
||||||
[32] 123101.7871 ('2103-02-18 15:53:42.748','25b804d3-a73e-ed14-e6e1-eafb0d9473cd')
|
[] 208972.3779 ('2034-03-05 22:29:21.994','1069d77c-dfd2-912e-60b8-3c5b964f7e11')
|
||||||
[-115,-85] -208371.1662 ('2039-10-22 18:06:50.235','41fba85a-5080-48bb-e18d-a8af04a890aa')
|
[-32] 167938.5050 ('2093-09-10 20:39:39.050','9d1025b6-2d0c-1d84-dafd-02668eb29270')
|
||||||
[-12,101] -7791.5577 ('2073-07-27 11:35:45.239','c00be55f-59ac-762c-af0a-9e33cf30a1f4')
|
[] 153744.6987 ('2088-10-02 11:02:11.024','a88e6cb7-2210-5ce5-6bcf-24afc0eca5b6')
|
||||||
[-127] -18602.1918 ('2024-02-05 19:54:00.798','ea85cbed-66f2-197b-4e63-dfbdcd306cce')
|
[67] -74220.6650 ('2074-12-30 18:43:40.817','68096065-18c8-8aca-fd21-15330ead669d')
|
||||||
[-78,125] -112158.3556 ('2016-08-12 06:46:17.173','15809e25-b003-010b-c63c-9e880568736a')
|
[6] 66759.8938 ('2091-09-01 19:07:18.219','bb14f4cc-0b54-9a8c-e835-71333b28c03b')
|
||||||
[] 151191.1081 ('1982-11-12 17:59:10.171','d6bbbe2c-fca0-53c8-22a6-de9e0715d3cc')
|
[-28,-82,9] 168625.3131 ('2002-03-20 21:02:30.321','405bb877-6e28-8b91-cb62-bd82a3fa797c')
|
||||||
[46] 58689.9611 ('2085-04-13 09:13:07.230','fe4be382-eb78-4cf9-fa57-c6eccf955419')
|
[] -19760.1670 ('2044-11-08 07:52:03.325','13769348-9e58-0e75-3972-8bbadc150715')
|
||||||
[-4,42] -88292.1046 ('1980-03-06 08:29:12.503','2633522e-ff9c-b837-1b9b-6559875c13b0')
|
[] 160663.7797 ('2025-04-12 13:17:53.501','e6370321-94f5-97e6-0348-a84e72ff5b42')
|
||||||
[-2] 64983.6649 ('2034-07-07 11:20:23.903','d19f5c4d-e444-2e5b-a55d-5280d1760b94')
|
[-17,18] 99105.9856 ('1972-05-01 12:23:11.688','02618b9e-97cd-4698-d2e8-3f52f4c5a09a')
|
||||||
[-31,-110] -25638.6649 ('2025-05-17 21:45:25.519','3654a15e-bfa3-6075-b5b8-07e25310de1f')
|
[86,77] -116990.3914 ('1981-12-31 05:06:54.198','3ac42bb4-8652-b1a8-10bb-98f0337261f8')
|
||||||
[25] -140469.2476 ('2083-12-13 23:55:25.450','940f7441-ae40-d810-f6c3-e2fff468050c')
|
[-109,69,-63] -151527.3587 ('2001-01-17 11:19:56.504','77fe7ee2-f279-2855-bfd2-a7d7cee678cc')
|
||||||
[-99] 128186.7318 ('1995-09-15 04:26:33.803','4cfd264f-ff00-4190-929c-b675826607d3')
|
[] -57762.3928 ('1978-08-16 18:47:37.660','ab9a110a-fd8d-3c4c-5a49-34c2005536ce')
|
||||||
[] -52961.0340 ('2046-03-19 14:15:50.245','314de821-308b-c61c-e256-9f6afed5d4f3')
|
[-77] 107274.6407 ('2017-01-12 12:03:02.657','c1ad4f17-cc54-45f3-9410-9c1011653f6d')
|
||||||
[] -155852.9334 ('2052-01-29 06:31:08.957','5be2ccd2-b5bb-921d-5b5e-4a0e22385de7')
|
[] 107133.6410 ('2050-10-05 06:29:27.154','36e576aa-c77f-994e-1925-4a4c40da3a0f')
|
||||||
[-74,81] 29366.0091 ('1978-03-05 19:24:49.193','ab9b6a39-89ac-9280-c76f-60d598ce65c6')
|
[] 46672.2176 ('2094-01-21 20:25:39.144','e9ba850d-604e-bc7d-417c-1078e89d4615')
|
||||||
[9] 56134.8951 ('2104-04-03 10:27:33.053','339a7f6d-0e0b-e039-78c0-2d045457d821')
|
[-87,-122,-65] -86258.4663 ('2081-06-17 03:37:45.498','64795221-9719-7937-b4d2-be5f30065ece')
|
||||||
[-61] 68841.1188 ('2059-07-26 12:14:33.197','c817bbb3-d091-b73c-1b9a-53f8a03bffb6')
|
[-53] -48672.1424 ('1992-06-27 17:27:23.602','7c67bc31-c7bb-6197-fdca-f73329b976f2')
|
||||||
[89] 168034.0459 ('2028-05-29 08:02:02.393','bc217a73-e802-1772-80b5-d8c827295799')
|
[34] -108954.7820 ('2096-07-03 23:06:30.632','9c1b37d7-4ced-9428-a0ae-34c5436b14c4')
|
||||||
[124] 11648.6762 ('2084-12-25 12:10:35.676','77390177-1dd6-a5c0-dd35-4f85e38bcb2c')
|
[] -168124.2364 ('1987-06-03 06:47:12.945','d1c39af4-f920-5095-b8e2-0f878950167b')
|
||||||
[-47,-125] -120893.6705 ('2012-10-18 22:52:57.524','472814b2-4033-c5a5-7d86-fb36079e88fb')
|
[] -112431.4799 ('2021-07-26 07:04:58.527','da07a72d-7e1f-8890-4c4b-326835d11b39')
|
||||||
[35] 153250.6252 ('2006-11-06 00:05:25.456','de0d6ed9-eca6-e01e-eb1c-c46c8ad6e33e')
|
[-35,-95,58] -181254.9139 ('2086-11-12 17:17:14.473','22f74d0b-dfc0-3f7a-33f4-8055d8fa7846')
|
||||||
[-43,70] -141086.3184 ('2013-02-03 23:07:11.759','65d48b24-cdc0-f7db-cb16-d0ad03279bcc')
|
[98,119] 11468.5238 ('2092-02-25 11:07:07.695','a1fb97bf-1885-6715-c233-b88a6cd111e4')
|
||||||
[120,-57] -93351.1404 ('2000-02-03 14:39:00.466','6991722b-90dc-e9dd-c5e7-f28bd1d4f0d8')
|
[] 82333.8963 ('1989-11-23 01:38:57.012','a2b82b5b-8331-555c-579b-de4b0eeb7e81')
|
||||||
[34,43] 187780.4567 ('2014-02-10 05:22:19.250','3db77bc5-d877-b22e-6667-955bf36d2e08')
|
[-5,-66,69] 32055.8376 ('2040-12-17 16:49:08.704','4537d25e-a2db-ea9a-8e24-a16ed7e0c6e4')
|
||||||
[73] -90148.5697 ('2014-10-05 18:34:31.419','5a0f919e-38c9-0a68-e805-977db04d0acb')
|
[81,-84,-24] -210815.2512 ('2047-06-09 13:30:06.922','ac3c5b5f-f977-2830-c398-d10a6076a498')
|
||||||
[] -179121.0029 ('2077-01-23 07:57:55.365','fcf79336-a6dc-44fd-8c78-7e74e07b60fa')
|
[84,-105] -175413.7733 ('1998-11-03 04:30:21.191','c535feac-1943-c0a1-23f0-645d5406db24')
|
||||||
[-69,120] 119321.8003 ('1989-07-01 13:11:35.185','92f6a362-250c-cfcd-acd7-99399cbf88ad')
|
[58,31] -335.8512 ('1973-07-09 12:21:10.444','24a7dd3d-2565-1de3-05d9-e45fd8ba7729')
|
||||||
[] 208864.8324 ('1991-02-17 03:04:00.682','b0dc8e88-ea6f-c2da-c116-3e4873dc8d54')
|
[-49,-47] 177399.2836 ('2049-03-15 15:33:00.190','e4432b9b-61e9-d451-dc87-ae3b9da6fd35')
|
||||||
[22,-14] -127735.4391 ('2036-08-10 08:33:03.806','5ab1ab2b-913d-ff8a-6f8f-86387e77ed5c')
|
[] 211525.2349 ('2106-01-11 10:44:18.918','23315435-7132-05b5-5a9b-c2c738433a87')
|
||||||
[83,-70] -142476.9847 ('2074-11-22 19:27:13.085','51b9d30a-3b10-265c-4086-1ac35b634ec7')
|
[45,-95,-39] -15314.9732 ('2055-10-29 13:51:12.182','833b2efa-8c72-f5f6-3040-cb4831e8ceb9')
|
||||||
[] -128052.2443 ('2088-01-02 10:58:36.999','745e8226-d906-7fb3-33f4-9a079037bdcd')
|
[] 213384.5774 ('2067-02-10 22:02:42.113','0cd7f438-caa7-0d21-867c-1fdb6d67d797')
|
||||||
[12,-116] -88390.1399 ('2074-02-18 17:46:45.208','fb5f827e-1809-6cab-2855-d45df20ecd92')
|
[99] -147316.5599 ('2000-05-09 21:37:34.776','a3ea6796-38d5-72ff-910d-8b4300831916')
|
||||||
[] -84110.2097 ('2039-03-24 17:08:15.660','88e18c93-6276-d176-dad1-7db72e340ca7')
|
[] 8828.2471 ('1993-11-30 16:53:22.503','7209213f-38bb-cfed-1955-f1fad5a9577a')
|
||||||
[] 202866.8175 ('2104-01-25 13:42:41.758','10faa33e-d383-c6b3-399d-44c06ebb00f5')
|
[117,9,-35] -134812.6269 ('2065-09-04 23:47:26.589','d33d0d6f-b9c0-2850-4593-cfc9f1e20a4d')
|
||||||
[-21] 151775.1601 ('1995-10-20 15:44:53.296','7ccaf135-787d-2ac0-09c0-7545c798ee14')
|
[-35,-58,-101] -9101.5369 ('2023-08-24 20:56:11.695','87fbe3f9-b1f0-c030-a4c0-8662045923b4')
|
||||||
[-19] -15498.5738 ('2097-08-02 18:34:16.406','cf97f268-02c0-24fc-bbf3-c7b272632c14')
|
[-58,87] 122510.9099 ('2019-08-09 17:40:29.849','c1d3a2cc-878f-c2c3-4a0b-10e98cda8b4a')
|
||||||
[116] -72670.9713 ('2020-08-31 18:10:41.904','f9cdd931-e2ed-0584-d4b9-67a6df717a4c')
|
[4,19,58] -13496.8672 ('2027-05-01 09:11:48.659','8996ae31-d670-cbfe-b735-b16b7c3b3476')
|
||||||
[] 124014.7040 ('1975-07-23 11:17:25.176','ccf33ba5-8fd8-c8b5-ccc4-a9cb892d4b55')
|
[23,-75,-89] -51218.2860 ('2010-06-02 02:49:03.396','d32b8b61-cc3e-31fa-2a2a-abefa60bfcee')
|
||||||
[-56] -204745.8115 ('2037-11-13 01:03:12.923','6dc83c7b-7782-57b4-a293-18ca8aba331d')
|
[50] -45297.4315 ('2087-04-15 06:46:08.247','04fe9603-97fc-07a4-6248-0f21e408c884')
|
||||||
[] -28535.2534 ('2105-04-07 20:51:09.990','0d9f3a2f-d4f2-a330-7b6e-001ea3aacbde')
|
[-23,17,63] 89185.9462 ('2065-10-26 08:27:12.817','a5fbf764-70b4-8b65-4a8f-7550abca3859')
|
||||||
[-124,-128] -31519.7583 ('1993-02-14 23:06:10.338','a073dafb-6f1f-273e-acf9-88200f82af6d')
|
[-6] -129925.3690 ('2013-11-05 07:44:45.233','11db26b3-e2b5-b9fa-6b0e-79c43a2e67ab')
|
||||||
[46] -154950.9257 ('2032-06-04 23:16:16.051','e6aa3b80-9f53-6c10-0cc8-622622f964b4')
|
[-72,-108] 203171.5475 ('2000-01-28 09:34:58.032','14d5399e-7949-20c7-0e47-85e2fce5836c')
|
||||||
[] 206914.3454 ('2003-10-05 10:44:30.786','137ed3be-2d40-d1c1-7aff-b32f7e21c0da')
|
[-73,34,-27] 2676.7265 ('2057-10-25 14:37:10.049','00049a92-4350-badb-3764-dd7f019b9b31')
|
||||||
[-47] 91521.1349 ('2006-09-01 04:06:32.496','52e4ef43-9379-4864-8f63-8e205875a096')
|
[65,-7] -153472.9461 ('1973-04-12 02:34:41.245','e0a0324d-1552-d11e-f3a5-fbd822d206c5')
|
||||||
[121] 161456.7813 ('2027-11-03 10:20:30.670','f1abbd17-f399-657c-1a47-1dd627578b53')
|
[] 81837.7838 ('2041-09-20 20:56:39.712','f7923f2c-e526-1706-79b9-58045d9deaa7')
|
||||||
[99] -127959.4741 ('2084-08-18 06:04:41.942','2a3b92c3-75ed-bd20-5a77-b77cbe1ce479')
|
[-113,8] 173192.6905 ('2066-04-02 09:59:59.356','e3013e5c-92e3-c03c-b57a-e1939e00a1a7')
|
||||||
[-97] 82020.4570 ('2061-10-25 06:16:50.814','8625d479-6e81-318f-5077-a9deb13c50e0')
|
[107] 9694.1102 ('1984-11-02 13:11:34.034','e973db18-07b7-2117-f3ba-e7002adfa939')
|
||||||
[71] -121599.1388 ('2010-04-02 11:05:18.877','0ec279cf-c9b2-dc65-40c0-2d0f390b1102')
|
[] -76460.9664 ('2051-02-10 09:54:42.143','b8344c22-9e8a-7052-c644-9c3e5989cdf1')
|
||||||
[] 98975.6469 ('2049-03-06 08:56:25.010','845340d7-a1df-9ddf-b737-9eb90ca6344c')
|
[59,59,0] 27041.7606 ('2083-02-17 18:21:22.547','4d6b137b-a3e1-f36d-2c0c-c8d718dda388')
|
||||||
[92,81] 135864.7854 ('2040-12-30 21:17:28.184','ea224755-198e-c9ae-c59b-0517a7459d7c')
|
[-114] 133673.9630 ('2005-10-02 20:34:27.452','04785b75-30e5-af8b-547e-d15bcb7f49fb')
|
||||||
[81] -154620.5037 ('1984-06-07 02:36:28.734','52d3b727-043f-1d43-6f48-51e8abdc2127')
|
[43] -169861.2000 ('2006-12-13 09:26:13.923','cb865d38-d961-d7f9-acbb-583b9f31252f')
|
||||||
[38] 33379.3375 ('2057-10-19 17:03:44.317','e709bfc2-0915-9e4e-4d01-c10b24795e30')
|
[] 197115.2174 ('2060-04-08 04:17:00.488','0f26c4b4-b24c-1fd5-c619-31bcf71a4831')
|
||||||
[] 7491.1071 ('1971-04-29 09:30:25.245','26bcd2ab-6d0b-fc20-27eb-084c4248af7d')
|
[-25] -200081.9506 ('2055-12-25 02:30:16.276','0b32ad69-2c84-4269-9718-e3171482878a')
|
||||||
[-122] -135635.3813 ('2010-03-04 23:05:25.982','66ed96eb-fc6e-653e-0353-ac4477ea60a6')
|
[14,110] -40196.4463 ('2084-08-13 19:37:07.588','ed882071-acba-b3ab-5d77-d79a9544a834')
|
||||||
[] -174748.4115 ('2020-10-28 07:39:33.461','e17fa9ba-2595-c0f9-2f85-d6bbdc2f6f6a')
|
[-62,-71,-82] -154958.9747 ('2100-07-08 02:32:53.741','7711c7c1-0d22-e302-fc86-61ef5e68db96')
|
||||||
[72,106] 25749.2190 ('2008-06-15 04:03:39.682','0e47b616-da80-091e-664d-2a35bc57a480')
|
[96,-114,-101] 78910.3320 ('2100-07-19 15:02:27.109','756bfd26-c4b3-94b8-e991-c7ab7a833b76')
|
||||||
[-84,97] 109277.9244 ('1998-10-27 10:40:00.442','9488bce4-46d7-8249-78aa-540b8be43937')
|
[49] 80117.2267 ('1970-07-04 03:50:56.748','aebac019-9054-4a77-2ccd-8801fc4a7496')
|
||||||
[-120,-107] -64113.5210 ('2091-12-03 06:46:11.903','325fcb1c-8552-b434-b349-732d62be19f1')
|
[] 102078.4801 ('2055-01-07 01:22:33.624','21f2e59a-a1ca-5df3-27fd-aa95456cfbe5')
|
||||||
[] -66141.6000 ('2085-10-05 08:08:11.830','4c66022b-75b9-b0a8-3897-b9de8ea851f1')
|
[-106] -108728.4237 ('2020-05-27 11:56:18.121','6b7b6674-9342-2360-4cc0-f7ef8a2404de')
|
||||||
[-34,-102] -142314.4437 ('2038-01-27 12:04:29.739','91e9eb11-5679-02ef-6ea6-2c9fdcb12ed9')
|
[] 173213.5631 ('2034-01-18 19:04:16.059','2dc0038d-67c1-f0ee-280b-f3f0f536b01a')
|
||||||
[103] 96187.7213 ('1978-10-07 13:57:43.616','7c02e8e3-9e98-5043-8029-34e32ad1af61')
|
[42] 139872.2503 ('2001-07-16 11:09:28.754','d6487da6-1077-1053-f314-9a1079f5df15')
|
||||||
[] -21344.8423 ('2085-01-13 00:10:52.538','52cb36f8-987a-f414-7e0f-93ddccc5c377')
|
[] 1107.5244 ('2031-02-26 15:06:00.846','b32bee8f-85b7-3c71-bb24-9a0093e6a08c')
|
||||||
[16] -95098.4107 ('2074-02-19 18:56:00.878','821e4b10-f70a-4bee-ef0c-ac12eab994f3')
|
[] 85892.8913 ('2088-04-13 14:54:18.514','84f3b59b-8d23-78a6-3032-91392344584f')
|
||||||
[21,86] 27954.7748 ('2033-10-18 03:15:38.815','bfe4d932-c5ed-45c0-9f50-72a6394d49af')
|
[43] -109644.2714 ('1974-07-04 14:45:43.139','cf722ca8-15f5-6fe2-997c-0cf88e95e902')
|
||||||
[] 149788.2085 ('2073-09-10 20:42:48.693','5e7d825e-5c88-7c89-4235-0e7934739a12')
|
[] 212557.3762 ('2069-03-03 07:21:08.439','9e676cac-36e6-2962-f7b1-578214f0dfbd')
|
||||||
[33,116] -148302.8732 ('2044-08-10 22:05:18.943','a53d4b07-5529-7472-3cca-3770f52b3648')
|
[-128,55] 80471.0777 ('1970-04-01 18:54:40.257','ca358854-416b-9c95-0b9b-c7fed7bb7cb5')
|
||||||
[] -98384.4505 ('2070-01-28 05:17:35.804','4833b839-51a3-87b8-7709-30676f697aa4')
|
[-30,-54] -132205.4512 ('2017-12-15 22:54:15.750','3558faa4-2d2f-c533-437f-1e03d3600f1d')
|
||||||
[] -75597.1523 ('2075-02-04 19:24:01.477','d64becff-5c08-b0a0-e7f1-b86eaf5f1913')
|
[-116,-72] -91499.6670 ('2105-09-23 21:06:17.755','07bb6e47-3234-c268-40d7-332388dc06f8')
|
||||||
[] 179005.6113 ('2100-05-27 21:54:12.965','d87ce81c-c471-b6b3-93b7-05225cb577be')
|
[] -201636.5228 ('2085-01-27 07:54:42.717','86c3bdc3-ff0f-1723-07c2-845aa3c02370')
|
||||||
[] -134366.9213 ('2054-11-16 18:19:00.801','c348fced-6700-f0f6-cda0-14aef7ea6948')
|
[-103,-39] 44330.7722 ('2064-07-02 11:08:28.068','0869c79d-6bdd-5d2d-a3d1-ffe13f6aa810')
|
||||||
[10] 82182.0343 ('2017-03-04 09:41:21.249','e19f0022-49ab-2d41-872d-be35669a79bc')
|
[99] -31035.5391 ('2093-07-26 01:50:23.026','aeb59338-254f-dc09-fbd7-263da415e211')
|
||||||
[-28] 90333.8564 ('2032-11-19 01:23:37.107','e2586be2-e968-21d0-d1b1-b438c55a59a3')
|
[101] 157961.4729 ('2036-05-04 02:35:07.845','8b6221a9-8dad-4655-7460-6b3031b06893')
|
||||||
[-73] 185647.6735 ('2001-01-23 16:20:26.442','24b04f39-f272-24ff-538d-41e636a1a37a')
|
[111] 84732.4403 ('1997-04-06 16:10:18.624','08806a79-59f4-c833-eedc-a200bb851767')
|
||||||
[-79,7] -87628.8007 ('2005-03-25 04:17:49.969','38a10e9d-7086-f358-8e50-c72b278bec42')
|
[9,-48] -190491.5590 ('2031-11-03 19:47:03.757','914e6166-c96e-e0e4-101a-0bb516cf5a2f')
|
||||||
[119,-55] -208591.8591 ('1976-11-14 15:17:57.569','d0935dc7-7f56-71db-67f2-1b4e52770ba9')
|
[-41] -132501.8311 ('2089-11-21 21:38:28.848','6de6cc8d-3c49-641e-fb12-87ed5ecb97b0')
|
||||||
[-108,-124] 181408.0349 ('2056-10-27 05:07:32.393','29d655c1-c35a-1245-25e2-65b4f233cb9c')
|
[77] 64903.6579 ('1985-04-17 17:08:03.998','26484b8a-f3f1-587f-7777-bc7a57a689c3')
|
||||||
-
|
-
|
||||||
|
@ -1,5 +1,5 @@
|
|||||||
DROP TABLE IF EXISTS test_table;
|
DROP TABLE IF EXISTS test_table;
|
||||||
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Generate();
|
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=GenerateRandom();
|
||||||
SELECT COUNT(*) FROM (SELECT * FROM test_table LIMIT 100);
|
SELECT COUNT(*) FROM (SELECT * FROM test_table LIMIT 100);
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test_table;
|
DROP TABLE IF EXISTS test_table;
|
||||||
@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test_table;
|
|||||||
SELECT '-';
|
SELECT '-';
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test_table_2;
|
DROP TABLE IF EXISTS test_table_2;
|
||||||
CREATE TABLE test_table_2(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Generate(3, 5, 10);
|
CREATE TABLE test_table_2(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=GenerateRandom(10, 5, 3);
|
||||||
|
|
||||||
SELECT * FROM test_table_2 LIMIT 100;
|
SELECT * FROM test_table_2 LIMIT 100;
|
||||||
|
|
||||||
|
@ -1,238 +1,238 @@
|
|||||||
UInt64 Int64 UInt32 Int32 UInt16 Int16 UInt8 Int8
|
UInt64 Int64 UInt32 Int32 UInt16 Int16 UInt8 Int8
|
||||||
5443401583997919274 956654340036924402 2956613447 2041372187 46025 26509 247 -34
|
2804162938822577320 -2776833771540858 3467776823 1163715250 23903 13655 137 -41
|
||||||
14051730854243326159 340055300607421421 579798001 915264595 58925 22498 36 -57
|
7885388429666205427 -1363628932535403038 484159052 -308788249 56810 -22227 51 -41
|
||||||
12126660396637528292 -9182366379883086416 535113873 -1583603936 45790 6066 230 91
|
4357435422797280898 1355609803008819271 4126129912 -852056475 64304 -11401 139 86
|
||||||
5198178071978083704 -3549936112074464250 3354362520 -1732019372 41330 -27737 13 -47
|
5935810273536892891 -804738887697332962 3109335413 -80126721 258 12889 18 88
|
||||||
9045663333607591872 -5069075924065328373 741246230 -1830932765 29642 -11720 41 7
|
368066018677693974 -4927165984347126295 1015254922 2026080544 44305 21973 16 0
|
||||||
18192666371709191624 -5005976579831091773 671021725 1851158245 38613 -27838 57 3
|
8124171311239967992 -1179703908046100129 1720727300 -138469036 61343 10573 252 -32
|
||||||
4333039311970693040 -7294587049092886539 2106347821 2101852759 24058 9107 85 94
|
15657812979985370729 -5733276247123822513 3254757884 -500590428 45913 19153 105 -102
|
||||||
1398111012802844853 1131449717368086026 1687614855 -1193084417 9803 -18141 198 115
|
18371568619324220532 -6793779541583578394 1686821450 -455892108 49050 -28603 248 80
|
||||||
15838944643191192696 6226099517671026657 1300309956 468322781 17216 -2375 184 -102
|
821735343441964030 3148260644406230976 256251035 -885069056 58858 -29361 58 61
|
||||||
15170414162889419078 3337938833953948518 3603117877 -1297530274 25534 8264 36 16
|
9558594037060121162 -2907172753635797124 4276198376 1947296644 26801 -13531 204 -66
|
||||||
-
|
-
|
||||||
Enum8(\'hello\' = 1, \'world\' = 5)
|
Enum8(\'hello\' = 1, \'world\' = 5)
|
||||||
|
hello
|
||||||
world
|
world
|
||||||
world
|
hello
|
||||||
world
|
world
|
||||||
hello
|
hello
|
||||||
hello
|
hello
|
||||||
world
|
world
|
||||||
world
|
|
||||||
world
|
|
||||||
hello
|
hello
|
||||||
world
|
hello
|
||||||
|
hello
|
||||||
-
|
-
|
||||||
Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))
|
Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))
|
||||||
['world','world','hello','hello','world','world','world']
|
['hello','hello','world','hello',NULL,'world','world','hello','hello','hello']
|
||||||
|
['world','hello']
|
||||||
|
['hello','hello','world','hello','world',NULL,NULL,NULL]
|
||||||
|
['world','world','hello','world','world','world',NULL,'hello','world','world']
|
||||||
|
[]
|
||||||
|
['hello','hello','world','hello','hello','world','hello','world']
|
||||||
|
['world',NULL,'world','world','hello','hello','world']
|
||||||
['world']
|
['world']
|
||||||
['world','hello','world']
|
['world','world']
|
||||||
[]
|
[NULL,'hello','hello','world','world','world','hello','world','world','world']
|
||||||
[]
|
|
||||||
['world','hello','hello','hello','world']
|
|
||||||
['hello']
|
|
||||||
['world','hello','hello','world','hello']
|
|
||||||
['hello','world','hello','hello','world','world']
|
|
||||||
['world','hello','world','hello','hello','world','world']
|
|
||||||
-
|
-
|
||||||
Nullable(Enum16(\'o\' = -200, \'h\' = 1, \'w\' = 5))
|
Nullable(Enum16(\'o\' = -200, \'h\' = 1, \'w\' = 5))
|
||||||
w
|
|
||||||
h
|
|
||||||
h
|
|
||||||
o
|
o
|
||||||
w
|
h
|
||||||
|
h
|
||||||
|
h
|
||||||
|
h
|
||||||
w
|
w
|
||||||
o
|
o
|
||||||
w
|
|
||||||
h
|
h
|
||||||
|
\N
|
||||||
o
|
o
|
||||||
-
|
-
|
||||||
Date DateTime DateTime(\'Europe/Moscow\')
|
Date DateTime DateTime(\'Europe/Moscow\')
|
||||||
2031-03-05 2034-09-09 02:49:47 2061-06-26 03:46:01
|
2106-02-07 2050-12-17 02:46:35 2096-02-16 22:18:22
|
||||||
1972-10-06 1999-01-02 11:09:55 2064-03-18 05:47:09
|
2106-02-07 2013-10-17 23:35:26 1976-01-24 12:52:48
|
||||||
2004-01-16 2055-12-02 15:29:20 2090-08-18 23:04:46
|
2039-08-16 1974-11-17 23:22:46 1980-03-04 21:02:50
|
||||||
2061-07-14 2051-03-20 20:58:44 1973-04-20 21:20:34
|
1997-04-11 1972-09-18 23:44:08 2040-07-10 14:46:42
|
||||||
2063-04-13 2048-01-31 01:02:11 2051-02-07 03:11:54
|
2103-11-03 2044-11-23 20:57:12 1970-10-09 02:30:14
|
||||||
2106-02-07 2028-08-29 13:37:25 2054-10-20 03:48:21
|
2066-11-19 2029-12-10 03:13:55 2106-01-30 21:52:44
|
||||||
2026-11-24 2036-08-09 02:59:19 2065-10-12 06:39:38
|
2064-08-14 2016-07-14 11:33:45 2096-12-12 00:40:50
|
||||||
2106-02-07 2068-04-17 13:07:59 2101-04-03 08:48:59
|
2046-09-13 2085-07-10 18:51:14 2096-01-15 16:31:33
|
||||||
1997-11-15 1984-11-03 12:39:41 1998-04-01 17:38:08
|
2008-03-16 2047-05-16 23:28:36 2103-02-11 16:44:39
|
||||||
2008-09-11 2064-12-25 16:23:42 2031-10-18 03:20:14
|
2000-07-07 2105-07-19 19:29:06 1980-01-02 05:18:22
|
||||||
-
|
-
|
||||||
DateTime64(3) DateTime64(6) DateTime64(6, \'Europe/Moscow\')
|
DateTime64(3) DateTime64(6) DateTime64(6, \'Europe/Moscow\')
|
||||||
1988-05-16 19:00:01.447 2064-03-18 05:47:09.972361 2104-06-20 09:26:44.845879
|
1978-06-07 23:50:57.320 2013-08-28 10:21:54.010758 1991-08-25 16:23:26.140215
|
||||||
2076-04-17 18:22:00.873 1973-04-20 21:20:34.769886 2052-08-01 07:14:05.921510
|
1978-08-25 17:07:25.427 2034-05-02 20:49:42.148578 2015-08-26 15:26:31.783160
|
||||||
1991-04-07 13:55:25.230 2054-10-20 03:48:21.341514 2013-02-07 18:37:45.437737
|
2037-04-04 10:50:56.898 2055-05-28 11:12:48.819271 2068-12-26 09:58:49.635722
|
||||||
2023-06-24 16:54:15.821 2101-04-03 08:48:59.544378 2039-07-05 08:51:02.770005
|
2041-09-02 07:07:24.891 2051-08-01 14:15:40.218654 2081-10-19 15:55:40.057084
|
||||||
2084-03-05 21:04:37.956 2031-10-18 03:20:14.437888 2076-03-16 14:08:20.993528
|
1976-07-15 23:59:41.974 2075-01-29 20:34:10.425321 1996-12-31 10:51:28.562331
|
||||||
1999-01-02 11:09:55.187 2054-01-01 16:49:22.580109 1997-01-09 20:11:35.889758
|
1974-11-03 08:09:51.992 2010-04-19 04:09:03.451487 1994-05-15 15:42:53.162162
|
||||||
2051-03-20 20:58:44.360 1975-02-11 06:38:15.042546 2015-10-21 23:47:13.191963
|
2061-10-11 20:14:02.729 1981-07-22 10:13:45.729103 2084-05-27 08:59:37.746021
|
||||||
2028-08-29 13:37:25.531 1975-02-14 07:25:38.319928 2103-09-16 20:57:23.033927
|
1989-12-13 02:01:16.532 1992-10-05 07:07:57.973222 2037-10-24 18:53:50.985504
|
||||||
2068-04-17 13:07:59.759 2024-03-06 21:42:43.711891 2045-04-22 19:38:11.140126
|
1992-12-28 12:26:04.030 1971-07-29 09:20:38.230976 1980-03-26 18:49:55.428516
|
||||||
2064-12-25 16:23:42.781 2025-08-18 15:44:56.149625 2093-09-26 16:30:56.744858
|
2051-12-11 10:09:13.162 1982-01-12 03:25:45.754492 2010-05-17 11:01:28.452864
|
||||||
-
|
-
|
||||||
Float32 Float64
|
Float32 Float64
|
||||||
2.3424705e38 5.304765772621186e307
|
-1.3551149e32 1.2262973812461839e235
|
||||||
4.5936326e37 1.3693852957827914e308
|
1.6263936e-15 -1.4319274895836525e122
|
||||||
4.2396088e37 1.1817811347484115e308
|
-8.991488e-33 -3.587091060722666e303
|
||||||
2.6575997e38 5.065787759860024e307
|
4.6137895e27 9.96990958623199e-254
|
||||||
5.8727575e37 8.815282962741328e307
|
9.749564e-13 -3.014080971435583e-286
|
||||||
5.3163816e37 1.7729324649694315e308
|
0.01518069 nan
|
||||||
1.6688205e38 4.2226828718895e307
|
-2.5833165e-24 -2.6774132404843463e217
|
||||||
1.3370661e38 1.3625030842560206e307
|
240769800000 4.559039863342969e-218
|
||||||
1.0302116e38 1.5435548915708008e308
|
2.0838264e-33 -6.156499824044965e254
|
||||||
2.8546838e38 1.4784044970034722e308
|
7.317837e-36 -1.6511853645079817e-21
|
||||||
-
|
-
|
||||||
Decimal32(4) Decimal64(8) Decimal64(8)
|
Decimal32(4) Decimal64(8) Decimal64(8)
|
||||||
-133835.3849 87676267830.44260947 10041303591043480341650.6377217747572943
|
-18731.5032 81241713112.39967992 -10576027963457111164764.0798899532879521
|
||||||
57979.8001 -68015271123.73929132 -11658496611537681782723.8256877955807880
|
65289.5061 -27889310937.24180887 5807515838469365530027.7612329616030438
|
||||||
53511.3873 -78637963449.98695195 16686303649199763212696.4854950355256776
|
-197586.1517 -751754543.85331084 3835903211857734974086.0358362773591932
|
||||||
-94060.4776 90273888640.14252543 7993046724924589483272.0796323974797493
|
183596.0063 8217353434.41964030 13633006218585943284268.9826084812209912
|
||||||
74124.6230 20114310313.64207198 -4810540869033768101015.4448286464595642
|
73041.2674 -88881500366.49430454 -148702703925022894263.3187064158377476
|
||||||
67102.1725 -60472921957.85611731 1764715777766465744700.9237855716355053
|
101454.4494 -27768337.71540858 -634829280961262229789.4961995996929358
|
||||||
210634.7821 -20967919098.37725326 -16938476260073815366594.8118263905360890
|
-174012.0101 -13636289325.35403038 -3611949395160064991369.2765012316944096
|
||||||
168761.4855 -74544559691.08355371 -9350794626143586522954.2962771754340925
|
138203.8526 13556098030.08819271 134470734594381953531.9736002591779584
|
||||||
130030.9956 -54650148153.48939189 -13456138041801265081736.4812607484010998
|
15395.1766 -8047388876.97332962 16804394201271843589306.4234533639925009
|
||||||
-69184.9419 38286965773.25360062 11485126437992390872631.7990315807376230
|
8569.7048 -49271659843.47126295 -14851374957489266092927.8687987539036841
|
||||||
-
|
-
|
||||||
Tuple(Int32, Int64)
|
Tuple(Int32, Int64)
|
||||||
(-1338353849,5443401583997919274)
|
(-187315032,8124171311239967992)
|
||||||
(579798001,-4395013219466225457)
|
(652895061,-2788931093724180887)
|
||||||
(535113873,-6320083677072023324)
|
(-1975861517,-75175454385331084)
|
||||||
(-940604776,5198178071978083704)
|
(1835960063,821735343441964030)
|
||||||
(741246230,9045663333607591872)
|
(730412674,-8888150036649430454)
|
||||||
(671021725,-254077702000359992)
|
(1014544494,-2776833771540858)
|
||||||
(2106347821,4333039311970693040)
|
(-1740120101,-1363628932535403038)
|
||||||
(1687614855,1398111012802844853)
|
(1382038526,1355609803008819271)
|
||||||
(1300309956,-2607799430518358920)
|
(153951766,-804738887697332962)
|
||||||
(-691849419,-3276329910820132538)
|
(85697048,-4927165984347126295)
|
||||||
-
|
-
|
||||||
Array(Int8)
|
Array(Int8)
|
||||||
[27,83,32,84,-29,-27,87]
|
[-122,110,114,1,124,34,-10,-1,-30,61]
|
||||||
[-1]
|
[-56,-18]
|
||||||
[-35,94,-55]
|
[-75,106,19,-19,71,-64,96,-6]
|
||||||
|
[-18,23,-48,18,30,-123,-37,-51,62,-2]
|
||||||
[]
|
[]
|
||||||
[]
|
[-44,-12,-23,5,-102,121,-92,48]
|
||||||
[45,-34,114,-54,-43]
|
[-97,-69,95,-87,-59,-60,-117]
|
||||||
[-6]
|
[-39]
|
||||||
[75,64,-66,-115,-30]
|
[-96,-17]
|
||||||
[-78,-89,56,66,-109,35]
|
[79,-8,-1,34,-95,79,111,-80,-26,58]
|
||||||
[-71,72,-9,36,-26,13,41]
|
|
||||||
-
|
-
|
||||||
Array(Nullable(Int32))
|
Array(Nullable(Int32))
|
||||||
[2041372187,915264595,-1583603936,-1732019372,-1830932765,1851158245,2101852759]
|
[24276614,-646532,-288866846,-317494603,-94322617,NULL,-841251554,-187367874,2040137193,-1147195228]
|
||||||
[-1193084417]
|
[-993679009,-274671221]
|
||||||
[468322781,-1297530274,-1407994935]
|
[587200591,-1334882399,2107128550,-1581800064,976027584,733011552,-1898440836,-676878904]
|
||||||
|
[-827190473,NULL,-168837384,-1185631883,1015254922,1720727300,-1040209412,1686821450,256251035,-18768920]
|
||||||
[]
|
[]
|
||||||
[]
|
[1163715250,-308788249,-852056475,NULL,2026080544,-138469036,-500590428,-455892108]
|
||||||
[-1321933267,-488197410,104178034,-1735625782,-1618897195]
|
[-885069056,1947296644,-571843233,16972592,-274748143,-1080380583,1756489194]
|
||||||
[-1272422918]
|
[2121012739]
|
||||||
[-153016757,891437888,1950049214,6580109,-1644079134]
|
[NULL,NULL]
|
||||||
[790042546,161321895,1074319928,161583938,515711891,1709750563]
|
[NULL,-1874507055,-886731441,821482880,311112585,-127271920,873843770,NULL,1482086359,1352327168]
|
||||||
[-149817671,1755521096,815845879,-51580892,1361921510,-1688868851,-1185529559]
|
|
||||||
-
|
-
|
||||||
Tuple(Int32, Array(Int64))
|
Tuple(Int32, Array(Int64))
|
||||||
(-1338353849,[5443401583997919274,-4395013219466225457,-6320083677072023324,5198178071978083704,9045663333607591872,-254077702000359992,4333039311970693040])
|
(-187315032,[-1179703908046100129,-5733276247123822513,-6793779541583578394,3148260644406230976,-2907172753635797124,2079447297870140215,-5092250158453768456,7390467479849635722])
|
||||||
(579798001,[1398111012802844853,-2607799430518358920,-3276329910820132538,956654340036924402,340055300607421421])
|
(652895061,[7244842965196057084,-80611897324989285,-1326235429680389454,-344141642787805595,-594719979102566112,-1958041690570123100,8363575405000452864])
|
||||||
(535113873,[])
|
(-1975861517,[72896731291475295])
|
||||||
(-940604776,[-9182366379883086416,-3549936112074464250,-5069075924065328373,-5005976579831091773])
|
(1835960063,[-4640199267198194415,9109680350160872938])
|
||||||
(741246230,[-7294587049092886539])
|
(730412674,[3628163485716526423,-8050946496653339179,3528242107232128335,-546628733788015735,-4538379399781299142,5808200961534384087,808655365001887293,-8844185874661444452,643019962728680518,8250477176286620863])
|
||||||
(671021725,[1131449717368086026,6226099517671026657,3337938833953948518,-104956130729581604,515805789944032293])
|
(1014544494,[-287233184700985380,1749427670542803376,-6435907283441313909,-398230413002921126,7257457516659393153,1409595563647451721,-946856126400551895,-8238390188888204749,8805947767553724527])
|
||||||
(2106347821,[2731028582309582302,-8197314279937271385,7439592879615992239,-8726726222408049230,-4046170041070917399,-8162695179087422573,7147712321550951494,-2473105312361834401,2871941532606538254])
|
(-1740120101,[4566168555161068712,2303771336793744574,-2858308093688847501,-674013896865039545,597627259745635607,1493493065813843889])
|
||||||
(1687614855,[7045950974355040215,8128475529675984757,3862453874713979777,8584893221699499395,-4344095019439049735,7221768832555831190,5712009283210486481,8657278465574644253,-4620821897447975309])
|
(1382038526,[8163231169061670909])
|
||||||
(1300309956,[-3580736586972265629])
|
(153951766,[-8934224507121259258,-5682982952279561296,-7665466011309768105,1158449958889177529,-5943902981558064139])
|
||||||
(-691849419,[7980379733974797651,-548434416689229144])
|
(85697048,[2092132020040612180,-7829994932198211729,5992776369613298329,-2557419325779681965,-2080080576758631926,1226592045800496749,1791516122484114661,-6512855691472390097,764717459130233392,4903403636828862838])
|
||||||
-
|
-
|
||||||
Nullable(String)
|
Nullable(String)
|
||||||
;\\Sm\'sH
|
fSRH40d6sX
|
||||||
T
|
|
||||||
@^1
|
|
||||||
|
|
||||||
|
=@ep]Vw~
|
||||||
7-f)$
|
b\'
|
||||||
9
|
6xGwg|(&Q
|
||||||
)&}y3
|
^ipx|,=a5N
|
||||||
w5>+\'@
|
(U]p\'l`
|
||||||
+g+N^g$
|
U6
|
||||||
|
\'%Y~t9
|
||||||
|
RL,{Xs\\tw
|
||||||
-
|
-
|
||||||
Array(String)
|
Array(String)
|
||||||
['Y9n(%ub','\\\'f%7','','X0O@','D','4^,~q','a(gmt6#{X']
|
['6xGwg|(&Q','^ipx|,=a5N','(U]p\'l`','U6','\'%Y~t9','RL,{Xs\\tw','`xbguF','?/;UTko','k3Igp@','']
|
||||||
['@#q4?Q%\'.']
|
['TlL','UeS}D']
|
||||||
['h','#B','{']
|
['0Z3|h','-]','&b!M-e;7','Dj7peUH{T','=D[','_@}a ','_}!','O,9V']
|
||||||
|
['r;5qbK&t+','4?a]25n#','_K','4S9,;m','RM nh|E7*','-L[3','','Fm?\'','/D$','.7^Jp5sba$']
|
||||||
[]
|
[]
|
||||||
[]
|
['CB','TaI&szh','Hnc?lApSP','2O"ms26O>','bX?}ix [','UlI+1','U','NQTpY#']
|
||||||
['~vYP/4f9.',':,7u.0',';e.<','^O,i','3']
|
['8+>','# ;M<:{M','t<iT','','p','dqC','D[6,P']
|
||||||
['!y1/Z\'5D']
|
['B5']
|
||||||
['&- KDN%>[','>-xM./ B','?+//','M,.71QR#_','~N']
|
['{dOzr.rb*q','aw7*u}BD']
|
||||||
['z9P/%m','7q\'!k','Q%] #.*3','U:&XeP{*',',','s.3']
|
['Mmj)WC\'','3Sb','',';URVOy\\','VE]','Z','Ti~3)N)<','Sx>b:^UG','?AaqQ?$Ee\'','PbE.6x]^']
|
||||||
['+k.=%','8\'nb=P','-uY ,h8(w','=\'W$','','m<+%l','<~+@ Vw']
|
|
||||||
-
|
-
|
||||||
UUID
|
UUID
|
||||||
4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf
|
26ea6355-f4d5-cca8-6d6e-86ff8a3abef3
|
||||||
a84a8d61-8560-a0e4-4823-a36a537a8578
|
3c78b86e-2b89-3682-5260-3bfe9847dfdb
|
||||||
7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8
|
051ba218-092d-1e16-70be-dac38ce0b4f8
|
||||||
3c220c4b-fbd0-efb0-1367-168f02acd8b5
|
d94bb9f2-5787-4e69-fef4-ec529188a474
|
||||||
dbcf3c3c-127c-bc78-d288-234e0a942946
|
0b6763f0-ed3d-ebfe-84a6-f2a21322b24a
|
||||||
0d46b80d-bebc-93f2-04b8-1e7e84b1abed
|
fff6227c-0172-6e86-ed13-6ab5eec83de2
|
||||||
8091ae6d-8194-3db0-cebc-17ea18786406
|
12d017ee-fa60-c047-f4d4-fe3ecddb851e
|
||||||
b9a7064d-de99-e30b-ba87-32d2cd3e2dc3
|
bb9f30a4-799a-05e9-efa0-d98bc4c5a95f
|
||||||
9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a
|
b06f4fa1-22ff-f84f-a1b7-a5807d983ae6
|
||||||
56678c86-2703-2fe1-2e52-bdaf2fce8366
|
2bb0de60-3a2c-ffc0-d7a7-a5c88ed8177c
|
||||||
-
|
-
|
||||||
Array(Nullable(UUID))
|
Array(Nullable(UUID))
|
||||||
['4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf','a84a8d61-8560-a0e4-4823-a36a537a8578','7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8','3c220c4b-fbd0-efb0-1367-168f02acd8b5','dbcf3c3c-127c-bc78-d288-234e0a942946','0d46b80d-bebc-93f2-04b8-1e7e84b1abed','8091ae6d-8194-3db0-cebc-17ea18786406']
|
['fff6227c-0172-6e86-ed13-6ab5eec83de2','12d017ee-fa60-c047-f4d4-fe3ecddb851e','bb9f30a4-799a-05e9-efa0-d98bc4c5a95f','b06f4fa1-22ff-f84f-a1b7-a5807d983ae6','2bb0de60-3a2c-ffc0-d7a7-a5c88ed8177c','1cdbae4c-ceb2-1337-b954-b175f5efbef8','66903704-3c83-8f8a-648a-da4ac1ffa9fc','fee19be8-0f46-149b-ed98-43e7455ce2b2','fb395cff-cd36-a665-f7bf-215478c38920','e4d3a374-e229-98a4-7411-6384cb3eeb00']
|
||||||
['b9a7064d-de99-e30b-ba87-32d2cd3e2dc3']
|
[NULL,'7e6c1603-68b1-e5ea-3259-d377a92d3557']
|
||||||
['9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a','56678c86-2703-2fe1-2e52-bdaf2fce8366','fe8b1ef4-86dd-23dc-0728-82a212b42c25']
|
['90454ad1-294d-55d5-30f6-d580cb258d4f','f869fc10-128b-3389-c104-6ffb3415cc3a','509ae000-5856-d7d7-0b38-ebc520a6be3d','854323ce-3546-f49c-08ec-773fcded8c46','727f9547-6808-2cbf-fc03-8af0ba318fdc','18473732-8ef1-61b0-a6af-106cb321978b','fa79338c-9e34-2b5a-64b7-ab28e4f8c281',NULL]
|
||||||
|
['8dab5bc5-a641-5a33-7a35-02d6b3af106f','3f5e4a1b-baef-24a8-1ff8-a3d32d5164be','d8553fee-c90d-6373-f6a5-6bee8faebb47','084b32d3-8d3a-8d17-14b9-f407ce57dfb1','71499f7f-6351-4ffd-8403-42257572ad06','b121fd1d-58c6-abb0-959e-cb46ae0d0257','1013a419-91a7-c1b9-ad83-03bafcd8abf5','1d08befc-d5bd-b954-9356-451821377f6f','532a9e5b-6927-da99-dc82-38ac492edd53','e32211bc-761c-b20a-1105-baf786421e6d']
|
||||||
[]
|
[]
|
||||||
[]
|
['18dcbe6b-ea0d-04e5-a59d-b0415468382f','0a9cd27a-38ff-3230-440c-63a0db8af976','dff6ee08-7533-ac1f-3f4b-a6450dffffb3','20fc45e0-8b0d-3652-1ef9-6a69cfab4ad8','09f997b7-ff78-73cf-d2e1-014279fc877b','3e96646a-f80c-c4b5-b6ea-ac9de22a1c1d','b9804d77-da87-b21e-33e8-a148791c24c3','4763e2bd-c2ec-9d95-8e8d-78d0fdcd6100']
|
||||||
['25e69006-a800-55de-8e3d-4a17f81a19a7','673ebe4e-af09-61af-86e4-70bca5481db2','c7d91dc9-0123-e8e9-8eb8-47fb80b35b93','6331c67c-7aba-2446-ddad-c3d24c8a985f','27db2f9b-92b4-220e-61c8-3f14833fe7d7']
|
['150955fb-d228-6700-f1ee-9c8579cfdf10',NULL,NULL,'706b1793-0b9b-b007-e892-8bb3f13ed5aa','d48ec551-5c15-5f80-09cf-5f3a09b4232e','11f16f65-b07b-8ca6-9037-c400f4cee06f','fd5cb198-44c1-631c-545a-b493d3456ca4']
|
||||||
['70ce256d-7fca-cf75-359a-3155bba86b81']
|
['8978d7e4-5b0b-ce2d-c714-66d3728f8226']
|
||||||
['7723aae6-820e-b583-c3b6-ac9887cab3f9','6438e07d-9a7a-4f96-4f45-222df77ea2d1','7824d4e5-6e37-2a1d-bfdf-8af768445673','ce4eab0b-b346-0363-6ec0-0116104b4d53','f86391cd-c2a3-d2a8-ea85-27ed3508504c']
|
['716309e3-bc54-8fae-d159-077e7f58328a','80413bd9-30b3-e071-8079-0f198e1952bf']
|
||||||
['b87d39f3-e56c-7128-7a66-4e516e2ce1c0','af25bac3-d662-673a-3516-022e687643ed','657c9997-3b29-f51c-7193-6a3b9a0c18eb','19bb38b5-6f97-a81c-2d56-57f189119a1a','0c7e416d-c669-dc04-1130-ff950fbbf44b','9705bc44-8d08-c734-6f47-8edcc1608a81']
|
['a55486af-06c3-4d55-fb6e-54559812b2c0','9f93002e-efb6-c6fd-3928-154e7bbe640b','1f197019-cde5-8fe3-16c0-45d857e32f5d','f2532c84-e28e-d4b8-89a5-335907448f4e','09c22b07-180f-4acb-d898-8d6c84667cf3','d11f8791-0962-9a60-d28c-2f3763793435','9aa152ff-1f38-4172-909e-a75c2f6cd244','7a5e4b17-0178-0a6d-7955-f6e5488d4da2','2bf6aea1-9cb5-1726-8dac-c7144c65e4a9','bfddcefd-68fa-85d8-a550-87605cd540cc']
|
||||||
['7c20103f-659a-f845-399c-abdc8dc88ba0','bf479e85-1a0f-66c3-66fa-f6029e4ee2a8','f14af1a0-823c-b414-eb8e-e6b05b019868','ce353e45-2a9e-492c-1c54-d50459160ecf','1e232279-77ad-db7e-82f6-b4b3e30cdc2e','991111af-30a3-1ff7-e15a-023dfa0a8a6e','a749ef39-dc02-d05c-e8b2-129a7cccfd24']
|
|
||||||
-
|
-
|
||||||
FixedString(4)
|
FixedString(4)
|
||||||
Gñ‘˜
|
A8CCD5F4
|
||||||
<16>-‡
|
5563EA26
|
||||||
Ä5S
|
F3BE3A8A
|
||||||
Tãå
|
FF866E6D
|
||||||
WÿÝ^
|
8236892B
|
||||||
É-Þr
|
6EB8783C
|
||||||
ÊÕúK
|
DBDF4798
|
||||||
@¾<>â
|
FE3B6052
|
||||||
²§8B
|
161E2D09
|
||||||
“#¹H
|
18A21B05
|
||||||
-
|
-
|
||||||
String
|
String
|
||||||
;\\Sm\'sH
|
fSRH40d6sX
|
||||||
T
|
|
||||||
@^1
|
|
||||||
|
|
||||||
|
=@ep]Vw~
|
||||||
7-f)$
|
b\'
|
||||||
9
|
6xGwg|(&Q
|
||||||
)&}y3
|
^ipx|,=a5N
|
||||||
w5>+\'@
|
(U]p\'l`
|
||||||
+g+N^g$
|
U6
|
||||||
|
\'%Y~t9
|
||||||
|
RL,{Xs\\tw
|
||||||
-
|
-
|
||||||
[27] -119308.4417 ('1998-04-01 17:38:08.539','4b8ad8e6-77fe-a02a-c301-c6c33e91d8cf')
|
[] -27467.1221 ('2021-03-08 03:39:14.331','08ec773f-cded-8c46-727f-954768082cbf')
|
||||||
[83] 46832.2781 ('1970-03-18 06:48:29.214','a84a8d61-8560-a0e4-4823-a36a537a8578')
|
[] 204013.7193 ('2026-05-05 05:20:23.160','30f6d580-cb25-8d4f-f869-fc10128b3389')
|
||||||
[32] -129753.0274 ('1995-01-14 03:15:46.162','7d88a6e0-3d74-cfc0-fc79-55ad1aebc5c8')
|
[-122] -9432.2617 ('2001-08-23 08:05:41.222','f7bf2154-78c3-8920-e4d3-a374e22998a4')
|
||||||
[] -140799.4935 ('2004-01-17 09:12:08.895','3c220c4b-fbd0-efb0-1367-168f02acd8b5')
|
[-30,61] -133488.2399 ('2048-05-14 09:05:06.021','a6af106c-b321-978b-fa79-338c9e342b5a')
|
||||||
[] -132193.3267 ('1986-05-06 01:18:11.938','dbcf3c3c-127c-bc78-d288-234e0a942946')
|
[-1] 58720.0591 ('1976-06-07 23:26:18.162','fc038af0-ba31-8fdc-1847-37328ef161b0')
|
||||||
[84] -48819.7410 ('2101-05-10 09:27:05.563','0d46b80d-bebc-93f2-04b8-1e7e84b1abed')
|
[1] -18736.7874 ('1977-03-10 04:41:16.215','3259d377-a92d-3557-9045-4ad1294d55d5')
|
||||||
[-29] 10417.8034 ('1995-11-08 18:51:19.096','8091ae6d-8194-3db0-cebc-17ea18786406')
|
[34,-10] -99367.9009 ('2031-05-08 10:00:41.084','0b38ebc5-20a6-be3d-8543-23ce3546f49c')
|
||||||
[-27] -173562.5782 ('2013-02-27 03:31:50.404','b9a7064d-de99-e30b-ba87-32d2cd3e2dc3')
|
[110] 31562.7502 ('2045-02-27 11:46:14.976','74116384-cb3e-eb00-0102-fb30ddea5d5f')
|
||||||
[] -161889.7195 ('2068-07-13 23:42:17.445','9ac46bba-2ba7-4ff5-0fb3-b785f4f0de0a')
|
[114] -84125.1554 ('2023-06-06 06:55:06.492','bf9ab359-ef9f-ad11-7e6c-160368b1e5ea')
|
||||||
[87] -127242.2918 ('2033-08-04 15:06:45.865','56678c86-2703-2fe1-2e52-bdaf2fce8366')
|
[124] -114719.5228 ('2010-11-11 22:57:23.722','c1046ffb-3415-cc3a-509a-e0005856d7d7')
|
||||||
-
|
-
|
||||||
[] 3608695403 ZL 109414.2847 h 2.2986075276244747e306 ('1985-05-10','2009-10-28 20:06:11','1993-01-03 17:51:52.981','b13ff007-c245-d737-85b2-1fa003e57127') .ü
|
[] 1900051923 { -189530.5846 h -5.6279699579452485e47 ('1984-12-06','2028-08-17 06:05:01','2036-04-02 23:52:28.468','4b3d498c-dd44-95c1-5b75-921504ec5d8d') F743
|
||||||
[85] 4204173796 ], -199466.5471 h 1.1231803213254798e308 ('2075-04-03','1983-02-12 23:57:05','2060-06-06 20:15:08.751','a2f2cbf4-b11b-6976-7b91-14b6964acbe2') *þ
|
[-102,-118] 392272782 Eb -14818.0200 o -2.664492247169164e59 ('2082-12-26','2052-09-09 06:50:50','2088-04-21 05:07:08.245','aeb9c26e-0ee7-2b8e-802b-2a96319b8e60') CBF4
|
||||||
[-94,100] 32713522 8D$ 102255.5602 h 1.738807291208415e308 ('2029-07-12','2056-08-07 23:18:32','2081-01-25 13:13:30.589','445a77b5-0a27-3485-8dd8-c7cc35d2692f') á’
|
[-71] 775049089 \N -158115.1178 w 4.1323844687113747e-305 ('2106-02-07','2090-07-31 16:45:26','2076-07-10 09:11:06.385','57c69bc6-dddd-0975-e932-a7b5173a1304') EB1D
|
||||||
[] 4117557956 0b>+ 65942.4942 w 5.949505844751135e307 ('2048-03-05','2074-01-22 02:32:44','2073-12-04 05:05:06.955','c12095e6-b82c-d81c-4629-acd80e02b080') <19>
|
[-28,100] 3675466147 { -146685.1749 h 3.6676044396877755e142 ('2017-10-25','2100-02-28 18:07:18','2055-10-14 06:36:20.056','14949dae-dfa8-a124-af83-887348b2f609') 6D88
|
||||||
[] 1511604199 Il= -96352.6064 o 1.6472659147355216e308 ('2024-06-01','2024-12-26 00:54:40','2038-04-14 05:21:44.387','ebbbe70a-0321-ff18-89de-2bc9a9e4c454') ¶Q
|
[-23] 2514120753 (`u, -119659.6174 w 1.3231258347475906e34 ('2106-02-07','2074-08-10 06:25:12','1976-12-04 18:31:55.745','86a9b3c1-4593-4d56-7762-3aa1dd22cbbf') AD43
|
||||||
[-18] 2278197196 ~ 193977.7666 o 1.213689191969361e308 ('2060-10-04','1992-10-24 16:31:53','1983-06-10 08:51:48.294','805b0a62-9ada-a47e-2d5e-63cb5923549c') \tÃ
|
[11,-36] 3308237300 \N 171205.1896 \N 5.634708707075817e195 ('1974-10-31','1993-12-24 09:38:45','2038-07-15 05:22:51.805','63d999b8-8cca-e237-c4a4-4dd7d0096f65') 609E
|
||||||
[] 3761265784 N"(6 -59230.0369 o 1.2102282609858645e308 ('2106-02-07','2060-07-09 20:14:59','2007-03-17 04:51:09.288','429df3a4-ff18-28d5-9ad8-dcdd78e8b1ae') Y|
|
[39] 1614362420 `4A8P 157144.0630 o -1.1843143253872814e-255 ('2106-02-07','2072-09-28 18:27:27','2073-07-10 12:19:58.146','6483f5c0-8733-364c-4fa0-9948d32e8903') A886
|
||||||
[] 66606254 6x&+ 130635.2269 o 1.1958868988757417e308 ('2088-10-07','2070-03-01 21:30:45','1978-05-22 14:28:52.523','d63c5cbb-9418-ce59-000c-056f88157bfa') у
|
[48,-120] 3848918261 1<Lu3 91487.2852 h -1.9300793134783347e263 ('2050-12-04','2076-04-05 09:33:05','2103-12-13 23:48:44.066','e522b794-b8fa-3f11-003b-3b6b088ff941') 556E
|
||||||
[-27,-12] 4089193163 )+.8 -111081.7896 o 1.464035857434812e308 ('2106-02-07','2007-04-27 23:04:36','1987-07-21 04:32:01.821','2bc4860a-7214-300a-851e-b61011c346ef') #Õ
|
[55] 3047524030 li&lF 93462.3661 h 2.8979254388809897e54 ('1976-01-10','1987-07-14 00:25:51','2021-11-19 04:44:08.986','486e5b26-5fe8-fe3e-12ef-09aee40643e0') 9E75
|
||||||
[14,-43] 3638976325 #" 116961.4294 o 9.260305126207595e307 ('2042-06-11','2087-12-28 00:21:16','2071-04-01 21:44:13.058','c00d218a-913f-b657-1ff9-99927741f7ab') Fx
|
[100,-42] 3999367674 -112975.9852 h 2.658098863752086e-160 ('2081-05-13','2071-08-07 13:34:33','1980-11-11 12:00:44.669','9754e8ac-5145-befb-63d9-a12dd1cf1f3a') DF63
|
||||||
-
|
-
|
||||||
|
@ -3,176 +3,176 @@ SELECT
|
|||||||
toTypeName(ui32), toTypeName(i32),
|
toTypeName(ui32), toTypeName(i32),
|
||||||
toTypeName(ui16), toTypeName(i16),
|
toTypeName(ui16), toTypeName(i16),
|
||||||
toTypeName(ui8), toTypeName(i8)
|
toTypeName(ui8), toTypeName(i8)
|
||||||
FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8')
|
FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
ui64, i64,
|
ui64, i64,
|
||||||
ui32, i32,
|
ui32, i32,
|
||||||
ui16, i16,
|
ui16, i16,
|
||||||
ui8, i8
|
ui8, i8
|
||||||
FROM generate('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 10, 10, 1)
|
FROM generateRandom('ui64 UInt64, i64 Int64, ui32 UInt32, i32 Int32, ui16 UInt16, i16 Int16, ui8 UInt8, i8 Int8', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)')
|
FROM generateRandom('i Enum8(\'hello\' = 1, \'world\' = 5)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Enum8(\'hello\' = 1, \'world\' = 5)', 10, 10, 1)
|
FROM generateRandom('i Enum8(\'hello\' = 1, \'world\' = 5)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))')
|
FROM generateRandom('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 10, 10, 1)
|
FROM generateRandom('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)s
|
toTypeName(i)s
|
||||||
FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))')
|
FROM generateRandom('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 10, 10, 1)
|
FROM generateRandom('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)))', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(d), toTypeName(dt), toTypeName(dtm)
|
toTypeName(d), toTypeName(dt), toTypeName(dtm)
|
||||||
FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')')
|
FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
d, dt, dtm
|
d, dt, dtm
|
||||||
FROM generate('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 10, 10, 1)
|
FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(dt64), toTypeName(dts64), toTypeName(dtms64)
|
toTypeName(dt64), toTypeName(dts64), toTypeName(dtms64)
|
||||||
FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')')
|
FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
dt64, dts64, dtms64
|
dt64, dts64, dtms64
|
||||||
FROM generate('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 10, 10, 1)
|
FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(f32), toTypeName(f64)
|
toTypeName(f32), toTypeName(f64)
|
||||||
FROM generate('f32 Float32, f64 Float64')
|
FROM generateRandom('f32 Float32, f64 Float64')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
f32, f64
|
f32, f64
|
||||||
FROM generate('f32 Float32, f64 Float64', 10, 10, 1)
|
FROM generateRandom('f32 Float32, f64 Float64', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(d32), toTypeName(d64), toTypeName(d64)
|
toTypeName(d32), toTypeName(d64), toTypeName(d64)
|
||||||
FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)')
|
FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
d32, d64, d128
|
d32, d64, d128
|
||||||
FROM generate('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 10, 10, 1)
|
FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Tuple(Int32, Int64)')
|
FROM generateRandom('i Tuple(Int32, Int64)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Tuple(Int32, Int64)', 10, 10, 1)
|
FROM generateRandom('i Tuple(Int32, Int64)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Array(Int8)')
|
FROM generateRandom('i Array(Int8)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Array(Int8)', 10, 10, 1)
|
FROM generateRandom('i Array(Int8)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Array(Nullable(Int32))')
|
FROM generateRandom('i Array(Nullable(Int32))')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Array(Nullable(Int32))', 10, 10, 1)
|
FROM generateRandom('i Array(Nullable(Int32))', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Tuple(Int32, Array(Int64))')
|
FROM generateRandom('i Tuple(Int32, Array(Int64))')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Tuple(Int32, Array(Int64))', 10, 10, 1)
|
FROM generateRandom('i Tuple(Int32, Array(Int64))', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Nullable(String)', 1)
|
FROM generateRandom('i Nullable(String)', 1)
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Nullable(String)', 10, 10, 1)
|
FROM generateRandom('i Nullable(String)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Array(String)')
|
FROM generateRandom('i Array(String)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Array(String)', 10, 10, 1)
|
FROM generateRandom('i Array(String)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
|
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i UUID')
|
FROM generateRandom('i UUID')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i UUID', 10, 10, 1)
|
FROM generateRandom('i UUID', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i Array(Nullable(UUID))')
|
FROM generateRandom('i Array(Nullable(UUID))')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i Array(Nullable(UUID))', 10, 10, 1)
|
FROM generateRandom('i Array(Nullable(UUID))', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i FixedString(4)')
|
FROM generateRandom('i FixedString(4)')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
hex(i)
|
||||||
FROM generate('i FixedString(4)', 10, 10, 1)
|
FROM generateRandom('i FixedString(4)', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
SELECT
|
SELECT
|
||||||
toTypeName(i)
|
toTypeName(i)
|
||||||
FROM generate('i String')
|
FROM generateRandom('i String')
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
SELECT
|
SELECT
|
||||||
i
|
i
|
||||||
FROM generate('i String', 10, 10, 1)
|
FROM generateRandom('i String', 1, 10, 10)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
DROP TABLE IF EXISTS test_table;
|
DROP TABLE IF EXISTS test_table;
|
||||||
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Memory;
|
CREATE TABLE test_table(a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)) ENGINE=Memory;
|
||||||
INSERT INTO test_table SELECT * FROM generate('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 2, 10, 1)
|
INSERT INTO test_table SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 1, 10, 2)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
|
|
||||||
SELECT * FROM test_table;
|
SELECT * FROM test_table ORDER BY a, d, c;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test_table;
|
DROP TABLE IF EXISTS test_table;
|
||||||
|
|
||||||
@ -180,10 +180,10 @@ SELECT '-';
|
|||||||
|
|
||||||
DROP TABLE IF EXISTS test_table_2;
|
DROP TABLE IF EXISTS test_table_2;
|
||||||
CREATE TABLE test_table_2(a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)) ENGINE=Memory;
|
CREATE TABLE test_table_2(a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16('h' = 1, 'w' = 5 , 'o' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)) ENGINE=Memory;
|
||||||
INSERT INTO test_table_2 SELECT * FROM generate('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)', 3, 5, 10)
|
INSERT INTO test_table_2 SELECT * FROM generateRandom('a Array(Int8), b UInt32, c Nullable(String), d Decimal32(4), e Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200)), f Float64, g Tuple(Date, DateTime, DateTime64, UUID), h FixedString(2)', 10, 5, 3)
|
||||||
LIMIT 10;
|
LIMIT 10;
|
||||||
|
|
||||||
SELECT * FROM test_table_2;
|
SELECT a, b, c, d, e, f, g, hex(h) FROM test_table_2 ORDER BY a, b, c, d, e, f, g, h;
|
||||||
SELECT '-';
|
SELECT '-';
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test_table_2;
|
DROP TABLE IF EXISTS test_table_2;
|
||||||
|
@ -1,4 +1,4 @@
|
|||||||
# docker build -t yandex/clickhouse-stateful-test .
|
# docker build -t yandex/clickhouse-stateful-test-with-coverage .
|
||||||
FROM yandex/clickhouse-stateless-test
|
FROM yandex/clickhouse-stateless-test
|
||||||
|
|
||||||
RUN echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 main" >> /etc/apt/sources.list
|
RUN echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 main" >> /etc/apt/sources.list
|
||||||
|
@ -71,7 +71,11 @@ start_clickhouse
|
|||||||
|
|
||||||
sleep 5
|
sleep 5
|
||||||
|
|
||||||
/s3downloader --dataset-names $DATASETS
|
if ! /s3downloader --dataset-names $DATASETS; then
|
||||||
|
echo "Cannot download datatsets"
|
||||||
|
exit 1
|
||||||
|
fi
|
||||||
|
|
||||||
|
|
||||||
chmod 777 -R /var/lib/clickhouse
|
chmod 777 -R /var/lib/clickhouse
|
||||||
|
|
||||||
|
@ -1,6 +1,6 @@
|
|||||||
# Generate {#table_engines-generate}
|
# GenerateRandom {#table_engines-generate}
|
||||||
|
|
||||||
The Generate table engine produces random data for given table schema.
|
The GenerateRandom table engine produces random data for given table schema.
|
||||||
|
|
||||||
Usage examples:
|
Usage examples:
|
||||||
|
|
||||||
@ -10,7 +10,7 @@ Usage examples:
|
|||||||
## Usage in ClickHouse Server
|
## Usage in ClickHouse Server
|
||||||
|
|
||||||
```sql
|
```sql
|
||||||
Generate(max_array_length, max_string_length, random_seed)
|
ENGINE = GenerateRandom(random_seed, max_string_length, max_array_length)
|
||||||
```
|
```
|
||||||
|
|
||||||
The `max_array_length` and `max_string_length` parameters specify maximum length of all
|
The `max_array_length` and `max_string_length` parameters specify maximum length of all
|
||||||
@ -25,7 +25,7 @@ It supports all [DataTypes](../../data_types/index.md) that can be stored in a t
|
|||||||
**1.** Set up the `generate_engine_table` table:
|
**1.** Set up the `generate_engine_table` table:
|
||||||
|
|
||||||
```sql
|
```sql
|
||||||
CREATE TABLE generate_engine_table (name String, value UInt32) ENGINE=Generate(3, 5, 1)
|
CREATE TABLE generate_engine_table (name String, value UInt32) ENGINE = GenerateRandom(1, 5, 3)
|
||||||
```
|
```
|
||||||
|
|
||||||
**2.** Query the data:
|
**2.** Query the data:
|
||||||
|
@ -1,11 +1,11 @@
|
|||||||
# generate
|
# generateRandom
|
||||||
|
|
||||||
Generates random data with given schema.
|
Generates random data with given schema.
|
||||||
Allows to populate test tables with data.
|
Allows to populate test tables with data.
|
||||||
Supports all data types that can be stored in table except `LowCardinality` and `AggregateFunction`.
|
Supports all data types that can be stored in table except `LowCardinality` and `AggregateFunction`.
|
||||||
|
|
||||||
```sql
|
```sql
|
||||||
generate('name TypeName[, name TypeName]...', 'limit'[, 'max_array_length'[, 'max_string_length'[, 'random_seed']]]);
|
generateRandom('name TypeName[, name TypeName]...', [, 'random_seed'[, 'max_string_length'[, 'max_array_length']]]);
|
||||||
```
|
```
|
||||||
|
|
||||||
**Parameters**
|
**Parameters**
|
||||||
@ -15,7 +15,7 @@ generate('name TypeName[, name TypeName]...', 'limit'[, 'max_array_length'[, 'ma
|
|||||||
- `limit` — Number of rows to generate.
|
- `limit` — Number of rows to generate.
|
||||||
- `max_array_length` — Maximum array length for all generated arrays. Defaults to `10`.
|
- `max_array_length` — Maximum array length for all generated arrays. Defaults to `10`.
|
||||||
- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`.
|
- `max_string_length` — Maximum string length for all generated strings. Defaults to `10`.
|
||||||
- `random_seed` — Specify random seed manually to produce stable results. Defaults to `0` — seed is randomly generated.
|
- `random_seed` — Specify random seed manually to produce stable results. If NULL — seed is randomly generated.
|
||||||
|
|
||||||
**Returned Value**
|
**Returned Value**
|
||||||
|
|
||||||
@ -25,7 +25,7 @@ A table object with requested schema.
|
|||||||
|
|
||||||
|
|
||||||
```sql
|
```sql
|
||||||
SELECT * FROM generate('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 3, 2, 10, 1);
|
SELECT * FROM generateRandom('a Array(Int8), d Decimal32(4), c Tuple(DateTime64(3), UUID)', 1, 10, 2);
|
||||||
```
|
```
|
||||||
```text
|
```text
|
||||||
┌─a────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
|
┌─a────────┬────────────d─┬─c──────────────────────────────────────────────────────────────────┐
|
||||||
|
@ -75,7 +75,7 @@ nav:
|
|||||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||||
- 'Memory': 'operations/table_engines/memory.md'
|
- 'Memory': 'operations/table_engines/memory.md'
|
||||||
- 'Buffer': 'operations/table_engines/buffer.md'
|
- 'Buffer': 'operations/table_engines/buffer.md'
|
||||||
- 'Generate': 'operations/table_engines/generate.md'
|
- 'GenerateRandom': 'operations/table_engines/generate.md'
|
||||||
|
|
||||||
- 'Database Engines':
|
- 'Database Engines':
|
||||||
- 'Introduction': 'database_engines/index.md'
|
- 'Introduction': 'database_engines/index.md'
|
||||||
@ -144,7 +144,7 @@ nav:
|
|||||||
- 'odbc': 'query_language/table_functions/odbc.md'
|
- 'odbc': 'query_language/table_functions/odbc.md'
|
||||||
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
||||||
- 'input': 'query_language/table_functions/input.md'
|
- 'input': 'query_language/table_functions/input.md'
|
||||||
- 'generate': 'query_language/table_functions/generate.md'
|
- 'generateRandom': 'query_language/table_functions/generate.md'
|
||||||
- 'Dictionaries':
|
- 'Dictionaries':
|
||||||
- 'Introduction': 'query_language/dicts/index.md'
|
- 'Introduction': 'query_language/dicts/index.md'
|
||||||
- 'External Dictionaries':
|
- 'External Dictionaries':
|
||||||
|
@ -109,7 +109,7 @@ nav:
|
|||||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||||
- 'Memory': 'operations/table_engines/memory.md'
|
- 'Memory': 'operations/table_engines/memory.md'
|
||||||
- 'Buffer': 'operations/table_engines/buffer.md'
|
- 'Buffer': 'operations/table_engines/buffer.md'
|
||||||
- 'Generate': 'operations/table_engines/generate.md'
|
- 'GenerateRandom': 'operations/table_engines/generate.md'
|
||||||
|
|
||||||
- 'SQL Reference':
|
- 'SQL Reference':
|
||||||
- 'hidden': 'query_language/index.md'
|
- 'hidden': 'query_language/index.md'
|
||||||
@ -171,7 +171,7 @@ nav:
|
|||||||
- 'odbc': 'query_language/table_functions/odbc.md'
|
- 'odbc': 'query_language/table_functions/odbc.md'
|
||||||
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
||||||
- 'input': 'query_language/table_functions/input.md'
|
- 'input': 'query_language/table_functions/input.md'
|
||||||
- 'generate': 'query_language/table_functions/generate.md'
|
- 'generateRandom': 'query_language/table_functions/generate.md'
|
||||||
- 'Dictionaries':
|
- 'Dictionaries':
|
||||||
- 'Introduction': 'query_language/dicts/index.md'
|
- 'Introduction': 'query_language/dicts/index.md'
|
||||||
- 'External Dictionaries':
|
- 'External Dictionaries':
|
||||||
|
@ -79,7 +79,8 @@ nav:
|
|||||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||||
- 'Memory': 'operations/table_engines/memory.md'
|
- 'Memory': 'operations/table_engines/memory.md'
|
||||||
- 'Buffer': 'operations/table_engines/buffer.md'
|
- 'Buffer': 'operations/table_engines/buffer.md'
|
||||||
- 'Generate': 'operations/table_engines/generate.md'
|
- 'GenerateRandom': 'operations/table_engines/generate.md'
|
||||||
|
|
||||||
- 'SQL Reference':
|
- 'SQL Reference':
|
||||||
- 'hidden': 'query_language/index.md'
|
- 'hidden': 'query_language/index.md'
|
||||||
- 'Syntax': 'query_language/syntax.md'
|
- 'Syntax': 'query_language/syntax.md'
|
||||||
@ -142,7 +143,8 @@ nav:
|
|||||||
- 'odbc': 'query_language/table_functions/odbc.md'
|
- 'odbc': 'query_language/table_functions/odbc.md'
|
||||||
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
||||||
- 'input': 'query_language/table_functions/input.md'
|
- 'input': 'query_language/table_functions/input.md'
|
||||||
- 'generate': 'query_language/table_functions/generate.md'
|
- 'generateRandom': 'query_language/table_functions/generate.md'
|
||||||
|
|
||||||
- 'Dictionaries':
|
- 'Dictionaries':
|
||||||
- 'Introduction': 'query_language/dicts/index.md'
|
- 'Introduction': 'query_language/dicts/index.md'
|
||||||
- 'External Dictionaries':
|
- 'External Dictionaries':
|
||||||
|
@ -80,7 +80,7 @@ nav:
|
|||||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||||
- 'Memory': 'operations/table_engines/memory.md'
|
- 'Memory': 'operations/table_engines/memory.md'
|
||||||
- 'Buffer': 'operations/table_engines/buffer.md'
|
- 'Buffer': 'operations/table_engines/buffer.md'
|
||||||
- 'Generate': 'operations/table_engines/generate.md'
|
- 'GenerateRandom': 'operations/table_engines/generate.md'
|
||||||
|
|
||||||
- 'Справка по SQL':
|
- 'Справка по SQL':
|
||||||
- 'hidden': 'query_language/index.md'
|
- 'hidden': 'query_language/index.md'
|
||||||
@ -144,7 +144,7 @@ nav:
|
|||||||
- 'odbc': 'query_language/table_functions/odbc.md'
|
- 'odbc': 'query_language/table_functions/odbc.md'
|
||||||
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
||||||
- 'input': 'query_language/table_functions/input.md'
|
- 'input': 'query_language/table_functions/input.md'
|
||||||
- 'generate': 'query_language/table_functions/generate.md'
|
- 'generateRandom': 'query_language/table_functions/generate.md'
|
||||||
- 'Словари':
|
- 'Словари':
|
||||||
- 'Введение': 'query_language/dicts/index.md'
|
- 'Введение': 'query_language/dicts/index.md'
|
||||||
- 'Внешние словари':
|
- 'Внешние словари':
|
||||||
|
@ -109,7 +109,7 @@ nav:
|
|||||||
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
- 'MaterializedView': 'operations/table_engines/materializedview.md'
|
||||||
- 'Memory': 'operations/table_engines/memory.md'
|
- 'Memory': 'operations/table_engines/memory.md'
|
||||||
- 'Buffer': 'operations/table_engines/buffer.md'
|
- 'Buffer': 'operations/table_engines/buffer.md'
|
||||||
- 'Generate': 'operations/table_engines/generate.md'
|
- 'GenerateRandom': 'operations/table_engines/generate.md'
|
||||||
|
|
||||||
- 'SQL语法':
|
- 'SQL语法':
|
||||||
- 'hidden': 'query_language/index.md'
|
- 'hidden': 'query_language/index.md'
|
||||||
@ -171,7 +171,7 @@ nav:
|
|||||||
- 'odbc': 'query_language/table_functions/odbc.md'
|
- 'odbc': 'query_language/table_functions/odbc.md'
|
||||||
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
- 'hdfs': 'query_language/table_functions/hdfs.md'
|
||||||
- 'input': 'query_language/table_functions/input.md'
|
- 'input': 'query_language/table_functions/input.md'
|
||||||
- 'generate': 'query_language/table_functions/generate.md'
|
- 'generateRandom': 'query_language/table_functions/generate.md'
|
||||||
- '字典':
|
- '字典':
|
||||||
- '介绍': 'query_language/dicts/index.md'
|
- '介绍': 'query_language/dicts/index.md'
|
||||||
- '外部字典':
|
- '外部字典':
|
||||||
|
Loading…
Reference in New Issue
Block a user