mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
add comment section in column declaration.
add comment column type in alter method. add comment section in system.columns.
This commit is contained in:
parent
8f7c2b5821
commit
ab050c8466
@ -716,7 +716,7 @@ bool TCPHandler::receiveData()
|
||||
{
|
||||
NamesAndTypesList columns = block.getNamesAndTypesList();
|
||||
storage = StorageMemory::create(external_table_name,
|
||||
ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}});
|
||||
ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, ColumnComments{}});
|
||||
storage->startup();
|
||||
query_context.addExternalTable(external_table_name, storage);
|
||||
}
|
||||
|
@ -166,13 +166,15 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
|
||||
|
||||
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
||||
using ParsedColumns = std::tuple<NamesAndTypesList, ColumnDefaults, ColumnComments>;
|
||||
|
||||
/// AST to the list of columns with types. Columns of Nested type are expanded into a list of real columns.
|
||||
static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast, const Context & context)
|
||||
static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, const Context & context)
|
||||
{
|
||||
/// list of table columns in correct order
|
||||
NamesAndTypesList columns{};
|
||||
ColumnDefaults defaults{};
|
||||
ColumnComments comments{};
|
||||
|
||||
/// Columns requiring type-deduction or default_expression type-check
|
||||
std::vector<std::pair<NameAndTypePair *, ASTColumnDeclaration *>> defaulted_columns{};
|
||||
@ -216,6 +218,11 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
|
||||
else
|
||||
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
||||
}
|
||||
|
||||
if (col_decl.comment_expression)
|
||||
{
|
||||
comments.emplace(col_decl.name, ColumnComment{col_decl.comment_expression});
|
||||
}
|
||||
}
|
||||
|
||||
/// set missing types and wrap default_expression's in a conversion-function if necessary
|
||||
@ -261,7 +268,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
|
||||
}
|
||||
}
|
||||
|
||||
return {Nested::flatten(columns), defaults};
|
||||
return {Nested::flatten(columns), defaults, comments};
|
||||
}
|
||||
|
||||
|
||||
@ -329,11 +336,17 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
||||
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
|
||||
column_declaration->type->owned_string = type_name;
|
||||
|
||||
const auto it = columns.defaults.find(column.name);
|
||||
if (it != std::end(columns.defaults))
|
||||
const auto defaults_it = columns.defaults.find(column.name);
|
||||
if (defaults_it != std::end(columns.defaults))
|
||||
{
|
||||
column_declaration->default_specifier = toString(it->second.kind);
|
||||
column_declaration->default_expression = it->second.expression->clone();
|
||||
column_declaration->default_specifier = toString(defaults_it->second.kind);
|
||||
column_declaration->default_expression = defaults_it->second.expression->clone();
|
||||
}
|
||||
|
||||
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();
|
||||
}
|
||||
|
||||
columns_list->children.push_back(column_declaration_ptr);
|
||||
@ -347,11 +360,13 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres
|
||||
{
|
||||
ColumnsDescription res;
|
||||
|
||||
auto && columns_and_defaults = parseColumns(columns, context);
|
||||
auto && parsed_columns = parseColumns(columns, context);
|
||||
auto columns_and_defaults = std::make_pair(std::move(std::get<0>(parsed_columns)), std::move(std::get<1>(parsed_columns)));
|
||||
res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized);
|
||||
res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias);
|
||||
res.ordinary = std::move(columns_and_defaults.first);
|
||||
res.defaults = std::move(columns_and_defaults.second);
|
||||
res.comments = std::move(std::get<2>(parsed_columns));
|
||||
|
||||
if (res.ordinary.size() + res.materialized.size() == 0)
|
||||
throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED};
|
||||
|
@ -44,7 +44,7 @@ public:
|
||||
internal = internal_;
|
||||
}
|
||||
|
||||
/// Obtain information about columns, their types and default values, for case when columns in CREATE query is specified explicitly.
|
||||
/// Obtain information about columns, their types, default values and column comments, for case when columns in CREATE query is specified explicitly.
|
||||
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context);
|
||||
/// Check that column types are allowed for usage in table according to settings.
|
||||
static void checkSupportedTypes(const ColumnsDescription & columns, const Context & context);
|
||||
|
@ -141,6 +141,13 @@ void ASTAlterCommand::formatImpl(
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : "");
|
||||
predicate->formatImpl(settings, state, frame);
|
||||
}
|
||||
else if (type == ASTAlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
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);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
@ -14,6 +14,7 @@ namespace DB
|
||||
* DROP COLUMN col_drop [FROM PARTITION partition],
|
||||
* MODIFY COLUMN col_name type,
|
||||
* DROP PARTITION partition,
|
||||
* COMMENT_COLUMN col_name 'comment',
|
||||
*/
|
||||
|
||||
class ASTAlterCommand : public IAST
|
||||
@ -25,6 +26,7 @@ public:
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
COMMENT_COLUMN,
|
||||
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
@ -66,6 +68,9 @@ public:
|
||||
/// A list of expressions of the form `column = expr` for the UPDATE command.
|
||||
ASTPtr update_assignments;
|
||||
|
||||
/// A column comment
|
||||
ASTPtr comment;
|
||||
|
||||
bool detach = false; /// true for DETACH PARTITION
|
||||
|
||||
bool part = false; /// true for ATTACH PART
|
||||
|
@ -5,7 +5,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Name, type, default-specifier, default-expression.
|
||||
/** Name, type, default-specifier, default-expression, comment-expression.
|
||||
* The type is optional if default-expression is specified.
|
||||
*/
|
||||
class ASTColumnDeclaration : public IAST
|
||||
@ -38,13 +38,12 @@ public:
|
||||
|
||||
if (comment_expression) {
|
||||
res->comment_expression = comment_expression->clone();
|
||||
res->children.push_back(res->comment_expression); // TODO: понять, зачем это нужно.
|
||||
res->children.push_back(res->comment_expression);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
protected:
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
|
||||
{
|
||||
frame.need_parens = false;
|
||||
@ -63,8 +62,8 @@ protected:
|
||||
default_expression->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
// TODO: понять, почему не отрицание
|
||||
if (comment_expression) {
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
comment_expression->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_clear_column("CLEAR COLUMN");
|
||||
ParserKeyword s_modify_column("MODIFY COLUMN");
|
||||
ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY");
|
||||
ParserKeyword s_comment_column("COMMENT COLUMN");
|
||||
|
||||
ParserKeyword s_attach_partition("ATTACH PARTITION");
|
||||
ParserKeyword s_detach_partition("DETACH PARTITION");
|
||||
@ -220,6 +221,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
command->type = ASTAlterCommand::UPDATE;
|
||||
}
|
||||
else if (s_comment_column.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
if (!parser_string_literal.parse(pos, command->comment, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::COMMENT_COLUMN;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
|
@ -13,6 +13,7 @@ namespace DB
|
||||
* [CLEAR COLUMN col_to_clear [IN PARTITION partition],]
|
||||
* [MODIFY COLUMN col_to_modify type, ...]
|
||||
* [MODIFY PRIMARY KEY (a, b, c...)]
|
||||
* [COMMENT COLUMN col_name string]
|
||||
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
|
||||
* [FETCH PARTITION partition FROM ...]
|
||||
* [FREEZE PARTITION]
|
||||
|
@ -113,6 +113,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserKeyword s_alias{"ALIAS"};
|
||||
ParserKeyword s_comment{"COMMENT"};
|
||||
ParserTernaryOperatorExpression expr_parser;
|
||||
ParserStringLiteral string_literal_parser;
|
||||
|
||||
/// mandatory column name
|
||||
ASTPtr name;
|
||||
@ -120,14 +121,13 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
return false;
|
||||
|
||||
/** column name should be followed by type name if it
|
||||
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT}
|
||||
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS}
|
||||
*/
|
||||
ASTPtr type;
|
||||
const auto fallback_pos = pos;
|
||||
if (!s_default.check(pos, expected) &&
|
||||
!s_materialized.check(pos, expected) &&
|
||||
!s_alias.check(pos, expected) &&
|
||||
!s_comment.check(pos, expected))
|
||||
!s_alias.check(pos, expected))
|
||||
{
|
||||
type_parser.parse(pos, type, expected);
|
||||
}
|
||||
@ -151,15 +151,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
else if (!type)
|
||||
return false; /// reject sole column name without type
|
||||
|
||||
String comment_specifier;
|
||||
ASTPtr comment_expression;
|
||||
pos_before_specifier = pos;
|
||||
if (s_comment.ignore(pos, expected))
|
||||
{
|
||||
comment_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_specifier->end});
|
||||
if (!expr_parser.parse(pos, comment_expression, expected)) {
|
||||
return false;
|
||||
}
|
||||
string_literal_parser.parse(pos, comment_expression, expected);
|
||||
}
|
||||
|
||||
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
||||
|
@ -90,6 +90,15 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
command.primary_key = command_ast->primary_key;
|
||||
return command;
|
||||
}
|
||||
else if (command_ast->type == ASTAlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
AlterCommand command;
|
||||
command.type = COMMENT_COLUMN;
|
||||
const auto & ast_identifier = typeid_cast<ASTIdentifier&>(*command_ast->column);
|
||||
command.column_name = ast_identifier.name;
|
||||
command.comment_expression = command_ast->comment;
|
||||
return command;
|
||||
}
|
||||
else
|
||||
return {};
|
||||
}
|
||||
@ -237,6 +246,11 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const
|
||||
/// This have no relation to changing the list of columns.
|
||||
/// TODO Check that all columns exist, that only columns with constant defaults are added.
|
||||
}
|
||||
else if (type == COMMENT_COLUMN)
|
||||
{
|
||||
|
||||
columns_description.comments[column_name].expression = comment_expression;
|
||||
}
|
||||
else
|
||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
@ -353,6 +367,15 @@ void AlterCommands::validate(const IStorage & table, const Context & context)
|
||||
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else if (command.type == AlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
const auto column_it = std::find_if(std::begin(all_columns), std::end(all_columns),
|
||||
std::bind(namesEqual, std::cref(command.column_name), std::placeholders::_1));
|
||||
if (column_it == std::end(all_columns))
|
||||
{
|
||||
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Existing defaulted columns may require default expression extensions with a type conversion,
|
||||
|
@ -21,6 +21,7 @@ struct AlterCommand
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
MODIFY_PRIMARY_KEY,
|
||||
COMMENT_COLUMN,
|
||||
};
|
||||
|
||||
Type type;
|
||||
@ -35,6 +36,7 @@ struct AlterCommand
|
||||
|
||||
ColumnDefaultKind default_kind{};
|
||||
ASTPtr default_expression{};
|
||||
ASTPtr comment_expression;
|
||||
|
||||
/// 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;
|
||||
@ -45,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 String & after_column = String{}, const ASTPtr & comment_expression = nullptr)
|
||||
: type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind},
|
||||
default_expression{default_expression}, after_column{after_column}
|
||||
default_expression{default_expression}, comment_expression(comment_expression), after_column{after_column}
|
||||
{}
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
7
dbms/src/Storages/ColumnComment.cpp
Normal file
7
dbms/src/Storages/ColumnComment.cpp
Normal file
@ -0,0 +1,7 @@
|
||||
#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);
|
||||
}
|
19
dbms/src/Storages/ColumnComment.h
Normal file
19
dbms/src/Storages/ColumnComment.h
Normal file
@ -0,0 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ColumnComment {
|
||||
ASTPtr expression;
|
||||
};
|
||||
|
||||
bool operator== (const ColumnComment& lhs, const ColumnComment& rhs);
|
||||
|
||||
using ColumnComments = std::unordered_map<std::string, ColumnComment>;
|
||||
|
||||
}
|
@ -75,22 +75,32 @@ String ColumnsDescription::toString() const
|
||||
{
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
const auto it = defaults.find(column.name);
|
||||
const auto defaults_it = defaults.find(column.name);
|
||||
const auto comments_it = comments.find(column.name);
|
||||
|
||||
writeBackQuotedString(column.name, buf);
|
||||
writeChar(' ', buf);
|
||||
writeText(column.type->getName(), buf);
|
||||
if (it == std::end(defaults))
|
||||
{
|
||||
writeChar('\n', buf);
|
||||
continue;
|
||||
}
|
||||
else
|
||||
writeChar('\t', buf);
|
||||
|
||||
writeText(DB::toString(it->second.kind), buf);
|
||||
writeChar('\t', buf);
|
||||
writeText(queryToString(it->second.expression), buf);
|
||||
const bool exist_comment = comments_it != std::end(comments) && !comments_it->second.expression;
|
||||
if (defaults_it != std::end(defaults))
|
||||
{
|
||||
writeChar('\t', buf);
|
||||
writeText(DB::toString(defaults_it->second.kind), buf);
|
||||
writeChar('\t', buf);
|
||||
writeText(queryToString(defaults_it->second.expression), buf);
|
||||
}
|
||||
else if (exist_comment)
|
||||
{
|
||||
writeChar('\t', buf);
|
||||
}
|
||||
|
||||
if (exist_comment)
|
||||
{
|
||||
writeChar('\t', buf);
|
||||
writeText(queryToString(comments_it->second.expression), buf);
|
||||
}
|
||||
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
};
|
||||
@ -102,6 +112,55 @@ String ColumnsDescription::toString() const
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
struct ParsedDefaultInfo
|
||||
{
|
||||
ColumnDefaultKind default_kind;
|
||||
ASTPtr default_expr_str;
|
||||
};
|
||||
|
||||
std::optional<ParsedDefaultInfo> parseDefaulfInfo(ReadBufferFromString & buf)
|
||||
{
|
||||
if (*buf.position() == '\n')
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
assertChar('\t', buf);
|
||||
if (*buf.position() == '\t')
|
||||
{
|
||||
assertChar('\t', buf);
|
||||
return {};
|
||||
}
|
||||
|
||||
String default_kind_str;
|
||||
readText(default_kind_str, buf);
|
||||
const auto default_kind = columnDefaultKindFromString(default_kind_str);
|
||||
assertChar('\t', buf);
|
||||
|
||||
ParserExpression expr_parser;
|
||||
String default_expr_str;
|
||||
readText(default_expr_str, buf);
|
||||
const char * begin = default_expr_str.data();
|
||||
const auto end = begin + default_expr_str.size();
|
||||
ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default_expression", 0);
|
||||
return ParsedDefaultInfo{default_kind, std::move(default_expr)};
|
||||
}
|
||||
|
||||
ASTPtr parseCommentExpr(ReadBufferFromString& buf)
|
||||
{
|
||||
if (*buf.position() == '\n')
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
ColumnsDescription ColumnsDescription::parse(const String & str)
|
||||
{
|
||||
@ -132,29 +191,31 @@ ColumnsDescription ColumnsDescription::parse(const String & str)
|
||||
result.ordinary.emplace_back(column_name, std::move(type));
|
||||
continue;
|
||||
}
|
||||
|
||||
assertChar('\t', buf);
|
||||
|
||||
String default_kind_str;
|
||||
readText(default_kind_str, buf);
|
||||
const auto default_kind = columnDefaultKindFromString(default_kind_str);
|
||||
assertChar('\t', buf);
|
||||
const auto default_info = parseDefaulfInfo(buf);
|
||||
if (default_info)
|
||||
{
|
||||
const auto & default_kind = default_info->default_kind;
|
||||
const auto & default_expr = default_info->default_expr_str;
|
||||
if (ColumnDefaultKind::Default == default_kind)
|
||||
result.ordinary.emplace_back(column_name, std::move(type));
|
||||
else if (ColumnDefaultKind::Materialized == default_kind)
|
||||
result.materialized.emplace_back(column_name, std::move(type));
|
||||
else if (ColumnDefaultKind::Alias == default_kind)
|
||||
result.aliases.emplace_back(column_name, std::move(type));
|
||||
|
||||
result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr});
|
||||
}
|
||||
|
||||
const auto comment_expr = parseCommentExpr(buf);
|
||||
if (comment_expr)
|
||||
{
|
||||
result.comments.emplace(column_name, ColumnComment{comment_expr});
|
||||
}
|
||||
|
||||
String default_expr_str;
|
||||
readText(default_expr_str, buf);
|
||||
assertChar('\n', buf);
|
||||
|
||||
const char * begin = default_expr_str.data();
|
||||
const auto end = begin + default_expr_str.size();
|
||||
ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression", 0);
|
||||
|
||||
if (ColumnDefaultKind::Default == default_kind)
|
||||
result.ordinary.emplace_back(column_name, std::move(type));
|
||||
else if (ColumnDefaultKind::Materialized == default_kind)
|
||||
result.materialized.emplace_back(column_name, std::move(type));
|
||||
else if (ColumnDefaultKind::Alias == default_kind)
|
||||
result.aliases.emplace_back(column_name, std::move(type));
|
||||
|
||||
result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr});
|
||||
}
|
||||
|
||||
assertEOF(buf);
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Storages/ColumnComment.h>
|
||||
#include <Storages/ColumnDefault.h>
|
||||
#include <Core/Block.h>
|
||||
|
||||
@ -15,6 +16,7 @@ struct ColumnsDescription
|
||||
NamesAndTypesList materialized;
|
||||
NamesAndTypesList aliases;
|
||||
ColumnDefaults defaults;
|
||||
ColumnComments comments;
|
||||
|
||||
ColumnsDescription() = default;
|
||||
|
||||
@ -22,11 +24,13 @@ struct ColumnsDescription
|
||||
NamesAndTypesList ordinary_,
|
||||
NamesAndTypesList materialized_,
|
||||
NamesAndTypesList aliases_,
|
||||
ColumnDefaults defaults_)
|
||||
ColumnDefaults defaults_,
|
||||
ColumnComments comments_ = {})
|
||||
: ordinary(std::move(ordinary_))
|
||||
, materialized(std::move(materialized_))
|
||||
, aliases(std::move(aliases_))
|
||||
, defaults(std::move(defaults_))
|
||||
, comments(std::move(comments_))
|
||||
{}
|
||||
|
||||
explicit ColumnsDescription(NamesAndTypesList ordinary_) : ordinary(std::move(ordinary_)) {}
|
||||
@ -36,7 +40,8 @@ struct ColumnsDescription
|
||||
return ordinary == other.ordinary
|
||||
&& materialized == other.materialized
|
||||
&& aliases == other.aliases
|
||||
&& defaults == other.defaults;
|
||||
&& defaults == other.defaults
|
||||
&& comments == other.comments;
|
||||
}
|
||||
|
||||
bool operator!=(const ColumnsDescription & other) const { return !(*this == other); }
|
||||
|
@ -4,6 +4,8 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/RWLockFIFO.h>
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Storages/ITableDeclaration.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <shared_mutex>
|
||||
@ -233,9 +235,19 @@ public:
|
||||
* This method must fully execute the ALTER query, taking care of the locks itself.
|
||||
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
|
||||
*/
|
||||
virtual void alter(const AlterCommands & /*params*/, const String & /*database_name*/, const String & /*table_name*/, const Context & /*context*/)
|
||||
virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
|
||||
{
|
||||
throw Exception("Method alter is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
for (const auto & param : params)
|
||||
{
|
||||
if (param.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
throw Exception("Method alter only supports change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
auto lock = lockStructureForAlter(__PRETTY_FUNCTION__);
|
||||
auto new_columns = getColumns();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
/** Execute CLEAR COLUMN ... IN PARTITION query which removes column from given partition. */
|
||||
|
@ -909,6 +909,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands)
|
||||
|
||||
for (const AlterCommand & command : commands)
|
||||
{
|
||||
if (command.type == AlterCommand::COMMENT_COLUMN)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
if (columns_alter_forbidden.count(command.column_name))
|
||||
throw Exception("trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
|
@ -37,6 +37,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_)
|
||||
{ "data_compressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "data_uncompressed_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "marks_bytes", std::make_shared<DataTypeUInt64>() },
|
||||
{ "comment", std::make_shared<DataTypeString>() },
|
||||
}));
|
||||
}
|
||||
|
||||
@ -81,6 +82,7 @@ protected:
|
||||
|
||||
NamesAndTypesList columns;
|
||||
ColumnDefaults column_defaults;
|
||||
ColumnComments column_comments;
|
||||
MergeTreeData::ColumnSizeByName column_sizes;
|
||||
|
||||
{
|
||||
@ -106,6 +108,7 @@ protected:
|
||||
|
||||
columns = storage->getColumns().getAll();
|
||||
column_defaults = storage->getColumns().defaults;
|
||||
column_comments = storage->getColumns().comments;
|
||||
|
||||
/** Info about sizes of columns for tables of MergeTree family.
|
||||
* NOTE: It is possible to add getter for this info to IStorage interface.
|
||||
@ -174,6 +177,21 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
const auto it = column_comments.find(column.name);
|
||||
if (it == std::end(column_comments))
|
||||
{
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
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>());
|
||||
}
|
||||
}
|
||||
|
||||
++rows_count;
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,28 @@
|
||||
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
|
||||
┌─table──────────────────────┬─name──────────┬─comment────────┐
|
||||
│ check_query_comment_column │ first_column │ first comment │
|
||||
│ check_query_comment_column │ fourth_column │ fourth comment │
|
||||
│ check_query_comment_column │ fifth_column │ │
|
||||
│ 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
|
||||
┌─table──────────────────────┬─name──────────┬─comment───────────────┐
|
||||
│ check_query_comment_column │ first_column │ another first column │
|
||||
│ check_query_comment_column │ fourth_column │ another fourth column │
|
||||
│ check_query_comment_column │ fifth_column │ another fifth column │
|
||||
│ 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)
|
||||
┌─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)
|
||||
┌─table──────────────────────┬─name──────────┬─comment────────────────┐
|
||||
│ check_query_comment_column │ first_column │ another first comment │
|
||||
│ check_query_comment_column │ second_column │ another second comment │
|
||||
│ check_query_comment_column │ third_column │ another third comment │
|
||||
└────────────────────────────┴───────────────┴────────────────────────┘
|
60
dbms/tests/queries/0_stateless/00725_comment_columns.sql
Normal file
60
dbms/tests/queries/0_stateless/00725_comment_columns.sql
Normal file
@ -0,0 +1,60 @@
|
||||
DROP TABLE IF EXISTS check_query_comment_column;
|
||||
|
||||
CREATE TABLE check_query_comment_column
|
||||
(
|
||||
first_column UInt8 DEFAULT 1 COMMENT 'first comment',
|
||||
second_column UInt8 MATERIALIZED first_column COMMENT 'second comment',
|
||||
third_column UInt8 ALIAS second_column COMMENT 'third comment',
|
||||
fourth_column UInt8 COMMENT 'fourth comment',
|
||||
fifth_column UInt8
|
||||
) ENGINE = TinyLog;
|
||||
|
||||
SHOW CREATE TABLE check_query_comment_column;
|
||||
|
||||
SELECT table, name, comment
|
||||
FROM system.columns
|
||||
WHERE table = 'check_query_comment_column'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
ALTER TABLE 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;
|
||||
|
||||
SELECT table, name, comment
|
||||
FROM system.columns
|
||||
WHERE table = 'check_query_comment_column'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
DROP TABLE IF EXISTS check_query_comment_column;
|
||||
|
||||
|
||||
CREATE TABLE 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;
|
||||
|
||||
SELECT table, name, comment
|
||||
FROM system.columns
|
||||
WHERE table = 'check_query_comment_column'
|
||||
FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
ALTER TABLE 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;
|
||||
|
||||
SELECT table, name, comment
|
||||
FROM system.columns
|
||||
WHERE table = 'check_query_comment_column'
|
||||
FORMAT PrettyCompactNoEscapes;
|
Loading…
Reference in New Issue
Block a user