diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 2e46ff294cc..e3e1f86bc63 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -25,6 +26,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_DATABASE; } @@ -67,6 +69,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & return node; /// Skip table functions. + ///FIXME it's very surprising that function which evaluates smth as literal may return ASTFunction instead of ASTLiteral if (const auto * table_func_ptr = node->as()) if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name)) return node; @@ -82,6 +85,25 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, cons return evaluateConstantExpressionAsLiteral(node, context); } +ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const Context & context) +{ + ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(node, context); + auto & literal = res->as(); + if (literal.value.safeGet().empty()) + { + String current_database = context.getCurrentDatabase(); + if (current_database.empty()) + { + /// Table was created on older version of ClickHouse and CREATE contains not folded expression. + /// Current database is not set yet during server startup, so we cannot evaluate it correctly. + literal.value = context.getConfigRef().getString("default_database", "default"); + } + else + literal.value = current_database; + } + return res; +} + namespace { using Conjunction = ColumnsWithTypeAndName; diff --git a/dbms/src/Interpreters/evaluateConstantExpression.h b/dbms/src/Interpreters/evaluateConstantExpression.h index a84104c53f4..422afd33d4b 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.h +++ b/dbms/src/Interpreters/evaluateConstantExpression.h @@ -37,6 +37,12 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & */ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context); +/** The same as evaluateConstantExpressionOrIdentifierAsLiteral(...), + * but if result is an empty string, replace it with current database name + * or default database name. + */ +ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const Context & context); + /** Try to fold condition to countable set of constant values. * @param condition a condition that we try to fold. * @param target_expr expression evaluated over a set of constants. diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 2f1b633413d..de403c9ff5c 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -788,7 +788,7 @@ void registerStorageBuffer(StorageFactory & factory) " destination_database, destination_table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); + engine_args[0] = evaluateConstantExpressionForDatabaseName(engine_args[0], args.local_context); engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context); String destination_database = engine_args[0]->as().value.safeGet(); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 3dabc4aa701..0dba78f05a1 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -253,6 +253,10 @@ StorageDistributed::StorageDistributed( if (num_local_shards && remote_database == id_.database_name && remote_table == id_.table_name) throw Exception("Distributed table " + id_.table_name + " looks at itself", ErrorCodes::INFINITE_LOOP); } + if (remote_database.empty()) + { + LOG_WARNING(log, "Name of remote database is empty. Default database will be used implicitly."); + } } diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index d61d2a04fdf..8008b9fe39d 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -49,9 +49,10 @@ StoragePtr StorageFactory::get( bool has_force_restore_data_flag) const { String name; - ASTs args; ASTStorage * storage_def = query.storage; + bool has_engine_args = false; + if (query.is_view) { if (query.storage) @@ -89,7 +90,7 @@ StoragePtr StorageFactory::get( "Engine definition cannot take the form of a parametric function", ErrorCodes::FUNCTION_CANNOT_HAVE_PARAMETERS); if (engine_def.arguments) - args = engine_def.arguments->children; + has_engine_args = true; name = engine_def.name; @@ -162,10 +163,11 @@ StoragePtr StorageFactory::get( } } + ASTs empty_engine_args; Arguments arguments { .engine_name = name, - .engine_args = args, + .engine_args = has_engine_args ? storage_def->engine->arguments->children : empty_engine_args, .storage_def = storage_def, .query = query, .relative_data_path = relative_data_path, diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 1455556f366..c98e0e93669 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -509,7 +509,7 @@ void registerStorageMerge(StorageFactory & factory) " - name of source database and regexp for table names.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); + engine_args[0] = evaluateConstantExpressionForDatabaseName(engine_args[0], args.local_context); engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.local_context); String source_database = engine_args[0]->as().value.safeGet(); diff --git a/dbms/src/TableFunctions/TableFunctionMerge.cpp b/dbms/src/TableFunctions/TableFunctionMerge.cpp index b0c22c96117..a05c0ef4c8b 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.cpp +++ b/dbms/src/TableFunctions/TableFunctionMerge.cpp @@ -9,7 +9,7 @@ #include #include #include -#include "registerTableFunctions.h" +#include namespace DB @@ -62,7 +62,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co " - name of source database and regexp for table names.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context); + args[0] = evaluateConstantExpressionForDatabaseName(args[0], context); args[1] = evaluateConstantExpressionAsLiteral(args[1], context); String source_database = args[0]->as().value.safeGet(); diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index da078a0d3f0..28b7015ef05 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -65,8 +65,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C if (is_cluster_function) { - ASTPtr ast_name = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - cluster_name = ast_name->as().value.safeGet(); + args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); + cluster_name = args[arg_num]->as().value.safeGet(); } else { diff --git a/dbms/tests/queries/0_stateless/00987_distributed_stack_overflow.sql b/dbms/tests/queries/0_stateless/00987_distributed_stack_overflow.sql index 8bc7b6963b9..4baa6969b31 100644 --- a/dbms/tests/queries/0_stateless/00987_distributed_stack_overflow.sql +++ b/dbms/tests/queries/0_stateless/00987_distributed_stack_overflow.sql @@ -1,13 +1,18 @@ +DROP TABLE IF EXISTS distr0; DROP TABLE IF EXISTS distr1; DROP TABLE IF EXISTS distr2; CREATE TABLE distr (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr); -- { serverError 269 } +CREATE TABLE distr0 (x UInt8) ENGINE = Distributed(test_shard_localhost, '', distr0); +SELECT * FROM distr0; -- { serverError 306 } + CREATE TABLE distr1 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr2); CREATE TABLE distr2 (x UInt8) ENGINE = Distributed(test_shard_localhost, currentDatabase(), distr1); SELECT * FROM distr1; -- { serverError 306 } SELECT * FROM distr2; -- { serverError 306 } +DROP TABLE distr0; DROP TABLE distr1; DROP TABLE distr2; diff --git a/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference new file mode 100644 index 00000000000..fe3791fcf4c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference @@ -0,0 +1,6 @@ +CREATE TABLE test_01083.file (`n` Int8) ENGINE = File(\'TSVWithNamesAndTypes\') +CREATE TABLE test_01083.buffer (`n` Int8) ENGINE = Buffer(\'test_01083\', \'file\', 16, 10, 200, 10000, 1000000, 10000000, 1000000000) +CREATE TABLE test_01083.merge (`n` Int8) ENGINE = Merge(\'test_01083\', \'file\') +CREATE TABLE test_01083.merge_tf AS merge(\'test_01083\', \'.*\') +CREATE TABLE test_01083.distributed (`n` Int8) ENGINE = Distributed(test_cluster, \'test_01083\', \'file\') +CREATE TABLE test_01083.distributed_tf AS cluster(\'test_cluster\', \'test_01083\', \'file\') diff --git a/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql new file mode 100644 index 00000000000..833287fd86f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -0,0 +1,19 @@ +DROP DATABASE IF EXISTS test_01083; +CREATE DATABASE test_01083; +USE test_01083; + +CREATE TABLE file (n Int8) ENGINE = File(upper('tsv') || 'WithNames' || 'AndTypes'); +CREATE TABLE buffer (n Int8) ENGINE = Buffer(currentDatabase(), file, 16, 10, 200, 10000, 1000000, 10000000, 1000000000); +CREATE TABLE merge (n Int8) ENGINE = Merge('', lower('FILE')); +CREATE TABLE merge_tf as merge(currentDatabase(), '.*'); +CREATE TABLE distributed (n Int8) ENGINE = Distributed(test_cluster, currentDatabase(), 'fi' || 'le'); +CREATE TABLE distributed_tf as cluster('test' || '_' || 'cluster', currentDatabase(), 'fi' || 'le'); + +SHOW CREATE file; +SHOW CREATE buffer; +SHOW CREATE merge; +SHOW CREATE merge_tf; +SHOW CREATE distributed; +SHOW CREATE distributed_tf; + +DROP DATABASE test_01083;