提交 ab050c84 编写于 作者: S Sabyanin Maxim

add comment section in column declaration.

add comment column type in alter method.
add comment section in system.columns.
上级 8f7c2b58
......@@ -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(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->default_specifier = toString(it->second.kind);
column_declaration->default_expression = it->second.expression->clone();
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);
......
#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);
}
#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))
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('\n', buf);
continue;
writeChar('\t', buf);
}
else
if (exist_comment)
{
writeChar('\t', buf);
writeText(queryToString(comments_it->second.expression), buf);
}
writeText(DB::toString(it->second.kind), buf);
writeChar('\t', buf);
writeText(queryToString(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);
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);
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});
}
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));
const auto comment_expr = parseCommentExpr(buf);
if (comment_expr)
{
result.comments.emplace(column_name, ColumnComment{comment_expr});
}
result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr});
assertChar('\n', buf);
}
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;
}
}
......
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 │
└────────────────────────────┴───────────────┴────────────────────────┘
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;
\ No newline at end of file
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册