Merge pull request #72548 from alexkats/dict

Add source query validation for dictionaries
This commit is contained in:
Alexey Katsman 2024-11-29 11:25:40 +00:00 committed by GitHub
commit d7167bd103
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 127 additions and 15 deletions

View File

@ -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();
}

View File

@ -13,9 +13,9 @@ using TableNamesSet = std::unordered_set<QualifiedTableName>;
/// 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);
}

View File

@ -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);
}
}

View File

@ -16,7 +16,7 @@ using TableNamesSet = std::unordered_set<QualifiedTableName>;
/// 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<DDLLoadingDependencyVisitor, true>;

View File

@ -15,6 +15,8 @@
#include <Storages/NamedCollectionsHelpers.h>
#include <Common/isLocalAddress.h>
#include <Common/logger_useful.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#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;

View File

@ -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);
}

View File

@ -0,0 +1,40 @@
<clickhouse>
<dictionary>
<name>test.dictionary_with_insert_query</name>
<source>
<clickhouse>
<host>localhost</host>
<port>9000</port>
<user>default</user>
<password></password>
<query>insert into test.xml_dictionary_table values (432234, 28, 'smth')</query>
</clickhouse>
</source>
<lifetime>
<min>0</min>
<max>0</max>
</lifetime>
<layout>
<flat/>
</layout>
<structure>
<id>
<name>id</name>
</id>
<attribute>
<name>SomeValue1</name>
<type>UInt8</type>
<null_value>1</null_value>
</attribute>
<attribute>
<name>SomeValue2</name>
<type>String</type>
<null_value>''</null_value>
</attribute>
</structure>
</dictionary>
</clickhouse>

View File

@ -0,0 +1 @@
ATTACH DATABASE default ENGINE=Ordinary

View File

@ -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)

View File

@ -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")