mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Fix tests
This commit is contained in:
parent
41bc68e82e
commit
e9721804b6
@ -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);
|
||||
}
|
||||
|
@ -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>();
|
||||
|
||||
}
|
||||
|
||||
|
@ -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>();
|
||||
|
@ -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
|
||||
|
@ -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.
|
||||
|
@ -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>
|
||||
|
@ -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>
|
@ -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");
|
||||
|
@ -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'
|
||||
|
Loading…
Reference in New Issue
Block a user