Merge pull request #6209 from dimarub2000/values_list

VALUES list
This commit is contained in:
alexey-milovidov 2019-08-01 04:36:04 +03:00 committed by GitHub
commit a8793fbef3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 325 additions and 55 deletions

View File

@ -263,7 +263,7 @@ protected:
class ParserColumnsOrIndicesDeclarationList : public IParserBase
{
protected:
protected:
const char * getName() const override { return "columns or indices declaration list"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};

View File

@ -0,0 +1,28 @@
#include <Storages/IStorage.h>
#include <Storages/StorageValues.h>
#include <DataStreams/OneBlockInputStream.h>
namespace DB
{
StorageValues::StorageValues(const std::string & database_name_, const std::string & table_name_, const Block & res_block_)
: database_name(database_name_), table_name(table_name_), res_block(res_block_)
{
setColumns(ColumnsDescription(res_block.getNamesAndTypesList()));
}
BlockInputStreams StorageValues::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);
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(res_block));
}
}

View File

@ -0,0 +1,36 @@
#pragma once
#include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h>
namespace DB
{
/* One block storage used for values table function
* It's structure is similar to IStorageSystemOneBlock
*/
class StorageValues : public ext::shared_ptr_helper<StorageValues>, public IStorage
{
public:
std::string getName() const override { return "Values"; }
std::string getTableName() const override { return table_name; }
std::string getDatabaseName() const override { return database_name; }
BlockInputStreams read(
const Names & column_names,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
private:
std::string database_name;
std::string table_name;
Block res_block;
protected:
StorageValues(const std::string & database_name_, const std::string & table_name_, const Block & res_block_);
};
}

View File

@ -11,10 +11,10 @@ NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes()
void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
{
const auto & functions = TableFunctionFactory::instance().getAllTableFunctions();
for (const auto & pair : functions)
const auto & functions_names = TableFunctionFactory::instance().getAllRegisteredNames();
for (const auto & name : functions_names)
{
res_columns[0]->insert(pair.first);
res_columns[0]->insert(name);
}
}

View File

@ -1,14 +1,17 @@
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/ITableFunctionFileLike.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <Storages/StorageFile.h>
#include <DataTypes/DataTypeFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <boost/algorithm/string.hpp>
namespace DB
@ -21,7 +24,7 @@ namespace ErrorCodes
StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const
{
// Parse args
/// Parse args
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
@ -40,26 +43,12 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons
std::string format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
std::string structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
// Create sample block
std::vector<std::string> structure_vals;
boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on);
if (structure_vals.size() % 2 != 0)
throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR);
/// Create sample block
Block sample_block;
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
parseColumnsListFromString(structure, sample_block, context);
for (size_t i = 0, size = structure_vals.size(); i < size; i += 2)
{
ColumnWithTypeAndName column;
column.name = structure_vals[i];
column.type = data_type_factory.get(structure_vals[i + 1]);
column.column = column.type->createColumn();
sample_block.insert(std::move(column));
}
// Create table
/// Create table
StoragePtr storage = getStorage(filename, format, sample_block, const_cast<Context &>(context), table_name);
storage->startup();

View File

@ -1,11 +1,10 @@
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/Context.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
namespace DB
{
@ -17,11 +16,16 @@ namespace ErrorCodes
}
void TableFunctionFactory::registerFunction(const std::string & name, Creator creator)
void TableFunctionFactory::registerFunction(const std::string & name, Creator creator, CaseSensitiveness case_sensitiveness)
{
if (!functions.emplace(name, std::move(creator)).second)
if (!table_functions.emplace(name, creator).second)
throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
if (case_sensitiveness == CaseInsensitive
&& !case_insensitive_table_functions.emplace(Poco::toLower(name), creator).second)
throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
TableFunctionPtr TableFunctionFactory::get(
@ -31,8 +35,8 @@ TableFunctionPtr TableFunctionFactory::get(
if (context.getSettings().readonly == 1) /** For example, for readonly = 2 - allowed. */
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
auto it = functions.find(name);
if (it == functions.end())
auto res = tryGet(name, context);
if (!res)
{
auto hints = getHints(name);
if (!hints.empty())
@ -41,12 +45,29 @@ TableFunctionPtr TableFunctionFactory::get(
throw Exception("Unknown table function " + name, ErrorCodes::UNKNOWN_FUNCTION);
}
return it->second();
return res;
}
TableFunctionPtr TableFunctionFactory::tryGet(
const std::string & name_param,
const Context &) const
{
String name = getAliasToOrName(name_param);
auto it = table_functions.find(name);
if (table_functions.end() != it)
return it->second();
it = case_insensitive_table_functions.find(Poco::toLower(name));
if (case_insensitive_table_functions.end() != it)
return it->second();
return {};
}
bool TableFunctionFactory::isTableFunctionName(const std::string & name) const
{
return functions.count(name);
return table_functions.count(name);
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
#include <Common/IFactoryWithAliases.h>
#include <Common/NamePrompter.h>
#include <ext/singleton.h>
@ -16,51 +17,47 @@ namespace DB
class Context;
using TableFunctionCreator = std::function<TableFunctionPtr()>;
/** Lets you get a table function by its name.
*/
class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>, public IHints<1, TableFunctionFactory>
class TableFunctionFactory final: public ext::singleton<TableFunctionFactory>, public IFactoryWithAliases<TableFunctionCreator>
{
public:
using Creator = std::function<TableFunctionPtr()>;
using TableFunctions = std::unordered_map<std::string, Creator>;
/// Register a function by its name.
/// No locking, you must register all functions before usage of get.
void registerFunction(const std::string & name, Creator creator);
void registerFunction(const std::string & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
template <typename Function>
void registerFunction()
void registerFunction(CaseSensitiveness case_sensitiveness = CaseSensitive)
{
auto creator = [] () -> TableFunctionPtr
{
return std::make_shared<Function>();
};
registerFunction(Function::name, std::move(creator));
registerFunction(Function::name, std::move(creator), case_sensitiveness);
}
/// Throws an exception if not found.
TableFunctionPtr get(
const std::string & name,
const Context & context) const;
TableFunctionPtr get(const std::string & name, const Context & context) const;
/// Returns nullptr if not found.
TableFunctionPtr tryGet(const std::string & name, const Context & context) const;
bool isTableFunctionName(const std::string & name) const;
const TableFunctions & getAllTableFunctions() const
{
return functions;
}
std::vector<String> getAllRegisteredNames() const override
{
std::vector<String> result;
auto getter = [](const auto & pair) { return pair.first; };
std::transform(functions.begin(), functions.end(), std::back_inserter(result), getter);
return result;
}
private:
TableFunctions functions;
using TableFunctions = std::unordered_map<std::string, Creator>;
const TableFunctions & getCreatorMap() const override { return table_functions; }
const TableFunctions & getCaseInsensitiveCreatorMap() const override { return case_insensitive_table_functions; }
String getFactoryName() const override { return "TableFunctionFactory"; }
TableFunctions table_functions;
TableFunctions case_insensitive_table_functions;
};
}

View File

@ -0,0 +1,95 @@
#include <Common/typeid_cast.h>
#include <Common/Exception.h>
#include <Core/Block.h>
#include <Storages/StorageValues.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionValues.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Interpreters/convertFieldToType.h>
#include <Interpreters/evaluateConstantExpression.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args, const Block & sample_block, const Context & context)
{
if (res_columns.size() == 1) /// Parsing arguments as Fields
{
for (size_t i = 1; i < args.size(); ++i)
{
const auto & [value_field, value_type_ptr] = evaluateConstantExpression(args[i], context);
Field value = convertFieldToType(value_field, *sample_block.getByPosition(0).type, value_type_ptr.get());
res_columns[0]->insert(value);
}
}
else /// Parsing arguments as Tuples
{
for (size_t i = 1; i < args.size(); ++i)
{
const auto & [value_field, value_type_ptr] = evaluateConstantExpression(args[i], context);
const TupleBackend & value_tuple = value_field.safeGet<Tuple>().toUnderType();
if (value_tuple.size() != sample_block.columns())
throw Exception("Values size should match with number of columns", ErrorCodes::LOGICAL_ERROR);
for (size_t j = 0; j < value_tuple.size(); ++j)
{
Field value = convertFieldToType(value_tuple[j], *sample_block.getByPosition(j).type, value_type_ptr.get());
res_columns[j]->insert(value);
}
}
}
}
StoragePtr TableFunctionValues::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 2 or more arguments: structure and values.",
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>();
Block sample_block;
parseColumnsListFromString(structure, sample_block, context);
MutableColumns res_columns = sample_block.cloneEmptyColumns();
/// Parsing other arguments as values and inserting them into columns
parseAndInsertValues(res_columns, args, sample_block, context);
Block res_block = sample_block.cloneWithColumns(std::move(res_columns));
auto res = StorageValues::create(getDatabaseName(), table_name, res_block);
res->startup();
return res;
}
void registerTableFunctionValues(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionValues>(TableFunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,20 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
namespace DB
{
/* values(structure, values...) - creates a temporary storage filling columns with values
* values is case-insensitive table function
*/
class TableFunctionValues : public ITableFunction
{
public:
static constexpr auto name = "values";
std::string getName() const override { return name; }
private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
};
}

View File

@ -0,0 +1,44 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ParserCreateQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
void parseColumnsListFromString(const std::string & structure, Block & sample_block, const Context & context)
{
Expected expected;
Tokens tokens(structure.c_str(), structure.c_str() + structure.size());
TokenIterator token_iterator(tokens);
ParserColumnDeclarationList parser;
ASTPtr columns_list_raw;
if (!parser.parse(token_iterator, columns_list_raw, expected))
throw Exception("Cannot parse columns declaration list.", ErrorCodes::SYNTAX_ERROR);
auto * columns_list = dynamic_cast<ASTExpressionList *>(columns_list_raw.get());
if (!columns_list)
throw Exception("Could not cast AST to ASTExpressionList", ErrorCodes::LOGICAL_ERROR);
ColumnsDescription columns_desc = InterpreterCreateQuery::getColumnsDescription(*columns_list, context);
for (const auto & [name, type]: columns_desc.getAllPhysical())
{
ColumnWithTypeAndName column;
column.name = name;
column.type = type;
column.column = type->createColumn();
sample_block.insert(std::move(column));
}
}
}

View File

@ -0,0 +1,11 @@
#pragma once
#include <Core/Block.h>
namespace DB
{
/// Parses a common argument for table functions such as table structure given in string
void parseColumnsListFromString(const std::string & structure, Block & sample_block, const Context & context);
}

View File

@ -13,6 +13,7 @@ void registerTableFunctionNumbers(TableFunctionFactory & factory);
void registerTableFunctionCatBoostPool(TableFunctionFactory & factory);
void registerTableFunctionFile(TableFunctionFactory & factory);
void registerTableFunctionURL(TableFunctionFactory & factory);
void registerTableFunctionValues(TableFunctionFactory & factory);
#if USE_HDFS
void registerTableFunctionHDFS(TableFunctionFactory & factory);
@ -39,6 +40,7 @@ void registerTableFunctions()
registerTableFunctionCatBoostPool(factory);
registerTableFunctionFile(factory);
registerTableFunctionURL(factory);
registerTableFunctionValues(factory);
#if USE_HDFS
registerTableFunctionHDFS(factory);

View File

@ -0,0 +1,13 @@
1 one
2 two
3 three
1 one
2 two
3 three
2018-01-01 2018-01-01 00:00:00
abra
cadabra
abracadabra
23 23 23
24 24 24
1.6660 a b

View File

@ -0,0 +1,14 @@
DROP TABLE IF EXISTS values_list;
SELECT * FROM VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three'));
CREATE TABLE values_list AS VALUES('a UInt64, s String', (1, 'one'), (2, 'two'), (3, 'three'));
SELECT * FROM values_list;
SELECT subtractYears(date, 1), subtractYears(date_time, 1) FROM VALUES('date Date, date_time DateTime', (toDate('2019-01-01'), toDateTime('2019-01-01 00:00:00')));
SELECT * FROM VALUES('s String', ('abra'), ('cadabra'), ('abracadabra'));
SELECT * FROM VALUES('n UInt64, s String, ss String', (1 + 22, '23', toString(23)), (toUInt64('24'), '24', concat('2', '4')));
SELECT * FROM VALUES('a Decimal(4, 4), b String, c String', (divide(toDecimal32(5, 3), 3), 'a', 'b'));
DROP TABLE values_list;