mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Added draft version of random table function.
Currently unimplemented data generators
This commit is contained in:
parent
072043156f
commit
6597176388
@ -304,6 +304,10 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const
|
||||
return ColumnAggregateFunction::create(function);
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeAggregateFunction::createColumnWithRandomData(size_t) const
|
||||
{
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Create empty state
|
||||
Field DataTypeAggregateFunction::getDefault() const
|
||||
|
@ -63,6 +63,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -487,6 +487,13 @@ MutableColumnPtr DataTypeArray::createColumn() const
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr DataTypeArray::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
Field DataTypeArray::getDefault() const
|
||||
{
|
||||
return Array();
|
||||
|
@ -94,6 +94,7 @@ public:
|
||||
bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -41,6 +41,13 @@ MutableColumnPtr DataTypeDecimalBase<T>::createColumn() const
|
||||
return ColumnType::create(0, scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr DataTypeDecimalBase<T>::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimalBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
|
@ -83,6 +83,7 @@ public:
|
||||
|
||||
Field getDefault() const override;
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
|
@ -111,6 +111,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override { return ColumnType::create(); }
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
@ -268,6 +268,12 @@ MutableColumnPtr DataTypeFixedString::createColumn() const
|
||||
return ColumnFixedString::create(n);
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeFixedString::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
Field DataTypeFixedString::getDefault() const
|
||||
{
|
||||
return String();
|
||||
|
@ -70,6 +70,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -934,6 +934,12 @@ MutableColumnPtr DataTypeLowCardinality::createColumn() const
|
||||
return ColumnLowCardinality::create(std::move(dictionary), std::move(indexes));
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeLowCardinality::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
Field DataTypeLowCardinality::getDefault() const
|
||||
{
|
||||
return dictionary_type->getDefault();
|
||||
|
@ -68,6 +68,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -14,6 +14,14 @@ MutableColumnPtr DataTypeNothing::createColumn() const
|
||||
return ColumnNothing::create(0);
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr DataTypeNothing::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
void DataTypeNothing::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
|
||||
{
|
||||
size_t size = column.size();
|
||||
|
@ -19,6 +19,7 @@ public:
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Nothing; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
/// These methods read and write zero bytes just to allow to figure out size of column.
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
|
||||
|
@ -488,6 +488,11 @@ MutableColumnPtr DataTypeNullable::createColumn() const
|
||||
return ColumnNullable::create(nested_data_type->createColumn(), ColumnUInt8::create());
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeNullable::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
return ColumnNullable::create(nested_data_type->createColumnWithRandomData(limit), DataTypeUInt8().createColumnWithRandomData(limit));
|
||||
}
|
||||
|
||||
Field DataTypeNullable::getDefault() const
|
||||
{
|
||||
return Null();
|
||||
|
@ -76,6 +76,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -239,6 +239,13 @@ MutableColumnPtr DataTypeNumberBase<T>::createColumn() const
|
||||
return ColumnVector<T>::create();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr DataTypeNumberBase<T>::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool DataTypeNumberBase<T>::isValueRepresentedByInteger() const
|
||||
{
|
||||
|
@ -45,6 +45,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
|
||||
// Used for expressions analysis.
|
||||
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); }
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
// Used only for debugging, making it DUMPABLE
|
||||
Field getDefault() const override { return Tuple(); }
|
||||
|
@ -360,6 +360,11 @@ MutableColumnPtr DataTypeString::createColumn() const
|
||||
return ColumnString::create();
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeString::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
bool DataTypeString::equals(const IDataType & rhs) const
|
||||
{
|
||||
|
@ -54,6 +54,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
|
@ -454,6 +454,14 @@ MutableColumnPtr DataTypeTuple::createColumn() const
|
||||
return ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr DataTypeTuple::createColumnWithRandomData(size_t limit) const
|
||||
{
|
||||
(void)limit;
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
Field DataTypeTuple::getDefault() const
|
||||
{
|
||||
return Tuple(ext::map<Tuple>(elems, [] (const DataTypePtr & elem) { return elem->getDefault(); }));
|
||||
|
@ -81,6 +81,7 @@ public:
|
||||
void deserializeProtobuf(IColumn & column, ProtobufReader & reader, bool allow_add_row, bool & row_added) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
@ -287,6 +287,10 @@ public:
|
||||
*/
|
||||
virtual MutableColumnPtr createColumn() const = 0;
|
||||
|
||||
/** Create column for corresponding type and fill with random values.
|
||||
*/
|
||||
virtual MutableColumnPtr createColumnWithRandomData(size_t size) const = 0;
|
||||
|
||||
/** Create ColumnConst for corresponding type, with specified size and value.
|
||||
*/
|
||||
ColumnPtr createColumnConst(size_t size, const Field & field) const;
|
||||
|
@ -42,6 +42,11 @@ public:
|
||||
throw Exception("Method createColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
MutableColumnPtr createColumnWithRandomData(size_t) const override
|
||||
{
|
||||
throw Exception("Method createColumnWithRandomData() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
Field getDefault() const override
|
||||
{
|
||||
throw Exception("Method getDefault() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
69
dbms/src/TableFunctions/TableFunctionRandom.cpp
Normal file
69
dbms/src/TableFunctions/TableFunctionRandom.cpp
Normal file
@ -0,0 +1,69 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionRandom.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include "registerTableFunctions.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
StoragePtr TableFunctionRandom::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
|
||||
{
|
||||
ASTs & args_func = ast_function->children;
|
||||
|
||||
if (args_func.size() != 1)
|
||||
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
if (args.size() > 2)
|
||||
throw Exception("Table function '" + getName() + "' requires one or two arguments: structure (and limit).",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
/// Parsing first argument as table structure and creating a sample block
|
||||
std::string structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
UInt64 limit = 1;
|
||||
/// Parsing second argument if present
|
||||
if (args.size() == 2)
|
||||
limit = args[1]->as<ASTLiteral &>().value.safeGet<Uint64>();
|
||||
|
||||
if (!limit)
|
||||
throw Exception("Table function '" + getName() + "' limit should not be 0.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
ColumnsDescription columns = parseColumnsListFromString(structure, context);
|
||||
|
||||
Block res_block;
|
||||
for (const auto & name_type : columns.getOrdinary())
|
||||
Column c = name_type.type->createColumnWithRandomData(limit) ;
|
||||
res_block.insert({ c, name_type.type, name_type.name });
|
||||
|
||||
auto res = StorageValues::create(StorageID(getDatabaseName(), table_name), columns, res_block);
|
||||
res->startup();
|
||||
return res;
|
||||
}
|
||||
|
||||
void registerTableFunctionRandom(TableFunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<TableFunctionRandom>(TableFunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
20
dbms/src/TableFunctions/TableFunctionRandom.h
Normal file
20
dbms/src/TableFunctions/TableFunctionRandom.h
Normal file
@ -0,0 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/* random(structure, limit) - creates a temporary storage filling columns with random data
|
||||
* random is case-insensitive table function
|
||||
*/
|
||||
class TableFunctionRandom : public ITableFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "generate";
|
||||
std::string getName() const override { return name; }
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -15,6 +15,7 @@ void registerTableFunctions()
|
||||
registerTableFunctionURL(factory);
|
||||
registerTableFunctionValues(factory);
|
||||
registerTableFunctionInput(factory);
|
||||
registerTableFunctionRandom(factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
registerTableFunctionS3(factory);
|
||||
|
@ -12,6 +12,7 @@ void registerTableFunctionFile(TableFunctionFactory & factory);
|
||||
void registerTableFunctionURL(TableFunctionFactory & factory);
|
||||
void registerTableFunctionValues(TableFunctionFactory & factory);
|
||||
void registerTableFunctionInput(TableFunctionFactory & factory);
|
||||
void registerTableFunctionRandom(TableFunctionFactory & factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
void registerTableFunctionS3(TableFunctionFactory & factory);
|
||||
|
@ -0,0 +1 @@
|
||||
SELECT * FROM random(3)
|
Loading…
Reference in New Issue
Block a user