diff --git a/src/Databases/DDLDependencyVisitor.cpp b/src/Databases/DDLDependencyVisitor.cpp index 1bee2cd23cf..6cf01e4e87e 100644 --- a/src/Databases/DDLDependencyVisitor.cpp +++ b/src/Databases/DDLDependencyVisitor.cpp @@ -37,8 +37,8 @@ namespace { friend void tryVisitNestedSelect(const String & query, DDLDependencyVisitorData & data); public: - DDLDependencyVisitorData(const ContextPtr & global_context_, const QualifiedTableName & table_name_, const ASTPtr & ast_, const String & current_database_) - : create_query(ast_), table_name(table_name_), default_database(global_context_->getCurrentDatabase()), current_database(current_database_), global_context(global_context_) + DDLDependencyVisitorData(const ContextPtr & global_context_, const QualifiedTableName & table_name_, const ASTPtr & ast_, const String & current_database_, bool can_throw_) + : create_query(ast_), table_name(table_name_), default_database(global_context_->getCurrentDatabase()), current_database(current_database_), global_context(global_context_), can_throw(can_throw_) { } @@ -82,6 +82,7 @@ namespace String current_database; ContextPtr global_context; TableNamesSet dependencies; + bool can_throw; /// CREATE TABLE or CREATE DICTIONARY or CREATE VIEW or CREATE TEMPORARY TABLE or CREATE DATABASE query. void visitCreateQuery(const ASTCreateQuery & create) @@ -483,23 +484,26 @@ namespace } catch (...) { - tryLogCurrentException("DDLDependencyVisitor"); + if (data.can_throw) + throw; + else + tryLogCurrentException("DDLDependencyVisitor"); } } } -TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database) +TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database, bool can_throw) { - DDLDependencyVisitor::Data data{global_global_context, table_name, ast, current_database}; + DDLDependencyVisitor::Data data{global_global_context, table_name, ast, current_database, can_throw}; DDLDependencyVisitor::Visitor visitor{data}; visitor.visit(ast); return std::move(data).getDependencies(); } -TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database) +TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database, bool can_throw) { - DDLDependencyVisitor::Data data{global_context, table_name, ast, current_database}; + DDLDependencyVisitor::Data data{global_context, table_name, ast, current_database, can_throw}; tryVisitNestedSelect(select_query, data); return std::move(data).getDependencies(); } diff --git a/src/Databases/DDLDependencyVisitor.h b/src/Databases/DDLDependencyVisitor.h index 400e6b04108..82ccf2ca3f1 100644 --- a/src/Databases/DDLDependencyVisitor.h +++ b/src/Databases/DDLDependencyVisitor.h @@ -13,9 +13,9 @@ using TableNamesSet = std::unordered_set; /// Returns a list of all tables explicitly referenced in the create query of a specified table. /// For example, a column default expression can use dictGet() and thus reference a dictionary. /// Does not validate AST, works a best-effort way. -TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database); +TableNamesSet getDependenciesFromCreateQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & current_database, bool can_throw = false); /// Returns a list of all tables explicitly referenced in the select query specified as a dictionary source. -TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database); +TableNamesSet getDependenciesFromDictionaryNestedSelectQuery(const ContextPtr & global_context, const QualifiedTableName & table_name, const ASTPtr & ast, const String & select_query, const String & current_database, bool can_throw = false); } diff --git a/src/Databases/DDLLoadingDependencyVisitor.cpp b/src/Databases/DDLLoadingDependencyVisitor.cpp index b91aa84ecd3..a1fde25e597 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.cpp +++ b/src/Databases/DDLLoadingDependencyVisitor.cpp @@ -22,7 +22,7 @@ namespace DB using TableLoadingDependenciesVisitor = DDLLoadingDependencyVisitor::Visitor; -TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast) +TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast, bool can_throw) { assert(global_context == global_context->getGlobalContext()); TableLoadingDependenciesVisitor::Data data; @@ -30,6 +30,7 @@ TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, c data.create_query = ast; data.global_context = global_context; data.table_name = table; + data.can_throw = can_throw; TableLoadingDependenciesVisitor visitor{data}; visitor.visit(ast); data.dependencies.erase(table); @@ -125,7 +126,7 @@ void DDLLoadingDependencyVisitor::visit(const ASTFunctionWithKeyValueArguments & { /// We don't have a table name, we have a select query instead that will be executed during dictionary loading. /// We need to find all tables used in this select query and add them to dependencies. - auto select_query_dependencies = getDependenciesFromDictionaryNestedSelectQuery(data.global_context, data.table_name, data.create_query, info->query, data.default_database); + auto select_query_dependencies = getDependenciesFromDictionaryNestedSelectQuery(data.global_context, data.table_name, data.create_query, info->query, data.default_database, data.can_throw); data.dependencies.merge(select_query_dependencies); } } diff --git a/src/Databases/DDLLoadingDependencyVisitor.h b/src/Databases/DDLLoadingDependencyVisitor.h index a9e9f4d7a53..a4f9c55b5ad 100644 --- a/src/Databases/DDLLoadingDependencyVisitor.h +++ b/src/Databases/DDLLoadingDependencyVisitor.h @@ -16,7 +16,7 @@ using TableNamesSet = std::unordered_set; /// Returns a list of all tables which should be loaded before a specified table. /// For example, a local ClickHouse table should be loaded before a dictionary which uses that table as its source. /// Does not validate AST, works a best-effort way. -TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast); +TableNamesSet getLoadingDependenciesFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast, bool can_throw = false); class DDLMatcherBase @@ -39,6 +39,7 @@ public: ContextPtr global_context; ASTPtr create_query; QualifiedTableName table_name; + bool can_throw; }; using Visitor = ConstInDepthNodeVisitor; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index dda9e72afa7..41b974b606e 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "ExternalQueryBuilder.h" @@ -28,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int INCORRECT_QUERY; } namespace @@ -167,6 +170,14 @@ QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & qu auto context_copy = Context::createCopy(context); context_copy->makeQueryContext(); + const char * query_begin = query.data(); + const char * query_end = query.data() + query.size(); + ParserQuery parser(query_end); + ASTPtr ast = parseQuery(parser, query_begin, query_end, "Query for ClickHouse dictionary", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + if (!ast || ast->getQueryKind() != IAST::QueryKind::Select) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Only SELECT query can be used as a dictionary source"); + if (configuration.is_local) { pipeline = executeQuery(query, context_copy, QueryFlags{ .internal = true }).second.pipeline; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b1a15d101ef..9e81871579d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1445,8 +1445,8 @@ void addTableDependencies(const ASTCreateQuery & create, const ASTPtr & query_pt void checkTableCanBeAddedWithNoCyclicDependencies(const ASTCreateQuery & create, const ASTPtr & query_ptr, const ContextPtr & context) { QualifiedTableName qualified_name{create.getDatabase(), create.getTable()}; - auto ref_dependencies = getDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr, context->getCurrentDatabase()); - auto loading_dependencies = getLoadingDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr); + auto ref_dependencies = getDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr, context->getCurrentDatabase(), /*can_throw*/true); + auto loading_dependencies = getLoadingDependenciesFromCreateQuery(context->getGlobalContext(), qualified_name, query_ptr, /*can_throw*/ true); DatabaseCatalog::instance().checkTableCanBeAddedWithNoCyclicDependencies(qualified_name, ref_dependencies, loading_dependencies); } diff --git a/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_insert_query.xml b/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_insert_query.xml new file mode 100644 index 00000000000..f180cddb620 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_insert_query.xml @@ -0,0 +1,40 @@ + + + test.dictionary_with_insert_query + + + localhost + 9000 + default + + insert into test.xml_dictionary_table values (432234, 28, 'smth') + + + + + 0 + 0 + + + + + + + + + id + + + SomeValue1 + UInt8 + 1 + + + + SomeValue2 + String + '' + + + + diff --git a/tests/integration/test_dictionaries_ddl/node5/metadata/default.sql b/tests/integration/test_dictionaries_ddl/node5/metadata/default.sql new file mode 100644 index 00000000000..439de26d9f9 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/node5/metadata/default.sql @@ -0,0 +1 @@ +ATTACH DATABASE default ENGINE=Ordinary diff --git a/tests/integration/test_dictionaries_ddl/node5/metadata/default/existing_ddl_dictionary_with_insert_query.sql b/tests/integration/test_dictionaries_ddl/node5/metadata/default/existing_ddl_dictionary_with_insert_query.sql new file mode 100644 index 00000000000..b8dd18b1049 --- /dev/null +++ b/tests/integration/test_dictionaries_ddl/node5/metadata/default/existing_ddl_dictionary_with_insert_query.sql @@ -0,0 +1,10 @@ +CREATE DICTIONARY existing_ddl_dictionary_with_insert_query +( + `id` UInt64, + `SomeValue1` UInt8, + `SomeValue2` String +) +PRIMARY KEY id +LAYOUT(FLAT()) +SOURCE(CLICKHOUSE(QUERY `INSERT INTO test.xml_dictionary_table values (987123, 30, "sv")`)) +LIFETIME(MIN 0 MAX 0) diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index bbce196400d..91e7abd856e 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -47,6 +47,15 @@ node3 = cluster.add_instance( node4 = cluster.add_instance( "node4", user_configs=["configs/user_admin.xml", "configs/config_password.xml"] ) +node5 = cluster.add_instance( + "node5", + # Check that the cluster can start when the dictionary with the bad source query + # was created using DDL + clickhouse_path_dir="node5", + main_configs=["configs/allow_remote_node.xml"], + dictionaries=["configs/dictionaries/dictionary_with_insert_query.xml"], + user_configs=["configs/user_admin.xml"], +) def create_mysql_conn(user, password, hostname, port): @@ -71,7 +80,7 @@ def execute_mysql_query(connection, query): def started_cluster(): try: cluster.start() - for clickhouse in [node1, node2, node3, node4]: + for clickhouse in [node1, node2, node3, node4, node5]: clickhouse.query("CREATE DATABASE test", user="admin") clickhouse.query( "CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", @@ -322,6 +331,39 @@ def test_conflicting_name(started_cluster): ) == "17\n" +def test_with_insert_query(started_cluster): + try: + node5.query( + """ + CREATE DICTIONARY test.ddl_dictionary_with_insert_query ( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' QUERY 'insert into test.xml_dictionary_table values (234432, 29, "htms")')) + LIFETIME(MIN 0 MAX 0) + """ + ) + assert "Invalid dictionary should not be created" and False + except QueryRuntimeException as ex: + assert "(SYNTAX_ERROR)" in str(ex) + assert ( + "Syntax error (Query for ClickHouse dictionary test.ddl_dictionary_with_insert_query)" + in str(ex) + ) + + try: + node5.query( + """ + SELECT dictGet('test.dictionary_with_insert_query', 'SomeValue1', 432234) + """ + ) + except QueryRuntimeException as ex: + assert "Only SELECT query can be used as a dictionary source" in str(ex) + + def test_http_dictionary_restrictions(started_cluster): try: node3.query( @@ -612,3 +654,5 @@ def test_named_collection(started_cluster): node1.query( "select dictGetUInt8('test.clickhouse_named_collection', 'SomeValue1', toUInt64(23))" ) == "0\n" + + node1.query("DROP DICTIONARY test.clickhouse_named_collection")