Merge branch 'ClickHouse:master' into task_cancellation

This commit is contained in:
Yarik Briukhovetskyi 2024-11-03 12:11:03 +01:00 committed by GitHub
commit fea51136fd
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
23 changed files with 428 additions and 84 deletions

2
.gitmodules vendored
View File

@ -332,7 +332,7 @@
url = https://github.com/ClickHouse/usearch.git
[submodule "contrib/SimSIMD"]
path = contrib/SimSIMD
url = https://github.com/ashvardanian/SimSIMD.git
url = https://github.com/ClickHouse/SimSIMD.git
[submodule "contrib/FP16"]
path = contrib/FP16
url = https://github.com/Maratyszcza/FP16.git

2
contrib/SimSIMD vendored

@ -1 +1 @@
Subproject commit ff51434d90c66f916e94ff05b24530b127aa4cff
Subproject commit 935fef2964bc38e995c5f465b42259a35b8cf0d3

2
contrib/usearch vendored

@ -1 +1 @@
Subproject commit 1706420acafbd83d852c512dcf343af0a4059e48
Subproject commit 53799b84ca9ad708b060d0b1cfa5f039371721cd

View File

@ -17,7 +17,7 @@ anyLast(column) [RESPECT NULLS]
- `column`: The column name.
:::note
Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the first value passed, regardless of whether it is `NULL` or not.
Supports the `RESPECT NULLS` modifier after the function name. Using this modifier will ensure the function selects the last value passed, regardless of whether it is `NULL` or not.
:::
**Returned value**
@ -40,4 +40,4 @@ SELECT anyLast(city) FROM any_last_nulls;
┌─anyLast(city)─┐
│ Valencia │
└───────────────┘
```
```

View File

@ -117,6 +117,7 @@ GRANT SELECT ON db*.* TO john -- correct
GRANT SELECT ON *.my_table TO john -- wrong
GRANT SELECT ON foo*bar TO john -- wrong
GRANT SELECT ON *suffix TO john -- wrong
GRANT SELECT(foo) ON db.table* TO john -- wrong
```
## Privileges

View File

@ -95,7 +95,7 @@ sudo yum install -y clickhouse-server clickhouse-client
sudo systemctl enable clickhouse-server
sudo systemctl start clickhouse-server
sudo systemctl status clickhouse-server
clickhouse-client # илм "clickhouse-client --password" если установлен пароль
clickhouse-client # или "clickhouse-client --password" если установлен пароль
```
Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`.

View File

@ -15,6 +15,9 @@ public:
explicit Credentials() = default;
explicit Credentials(const String & user_name_);
Credentials(const Credentials &) = default;
Credentials(Credentials &&) = default;
virtual ~Credentials() = default;
const String & getUserName() const;

View File

@ -48,9 +48,9 @@ ASTPtr normalizeCreateWorkloadEntityQuery(const IAST & create_query)
/// Returns a type of a workload entity `ptr`
WorkloadEntityType getEntityType(const ASTPtr & ptr)
{
if (auto * res = typeid_cast<ASTCreateWorkloadQuery *>(ptr.get()))
if (auto * res = typeid_cast<ASTCreateWorkloadQuery *>(ptr.get()); res)
return WorkloadEntityType::Workload;
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(ptr.get()))
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(ptr.get()); res)
return WorkloadEntityType::Resource;
chassert(false);
return WorkloadEntityType::MAX;
@ -106,7 +106,7 @@ void forEachReference(
for (const String & resource : resources)
func(resource, res->getWorkloadName(), ReferenceType::ForResource);
}
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(source_entity.get()))
if (auto * res = typeid_cast<ASTCreateResourceQuery *>(source_entity.get()); res)
{
// RESOURCE has no references to be validated, we allow mentioned disks to be created later
}

View File

@ -65,6 +65,12 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"24.11",
{
{"distributed_cache_discard_connection_if_unread_data", true, true, "New setting"},
{"azure_check_objects_after_upload", false, false, "Check each uploaded object in azure blob storage to be sure that upload was successful"},
{"backup_restore_keeper_max_retries", 20, 1000, "Should be big enough so the whole operation BACKUP or RESTORE operation won't fail because of a temporary [Zoo]Keeper failure in the middle of it."},
{"backup_restore_failure_after_host_disconnected_for_seconds", 0, 3600, "New setting."},
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
}
},
{"24.10",
@ -113,12 +119,6 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"allow_reorder_prewhere_conditions", false, true, "New setting"},
{"input_format_parquet_bloom_filter_push_down", false, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and bloom filter in the Parquet metadata."},
{"date_time_64_output_format_cut_trailing_zeros_align_to_groups_of_thousands", false, false, "Dynamically trim the trailing zeros of datetime64 values to adjust the output scale to (0, 3, 6), corresponding to 'seconds', 'milliseconds', and 'microseconds'."},
{"azure_check_objects_after_upload", false, false, "Check each uploaded object in azure blob storage to be sure that upload was successful"},
{"backup_restore_keeper_max_retries", 20, 1000, "Should be big enough so the whole operation BACKUP or RESTORE operation won't fail because of a temporary [Zoo]Keeper failure in the middle of it."},
{"backup_restore_failure_after_host_disconnected_for_seconds", 0, 3600, "New setting."},
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
}
},
{"24.9",

View File

@ -155,6 +155,9 @@ namespace
for (auto & [access_flags, columns] : access_and_columns)
{
if (wildcard && !columns.empty())
return false;
AccessRightsElement element;
element.access_flags = access_flags;
element.columns = std::move(columns);

View File

@ -6,6 +6,7 @@
#include <Access/ExternalAuthenticators.h>
#include <Common/Base64.h>
#include <Common/HTTPHeaderFilter.h>
#include <Server/HTTPHandler.h>
#include <Server/HTTP/HTTPServerRequest.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/HTTPServerResponse.h>
@ -54,11 +55,13 @@ bool authenticateUserByHTTP(
HTTPServerResponse & response,
Session & session,
std::unique_ptr<Credentials> & request_credentials,
const HTTPHandlerConnectionConfig & connection_config,
ContextPtr global_context,
LoggerPtr log)
{
/// Get the credentials created by the previous call of authenticateUserByHTTP() while handling the previous HTTP request.
auto current_credentials = std::move(request_credentials);
const auto & config_credentials = connection_config.credentials;
/// The user and password can be passed by headers (similar to X-Auth-*),
/// which is used by load balancers to pass authentication information.
@ -70,6 +73,7 @@ bool authenticateUserByHTTP(
/// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name
/// extracted from the SSL certificate used for this connection instead of checking password.
bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on");
bool has_config_credentials = config_credentials.has_value();
/// User name and password can be passed using HTTP Basic auth or query parameters
/// (both methods are insecure).
@ -79,6 +83,10 @@ bool authenticateUserByHTTP(
std::string spnego_challenge;
SSLCertificateSubjects certificate_subjects;
if (config_credentials)
{
checkUserNameNotEmpty(config_credentials->getUserName(), "config authentication");
}
if (has_ssl_certificate_auth)
{
#if USE_SSL
@ -86,6 +94,8 @@ bool authenticateUserByHTTP(
checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers");
/// It is prohibited to mix different authorization schemes.
if (has_config_credentials)
throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication set in config");
if (!password.empty())
throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password");
if (has_http_credentials)
@ -109,6 +119,8 @@ bool authenticateUserByHTTP(
checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers");
/// It is prohibited to mix different authorization schemes.
if (has_config_credentials)
throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "authentication set in config");
if (has_http_credentials)
throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header");
if (has_credentials_in_query_params)
@ -117,6 +129,8 @@ bool authenticateUserByHTTP(
else if (has_http_credentials)
{
/// It is prohibited to mix different authorization schemes.
if (has_config_credentials)
throwMultipleAuthenticationMethods("Authorization HTTP header", "authentication set in config");
if (has_credentials_in_query_params)
throwMultipleAuthenticationMethods("Authorization HTTP header", "authentication via parameters");
@ -190,6 +204,10 @@ bool authenticateUserByHTTP(
return false;
}
}
else if (has_config_credentials)
{
current_credentials = std::make_unique<BasicCredentials>(*config_credentials);
}
else // I.e., now using user name and password strings ("Basic").
{
if (!current_credentials)

View File

@ -11,13 +11,22 @@ class HTMLForm;
class HTTPServerResponse;
class Session;
class Credentials;
class BasicCredentials;
struct HTTPHandlerConnectionConfig;
/// Authenticates a user via HTTP protocol and initializes a session.
///
/// Usually retrieves the name and the password for that user from either the request's headers or from the query parameters.
/// Returns true when the user successfully authenticated,
/// the session instance will be configured accordingly, and the request_credentials instance will be dropped.
/// Returns false when the user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header,
/// in this case the `request_credentials` instance must be preserved until the next request or until any exception.
/// You can also pass user/password explicitly via `config_credentials`.
///
/// Returns true when the user successfully authenticated:
/// - the session instance will be configured accordingly
/// - and the request_credentials instance will be dropped.
///
/// Returns false when the user is not authenticated yet:
/// - the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header
/// - the `request_credentials` instance must be preserved until the next request or until any exception.
///
/// Throws an exception if authentication failed.
bool authenticateUserByHTTP(
const HTTPServerRequest & request,
@ -25,6 +34,7 @@ bool authenticateUserByHTTP(
HTTPServerResponse & response,
Session & session,
std::unique_ptr<Credentials> & request_credentials,
const HTTPHandlerConnectionConfig & connection_config,
ContextPtr global_context,
LoggerPtr log);

View File

@ -1,6 +1,5 @@
#include <Server/HTTPHandler.h>
#include <Access/Credentials.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Core/ExternalTable.h>
@ -145,6 +144,15 @@ static std::chrono::steady_clock::duration parseSessionTimeout(
return std::chrono::seconds(session_timeout);
}
HTTPHandlerConnectionConfig::HTTPHandlerConnectionConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
if (config.has(config_prefix + ".handler.user") || config.has(config_prefix + ".handler.password"))
{
credentials.emplace(
config.getString(config_prefix + ".handler.user", "default"),
config.getString(config_prefix + ".handler.password", ""));
}
}
void HTTPHandler::pushDelayedResults(Output & used_output)
{
@ -182,11 +190,12 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
}
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_)
HTTPHandler::HTTPHandler(IServer & server_, const HTTPHandlerConnectionConfig & connection_config_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_)
: server(server_)
, log(getLogger(name))
, default_settings(server.context()->getSettingsRef())
, http_response_headers_override(http_response_headers_override_)
, connection_config(connection_config_)
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
@ -199,7 +208,7 @@ HTTPHandler::~HTTPHandler() = default;
bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response)
{
return authenticateUserByHTTP(request, params, response, *session, request_credentials, server.context(), log);
return authenticateUserByHTTP(request, params, response, *session, request_credentials, connection_config, server.context(), log);
}
@ -768,8 +777,12 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
}
DynamicQueryHandler::DynamicQueryHandler(
IServer & server_, const std::string & param_name_, const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "DynamicQueryHandler", http_response_headers_override_), param_name(param_name_)
IServer & server_,
const HTTPHandlerConnectionConfig & connection_config,
const std::string & param_name_,
const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, connection_config, "DynamicQueryHandler", http_response_headers_override_)
, param_name(param_name_)
{
}
@ -826,12 +839,13 @@ std::string DynamicQueryHandler::getQuery(HTTPServerRequest & request, HTMLForm
PredefinedQueryHandler::PredefinedQueryHandler(
IServer & server_,
const HTTPHandlerConnectionConfig & connection_config,
const NameSet & receive_params_,
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const HTTPResponseHeaderSetup & http_response_headers_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", http_response_headers_override_)
: HTTPHandler(server_, connection_config, "PredefinedQueryHandler", http_response_headers_override_)
, receive_params(receive_params_)
, predefined_query(predefined_query_)
, url_regex(url_regex_)
@ -923,10 +937,11 @@ HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server,
{
auto query_param_name = config.getString(config_prefix + ".handler.query_param_name", "query");
HTTPHandlerConnectionConfig connection_config(config, config_prefix);
HTTPResponseHeaderSetup http_response_headers_override = parseHTTPResponseHeaders(config, config_prefix);
auto creator = [&server, query_param_name, http_response_headers_override]() -> std::unique_ptr<DynamicQueryHandler>
{ return std::make_unique<DynamicQueryHandler>(server, query_param_name, http_response_headers_override); };
auto creator = [&server, query_param_name, http_response_headers_override, connection_config]() -> std::unique_ptr<DynamicQueryHandler>
{ return std::make_unique<DynamicQueryHandler>(server, connection_config, query_param_name, http_response_headers_override); };
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -968,6 +983,8 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
Poco::Util::AbstractConfiguration::Keys headers_name;
config.keys(config_prefix + ".headers", headers_name);
HTTPHandlerConnectionConfig connection_config(config, config_prefix);
for (const auto & header_name : headers_name)
{
auto expression = config.getString(config_prefix + ".headers." + header_name);
@ -1001,12 +1018,18 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
predefined_query,
regex,
headers_name_with_regex,
http_response_headers_override]
http_response_headers_override,
connection_config]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, regex,
headers_name_with_regex, http_response_headers_override);
server,
connection_config,
analyze_receive_params,
predefined_query,
regex,
headers_name_with_regex,
http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
@ -1019,18 +1042,21 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server,
analyze_receive_params,
predefined_query,
headers_name_with_regex,
http_response_headers_override]
http_response_headers_override,
connection_config]
-> std::unique_ptr<PredefinedQueryHandler>
{
return std::make_unique<PredefinedQueryHandler>(
server, analyze_receive_params, predefined_query, CompiledRegexPtr{},
headers_name_with_regex, http_response_headers_override);
server,
connection_config,
analyze_receive_params,
predefined_query,
CompiledRegexPtr{},
headers_name_with_regex,
http_response_headers_override);
};
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(std::move(creator));
factory->addFiltersFromConfig(config, config_prefix);
return factory;
}

View File

@ -12,6 +12,7 @@
#include <IO/CascadeWriteBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
#include <Common/re2.h>
#include <Access/Credentials.h>
#include "HTTPResponseHeaderWriter.h"
@ -26,17 +27,28 @@ namespace DB
{
class Session;
class Credentials;
class IServer;
struct Settings;
class WriteBufferFromHTTPServerResponse;
using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
struct HTTPHandlerConnectionConfig
{
std::optional<BasicCredentials> credentials;
/// TODO:
/// String quota;
/// String default_database;
HTTPHandlerConnectionConfig() = default;
HTTPHandlerConnectionConfig(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
};
class HTTPHandler : public HTTPRequestHandler
{
public:
HTTPHandler(IServer & server_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_);
HTTPHandler(IServer & server_, const HTTPHandlerConnectionConfig & connection_config_, const std::string & name, const HTTPResponseHeaderSetup & http_response_headers_override_);
~HTTPHandler() override;
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
@ -146,16 +158,7 @@ private:
// The request_credential instance may outlive a single request/response loop.
// This happens only when the authentication mechanism requires more than a single request/response exchange (e.g., SPNEGO).
std::unique_ptr<Credentials> request_credentials;
// Returns true when the user successfully authenticated,
// the session instance will be configured accordingly, and the request_credentials instance will be dropped.
// Returns false when the user is not authenticated yet, and the 'Negotiate' response is sent,
// the session and request_credentials instances are preserved.
// Throws an exception if authentication failed.
bool authenticateUser(
HTTPServerRequest & request,
HTMLForm & params,
HTTPServerResponse & response);
HTTPHandlerConnectionConfig connection_config;
/// Also initializes 'used_output'.
void processQuery(
@ -174,6 +177,13 @@ private:
Output & used_output);
static void pushDelayedResults(Output & used_output);
protected:
// @see authenticateUserByHTTP()
virtual bool authenticateUser(
HTTPServerRequest & request,
HTMLForm & params,
HTTPServerResponse & response);
};
class DynamicQueryHandler : public HTTPHandler
@ -184,6 +194,7 @@ private:
public:
explicit DynamicQueryHandler(
IServer & server_,
const HTTPHandlerConnectionConfig & connection_config,
const std::string & param_name_ = "query",
const HTTPResponseHeaderSetup & http_response_headers_override_ = std::nullopt);
@ -203,6 +214,7 @@ private:
public:
PredefinedQueryHandler(
IServer & server_,
const HTTPHandlerConnectionConfig & connection_config,
const NameSet & receive_params_,
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,

View File

@ -275,7 +275,7 @@ void addDefaultHandlersFactory(
auto dynamic_creator = [&server] () -> std::unique_ptr<DynamicQueryHandler>
{
return std::make_unique<DynamicQueryHandler>(server, "query");
return std::make_unique<DynamicQueryHandler>(server, HTTPHandlerConnectionConfig{}, "query");
};
auto query_handler = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(std::move(dynamic_creator));
query_handler->addFilter([](const auto & request)

View File

@ -7,6 +7,7 @@
#include <Server/HTTP/sendExceptionToHTTPClient.h>
#include <Server/IServer.h>
#include <Server/PrometheusMetricsWriter.h>
#include <Server/HTTPHandler.h>
#include "config.h"
#include <Access/Credentials.h>
@ -137,7 +138,7 @@ protected:
bool authenticateUser(HTTPServerRequest & request, HTTPServerResponse & response)
{
return authenticateUserByHTTP(request, *params, response, *session, request_credentials, server().context(), log());
return authenticateUserByHTTP(request, *params, response, *session, request_credentials, HTTPHandlerConnectionConfig{}, server().context(), log());
}
void makeContext(HTTPServerRequest & request)

View File

@ -17,9 +17,10 @@ class SimpleCluster:
cluster.start()
def add_instance(self, name, config_dir):
script_path = os.path.dirname(os.path.realpath(__file__))
return self.cluster.add_instance(
name, main_configs=[os.path.join(script_path, config_dir, "config.xml")]
name,
main_configs=[os.path.join(config_dir, "config.xml")],
user_configs=["users.d/users.yaml"],
)
@ -96,6 +97,16 @@ def test_dynamic_query_handler():
== res_custom_ct.headers["X-Test-Http-Response-Headers-Even-Multiple"]
)
assert cluster.instance.http_request(
"test_dynamic_handler_auth_with_password?query=select+currentUser()"
).content, "with_password"
assert cluster.instance.http_request(
"test_dynamic_handler_auth_with_password_fail?query=select+currentUser()"
).status_code, 403
assert cluster.instance.http_request(
"test_dynamic_handler_auth_without_password?query=select+currentUser()"
).content, "without_password"
def test_predefined_query_handler():
with contextlib.closing(
@ -177,6 +188,16 @@ def test_predefined_query_handler():
)
assert b"max_threads\t1\n" == res1.content
assert cluster.instance.http_request(
"test_predefined_handler_auth_with_password"
).content, "with_password"
assert cluster.instance.http_request(
"test_predefined_handler_auth_with_password_fail"
).status_code, 403
assert cluster.instance.http_request(
"test_predefined_handler_auth_without_password"
).content, "without_password"
def test_fixed_static_handler():
with contextlib.closing(

View File

@ -24,5 +24,32 @@
</http_response_headers>
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_dynamic_handler_auth_with_password</url>
<handler>
<type>dynamic_query_handler</type>
<user>with_password</user>
<password>password</password>
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_dynamic_handler_auth_with_password_fail</url>
<handler>
<type>dynamic_query_handler</type>
<user>with_password</user>
<!-- No password - authentication should fail -->
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_dynamic_handler_auth_without_password</url>
<handler>
<type>dynamic_query_handler</type>
<user>without_password</user>
</handler>
</rule>
</http_handlers>
</clickhouse>

View File

@ -33,5 +33,35 @@
<query>INSERT INTO test_table(id, data) SELECT {id:UInt32}, {_request_body:String}</query>
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_predefined_handler_auth_with_password</url>
<handler>
<type>predefined_query_handler</type>
<user>with_password</user>
<password>password</password>
<query>SELECT currentUser()</query>
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_predefined_handler_auth_with_password_fail</url>
<handler>
<type>predefined_query_handler</type>
<user>with_password</user>
<!-- No password - authentication should fail -->
<query>SELECT currentUser()</query>
</handler>
</rule>
<rule>
<methods>GET</methods>
<url>/test_predefined_handler_auth_without_password</url>
<handler>
<type>predefined_query_handler</type>
<user>without_password</user>
<query>SELECT currentUser()</query>
</handler>
</rule>
</http_handlers>
</clickhouse>

View File

@ -0,0 +1,7 @@
users:
with_password:
profile: default
password: password
without_password:
profile: default
no_password: 1

View File

@ -67,7 +67,7 @@ Expression (Projection)
Condition: true
Parts: 1/1
Granules: 4/4
-- Non-default quantization
-- Test all distance metrics x all quantization
1 [2,3.2] 2.3323807824711897
4 [2.4,5.2] 3.9999999046325727
2 [4.2,3.4] 4.427188573446585
@ -75,7 +75,7 @@ Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_f64)
ReadFromMergeTree (default.tab_l2_f64)
Indexes:
PrimaryKey
Condition: true
@ -93,7 +93,7 @@ Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_f32)
ReadFromMergeTree (default.tab_l2_f32)
Indexes:
PrimaryKey
Condition: true
@ -111,7 +111,7 @@ Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_f16)
ReadFromMergeTree (default.tab_l2_f16)
Indexes:
PrimaryKey
Condition: true
@ -129,7 +129,7 @@ Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_bf16)
ReadFromMergeTree (default.tab_l2_bf16)
Indexes:
PrimaryKey
Condition: true
@ -147,7 +147,97 @@ Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_i8)
ReadFromMergeTree (default.tab_l2_i8)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 3/4
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_cos_f64)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 3/4
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_cos_f32)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 3/4
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_cos_f16)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 3/4
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_cos_bf16)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 3/4
6 [1,9.3] 0.005731362878640178
4 [2.4,5.2] 0.09204062768384846
1 [2,3.2] 0.15200169244542905
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_cos_i8)
Indexes:
PrimaryKey
Condition: true

View File

@ -81,88 +81,181 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind
DROP TABLE tab;
SELECT '-- Non-default quantization';
CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
SELECT '-- Test all distance metrics x all quantization';
DROP TABLE IF EXISTS tab_l2_f64;
DROP TABLE IF EXISTS tab_l2_f32;
DROP TABLE IF EXISTS tab_l2_f16;
DROP TABLE IF EXISTS tab_l2_bf16;
DROP TABLE IF EXISTS tab_l2_i8;
DROP TABLE IF EXISTS tab_cos_f64;
DROP TABLE IF EXISTS tab_cos_f32;
DROP TABLE IF EXISTS tab_cos_f16;
DROP TABLE IF EXISTS tab_cos_bf16;
DROP TABLE IF EXISTS tab_cos_i8;
CREATE TABLE tab_l2_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_l2_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_l2_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_l2_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_l2_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_cos_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f64', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_cos_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f32', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_cos_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_cos_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'bf16', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_cos_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'i8', 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab_l2_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_l2_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_l2_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_l2_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_l2_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_cos_f64 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_cos_f32 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_cos_f16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_cos_bf16 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
INSERT INTO tab_cos_i8 VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f64
FROM tab_l2_f64
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f64
FROM tab_l2_f64
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f32
FROM tab_l2_f32
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f32
FROM tab_l2_f32
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f16
FROM tab_l2_f16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f16
FROM tab_l2_f16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_bf16
FROM tab_l2_bf16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_bf16
FROM tab_l2_bf16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_i8
FROM tab_l2_i8
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_i8
FROM tab_l2_i8
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
DROP TABLE tab_f64;
DROP TABLE tab_f32;
DROP TABLE tab_f16;
DROP TABLE tab_bf16;
DROP TABLE tab_i8;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f64
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f64
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f32
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f32
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f16
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_f16
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_bf16
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_bf16
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_i8
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, cosineDistance(vec, reference_vec)
FROM tab_cos_i8
ORDER BY cosineDistance(vec, reference_vec)
LIMIT 3;
DROP TABLE tab_l2_f64;
DROP TABLE tab_l2_f32;
DROP TABLE tab_l2_f16;
DROP TABLE tab_l2_bf16;
DROP TABLE tab_l2_i8;
DROP TABLE tab_cos_f64;
DROP TABLE tab_cos_f32;
DROP TABLE tab_cos_f16;
DROP TABLE tab_cos_bf16;
DROP TABLE tab_cos_i8;
SELECT '-- Index on Array(Float64) column';
CREATE TABLE tab(id Int32, vec Array(Float64), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;

View File

@ -19,4 +19,6 @@ REVOKE SELECT ON team*.* FROM user_03141;
SHOW GRANTS FOR user_03141;
SELECT '---';
GRANT SELECT(bar) ON foo.test* TO user_03141; -- { clientError SYNTAX_ERROR }
DROP USER user_03141;