diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 3bf03ec28b1..dc4b687c8dc 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -68,7 +68,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} , context(context_) - , is_local{isLocalAddress({host, port}, context.getTCPPort())} + , is_local{isLocalAddress({host, port}, secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort())} , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password)} , load_all_query{query_builder.composeLoadAllQuery()} { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index bdd1b23d19f..3d5014ac440 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -620,6 +620,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, return false; StoragePtr res; + /// NOTE: CREATE query may be rewritten by Storage creator or table function if (create.as_table_function) { const auto & table_function = create.as_table_function->as(); diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index e3e1f86bc63..23cdcbd9fd5 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -67,13 +67,6 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & /// If it's already a literal. if (node->as()) 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; - return std::make_shared(evaluateConstantExpression(node, context).first); } diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index 28b7015ef05..ffcec737fb7 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -75,8 +75,6 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C } ++arg_num; - args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - const auto * function = args[arg_num]->as(); if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) @@ -86,6 +84,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C } else { + args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); remote_database = args[arg_num]->as().value.safeGet(); ++arg_num; 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 index fe3791fcf4c..b114333f8dd 100644 --- a/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference +++ b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.reference @@ -1,6 +1,12 @@ 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 (`n` Int8) ENGINE = Merge(\'test_01083\', \'distributed\') 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\') +CREATE TABLE test_01083.url (`n` UInt64, `_path` String) ENGINE = URL(\'http://localhost:8123/?query=select+n,+_path+from+test_01083.file+format+CSV\', \'CSV\') +CREATE TABLE test_01083.rich_syntax AS remote(\'localhos{x|y|t}\', cluster(\'test_cluster\', remote(\'127.0.0.{1..4}\', \'test_01083\', \'url\'))) +1 +1 +1 +1 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 index 833287fd86f..302567317e1 100644 --- a/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql +++ b/dbms/tests/queries/0_stateless/01083_expressions_in_engine_arguments.sql @@ -4,10 +4,47 @@ 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 (n Int8) ENGINE = Merge('', lower('DISTRIBUTED')); 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'); +CREATE TABLE distributed_tf as cluster('test' || '_' || 'cluster', '', 'fi' || 'le'); + +INSERT INTO file VALUES (1); +CREATE TABLE url (n UInt64, _path String) ENGINE=URL +( + replace + ( + 'http://localhost:8123/?query=' || 'select n, _path from ' || currentDatabase() || '.file format CSV', ' ', '+' -- replace `file` with `merge` here after #9246 is fixed + ), + CSV +); + +-- The following line is needed just to disable checking stderr for emptiness +SELECT nonexistentsomething; -- { serverError 47 } + +CREATE DICTIONARY dict (n UInt64, _path String DEFAULT '42') PRIMARY KEY n +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9440 SECURE 1 USER 'default' TABLE 'url' DB 'test_01083')) LIFETIME(1) LAYOUT(CACHE(SIZE_IN_CELLS 1)); + +-- TODO make fuzz test from this +CREATE TABLE rich_syntax as remote +( + 'localhos{x|y|t}', + cluster + ( + 'test' || '_' || 'cluster', + remote + ( + '127.0.0.{1..4}', + if + ( + toString(40 + 2.0) NOT IN ('hello', dictGetString(currentDatabase() || '.dict', '_path', toUInt64('0001'))), + currentDatabase(), + 'FAIL' + ), + extract('123url456', '[a-z]+') + ) + ) +); SHOW CREATE file; SHOW CREATE buffer; @@ -15,5 +52,9 @@ SHOW CREATE merge; SHOW CREATE merge_tf; SHOW CREATE distributed; SHOW CREATE distributed_tf; +SHOW CREATE url; +SHOW CREATE rich_syntax; + +SELECT n from rich_syntax; DROP DATABASE test_01083;