Merge pull request #29899 from Enmk/Governance/dictionary_comment

Governance/dictionary comment
This commit is contained in:
Maksim Kita 2021-10-14 12:35:10 +03:00 committed by GitHub
commit f589084a94
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 169 additions and 49 deletions

View File

@ -80,8 +80,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
// Attach database metadata
if (impl && create.storage && create.storage->comment)
impl->setDatabaseComment(create.storage->comment->as<ASTLiteral>()->value.safeGet<String>());
if (impl && create.comment)
impl->setDatabaseComment(create.comment->as<ASTLiteral>()->value.safeGet<String>());
return impl;
}

View File

@ -79,7 +79,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const
create_query->storage->set(create_query->storage->engine, makeASTFunction(getEngineName()));
if (const auto comment_value = getDatabaseComment(); !comment_value.empty())
create_query->storage->set(create_query->storage->comment, std::make_shared<ASTLiteral>(comment_value));
create_query->set(create_query->comment, std::make_shared<ASTLiteral>(comment_value));
return create_query;
}

View File

@ -189,12 +189,12 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo
if (metadata.settings_changes)
storage_ast.set(storage_ast.settings, metadata.settings_changes);
}
}
if (metadata.comment.empty())
storage_ast.reset(storage_ast.comment);
ast_create_query.reset(ast_create_query.comment);
else
storage_ast.set(storage_ast.comment, std::make_shared<ASTLiteral>(metadata.comment));
}
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(metadata.comment));
}
@ -532,11 +532,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
{
auto & ast_create_query = ast->as<ASTCreateQuery &>();
// TODO(nemkov): this is a precaution and should never happen, remove if there are no failed tests on CI/CD.
if (!ast_create_query.storage)
throw Exception(ErrorCodes::LOGICAL_ERROR, "ASTCreateQuery lacks engine clause, but a comment is present.");
ast_create_query.storage->set(ast_create_query.storage->comment, std::make_shared<ASTLiteral>(database_comment));
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(database_comment));
}
return ast;

View File

@ -198,7 +198,7 @@ ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const
create_query->set(create_query->storage, database_engine_define);
if (const auto comment_value = getDatabaseComment(); !comment_value.empty())
create_query->storage->set(create_query->storage->comment, std::make_shared<ASTLiteral>(comment_value));
create_query->set(create_query->comment, std::make_shared<ASTLiteral>(comment_value));
return create_query;
}

View File

@ -349,7 +349,7 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const
create_query->set(create_query->storage, database_engine_define);
if (const auto comment_value = getDatabaseComment(); !comment_value.empty())
create_query->storage->set(create_query->storage->comment, std::make_shared<ASTLiteral>(comment_value));
create_query->set(create_query->comment, std::make_shared<ASTLiteral>(comment_value));
return create_query;
}

View File

@ -164,7 +164,7 @@ ASTPtr DatabaseSQLite::getCreateDatabaseQuery() const
create_query->set(create_query->storage, database_engine_define);
if (const auto comment_value = getDatabaseComment(); !comment_value.empty())
create_query->storage->set(create_query->storage->comment, std::make_shared<ASTLiteral>(comment_value));
create_query->set(create_query->comment, std::make_shared<ASTLiteral>(comment_value));
return create_query;
}

View File

@ -55,7 +55,11 @@ DictionaryPtr DictionaryFactory::create(
if (found != registered_layouts.end())
{
const auto & layout_creator = found->second.layout_create_function;
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
auto result = layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl);
if (config.hasProperty(config_prefix + ".comment"))
result->setDictionaryComment(config.getString(config_prefix + ".comment"));
return result;
}
}

View File

@ -216,12 +216,25 @@ struct IDictionary : public IExternalLoadable
return std::static_pointer_cast<const IDictionary>(IExternalLoadable::shared_from_this());
}
void setDictionaryComment(String new_comment)
{
std::lock_guard lock{name_mutex};
dictionary_comment = std::move(new_comment);
}
String getDictionaryComment() const
{
std::lock_guard lock{name_mutex};
return dictionary_comment;
}
private:
mutable std::mutex name_mutex;
mutable StorageID dictionary_id;
protected:
const String full_name;
String dictionary_comment;
};
}

View File

@ -573,6 +573,15 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte
if (query.dictionary->range)
buildRangeConfiguration(xml_document, structure_element, query.dictionary->range, all_attr_names_and_types);
if (query.comment)
{
AutoPtr<Element> comment_element(xml_document->createElement("comment"));
current_dictionary->appendChild(comment_element);
AutoPtr<Text> comment_value(xml_document->createTextNode(query.comment->as<ASTLiteral>()->value.safeGet<String>()));
comment_element->appendChild(comment_value);
}
conf->load(xml_document);
return conf;
}

View File

@ -44,7 +44,8 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration)
" SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))"
" LAYOUT(FLAT())"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
" RANGE(MIN second_column MAX third_column)"
" COMMENT 'hello world!'";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
@ -92,6 +93,9 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration)
/// layout
EXPECT_TRUE(config->has("dictionary.layout.flat"));
// comment
EXPECT_TRUE(config->has("dictionary.comment"));
}

View File

@ -32,9 +32,6 @@ ASTPtr ASTStorage::clone() const
if (settings)
res->set(res->settings, settings->clone());
if (comment)
res->set(res->comment, comment->clone());
return res;
}
@ -75,12 +72,6 @@ void ASTStorage::formatImpl(const FormatSettings & s, FormatState & state, Forma
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : "");
settings->formatImpl(s, state, frame);
}
if (comment)
{
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "COMMENT " << (s.hilite ? hilite_none : "");
comment->formatImpl(s, state, frame);
}
}
@ -217,6 +208,9 @@ ASTPtr ASTCreateQuery::clone() const
res->set(res->dictionary, dictionary->clone());
}
if (comment)
res->set(res->comment, comment->clone());
cloneOutputOptions(*res);
return res;
@ -245,6 +239,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (storage)
storage->formatImpl(settings, state, frame);
if (comment)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
}
return;
}
@ -405,6 +405,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH " << (settings.hilite ? hilite_none : "");
tables->formatImpl(settings, state, frame);
}
if (comment)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : "");
comment->formatImpl(settings, state, frame);
}
}
}

View File

@ -23,7 +23,6 @@ public:
IAST * order_by = nullptr;
IAST * sample_by = nullptr;
IAST * ttl_table = nullptr;
IAST * comment = nullptr;
ASTSetQuery * settings = nullptr;
@ -75,6 +74,7 @@ public:
String as_table;
ASTPtr as_table_function;
ASTSelectWithUnionQuery * select = nullptr;
IAST * comment = nullptr;
bool is_dictionary{false}; /// CREATE DICTIONARY
ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of

View File

@ -26,6 +26,20 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace
{
ASTPtr parseComment(IParser::Pos & pos, Expected & expected)
{
ParserKeyword s_comment("COMMENT");
ParserStringLiteral string_literal_parser;
ASTPtr comment;
s_comment.ignore(pos, expected) && string_literal_parser.parse(pos, comment, expected);
return comment;
}
}
bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserToken open(TokenType::OpeningRoundBracket);
@ -314,7 +328,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_sample_by("SAMPLE BY");
ParserKeyword s_ttl("TTL");
ParserKeyword s_settings("SETTINGS");
ParserKeyword s_comment("COMMENT");
ParserIdentifierWithOptionalParameters ident_with_optional_params_p;
ParserExpression expression_p;
@ -329,7 +342,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ASTPtr sample_by;
ASTPtr ttl_table;
ASTPtr settings;
ASTPtr comment_expression;
if (!s_engine.ignore(pos, expected))
return false;
@ -387,13 +399,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
return false;
}
if (s_comment.ignore(pos, expected))
{
/// should be followed by a string literal
if (!string_literal_parser.parse(pos, comment_expression, expected))
return false;
}
break;
}
@ -407,8 +412,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
storage->set(storage->settings, settings);
storage->set(storage->comment, comment_expression);
node = storage;
return true;
}
@ -568,7 +571,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
}
}
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();
node = query;
@ -590,6 +593,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
query->set(query->columns_list, columns_list);
query->set(query->storage, storage);
if (comment)
query->set(query->comment, comment);
if (query->storage && query->columns_list && query->columns_list->primary_key)
{
if (query->storage->primary_key)
@ -803,7 +809,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
}
storage_p.parse(pos, storage, expected);
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();
node = query;
@ -816,6 +822,8 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
query->cluster = cluster_str;
query->set(query->storage, storage);
if (comment)
query->set(query->comment, comment);
return true;
}
@ -934,6 +942,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!select_p.parse(pos, select, expected))
return false;
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();
node = query;
@ -958,6 +967,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->set(query->columns_list, columns_list);
query->set(query->storage, storage);
if (comment)
query->set(query->comment, comment);
tryGetIdentifierNameInto(as_database, query->as_database);
tryGetIdentifierNameInto(as_table, query->as_table);
@ -1039,6 +1050,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
return false;
}
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();
node = query;
query->is_dictionary = true;
@ -1056,6 +1069,9 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
query->set(query->dictionary, dictionary);
query->cluster = cluster_str;
if (comment)
query->set(query->comment, comment);
return true;
}

View File

@ -1,5 +1,6 @@
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/parseQuery.h>
@ -132,11 +133,6 @@ INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest,
"MODIFY COMMENT ''",
"MODIFY COMMENT ''",
},
// {
// // No comment - same as empty comment
// "MODIFY COMMENT NULL",
// "MODIFY COMMENT ''",
// },
{
// Non-empty comment value
"MODIFY COMMENT 'some comment value'",
@ -144,3 +140,33 @@ INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest,
}
}
)));
INSTANTIATE_TEST_SUITE_P(ParserCreateQuery_DICTIONARY_WITH_COMMENT, ParserTest,
::testing::Combine(
::testing::Values(std::make_shared<ParserAlterCommand>()),
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
{
R"sql(CREATE DICTIONARY 2024_dictionary_with_comment
(
id UInt64,
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table'))
LAYOUT(FLAT())
LIFETIME(MIN 0 MAX 1000)
COMMENT 'Test dictionary with comment';
)sql",
R"sql(CREATE DICTIONARY `2024_dictionary_with_comment`
(
`id` UInt64,
`value` String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table'))
LIFETIME(MIN 0 MAX 1000)
LAYOUT(FLAT())
COMMENT 'Test dictionary with comment')sql"
}}
)));

View File

@ -146,8 +146,8 @@ StoragePtr StorageFactory::get(
throw Exception("Unknown table engine " + name, ErrorCodes::UNKNOWN_STORAGE);
}
if (storage_def->comment)
comment = storage_def->comment->as<ASTLiteral &>().value.get<String>();
if (query.comment)
comment = query.comment->as<ASTLiteral &>().value.get<String>();
auto check_feature = [&](String feature_description, FeatureMatcherFn feature_matcher_fn)
{

View File

@ -47,7 +47,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes()
{"last_successful_update_time", std::make_shared<DataTypeDateTime>()},
{"loading_duration", std::make_shared<DataTypeFloat32>()},
//{ "creation_time", std::make_shared<DataTypeDateTime>() },
{"last_exception", std::make_shared<DataTypeString>()}
{"last_exception", std::make_shared<DataTypeString>()},
{"comment", std::make_shared<DataTypeString>()}
};
}
@ -140,6 +141,16 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt
else
res_columns[i++]->insertDefault();
if (dict_ptr)
res_columns[i++]->insert(dict_ptr->getDictionaryComment());
else
{
if (load_result.config && load_result.config->config->has("dictionary.comment"))
res_columns[i++]->insert(load_result.config->config->getString("dictionary.comment"));
else
res_columns[i++]->insertDefault();
}
/// Start fill virtual columns
res_columns[i++]->insert(dictionary_structure.getKeyDescription());

View File

@ -0,0 +1,2 @@
CREATE DICTIONARY default.`2024_dictionary_with_comment`\n(\n `id` UInt64,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT tcpPort() TABLE \'source_table\'))\nLIFETIME(MIN 0 MAX 1000)\nLAYOUT(FLAT())\nCOMMENT \'Test dictionary with comment\'
Test dictionary with comment

View File

@ -0,0 +1,33 @@
-----------------------------------------------------------------------------------
-- Check that `DICTIONARY` can be created with a `COMMENT` clause
-- and comment is visible both in `comment` column of `system.dictionaries`
-- and `SHOW CREATE DICTIONARY`.
-----------------------------------------------------------------------------------
-- prerequisites
CREATE TABLE source_table
(
id UInt64,
value String
) ENGINE = Memory();
INSERT INTO source_table VALUES (1, 'First');
INSERT INTO source_table VALUES (2, 'Second');
DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment;
CREATE DICTIONARY 2024_dictionary_with_comment
(
id UInt64,
value String
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table'))
LAYOUT(FLAT())
LIFETIME(MIN 0 MAX 1000)
COMMENT 'Test dictionary with comment';
SHOW CREATE DICTIONARY 2024_dictionary_with_comment;
SELECT comment FROM system.dictionaries WHERE name == '2024_dictionary_with_comment' AND database == currentDatabase();
DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment;