mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Fix tests
This commit is contained in:
parent
1d75f740d7
commit
d3db1dd6a7
@ -14,6 +14,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int POSTGRESQL_CONNECTION_FAILURE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
@ -70,6 +71,9 @@ ConnectionHolderPtr PoolWithFailover::get()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (replicas_with_priority.empty())
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "No address specified");
|
||||
|
||||
DB::WriteBufferFromOwnString error_message;
|
||||
for (size_t try_idx = 0; try_idx < max_tries; ++try_idx)
|
||||
{
|
||||
|
@ -330,7 +330,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
{
|
||||
validateNamedCollection(
|
||||
*named_collection,
|
||||
{"host", "port", "user", "password", "database", "table"},
|
||||
{"host", "port", "user", "password", "database"},
|
||||
{"schema", "on_conflict", "use_table_cache"});
|
||||
|
||||
configuration.host = named_collection->get<String>("host");
|
||||
@ -338,7 +338,6 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
configuration.username = named_collection->get<String>("user");
|
||||
configuration.password = named_collection->get<String>("password");
|
||||
configuration.database = named_collection->get<String>("database");
|
||||
configuration.table = named_collection->get<String>("table");
|
||||
configuration.schema = named_collection->getOrDefault<String>("schema", "");
|
||||
configuration.on_conflict = named_collection->getOrDefault<String>("on_conflict", "");
|
||||
use_table_cache = named_collection->getOrDefault<UInt64>("use_tables_cache", 0);
|
||||
@ -405,7 +404,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
{
|
||||
validateNamedCollection(
|
||||
*named_collection,
|
||||
{"host", "port", "user", "password", "database", "table"},
|
||||
{"host", "port", "user", "password", "database"},
|
||||
{"schema"});
|
||||
|
||||
configuration.host = named_collection->get<String>("host");
|
||||
@ -413,7 +412,6 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
configuration.username = named_collection->get<String>("user");
|
||||
configuration.password = named_collection->get<String>("password");
|
||||
configuration.database = named_collection->get<String>("database");
|
||||
configuration.table = named_collection->get<String>("table");
|
||||
configuration.schema = named_collection->getOrDefault<String>("schema", "");
|
||||
}
|
||||
else
|
||||
|
@ -8,11 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
NamedCollectionPtr tryGetNamedCollectionFromASTs(ASTs asts)
|
||||
@ -80,50 +75,6 @@ NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts)
|
||||
return collection_copy;
|
||||
}
|
||||
|
||||
void validateNamedCollection(
|
||||
const NamedCollection & collection,
|
||||
const std::unordered_set<std::string_view> & required_keys,
|
||||
const std::unordered_set<std::string_view> & optional_keys,
|
||||
const std::vector<std::regex> & optional_regex_keys)
|
||||
{
|
||||
const auto & keys = collection.getKeys();
|
||||
auto required_keys_copy = required_keys;
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
auto it = required_keys_copy.find(key);
|
||||
if (it != required_keys_copy.end())
|
||||
{
|
||||
required_keys_copy.erase(it);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (optional_keys.contains(key))
|
||||
continue;
|
||||
|
||||
auto match = std::find_if(
|
||||
optional_regex_keys.begin(), optional_regex_keys.end(),
|
||||
[&](const std::regex & regex) { return std::regex_search(key, regex); })
|
||||
!= optional_regex_keys.end();
|
||||
|
||||
if (!match)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key `{}` in named collection. Required keys: {}, optional keys: {}",
|
||||
key, fmt::join(required_keys, ", "), fmt::join(optional_keys, ", "));
|
||||
}
|
||||
}
|
||||
|
||||
if (!required_keys_copy.empty())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Required keys ({}) are not specified. All required keys: {}, optional keys: {}",
|
||||
fmt::join(required_keys_copy, ", "), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", "));
|
||||
}
|
||||
}
|
||||
|
||||
HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collection)
|
||||
{
|
||||
HTTPHeaderEntries headers;
|
||||
|
@ -1,23 +1,68 @@
|
||||
#pragma once
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
#include <Common/NamedCollections/NamedCollections_fwd.h>
|
||||
#include <Common/NamedCollections/NamedCollections.h>
|
||||
#include <unordered_set>
|
||||
#include <string_view>
|
||||
#include <fmt/format.h>
|
||||
#include <regex>
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts);
|
||||
|
||||
void validateNamedCollection(
|
||||
const NamedCollection & collection,
|
||||
const std::unordered_set<std::string_view> & required_keys,
|
||||
const std::unordered_set<std::string_view> & optional_keys,
|
||||
const std::vector<std::regex> & optional_regex_keys = {});
|
||||
|
||||
HTTPHeaderEntries getHeadersFromNamedCollection(const NamedCollection & collection);
|
||||
|
||||
template <typename RequiredKeys = std::unordered_set<std::string_view>,
|
||||
typename OptionalKeys = std::unordered_set<std::string_view>>
|
||||
void validateNamedCollection(
|
||||
const NamedCollection & collection,
|
||||
const RequiredKeys & required_keys,
|
||||
const OptionalKeys & optional_keys,
|
||||
const std::vector<std::regex> & optional_regex_keys = {})
|
||||
{
|
||||
const auto & keys = collection.getKeys();
|
||||
auto required_keys_copy = required_keys;
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (required_keys_copy.contains(key))
|
||||
{
|
||||
required_keys_copy.erase(key);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (optional_keys.contains(key))
|
||||
continue;
|
||||
|
||||
auto match = std::find_if(
|
||||
optional_regex_keys.begin(), optional_regex_keys.end(),
|
||||
[&](const std::regex & regex) { return std::regex_search(key, regex); })
|
||||
!= optional_regex_keys.end();
|
||||
|
||||
if (!match)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Unexpected key `{}` in named collection. Required keys: {}, optional keys: {}",
|
||||
key, fmt::join(required_keys, ", "), fmt::join(optional_keys, ", "));
|
||||
}
|
||||
}
|
||||
|
||||
if (!required_keys_copy.empty())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Required keys ({}) are not specified. All required keys: {}, optional keys: {}",
|
||||
fmt::join(required_keys_copy, ", "), fmt::join(required_keys, ", "), fmt::join(optional_keys, ", "));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -173,6 +173,13 @@ SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const Storage
|
||||
return std::make_shared<StorageMongoDBSink>(collection_name, database_name, metadata_snapshot, connection);
|
||||
}
|
||||
|
||||
struct KeysCmp
|
||||
{
|
||||
constexpr bool operator()(const auto & lhs, const auto & rhs) const
|
||||
{
|
||||
return lhs == rhs || ((lhs == "table") && (rhs == "collection")) || ((rhs == "table") && (lhs == "collection"));
|
||||
}
|
||||
};
|
||||
StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context)
|
||||
{
|
||||
Configuration configuration;
|
||||
@ -181,7 +188,7 @@ StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args,
|
||||
{
|
||||
validateNamedCollection(
|
||||
*named_collection,
|
||||
{"host", "port", "user", "password", "database", "collection", "table"},
|
||||
std::unordered_multiset<std::string_view, std::hash<std::string_view>, KeysCmp>{"host", "port", "user", "password", "database", "collection", "table"},
|
||||
{"options"});
|
||||
|
||||
configuration.host = named_collection->get<String>("host");
|
||||
@ -189,7 +196,7 @@ StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args,
|
||||
configuration.username = named_collection->get<String>("user");
|
||||
configuration.password = named_collection->get<String>("password");
|
||||
configuration.database = named_collection->get<String>("database");
|
||||
configuration.table = named_collection->getOrDefault<String>("collection", named_collection->get<String>("table"));
|
||||
configuration.table = named_collection->getOrDefault<String>("collection", named_collection->getOrDefault<String>("table", ""));
|
||||
configuration.options = named_collection->getOrDefault<String>("options", "");
|
||||
}
|
||||
else
|
||||
|
@ -403,6 +403,7 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::getConfiguration(ASTs engine
|
||||
{
|
||||
configuration.host = named_collection->get<String>("host");
|
||||
configuration.port = static_cast<UInt16>(named_collection->get<UInt64>("port"));
|
||||
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
|
||||
}
|
||||
|
||||
configuration.username = named_collection->get<String>("user");
|
||||
@ -415,9 +416,14 @@ StoragePostgreSQL::Configuration StoragePostgreSQL::getConfiguration(ASTs engine
|
||||
else
|
||||
{
|
||||
if (engine_args.size() < 5 || engine_args.size() > 7)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage PostgreSQL requires from 5 to 7 parameters: "
|
||||
"PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']. Got: {}",
|
||||
engine_args.size());
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Storage PostgreSQL requires from 5 to 7 parameters: "
|
||||
"PostgreSQL('host:port', 'database', 'table', 'username', 'password' "
|
||||
"[, 'schema', 'ON CONFLICT ...']. Got: {}",
|
||||
engine_args.size());
|
||||
}
|
||||
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
||||
|
@ -1,22 +1,19 @@
|
||||
<clickhouse>
|
||||
<named_collections>
|
||||
<named_collection_for_meili>
|
||||
<database>MeiliSearch</database>
|
||||
<url>http://meili1:7700</url>
|
||||
<table>new_table</table>
|
||||
<index>new_table</index>
|
||||
</named_collection_for_meili>
|
||||
|
||||
<named_collection_for_meili_secure>
|
||||
<database>MeiliSearch</database>
|
||||
<url>http://meili_secure:7700</url>
|
||||
<table>new_table</table>
|
||||
<password>password</password>
|
||||
<index>new_table</index>
|
||||
<key>password</key>
|
||||
</named_collection_for_meili_secure>
|
||||
|
||||
<named_collection_for_meili_secure_no_password>
|
||||
<database>MeiliSearch</database>
|
||||
<url>http://meili_secure:7700</url>
|
||||
<table>new_table</table>
|
||||
<index>new_table</index>
|
||||
</named_collection_for_meili_secure_no_password>
|
||||
</named_collections>
|
||||
</clickhouse>
|
||||
|
@ -598,7 +598,7 @@ def test_named_collection_secure(started_cluster):
|
||||
)
|
||||
|
||||
node.query(
|
||||
'CREATE TABLE combine_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili_secure_no_password, password="password" )'
|
||||
'CREATE TABLE combine_meili_table(id UInt64, data String) ENGINE = MeiliSearch( named_collection_for_meili_secure_no_password, key="password" )'
|
||||
)
|
||||
|
||||
assert node.query("SELECT COUNT() FROM simple_meili_table") == "100\n"
|
||||
|
Loading…
Reference in New Issue
Block a user