mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
more complex test and some fixes
This commit is contained in:
parent
c492ee93d9
commit
e1f5a620b5
@ -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()}
|
||||
{
|
||||
|
@ -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<ASTFunction &>();
|
||||
|
@ -67,13 +67,6 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context &
|
||||
/// If it's already a literal.
|
||||
if (node->as<ASTLiteral>())
|
||||
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<ASTFunction>())
|
||||
if (TableFunctionFactory::instance().isTableFunctionName(table_func_ptr->name))
|
||||
return node;
|
||||
|
||||
return std::make_shared<ASTLiteral>(evaluateConstantExpression(node, context).first);
|
||||
}
|
||||
|
||||
|
@ -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<ASTFunction>();
|
||||
|
||||
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<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
++arg_num;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user