Fix tests

This commit is contained in:
kssenii 2021-09-09 12:18:08 +03:00
parent 41bc68e82e
commit e9721804b6
9 changed files with 41 additions and 41 deletions

View File

@ -226,8 +226,10 @@ getURLBasedDataSourceConfiguration(const ASTs & args, ContextPtr context)
for (const auto & key : keys)
{
if (key == "url")
{
configuration.url = config.getString(config_prefix + ".url", "");
if (key == "headers")
}
else if (key == "headers")
{
Poco::Util::AbstractConfiguration::Keys header_keys;
config.keys(config_prefix + '.' + "headers", header_keys);
@ -272,7 +274,7 @@ getURLBasedDataSourceConfiguration(const ASTs & args, ContextPtr context)
if (configuration.url.empty() || configuration.format.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Storage requires {}", configuration.url.empty() ? "uri" : "format");
"Storage requires {}", configuration.url.empty() ? "url" : "format");
return std::make_tuple(configuration, non_common_args, true);
}

View File

@ -137,15 +137,15 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C
/// 27017 is the default MongoDB port.
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 27017);
configuration.host = parsed_host_port.first;
configuration.port = parsed_host_port.second;
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.collection = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
String options;
if (engine_args.size() >= 6)
options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
configuration.options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
}

View File

@ -747,9 +747,9 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
else if (arg_name == "secret_access_key")
configuration.secret_access_key = arg_value.safeGet<String>();
else
throw Exception(
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
arg_name);
}
}
else
@ -762,7 +762,7 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
configuration.url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 4)
{
configuration.access_key_id = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();

View File

@ -50,9 +50,10 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
else if (arg_name == "secret_access_key")
configuration.secret_access_key = arg_value.safeGet<String>();
else
throw Exception(
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Unknown key-value argument `{}` for StorageS3, expected: "
"url, [access_key_id, secret_access_key], name of used format, structure and [compression_method].",
arg_name);
}
}
else

View File

@ -2,7 +2,7 @@ import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/config_with_hosts.xml', 'configs/named_collections.xml'])
node1 = cluster.add_instance('node1', main_configs=['configs/config_with_hosts.xml'])
node2 = cluster.add_instance('node2', main_configs=['configs/config_with_only_primary_hosts.xml'])
node3 = cluster.add_instance('node3', main_configs=['configs/config_with_only_regexp_hosts.xml'])
node4 = cluster.add_instance('node4', main_configs=[]) # No `remote_url_allow_hosts` at all.

View File

@ -8,4 +8,22 @@
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
<named_collections>
<postgres1>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
<port>5432</port>
<database>postgres_database</database>
<table>test_table</table>
</postgres1>
<postgres2>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
<port>1111</port>
<database>postgres_database</database>
<table>test_table</table>
</postgres2>
</named_collections>
</yandex>

View File

@ -1,19 +0,0 @@
<yandex>
<named_collections>
<postgres1>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
<port>5432</port>
<database>postgres_database</database>
<table>test_table</table>
</postgres1>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
<port>1111</port>
<database>postgres_database</database>
<table>test_table</table>
<postgres2>
</named_collections>
</yandex>

View File

@ -14,7 +14,7 @@ import threading
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance',
main_configs = ['configs/log_conf.xml', 'configs/named_collections.xml'],
main_configs = ['configs/log_conf.xml'],
user_configs = ['configs/users.xml'],
with_postgres=True, stay_alive=True)
@ -149,12 +149,10 @@ def check_tables_are_synchronized(table_name, order_by='key', postgres_database=
def started_cluster():
try:
cluster.start()
conn = get_postgres_conn(ip=cluster.postgres_ip,
port=cluster.postgres_port)
conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port)
cursor = conn.cursor()
create_postgres_db(cursor, 'postgres_database')
create_clickhouse_postgres_db(ip=cluster.postgres_ip,
port=cluster.postgres_port)
create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port)
instance.query("DROP DATABASE IF EXISTS test_database")
yield cluster
@ -989,7 +987,7 @@ def test_predefined_connection_configuration(started_cluster):
conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True)
cursor = conn.cursor()
cursor.execute(f'DROP TABLE IF EXISTS test_table')
cursor.execute(f'CREATE TABLE test_table (a integer PRIMARY KEY, b integer)')
cursor.execute(f'CREATE TABLE test_table (key integer PRIMARY KEY, value integer)')
instance.query("CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1)")
check_tables_are_synchronized("test_table");

View File

@ -12,8 +12,8 @@ def started_cluster(request):
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node',
main_configs=["configs_secure/config.d/ssl_conf.xml", 'configs/named_collections.xml'],
with_mongo=True)
#with_mongo_secure=request.param)
with_mongo=True,
with_mongo_secure=request.param)
cluster.start()
yield cluster
finally:
@ -40,7 +40,7 @@ def test_simple_select(started_cluster):
node = started_cluster.instances['node']
node.query(
"create table simple_mongo_table(key uint64, data string) engine = mongodb('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')")
"create table simple_mongo_table(key UInt64, data String) engine = MongoDB('mongo1:27017', 'test', 'simple_table', 'root', 'clickhouse')")
assert node.query("select count() from simple_mongo_table") == '100\n'
assert node.query("select sum(key) from simple_mongo_table") == str(sum(range(0, 100))) + '\n'