mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
parser changed, test modified, comments added
This commit is contained in:
parent
30c35b1cbf
commit
f91c64bd28
@ -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;
|
||||
};
|
||||
|
@ -5,7 +5,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* One block storage used for values table function
|
||||
* It's structure is similar to IStorageSystemOneBlock
|
||||
*/
|
||||
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_)
|
||||
{
|
||||
|
@ -6,7 +6,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/* values(structure, values...) - creates a temporary storage filling columns with values
|
||||
* values is case-insensitive table function
|
||||
*/
|
||||
class StorageValues : public ext::shared_ptr_helper<StorageValues>, public IStorage
|
||||
{
|
||||
public:
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <TableFunctions/TableFunctionValues.h>
|
||||
@ -15,8 +16,7 @@
|
||||
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,6 +25,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
static void parseAndInsertValues(MutableColumns & res_columns, const ASTs & args, const Block & sample_block, const Context & context)
|
||||
@ -74,21 +75,30 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C
|
||||
/// Parsing first argument as table structure
|
||||
std::string structure = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
std::vector<std::string> structure_values;
|
||||
boost::split(structure_values, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on);
|
||||
Expected expected;
|
||||
|
||||
if (structure_values.size() % 2 != 0)
|
||||
throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR);
|
||||
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);
|
||||
|
||||
Block sample_block;
|
||||
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
|
||||
|
||||
for (size_t i = 0, size = structure_values.size(); i < size; i += 2)
|
||||
for (const auto & [name, type]: columns_desc.getAllPhysical())
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.name = structure_values[i];
|
||||
column.type = data_type_factory.get(structure_values[i + 1]);
|
||||
column.column = column.type->createColumn();
|
||||
column.name = name;
|
||||
column.type = type;
|
||||
column.column = type->createColumn();
|
||||
sample_block.insert(std::move(column));
|
||||
}
|
||||
|
||||
@ -101,7 +111,6 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, const C
|
||||
auto res = StorageValues::create(getDatabaseName(), table_name, res_block);
|
||||
res->startup();
|
||||
return res;
|
||||
|
||||
}
|
||||
|
||||
void registerTableFunctionValues(TableFunctionFactory & factory)
|
||||
|
@ -10,3 +10,4 @@ cadabra
|
||||
abracadabra
|
||||
23 23 23
|
||||
24 24 24
|
||||
1.6660 a b
|
||||
|
@ -9,4 +9,6 @@ SELECT subtractYears(date, 1), subtractYears(date_time, 1) FROM VALUES('date Dat
|
||||
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;
|
||||
|
Loading…
Reference in New Issue
Block a user