Fix checks

This commit is contained in:
kssenii 2021-09-10 14:11:52 +03:00
parent 00be148fce
commit ba1534f1df
7 changed files with 21 additions and 8 deletions

View File

@ -163,7 +163,8 @@ ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority(
{
.database = common_configuration.database,
.table = common_configuration.table,
.schema = common_configuration.schema
.schema = common_configuration.schema,
.replicas_configurations = {}
};
if (dict_config.has(dict_config_prefix + ".replica"))

View File

@ -31,7 +31,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}

View File

@ -93,9 +93,13 @@ StorageExternalDistributed::StorageExternalDistributed(
addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 5432);
ExternalDataSourceConfiguration configuration
{
.host = "",
.port = 0,
.username = username,
.password = password,
.database = remote_database,
.table = remote_table,
.schema = "",
.addresses = addresses
};

View File

@ -31,8 +31,10 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NETWORK_ERROR;
extern const int BAD_ARGUMENTS;
}
IStorageURLBase::IStorageURLBase(
const Poco::URI & uri_,
ContextPtr /*context_*/,
@ -397,7 +399,7 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex
illegal_args += ", ";
illegal_args += arg.first;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguements {} for table function URL", illegal_args);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args);
}
}
else

View File

@ -14,6 +14,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto & func_args = ast_function->as<ASTFunction &>();
@ -33,7 +38,7 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co
illegal_args += ", ";
illegal_args += arg.first;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguements {} for table function URL", illegal_args);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args);
}
filename = configuration.url;

View File

@ -89,6 +89,7 @@ def test_predefined_connection_configuration(started_cluster):
hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n")
assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n"
node1.query("drop table if exists WebHDFSStorageWithRedirect")
node1.query(
"create table WebHDFSStorageWithRedirect (id UInt32, name String, weight Float64) ENGINE = URL(url1, url='http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', format='TSV')")
assert node1.query("SET max_http_get_redirects=1; select * from WebHDFSStorageWithRedirect") == "1\tMark\t72.53\n"

View File

@ -11,7 +11,7 @@ def started_cluster(request):
try:
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node',
main_configs=["configs_secure/config.d/ssl_conf.xml", 'configs/named_collections.xml'],
main_configs=["configs_secure/config.d/ssl_conf.xml", "configs/named_collections.xml"],
with_mongo=True,
with_mongo_secure=request.param)
cluster.start()
@ -40,10 +40,10 @@ 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'
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'
assert node.query("SELECT data from simple_mongo_table where key = 42") == hex(42 * 42) + '\n'
node.query("DROP TABLE simple_mongo_table")
@ -125,6 +125,7 @@ def test_secure_connection(started_cluster):
node.query("DROP TABLE simple_mongo_table")
simple_mongo_table.drop()
@pytest.mark.parametrize('started_cluster', [False], indirect=['started_cluster'])
def test_predefined_connection_configuration(started_cluster):
mongo_connection = get_mongo_connection(started_cluster)
db = mongo_connection['test']