mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Add new function generateRandomStructure
This commit is contained in:
parent
dee3156e0c
commit
067bfb8844
@ -2609,3 +2609,73 @@ Result:
|
||||
│ 286 │
|
||||
└──────────────────────────┘
|
||||
```
|
||||
|
||||
## generateRandomStructure
|
||||
|
||||
Generates random table structure in a format `column1_name column1_type, column2_name column2_type, ...`.
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
generateRandomStructure([number_of_columns, seed])
|
||||
```
|
||||
|
||||
**Arguments**
|
||||
|
||||
- `number_of_columns` — The desired number of columns in the result table structure. If set to 0, the number of columns will be random from 1 to 128. Default value - 0.
|
||||
- `seed` - Random seed to produce stable results. If seed is not specified, it is randomly generated.
|
||||
|
||||
All arguments must be constant.
|
||||
|
||||
**Returned value**
|
||||
|
||||
- Randomly generated table structure.
|
||||
|
||||
Type: [String](../../sql-reference/data-types/string.md).
|
||||
|
||||
**Examples**
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT generateRandomStructure()
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─generateRandomStructure()─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
|
||||
│ c1 Decimal32(5), c2 Date, c3 Tuple(LowCardinality(String), Int128, UInt64, UInt16, UInt8, IPv6), c4 Array(UInt128), c5 UInt32, c6 IPv4, c7 Decimal256(64), c8 Decimal128(3), c9 UInt256, c10 UInt64, c11 DateTime │
|
||||
└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT generateRandomStructure(1)
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─generateRandomStructure(1)─┐
|
||||
│ c1 Map(UInt256, UInt16) │
|
||||
└────────────────────────────┘
|
||||
```
|
||||
|
||||
Query:
|
||||
|
||||
``` sql
|
||||
SELECT generateRandomStructure(0, 11)
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
``` text
|
||||
┌─generateRandomStructure(0, 11)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
|
||||
│ c1 Date32, c2 String, c3 IPv6, c4 DateTime, c5 UInt16, c6 Tuple(e1 UInt32, e2 Date, e3 Date, e4 IPv6, e5 Nested(e1 DateTime, e2 FixedString(110), e3 Int256, e4 Array(Decimal64(4)), e5 Decimal128(18), e6 Enum16('v0' = 0, 'v1' = 1, 'v2' = 2, 'v3' = 3, 'v4' = 4)), e6 DateTime64(4)), c7 DateTime, c8 DateTime64(6), c9 Bool │
|
||||
└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
This function can be used together with [generateRandom](../../sql-reference/table-functions/generate.md) to generate completely random tables.
|
||||
|
||||
|
@ -53,5 +53,19 @@ SELECT * FROM random;
|
||||
└──────────────────────────────┴──────────────┴────────────────────────────────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
In combination with [generateRandomStructure](../../sql-reference/functions/other-functions.md#generateRandomStructure):
|
||||
|
||||
```sql
|
||||
SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 3;
|
||||
```
|
||||
|
||||
```text
|
||||
┌─────────────────────────c1─┬─────c2─┬───────────────────c3─┬───────────────────────────────────────c4─┐
|
||||
│ 2085-07-05 23:48:43.345759 │ -20656 │ 1632406185424686785 │ -210464718903845545171230673454802.15238 │
|
||||
│ 1971-07-17 16:32:36.390777 │ -27071 │ -1553021742787219162 │ 1095158319964381336405161704296125.08074 │
|
||||
│ 2024-02-19 13:14:32.902513 │ 24913 │ 7727442383333447640 │ 1090748832613398997057187200834127.07109 │
|
||||
└────────────────────────────┴────────┴──────────────────────┴──────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
## Related content
|
||||
- Blog: [Generating random data in ClickHouse](https://clickhouse.com/blog/generating-random-test-distribution-data-for-clickhouse)
|
||||
|
@ -71,7 +71,7 @@ std::string extractTableName(const std::string & nested_name)
|
||||
}
|
||||
|
||||
|
||||
Block flatten(const Block & block)
|
||||
static Block flattenImpl(const Block & block, bool flatten_named_tuple)
|
||||
{
|
||||
Block res;
|
||||
|
||||
@ -114,7 +114,7 @@ Block flatten(const Block & block)
|
||||
else
|
||||
res.insert(elem);
|
||||
}
|
||||
else if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(elem.type.get()))
|
||||
else if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(elem.type.get()); type_tuple && flatten_named_tuple)
|
||||
{
|
||||
if (type_tuple->haveExplicitNames())
|
||||
{
|
||||
@ -143,6 +143,17 @@ Block flatten(const Block & block)
|
||||
return res;
|
||||
}
|
||||
|
||||
Block flatten(const Block & block)
|
||||
{
|
||||
return flattenImpl(block, true);
|
||||
}
|
||||
|
||||
|
||||
Block flattenArrayOfTuples(const Block & block)
|
||||
{
|
||||
return flattenImpl(block, false);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
|
@ -23,6 +23,9 @@ namespace Nested
|
||||
/// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ...
|
||||
Block flatten(const Block & block);
|
||||
|
||||
/// Same as flatten but only for Array with named Tuple element column.
|
||||
Block flattenArrayOfTuples(const Block & block);
|
||||
|
||||
/// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column.
|
||||
NamesAndTypesList collect(const NamesAndTypesList & names_and_types);
|
||||
|
||||
|
343
src/Functions/generateRandomStructure.cpp
Normal file
343
src/Functions/generateRandomStructure.cpp
Normal file
@ -0,0 +1,343 @@
|
||||
#include "config.h"
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
class FunctionGenerateRandomStructure : public IFunction
|
||||
{
|
||||
private:
|
||||
enum class SimpleTypes
|
||||
{
|
||||
Int8,
|
||||
UInt8,
|
||||
Bool,
|
||||
Int16,
|
||||
UInt16,
|
||||
Int32,
|
||||
UInt32,
|
||||
Int64,
|
||||
UInt64,
|
||||
Int128,
|
||||
UInt128,
|
||||
Int256,
|
||||
UInt256,
|
||||
Float32,
|
||||
Float64,
|
||||
DateTime64,
|
||||
Decimal32,
|
||||
Decimal64,
|
||||
Decimal128,
|
||||
Decimal256,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
String,
|
||||
FixedString,
|
||||
Enum8,
|
||||
Enum16,
|
||||
IPv4,
|
||||
IPv6,
|
||||
};
|
||||
|
||||
enum class ComplexTypes
|
||||
{
|
||||
Nullable,
|
||||
LowCardinality,
|
||||
Array,
|
||||
Tuple,
|
||||
Map,
|
||||
Nested,
|
||||
};
|
||||
|
||||
enum class MapKeyTypes
|
||||
{
|
||||
Int8,
|
||||
UInt8,
|
||||
Bool,
|
||||
Int16,
|
||||
UInt16,
|
||||
Int32,
|
||||
UInt32,
|
||||
Int64,
|
||||
UInt64,
|
||||
Int128,
|
||||
UInt128,
|
||||
Int256,
|
||||
UInt256,
|
||||
Date,
|
||||
Date32,
|
||||
DateTime,
|
||||
String,
|
||||
FixedString,
|
||||
};
|
||||
|
||||
static constexpr size_t MAX_NUMBER_OF_COLUMNS = 128;
|
||||
static constexpr size_t MAX_TUPLE_ELEMENTS = 16;
|
||||
static constexpr size_t MAX_DATETIME64_PRECISION = 9;
|
||||
static constexpr size_t MAX_DECIMAL32_PRECISION = 9;
|
||||
static constexpr size_t MAX_DECIMAL64_PRECISION = 18;
|
||||
static constexpr size_t MAX_DECIMAL128_PRECISION = 38;
|
||||
static constexpr size_t MAX_DECIMAL256_PRECISION = 76;
|
||||
static constexpr size_t MAX_DEPTH = 32;
|
||||
|
||||
public:
|
||||
static constexpr auto name = "generateRandomStructure";
|
||||
|
||||
static FunctionPtr create(ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_shared<FunctionGenerateRandomStructure>();
|
||||
}
|
||||
|
||||
String getName() const override { return name; }
|
||||
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
|
||||
bool isVariadic() const override { return true; }
|
||||
bool isDeterministic() const override { return false; }
|
||||
bool isDeterministicInScopeOfQuery() const override { return false; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() > 2)
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 0, 1 or 2.",
|
||||
getName(), arguments.size());
|
||||
|
||||
if (arguments.size() > 1 && !isUnsignedInteger(arguments[0]))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of the first argument of function {}, expected unsigned integer",
|
||||
arguments[0]->getName(),
|
||||
getName());
|
||||
}
|
||||
|
||||
if (arguments.size() > 2 && !isUnsignedInteger(arguments[1]))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of the second argument of function {}, expected unsigned integer",
|
||||
arguments[1]->getName(),
|
||||
getName());
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return false; }
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override
|
||||
{
|
||||
size_t seed = randomSeed();
|
||||
size_t number_of_columns = 0;
|
||||
|
||||
if (!arguments.empty())
|
||||
{
|
||||
const auto & first_arg = arguments[0];
|
||||
|
||||
if (!isUnsignedInteger(first_arg.type))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of the first argument of function {}, expected unsigned integer",
|
||||
first_arg.type->getName(),
|
||||
getName());
|
||||
|
||||
number_of_columns = first_arg.column->getUInt(0);
|
||||
if (number_of_columns > MAX_NUMBER_OF_COLUMNS)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Maximum allowed number of columns is {}, got {}", MAX_NUMBER_OF_COLUMNS, number_of_columns);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
const auto & second_arg = arguments[1];
|
||||
|
||||
if (!isUnsignedInteger(second_arg.type))
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of the second argument of function {}, expected unsigned integer",
|
||||
second_arg.type->getName(),
|
||||
getName());
|
||||
|
||||
seed = second_arg.column->getUInt(0);
|
||||
}
|
||||
}
|
||||
|
||||
pcg64 rng(seed);
|
||||
if (number_of_columns == 0)
|
||||
number_of_columns = generateNumberOfColumns(rng);
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
String generated_structure = "";
|
||||
for (size_t i = 0; i != number_of_columns; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
generated_structure += ", ";
|
||||
auto type = generateRandomType(rng);
|
||||
generated_structure += "c" + std::to_string(i + 1) + " " + type;
|
||||
}
|
||||
col_res->insert(generated_structure);
|
||||
return ColumnConst::create(std::move(col_res), input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
size_t generateNumberOfColumns(pcg64 & rng) const
|
||||
{
|
||||
return rng() % MAX_NUMBER_OF_COLUMNS + 1;
|
||||
}
|
||||
|
||||
String generateRandomType(pcg64 & rng, bool allow_complex_types = true, size_t depth = 0) const
|
||||
{
|
||||
constexpr size_t simple_types_size = magic_enum::enum_count<SimpleTypes>();
|
||||
constexpr size_t complex_types_size = magic_enum::enum_count<ComplexTypes>();
|
||||
size_t type_index;
|
||||
if (allow_complex_types)
|
||||
type_index = rng() % (simple_types_size + complex_types_size);
|
||||
else
|
||||
type_index = rng() % simple_types_size;
|
||||
|
||||
if (type_index < simple_types_size)
|
||||
{
|
||||
auto type = magic_enum::enum_value<SimpleTypes>(type_index);
|
||||
switch (type)
|
||||
{
|
||||
case SimpleTypes::FixedString:
|
||||
return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")";
|
||||
case SimpleTypes::DateTime64:
|
||||
return "DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION) + ")";
|
||||
case SimpleTypes::Decimal32:
|
||||
return "Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION) + ")";
|
||||
case SimpleTypes::Decimal64:
|
||||
return "Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION) + ")";
|
||||
case SimpleTypes::Decimal128:
|
||||
return "Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION) + ")";
|
||||
case SimpleTypes::Decimal256:
|
||||
return "Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION) + ")";
|
||||
case SimpleTypes::Enum8:
|
||||
return "Enum8(" + generateEnumValues(rng) + ")";
|
||||
case SimpleTypes::Enum16:
|
||||
return "Enum16(" + generateEnumValues(rng) + ")";
|
||||
default:
|
||||
return String(magic_enum::enum_name<SimpleTypes>(type));
|
||||
}
|
||||
}
|
||||
|
||||
auto complex_type = magic_enum::enum_value<ComplexTypes>(type_index - simple_types_size);
|
||||
switch (complex_type)
|
||||
{
|
||||
case ComplexTypes::LowCardinality:
|
||||
return "LowCardinality(" + generateLowCardinalityNestedType(rng) + ")";
|
||||
case ComplexTypes::Nullable:
|
||||
return "Nullable(" + generateRandomType(rng, false, depth + 1) + ")";
|
||||
case ComplexTypes::Array:
|
||||
return "Array(" + generateRandomType(rng, true, depth + 1) + ")";
|
||||
case ComplexTypes::Map:
|
||||
return "Map(" + generateMapKeyType(rng) + ", " + generateRandomType(rng, true, depth + 1) + ")";
|
||||
case ComplexTypes::Tuple:
|
||||
{
|
||||
size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1;
|
||||
bool named_tuple = rng() % 2;
|
||||
String tuple_type = "Tuple(";
|
||||
for (size_t i = 0; i != elements; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
tuple_type += ", ";
|
||||
if (named_tuple)
|
||||
tuple_type += "e" + std::to_string(i + 1) + " ";
|
||||
tuple_type += generateRandomType(rng, true, depth + 1);
|
||||
}
|
||||
return tuple_type + ")";
|
||||
}
|
||||
case ComplexTypes::Nested:
|
||||
{
|
||||
size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1;
|
||||
String nested_type = "Nested(";
|
||||
for (size_t i = 0; i != elements; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
nested_type += ", ";
|
||||
nested_type += "e" + std::to_string(i + 1) + " " + generateRandomType(rng, true, depth + 1);
|
||||
}
|
||||
return nested_type + ")";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String generateMapKeyType(pcg64 & rng) const
|
||||
{
|
||||
constexpr size_t map_keys_types_size = magic_enum::enum_count<MapKeyTypes>();
|
||||
auto type = magic_enum::enum_value<MapKeyTypes>(rng() % map_keys_types_size);
|
||||
if (type == MapKeyTypes::FixedString)
|
||||
return "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")";
|
||||
return String(magic_enum::enum_name<MapKeyTypes>(type));
|
||||
}
|
||||
|
||||
String generateLowCardinalityNestedType(pcg64 & rng) const
|
||||
{
|
||||
/// Support only String and FixedString.
|
||||
String nested_type;
|
||||
if (rng() % 2)
|
||||
nested_type = "String";
|
||||
else
|
||||
nested_type = "FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")";
|
||||
return rng() % 2 ? nested_type : "Nullable(" + nested_type + ")";
|
||||
}
|
||||
|
||||
String generateEnumValues(pcg64 & rng) const
|
||||
{
|
||||
/// Don't generate big enums, because it will lead to really big strings
|
||||
/// and slowness of this function, and it can lead to `Max query size exceeded`
|
||||
/// while using this function with generateRandom.
|
||||
ssize_t num_values = rng() % 16 + 1;
|
||||
String result;
|
||||
for (ssize_t i = 0; i != num_values; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
result += ", ";
|
||||
result += "'v" + std::to_string(i) + "' = " + std::to_string(i);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
REGISTER_FUNCTION(GenerateRandomStructure)
|
||||
{
|
||||
factory.registerFunction<FunctionGenerateRandomStructure>(
|
||||
{
|
||||
R"(
|
||||
Generates a random table structure.
|
||||
This function takes an optional constant argument, the number of column in the result structure.
|
||||
If argument is now specified, the number of columns is random. The maximum number of columns is 1024.
|
||||
The function returns a value of type String.
|
||||
)",
|
||||
Documentation::Examples{
|
||||
{"random", "SELECT generateRandomStructure()"},
|
||||
{"with specified number of arguments", "SELECT generateRandomStructure(10)"}},
|
||||
Documentation::Categories{"Random"}
|
||||
},
|
||||
FunctionFactory::CaseSensitive);
|
||||
}
|
||||
|
||||
}
|
@ -422,7 +422,7 @@ class GenerateSource : public ISource
|
||||
{
|
||||
public:
|
||||
GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_)
|
||||
: ISource(Nested::flatten(prepareBlockToFill(block_header_)))
|
||||
: ISource(Nested::flattenArrayOfTuples(prepareBlockToFill(block_header_)))
|
||||
, block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_)
|
||||
, block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {}
|
||||
|
||||
@ -437,7 +437,7 @@ protected:
|
||||
for (const auto & elem : block_to_fill)
|
||||
columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context));
|
||||
|
||||
columns = Nested::flatten(block_to_fill.cloneWithColumns(columns)).getColumns();
|
||||
columns = Nested::flattenArrayOfTuples(block_to_fill.cloneWithColumns(columns)).getColumns();
|
||||
return {std::move(columns), block_size};
|
||||
}
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <TableFunctions/TableFunctionGenerateRandom.h>
|
||||
#include <Interpreters/parseColumnsListForTableFunction.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
@ -28,7 +29,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||
}
|
||||
|
||||
void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/)
|
||||
void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context)
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
|
||||
@ -45,6 +46,9 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co
|
||||
"Table function '{}' requires at most four arguments: "
|
||||
" structure, [random_seed, max_string_length, max_array_length].", getName());
|
||||
|
||||
/// Allow constant expression for structure argument, it can be generated using generateRandomStructure function.
|
||||
args[0] = evaluateConstantExpressionAsLiteral(args[0], context);
|
||||
|
||||
// All the arguments must be literals.
|
||||
for (const auto & arg : args)
|
||||
{
|
||||
|
14
tests/queries/0_stateless/02584_compressor_codecs.reference
Normal file
14
tests/queries/0_stateless/02584_compressor_codecs.reference
Normal file
@ -0,0 +1,14 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
34
tests/queries/0_stateless/02584_compressor_codecs.sh
Executable file
34
tests/queries/0_stateless/02584_compressor_codecs.sh
Executable file
@ -0,0 +1,34 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
echo "Hello, World!" > 02584_test_data
|
||||
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Delta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Delta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Delta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Delta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out';
|
||||
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'DoubleDelta(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out';
|
||||
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Gorilla' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Gorilla([1,2])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'Gorilla(4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out';
|
||||
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'FPC' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'FPC(5)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "BAD_ARGUMENTS";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 1)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'FPC([1,2,3])' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_CODEC_PARAMETER";
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'FPC(5, 4)' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out';
|
||||
|
||||
|
||||
$CLICKHOUSE_COMPRESSOR --codec 'T64' --codec 'LZ4' --input '02584_test_data' --output '02584_test_out' 2>&1 | grep -c "ILLEGAL_SYNTAX_FOR_CODEC_TYPE";
|
||||
|
||||
rm 02584_test_data 02584_test_out
|
||||
|
@ -0,0 +1,4 @@
|
||||
c1 Int256, c2 Bool, c3 Int16, c4 Map(Int64, Array(Bool)), c5 Enum16(\'v0\' = 0, \'v1\' = 1, \'v2\' = 2, \'v3\' = 3, \'v4\' = 4, \'v5\' = 5, \'v6\' = 6, \'v7\' = 7, \'v8\' = 8, \'v9\' = 9, \'v10\' = 10)
|
||||
String
|
||||
Const(String)
|
||||
2085-07-05 23:48:43.345759 10105 1535011673144902513
|
@ -0,0 +1,10 @@
|
||||
select generateRandomStructure(5, 42);
|
||||
select toTypeName(generateRandomStructure(5, 42));
|
||||
select toColumnTypeName(generateRandomStructure(5, 42));
|
||||
SELECT * FROM generateRandom(generateRandomStructure(3, 24), 24) LIMIT 1;
|
||||
|
||||
select generateRandomStructure(5, 42, 42); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH}
|
||||
select generateRandomStructure('5'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select generateRandomStructure(5, '42'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT}
|
||||
select generateRandomStructure(materialize(5), 42); -- {serverError ILLEGAL_COLUMN}
|
||||
select generateRandomStructure(5, materialize(42)); -- {serverError ILLEGAL_COLUMN}
|
Loading…
Reference in New Issue
Block a user