diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 20bdfb12939..615c265fc6a 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -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); } diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 2e40f2a2b3f..111f5191de5 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -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().value.safeGet(), 27017); + configuration.host = parsed_host_port.first; + configuration.port = parsed_host_port.second; configuration.database = engine_args[1]->as().value.safeGet(); configuration.collection = engine_args[2]->as().value.safeGet(); configuration.username = engine_args[3]->as().value.safeGet(); configuration.password = engine_args[4]->as().value.safeGet(); - String options; - if (engine_args.size() >= 6) - options = engine_args[5]->as().value.safeGet(); + configuration.options = engine_args[5]->as().value.safeGet(); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 211850a837a..1a733a6f326 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -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(); 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().value.safeGet(); + configuration.url = engine_args[0]->as().value.safeGet(); if (engine_args.size() >= 4) { configuration.access_key_id = engine_args[1]->as().value.safeGet(); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index cbdedeac606..3d928f0f7a1 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -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(); 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 diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index ddfd953ca78..16be9e755bb 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -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. diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml b/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml index f9d15e572aa..6dc9fce900a 100644 --- a/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml +++ b/tests/integration/test_postgresql_replica_database_engine/configs/log_conf.xml @@ -8,4 +8,22 @@ /var/log/clickhouse-server/stderr.log /var/log/clickhouse-server/stdout.log + + + postgres + mysecretpassword + postgres1 + 5432 + postgres_database + test_table
+
+ + postgres + mysecretpassword + postgres1 + 1111 + postgres_database + test_table
+
+
diff --git a/tests/integration/test_postgresql_replica_database_engine/configs/named_collections.xml b/tests/integration/test_postgresql_replica_database_engine/configs/named_collections.xml deleted file mode 100644 index 10195c7867a..00000000000 --- a/tests/integration/test_postgresql_replica_database_engine/configs/named_collections.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - postgres - mysecretpassword - postgres1 - 5432 - postgres_database - test_table
-
- postgres - mysecretpassword - postgres1 - 1111 - postgres_database - test_table
- -
-
diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index bbbff2430b7..03ec53ecd64 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -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"); diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index 539130ea2fd..4981579d9ce 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -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'