mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fixes and test
This commit is contained in:
parent
e9784573ea
commit
45aebda677
@ -11,15 +11,16 @@ StorageValues::StorageValues(const std::string & database_name_, const std::stri
|
||||
setColumns(ColumnsDescription(res_block.getNamesAndTypesList()));
|
||||
}
|
||||
|
||||
BlockInputStreams StorageValues::read(const Names & column_names,
|
||||
const SelectQueryInfo &,
|
||||
const Context & /*context*/,
|
||||
QueryProcessingStage::Enum /*processed_stage*/,
|
||||
size_t /*max_block_size*/,
|
||||
unsigned /*num_streams*/)
|
||||
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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -13,12 +13,13 @@ public:
|
||||
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;
|
||||
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;
|
||||
|
@ -25,7 +25,7 @@ void TableFunctionFactory::registerFunction(const std::string & name, Creator cr
|
||||
|
||||
if (case_sensitiveness == CaseInsensitive
|
||||
&& !case_insensitive_table_functions.emplace(Poco::toLower(name), creator).second)
|
||||
throw Exception("TableFunctionFactory: the case insensitive aggregate function name '" + name + "' is not unique",
|
||||
throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique",
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Storages/StorageValues.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
@ -11,10 +12,12 @@
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionValues.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Core/Block.h>
|
||||
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,6 +27,37 @@ 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;
|
||||
@ -37,7 +71,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C
|
||||
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
|
||||
/// Parsing first argument as table structure
|
||||
std::string structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
std::vector<std::string> structure_values;
|
||||
@ -60,21 +94,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C
|
||||
|
||||
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
||||
|
||||
///Parsing other arguments as Fields
|
||||
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);
|
||||
}
|
||||
}
|
||||
parseAndInsertValues(res_columns, args, sample_block, context);
|
||||
|
||||
Block res_block = sample_block.cloneWithColumns(std::move(res_columns));
|
||||
|
||||
|
12
dbms/tests/queries/0_stateless/00975_values_list.reference
Normal file
12
dbms/tests/queries/0_stateless/00975_values_list.reference
Normal file
@ -0,0 +1,12 @@
|
||||
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
|
11
dbms/tests/queries/0_stateless/00975_values_list.sql
Normal file
11
dbms/tests/queries/0_stateless/00975_values_list.sql
Normal file
@ -0,0 +1,11 @@
|
||||
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')));
|
Loading…
Reference in New Issue
Block a user