mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Fix build for aarch64-darwin
This commit is contained in:
parent
37e4c531ba
commit
b23afdc533
@ -103,11 +103,19 @@ set (SRCS_CONTEXT
|
||||
)
|
||||
|
||||
if (ARCH_AARCH64)
|
||||
set (SRCS_CONTEXT ${SRCS_CONTEXT}
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/jump_arm64_aapcs_elf_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/make_arm64_aapcs_elf_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/ontop_arm64_aapcs_elf_gas.S"
|
||||
)
|
||||
if (OS_DARWIN)
|
||||
set (SRCS_CONTEXT ${SRCS_CONTEXT}
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/jump_arm64_aapcs_macho_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/make_arm64_aapcs_macho_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/ontop_arm64_aapcs_macho_gas.S"
|
||||
)
|
||||
else()
|
||||
set (SRCS_CONTEXT ${SRCS_CONTEXT}
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/jump_arm64_aapcs_elf_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/make_arm64_aapcs_elf_gas.S"
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/ontop_arm64_aapcs_elf_gas.S"
|
||||
)
|
||||
endif()
|
||||
elseif (ARCH_PPC64LE)
|
||||
set (SRCS_CONTEXT ${SRCS_CONTEXT}
|
||||
"${LIBRARY_DIR}/libs/context/src/asm/jump_ppc64_sysv_elf_gas.S"
|
||||
|
@ -79,8 +79,8 @@ public:
|
||||
ERROR = 4,
|
||||
};
|
||||
#endif
|
||||
static FiberInfo getCurrentFiberInfo();
|
||||
|
||||
static FiberInfo getCurrentFiberInfo();
|
||||
protected:
|
||||
/// Method that is called in resume() before actual fiber resuming.
|
||||
/// If it returns false, resume() will return immediately without actual fiber resuming.
|
||||
@ -124,6 +124,48 @@ private:
|
||||
std::unique_ptr<AsyncTask> task;
|
||||
};
|
||||
|
||||
/// Simple implementation for fiber local variable.
|
||||
template <typename T>
|
||||
struct FiberLocal
|
||||
{
|
||||
public:
|
||||
FiberLocal()
|
||||
{
|
||||
/// Initialize main instance for this thread.
|
||||
/// Contexts for fibers will inherit this instance
|
||||
/// (it could be changed before creating fibers).
|
||||
data[nullptr] = T();
|
||||
}
|
||||
|
||||
T & operator*()
|
||||
{
|
||||
return get();
|
||||
}
|
||||
|
||||
T * operator->()
|
||||
{
|
||||
return &get();
|
||||
}
|
||||
|
||||
private:
|
||||
T & get()
|
||||
{
|
||||
/// Get instance for current fiber.
|
||||
return getInstanceForFiber(AsyncTaskExecutor::getCurrentFiberInfo());
|
||||
}
|
||||
|
||||
T & getInstanceForFiber(FiberInfo info)
|
||||
{
|
||||
auto it = data.find(info.fiber);
|
||||
/// If it's the first request, we need to initialize instance for the fiber using instance from parent fiber.
|
||||
if (it == data.end())
|
||||
it = data.insert({info.fiber, getInstanceForFiber(*info.parent_fiber_info)}).first;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
std::unordered_map<const Fiber *, T> data;
|
||||
};
|
||||
|
||||
String getSocketTimeoutExceededMessageByTimeoutType(AsyncEventTimeoutType type, Poco::Timespan timeout, const String & socket_description);
|
||||
|
||||
}
|
||||
|
@ -14,48 +14,9 @@ namespace DB
|
||||
namespace OpenTelemetry
|
||||
{
|
||||
|
||||
/// This code can be executed inside several fibers in one thread,
|
||||
/// we should use fiber local tracing context.
|
||||
struct FiberLocalTracingContextOnThread
|
||||
{
|
||||
public:
|
||||
FiberLocalTracingContextOnThread()
|
||||
{
|
||||
/// Initialize main context for this thread.
|
||||
/// Contexts for fibers will inherit this main context.
|
||||
data[nullptr] = TracingContextOnThread();
|
||||
}
|
||||
|
||||
TracingContextOnThread & operator*()
|
||||
{
|
||||
return get();
|
||||
}
|
||||
|
||||
TracingContextOnThread * operator->()
|
||||
{
|
||||
return &get();
|
||||
}
|
||||
|
||||
private:
|
||||
TracingContextOnThread & get()
|
||||
{
|
||||
/// Get context for current fiber.
|
||||
return getContextForFiber(AsyncTaskExecutor::getCurrentFiberInfo());
|
||||
}
|
||||
|
||||
TracingContextOnThread & getContextForFiber(FiberInfo info)
|
||||
{
|
||||
auto it = data.find(info.fiber);
|
||||
/// If it's the first request, we need to initialize context for the fiber using context from parent fiber.
|
||||
if (it == data.end())
|
||||
it = data.insert({info.fiber, getContextForFiber(*info.parent_fiber_info)}).first;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
std::unordered_map<const Fiber *, TracingContextOnThread> data;
|
||||
};
|
||||
|
||||
thread_local FiberLocalTracingContextOnThread current_fiber_trace_context;
|
||||
///// This code can be executed inside several fibers in one thread,
|
||||
///// we should use fiber local tracing context.
|
||||
thread_local FiberLocal<TracingContextOnThread> current_fiber_trace_context;
|
||||
|
||||
bool Span::addAttribute(std::string_view name, UInt64 value) noexcept
|
||||
{
|
||||
|
457
src/Functions/FunctionGenerateRandomStructure.cpp
Normal file
457
src/Functions/FunctionGenerateRandomStructure.cpp
Normal file
@ -0,0 +1,457 @@
|
||||
#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 BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
class FunctionGenerateRandomStructure : public IFunction
|
||||
{
|
||||
private:
|
||||
static constexpr std::array<TypeIndex, 29> simple_types
|
||||
{
|
||||
TypeIndex::Int8,
|
||||
TypeIndex::UInt8,
|
||||
TypeIndex::Int16,
|
||||
TypeIndex::UInt16,
|
||||
TypeIndex::Int32,
|
||||
TypeIndex::UInt32,
|
||||
TypeIndex::Int64,
|
||||
TypeIndex::UInt64,
|
||||
TypeIndex::Int128,
|
||||
TypeIndex::UInt128,
|
||||
TypeIndex::Int256,
|
||||
TypeIndex::UInt256,
|
||||
TypeIndex::Float32,
|
||||
TypeIndex::Float64,
|
||||
TypeIndex::Decimal32,
|
||||
TypeIndex::Decimal64,
|
||||
TypeIndex::Decimal128,
|
||||
TypeIndex::Decimal256,
|
||||
TypeIndex::Date,
|
||||
TypeIndex::Date32,
|
||||
TypeIndex::DateTime,
|
||||
TypeIndex::DateTime64,
|
||||
TypeIndex::String,
|
||||
TypeIndex::FixedString,
|
||||
TypeIndex::Enum8,
|
||||
TypeIndex::Enum16,
|
||||
TypeIndex::IPv4,
|
||||
TypeIndex::IPv6,
|
||||
TypeIndex::UUID,
|
||||
};
|
||||
|
||||
static constexpr std::array<TypeIndex, 5> complex_types
|
||||
{
|
||||
TypeIndex::Nullable,
|
||||
TypeIndex::LowCardinality,
|
||||
TypeIndex::Array,
|
||||
TypeIndex::Tuple,
|
||||
TypeIndex::Map,
|
||||
};
|
||||
|
||||
static constexpr std::array<TypeIndex, 22> map_key_types
|
||||
{
|
||||
TypeIndex::Int8,
|
||||
TypeIndex::UInt8,
|
||||
TypeIndex::Int16,
|
||||
TypeIndex::UInt16,
|
||||
TypeIndex::Int32,
|
||||
TypeIndex::UInt32,
|
||||
TypeIndex::Int64,
|
||||
TypeIndex::UInt64,
|
||||
TypeIndex::Int128,
|
||||
TypeIndex::UInt128,
|
||||
TypeIndex::Int256,
|
||||
TypeIndex::UInt256,
|
||||
TypeIndex::Date,
|
||||
TypeIndex::Date32,
|
||||
TypeIndex::DateTime,
|
||||
TypeIndex::String,
|
||||
TypeIndex::FixedString,
|
||||
TypeIndex::IPv4,
|
||||
TypeIndex::Enum8,
|
||||
TypeIndex::Enum16,
|
||||
TypeIndex::UUID,
|
||||
TypeIndex::LowCardinality,
|
||||
};
|
||||
|
||||
static constexpr std::array<TypeIndex, 22> suspicious_lc_types
|
||||
{
|
||||
TypeIndex::Int8,
|
||||
TypeIndex::UInt8,
|
||||
TypeIndex::Int16,
|
||||
TypeIndex::UInt16,
|
||||
TypeIndex::Int32,
|
||||
TypeIndex::UInt32,
|
||||
TypeIndex::Int64,
|
||||
TypeIndex::UInt64,
|
||||
TypeIndex::Int128,
|
||||
TypeIndex::UInt128,
|
||||
TypeIndex::Int256,
|
||||
TypeIndex::UInt256,
|
||||
TypeIndex::Float32,
|
||||
TypeIndex::Float64,
|
||||
TypeIndex::Date,
|
||||
TypeIndex::Date32,
|
||||
TypeIndex::DateTime,
|
||||
TypeIndex::String,
|
||||
TypeIndex::FixedString,
|
||||
TypeIndex::IPv4,
|
||||
TypeIndex::IPv6,
|
||||
TypeIndex::UUID,
|
||||
};
|
||||
|
||||
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";
|
||||
|
||||
explicit FunctionGenerateRandomStructure(bool allow_suspicious_lc_types_) : allow_suspicious_lc_types(allow_suspicious_lc_types_)
|
||||
{
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionGenerateRandomStructure>(context->getSettingsRef().allow_suspicious_low_cardinality_types.value);
|
||||
}
|
||||
|
||||
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}; }
|
||||
bool useDefaultImplementationForConstants() const override { return false; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
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 {}, expected from 0 to 2",
|
||||
getName(), arguments.size());
|
||||
|
||||
|
||||
for (size_t i = 0; i != arguments.size(); ++i)
|
||||
{
|
||||
if (!isUnsignedInteger(arguments[i]) && !arguments[i]->onlyNull())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of the {} argument of function {}, expected unsigned integer or Null",
|
||||
arguments[i]->getName(),
|
||||
i + 1,
|
||||
getName());
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
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() && !arguments[0].column->onlyNull())
|
||||
{
|
||||
number_of_columns = arguments[0].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() > 1 && !arguments[1].column->onlyNull())
|
||||
seed = arguments[1].column->getUInt(0);
|
||||
|
||||
pcg64 rng(seed);
|
||||
if (number_of_columns == 0)
|
||||
number_of_columns = generateNumberOfColumns(rng);
|
||||
|
||||
auto col_res = ColumnString::create();
|
||||
auto & string_column = assert_cast<ColumnString &>(*col_res);
|
||||
auto & chars = string_column.getChars();
|
||||
WriteBufferFromVector buf(chars);
|
||||
writeRandomStructure(rng, number_of_columns, buf, allow_suspicious_lc_types);
|
||||
buf.finalize();
|
||||
chars.push_back(0);
|
||||
string_column.getOffsets().push_back(chars.size());
|
||||
return ColumnConst::create(std::move(col_res), input_rows_count);
|
||||
}
|
||||
|
||||
static String getRandomStructure(size_t seed, const ContextPtr & context)
|
||||
{
|
||||
pcg64 rng(seed);
|
||||
size_t number_of_columns = generateNumberOfColumns(rng);
|
||||
WriteBufferFromOwnString buf;
|
||||
writeRandomStructure(rng, number_of_columns, buf, context->getSettingsRef().allow_suspicious_low_cardinality_types);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
static size_t generateNumberOfColumns(pcg64 & rng)
|
||||
{
|
||||
return rng() % MAX_NUMBER_OF_COLUMNS + 1;
|
||||
}
|
||||
|
||||
static void writeRandomStructure(pcg64 & rng, size_t number_of_columns, WriteBuffer & buf, bool allow_suspicious_lc_types)
|
||||
{
|
||||
for (size_t i = 0; i != number_of_columns; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(", ", buf);
|
||||
String column_name = "c" + std::to_string(i + 1);
|
||||
writeString(column_name, buf);
|
||||
writeChar(' ', buf);
|
||||
writeRandomType(column_name, rng, buf, allow_suspicious_lc_types);
|
||||
}
|
||||
}
|
||||
|
||||
template <bool allow_complex_types = true>
|
||||
static void writeRandomType(const String & column_name, pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types, size_t depth = 0)
|
||||
{
|
||||
if (allow_complex_types && depth > MAX_DEPTH)
|
||||
writeRandomType<false>(column_name, rng, buf, depth);
|
||||
|
||||
constexpr auto all_types = getAllTypes<allow_complex_types>();
|
||||
auto type = all_types[rng() % all_types.size()];
|
||||
|
||||
switch (type)
|
||||
{
|
||||
case TypeIndex::UInt8:
|
||||
if (rng() % 2)
|
||||
writeCString("UInt8", buf);
|
||||
else
|
||||
writeCString("Bool", buf);
|
||||
return;
|
||||
case TypeIndex::FixedString:
|
||||
writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::DateTime64:
|
||||
writeString("DateTime64(" + std::to_string(rng() % MAX_DATETIME64_PRECISION + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::Decimal32:
|
||||
writeString("Decimal32(" + std::to_string(rng() % MAX_DECIMAL32_PRECISION + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::Decimal64:
|
||||
writeString("Decimal64(" + std::to_string(rng() % MAX_DECIMAL64_PRECISION + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::Decimal128:
|
||||
writeString("Decimal128(" + std::to_string(rng() % MAX_DECIMAL128_PRECISION + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::Decimal256:
|
||||
writeString("Decimal256(" + std::to_string(rng() % MAX_DECIMAL256_PRECISION + 1) + ")", buf);
|
||||
return;
|
||||
case TypeIndex::Enum8:
|
||||
writeCString("Enum8(", buf);
|
||||
writeEnumValues(column_name, rng, buf, INT8_MAX);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
case TypeIndex::Enum16:
|
||||
writeCString("Enum16(", buf);
|
||||
writeEnumValues(column_name, rng, buf, INT16_MAX);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
case TypeIndex::LowCardinality:
|
||||
writeCString("LowCardinality(", buf);
|
||||
writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
case TypeIndex::Nullable:
|
||||
{
|
||||
writeCString("Nullable(", buf);
|
||||
writeRandomType<false>(column_name, rng, buf, allow_suspicious_lc_types, depth + 1);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
}
|
||||
case TypeIndex::Array:
|
||||
{
|
||||
writeCString("Array(", buf);
|
||||
writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
}
|
||||
case TypeIndex::Map:
|
||||
{
|
||||
writeCString("Map(", buf);
|
||||
writeMapKeyType(rng, buf, allow_suspicious_lc_types);
|
||||
writeCString(", ", buf);
|
||||
writeRandomType(column_name, rng, buf, allow_suspicious_lc_types, depth + 1);
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
}
|
||||
case TypeIndex::Tuple:
|
||||
{
|
||||
size_t elements = rng() % MAX_TUPLE_ELEMENTS + 1;
|
||||
bool generate_nested = rng() % 2;
|
||||
bool generate_named_tuple = rng() % 2;
|
||||
if (generate_nested)
|
||||
writeCString("Nested(", buf);
|
||||
else
|
||||
writeCString("Tuple(", buf);
|
||||
|
||||
for (size_t i = 0; i != elements; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(", ", buf);
|
||||
|
||||
String element_name = "e" + std::to_string(i + 1);
|
||||
if (generate_named_tuple || generate_nested)
|
||||
{
|
||||
writeString(element_name, buf);
|
||||
writeChar(' ', buf);
|
||||
}
|
||||
writeRandomType(element_name, rng, buf, allow_suspicious_lc_types, depth + 1);
|
||||
}
|
||||
writeChar(')', buf);
|
||||
return;
|
||||
}
|
||||
default:
|
||||
writeString(magic_enum::enum_name<TypeIndex>(type), buf);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
static void writeMapKeyType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types)
|
||||
{
|
||||
TypeIndex type = map_key_types[rng() % map_key_types.size()];
|
||||
if (type == TypeIndex::FixedString)
|
||||
{
|
||||
writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf);
|
||||
}
|
||||
else if (type == TypeIndex::LowCardinality)
|
||||
{
|
||||
writeCString("LowCardinality(", buf);
|
||||
writeLowCardinalityNestedType(rng, buf, allow_suspicious_lc_types);
|
||||
writeChar(')', buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeString(magic_enum::enum_name<TypeIndex>(type), buf);
|
||||
}
|
||||
}
|
||||
|
||||
static void writeLowCardinalityNestedType(pcg64 & rng, WriteBuffer & buf, bool allow_suspicious_lc_types)
|
||||
{
|
||||
bool make_nullable = rng() % 2;
|
||||
if (make_nullable)
|
||||
writeCString("Nullable(", buf);
|
||||
|
||||
if (allow_suspicious_lc_types)
|
||||
{
|
||||
TypeIndex type = suspicious_lc_types[rng() % map_key_types.size()];
|
||||
if (type == TypeIndex::FixedString)
|
||||
writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf);
|
||||
else
|
||||
writeString(magic_enum::enum_name<TypeIndex>(type), buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Support only String and FixedString.
|
||||
if (rng() % 2)
|
||||
writeCString("String", buf);
|
||||
else
|
||||
writeString("FixedString(" + std::to_string(rng() % MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS + 1) + ")", buf);
|
||||
}
|
||||
|
||||
if (make_nullable)
|
||||
writeChar(')', buf);
|
||||
}
|
||||
|
||||
static void writeEnumValues(const String & column_name, pcg64 & rng, WriteBuffer & buf, ssize_t max_value)
|
||||
{
|
||||
/// Don't generate big enums, because it will lead to really big result
|
||||
/// and slowness of this function, and it can lead to `Max query size exceeded`
|
||||
/// while using this function with generateRandom.
|
||||
size_t num_values = rng() % 16 + 1;
|
||||
std::vector<Int16> values(num_values);
|
||||
|
||||
/// Generate random numbers from range [-(max_value + 1), max_value - num_values + 1]
|
||||
for (Int16 & x : values)
|
||||
x = rng() % (2 * (max_value + 1) - num_values) - max_value - 1;
|
||||
/// Make all numbers unique.
|
||||
std::sort(values.begin(), values.end());
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
values[i] += i;
|
||||
std::shuffle(values.begin(), values.end(), rng);
|
||||
for (size_t i = 0; i != num_values; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
writeCString(", ", buf);
|
||||
writeString("'" + column_name + "V" + std::to_string(values[i]) + "' = " + std::to_string(i), buf);
|
||||
}
|
||||
}
|
||||
|
||||
template <bool allow_complex_types>
|
||||
static constexpr auto getAllTypes()
|
||||
{
|
||||
constexpr size_t complex_types_size = complex_types.size() * allow_complex_types;
|
||||
constexpr size_t result_size = simple_types.size() + complex_types_size;
|
||||
std::array<TypeIndex, result_size> result;
|
||||
size_t index = 0;
|
||||
|
||||
for (size_t i = 0; i != simple_types.size(); ++i, ++index)
|
||||
result[index] = simple_types[i];
|
||||
|
||||
for (size_t i = 0; i != complex_types_size; ++i, ++index)
|
||||
result[index] = complex_types[i];
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
bool allow_suspicious_lc_types;
|
||||
};
|
||||
|
||||
|
||||
REGISTER_FUNCTION(GenerateRandomStructure)
|
||||
{
|
||||
factory.registerFunction<FunctionGenerateRandomStructure>(
|
||||
{
|
||||
R"(
|
||||
Generates a random table structure.
|
||||
This function takes 2 optional constant arguments:
|
||||
the number of columns in the result structure (random by default) and random seed (random by default)
|
||||
The maximum number of columns is 128.
|
||||
The function returns a value of type String.
|
||||
)",
|
||||
Documentation::Examples{
|
||||
{"random", "SELECT generateRandomStructure()"},
|
||||
{"with specified number of columns", "SELECT generateRandomStructure(10)"},
|
||||
{"with specified seed", "SELECT generateRandomStructure(10, 42)"},
|
||||
},
|
||||
Documentation::Categories{"Random"}
|
||||
},
|
||||
FunctionFactory::CaseSensitive);
|
||||
}
|
||||
|
||||
}
|
45
src/Functions/FunctionGenerateRandomStructure.h
Normal file
45
src/Functions/FunctionGenerateRandomStructure.h
Normal file
@ -0,0 +1,45 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionGenerateRandomStructure : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "generateRandomStructure";
|
||||
|
||||
explicit FunctionGenerateRandomStructure(bool allow_suspicious_lc_types_) : allow_suspicious_lc_types(allow_suspicious_lc_types_)
|
||||
{
|
||||
}
|
||||
|
||||
static FunctionPtr create(ContextPtr context)
|
||||
{
|
||||
return std::make_shared<FunctionGenerateRandomStructure>(context->getSettingsRef().allow_suspicious_low_cardinality_types.value);
|
||||
}
|
||||
|
||||
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}; }
|
||||
bool useDefaultImplementationForConstants() const override { return false; }
|
||||
bool useDefaultImplementationForNulls() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override;
|
||||
|
||||
static String generateRandomStructure(size_t seed, const ContextPtr & context);
|
||||
|
||||
private:
|
||||
bool allow_suspicious_lc_types;
|
||||
};
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user