diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index fc428aae9ac..d997977cc3a 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -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()->value.safeGet()); + if (impl && create.comment) + impl->setDatabaseComment(create.comment->as()->value.safeGet()); return impl; } diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index ff4021cf6d2..87e5f86c893 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -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(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index ea9635530c7..97e59f53f64 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -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); - else - storage_ast.set(storage_ast.comment, std::make_shared(metadata.comment)); } + + if (metadata.comment.empty()) + ast_create_query.reset(ast_create_query.comment); + else + ast_create_query.set(ast_create_query.comment, std::make_shared(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(); - // 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(database_comment)); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index a30341ae927..2377b7da809 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -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(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 98f868dadc2..430caa0f4a3 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -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(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 7d7cd523179..05bc2f24834 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -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(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 9b7fbbf7c68..92ff269a5ac 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -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; } } diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index a62fb4a788e..b40dc418c66 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -216,12 +216,25 @@ struct IDictionary : public IExternalLoadable return std::static_pointer_cast(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; }; } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 341a89bfc06..0c6944eeccc 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -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 comment_element(xml_document->createElement("comment")); + current_dictionary->appendChild(comment_element); + AutoPtr comment_value(xml_document->createTextNode(query.comment->as()->value.safeGet())); + + comment_element->appendChild(comment_value); + } + conf->load(xml_document); return conf; } diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 8542a43e202..6e97910c6e1 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -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")); } diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index be973a988bd..2c048886247 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -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); + } } } diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index e4f2b628886..51c4ac15e67 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -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 diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 1da1bfba491..af2966948fb 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -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(); 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(); 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(); 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(); 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; } diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index f97bc77272c..5ebea834a91 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -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()), + ::testing::ValuesIn(std::initializer_list{ + { + 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" + }} +))); diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index cfa50b95487..ba2ac3f72a3 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -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().value.get(); + if (query.comment) + comment = query.comment->as().value.get(); auto check_feature = [&](String feature_description, FeatureMatcherFn feature_matcher_fn) { diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 0826bb58473..d8f92d38081 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -47,7 +47,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() {"last_successful_update_time", std::make_shared()}, {"loading_duration", std::make_shared()}, //{ "creation_time", std::make_shared() }, - {"last_exception", std::make_shared()} + {"last_exception", std::make_shared()}, + {"comment", std::make_shared()} }; } @@ -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()); diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference b/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference new file mode 100644 index 00000000000..45f2c41f0b0 --- /dev/null +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference @@ -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 diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql new file mode 100644 index 00000000000..00557e4cf5c --- /dev/null +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql @@ -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;