fixed comment test.

remove ColumnComment struct.
style fixes
This commit is contained in:
Sabyanin Maxim 2018-11-06 16:26:43 +03:00
parent 73e9f5057d
commit bdcdc15b1e
14 changed files with 72 additions and 74 deletions

View File

@ -220,9 +220,9 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
}
if (col_decl.comment_expression)
if (!col_decl.comment.empty())
{
comments.emplace(col_decl.name, ColumnComment{col_decl.comment_expression});
comments.emplace(col_decl.name, col_decl.comment);
}
}
@ -351,7 +351,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
const auto comments_it = columns.comments.find(column.name);
if (comments_it != std::end(columns.comments))
{
column_declaration->comment_expression = comments_it->second.expression->clone();
column_declaration->comment = comments_it->second;
}
columns_list->children.push_back(column_declaration_ptr);

View File

@ -45,6 +45,10 @@ Block InterpreterDescribeQuery::getSampleBlock()
col.name = "default_expression";
block.insert(col);
// TODO: may be it unneeded
col.name = "comment_expression";
block.insert(col);
return block;
}
@ -55,6 +59,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
NamesAndTypesList columns;
ColumnDefaults column_defaults;
ColumnComments column_comments;
StoragePtr table;
auto table_expression = typeid_cast<const ASTTableExpression *>(ast.table_expression.get());
@ -101,6 +106,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__);
columns = table->getColumns().getAll();
column_defaults = table->getColumns().defaults;
column_comments = table->getColumns().comments;
}
Block sample_block = getSampleBlock();
@ -111,16 +117,26 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
res_columns[0]->insert(column.name);
res_columns[1]->insert(column.type->getName());
const auto it = column_defaults.find(column.name);
if (it == std::end(column_defaults))
const auto defaults_it = column_defaults.find(column.name);
if (defaults_it == std::end(column_defaults))
{
res_columns[2]->insertDefault();
res_columns[3]->insertDefault();
}
else
{
res_columns[2]->insert(toString(it->second.kind));
res_columns[3]->insert(queryToString(it->second.expression));
res_columns[2]->insert(toString(defaults_it->second.kind));
res_columns[3]->insert(queryToString(defaults_it->second.expression));
}
const auto comments_it = column_comments.find(column.name);
if (comments_it == std::end(column_comments))
{
res_columns[4]->insertDefault();
}
else
{
res_columns[4]->insert(comments_it->second);
}
}

View File

@ -146,7 +146,7 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
settings.ostr << " ";
comment->formatImpl(settings, state, frame);
// comment->formatImpl(settings, state, frame);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);

View File

@ -15,7 +15,7 @@ public:
ASTPtr type;
String default_specifier;
ASTPtr default_expression;
ASTPtr comment_expression;
String comment;
String getID() const override { return "ColumnDeclaration_" + name; }
@ -36,12 +36,6 @@ public:
res->children.push_back(res->default_expression);
}
if (comment_expression)
{
res->comment_expression = comment_expression->clone();
res->children.push_back(res->comment_expression);
}
return res;
}
@ -63,10 +57,9 @@ public:
default_expression->formatImpl(settings, state, frame);
}
if (comment_expression)
if (!comment.empty())
{
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ';
comment_expression->formatImpl(settings, state, frame);
settings.ostr << ' ' << (settings.hilite ? hilite_none : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ' << comment;
}
}
};

View File

@ -6,6 +6,7 @@
#include <Parsers/ASTNameTypePair.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/CommonParsers.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
@ -175,7 +176,8 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
if (comment_expression)
{
column_declaration->comment_expression = comment_expression;
auto & literal_value = typeid_cast<ASTLiteral &>(*comment_expression).value;
column_declaration->comment = literal_value.safeGet<String>();
column_declaration->children.push_back(std::move(comment_expression));
}

View File

@ -7,7 +7,6 @@
namespace DB
{
// TODO: возможно тут тоже надо разобраться
/** Query (EXISTS | SHOW CREATE) [TABLE] [db.]name [FORMAT format]
*/
class ParserTablePropertiesQuery : public IParserBase

View File

@ -94,9 +94,10 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
{
AlterCommand command;
command.type = COMMENT_COLUMN;
const auto & ast_identifier = typeid_cast<ASTIdentifier&>(*command_ast->column);
const auto & ast_identifier = typeid_cast<ASTIdentifier &>(*command_ast->column);
command.column_name = ast_identifier.name;
command.comment_expression = command_ast->comment;
const auto & ast_comment = typeid_cast<ASTLiteral &>(*command_ast->comment);
command.comment = ast_comment.value.get<String>();
return command;
}
else
@ -249,7 +250,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const
else if (type == COMMENT_COLUMN)
{
columns_description.comments[column_name].expression = comment_expression;
columns_description.comments[column_name] = comment;
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);

View File

@ -36,7 +36,7 @@ struct AlterCommand
ColumnDefaultKind default_kind{};
ASTPtr default_expression{};
ASTPtr comment_expression;
String comment;
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
String after_column;
@ -47,9 +47,9 @@ struct AlterCommand
AlterCommand() = default;
AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type,
const ColumnDefaultKind default_kind, const ASTPtr & default_expression,
const String & after_column = String{}, const ASTPtr & comment_expression = nullptr)
const String & after_column = String{}, const String & comment = "")
: type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind},
default_expression{default_expression}, comment_expression(comment_expression), after_column{after_column}
default_expression{default_expression}, comment(comment), after_column{after_column}
{}
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);

View File

@ -1,7 +0,0 @@
#include <Parsers/queryToString.h>
#include <Storages/ColumnComment.h>
bool DB::operator== (const DB::ColumnComment& lhs, const DB::ColumnComment& rhs)
{
return queryToString(lhs.expression) == queryToString(rhs.expression);
}

View File

@ -8,13 +8,6 @@
namespace DB
{
struct ColumnComment
{
ASTPtr expression;
};
bool operator== (const ColumnComment& lhs, const ColumnComment& rhs);
using ColumnComments = std::unordered_map<std::string, ColumnComment>;
using ColumnComments = std::unordered_map<std::string, String>;
}

View File

@ -1,4 +1,5 @@
#include <Storages/ColumnsDescription.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
@ -10,6 +11,7 @@
#include <IO/ReadBufferFromString.h>
#include <DataTypes/DataTypeFactory.h>
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <ext/collection_cast.h>
#include <ext/map.h>
@ -82,7 +84,7 @@ String ColumnsDescription::toString() const
writeChar(' ', buf);
writeText(column.type->getName(), buf);
const bool exist_comment = comments_it != std::end(comments) && !comments_it->second.expression;
const bool exist_comment = comments_it != std::end(comments);
if (defaults_it != std::end(defaults))
{
writeChar('\t', buf);
@ -98,7 +100,7 @@ String ColumnsDescription::toString() const
if (exist_comment)
{
writeChar('\t', buf);
writeText(queryToString(comments_it->second.expression), buf);
writeText(comments_it->second, buf);
}
writeChar('\n', buf);
@ -146,7 +148,7 @@ std::optional<ParsedDefaultInfo> parseDefaulfInfo(ReadBufferFromString & buf)
return ParsedDefaultInfo{default_kind, std::move(default_expr)};
}
ASTPtr parseCommentExpr(ReadBufferFromString& buf)
String parseComment(ReadBufferFromString& buf)
{
if (*buf.position() == '\n')
{
@ -154,12 +156,9 @@ ASTPtr parseCommentExpr(ReadBufferFromString& buf)
}
ParserExpression parser_expr;
String comment_expr_str;
readText(comment_expr_str, buf);
const char * begin = comment_expr_str.data();
const auto end = begin + comment_expr_str.size();
ASTPtr comment_expr = parseQuery(parser_expr, begin, end, "comment_expression", 0);
return comment_expr;
String comment;
readText(comment, buf); // This is wrong may be
return comment;
}
ColumnsDescription ColumnsDescription::parse(const String & str)
@ -209,10 +208,10 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr});
}
const auto comment_expr = parseCommentExpr(buf);
if (comment_expr)
const auto comment = parseComment(buf);
if (!comment.empty())
{
result.comments.emplace(column_name, ColumnComment{comment_expr});
result.comments.emplace(column_name, comment);
}
assertChar('\n', buf);

View File

@ -186,9 +186,8 @@ protected:
}
else
{
const auto & literal = typeid_cast<ASTLiteral*>(it->second.expression.get());
if (columns_mask[src_index++])
res_columns[res_index++]->insert(literal->value.get<String>());
res_columns[res_index++]->insert(it->second);
}
}

View File

@ -1,4 +1,4 @@
CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog
CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog
┌─table──────────────────────┬─name──────────┬─comment────────┐
│ check_query_comment_column │ first_column │ first comment │
│ check_query_comment_column │ fourth_column │ fourth comment │
@ -6,7 +6,7 @@ CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 C
│ check_query_comment_column │ second_column │ second comment │
│ check_query_comment_column │ third_column │ third comment │
└────────────────────────────┴───────────────┴────────────────┘
CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog
CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog
┌─table──────────────────────┬─name──────────┬─comment───────────────┐
│ check_query_comment_column │ first_column │ another first column │
│ check_query_comment_column │ fourth_column │ another fourth column │
@ -14,13 +14,13 @@ CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 C
│ check_query_comment_column │ second_column │ another second column │
│ check_query_comment_column │ third_column │ another third column │
└────────────────────────────┴───────────────┴───────────────────────┘
CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192)
CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192)
┌─table──────────────────────┬─name──────────┬─comment────────┐
│ check_query_comment_column │ first_column │ first comment │
│ check_query_comment_column │ second_column │ second comment │
│ check_query_comment_column │ third_column │ third comment │
└────────────────────────────┴───────────────┴────────────────┘
CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192)
CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192)
┌─table──────────────────────┬─name──────────┬─comment────────────────┐
│ check_query_comment_column │ first_column │ another first comment │
│ check_query_comment_column │ second_column │ another second comment │

View File

@ -1,6 +1,7 @@
DROP TABLE IF EXISTS check_query_comment_column;
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.check_query_comment_column;
CREATE TABLE check_query_comment_column
CREATE TABLE test.check_query_comment_column
(
first_column UInt8 DEFAULT 1 COMMENT 'first comment',
second_column UInt8 MATERIALIZED first_column COMMENT 'second comment',
@ -9,52 +10,54 @@ CREATE TABLE check_query_comment_column
fifth_column UInt8
) ENGINE = TinyLog;
SHOW CREATE TABLE check_query_comment_column;
SHOW CREATE TABLE test.check_query_comment_column;
SELECT table, name, comment
FROM system.columns
WHERE table = 'check_query_comment_column'
WHERE table = 'check_query_comment_column' AND database = 'test'
FORMAT PrettyCompactNoEscapes;
ALTER TABLE check_query_comment_column
ALTER TABLE test.check_query_comment_column
COMMENT COLUMN first_column 'another first column',
COMMENT COLUMN second_column 'another second column',
COMMENT COLUMN third_column 'another third column',
COMMENT COLUMN fourth_column 'another fourth column',
COMMENT COLUMN fifth_column 'another fifth column';
SHOW CREATE TABLE check_query_comment_column;
SHOW CREATE TABLE test.check_query_comment_column;
SELECT table, name, comment
FROM system.columns
WHERE table = 'check_query_comment_column'
WHERE table = 'check_query_comment_column' AND database = 'test'
FORMAT PrettyCompactNoEscapes;
DROP TABLE IF EXISTS check_query_comment_column;
DROP TABLE IF EXISTS test.check_query_comment_column;
CREATE TABLE check_query_comment_column
CREATE TABLE test.check_query_comment_column
(
first_column Date COMMENT 'first comment',
second_column UInt8 COMMENT 'second comment',
third_column UInt8 COMMENT 'third comment'
) ENGINE = MergeTree(first_column, (second_column, second_column), 8192);
SHOW CREATE TABLE check_query_comment_column;
SHOW CREATE TABLE test.check_query_comment_column;
SELECT table, name, comment
FROM system.columns
WHERE table = 'check_query_comment_column'
WHERE table = 'check_query_comment_column' AND database = 'test'
FORMAT PrettyCompactNoEscapes;
ALTER TABLE check_query_comment_column
ALTER TABLE test.check_query_comment_column
COMMENT COLUMN first_column 'another first comment',
COMMENT COLUMN second_column 'another second comment',
COMMENT COLUMN third_column 'another third comment';
SHOW CREATE TABLE check_query_comment_column;
SHOW CREATE TABLE test.check_query_comment_column;
SELECT table, name, comment
FROM system.columns
WHERE table = 'check_query_comment_column'
FORMAT PrettyCompactNoEscapes;
WHERE table = 'check_query_comment_column' and database = 'test'
FORMAT PrettyCompactNoEscapes;
DROP table test.check_query_comment_column;