more complex test and some fixes

This commit is contained in:
Alexander Tokmakov 2020-02-20 23:24:04 +03:00
parent c492ee93d9
commit e1f5a620b5
6 changed files with 53 additions and 13 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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