Merge branch 'master' into better_delete_outdated_parts

This commit is contained in:
alesapin 2022-05-09 12:40:35 +02:00
commit bf4ed9b7e8
49 changed files with 1080 additions and 233 deletions

View File

@ -0,0 +1,43 @@
#!/usr/bin/env bash
QUERIES_FILE="queries.sql"
TABLE=$1
TRIES=3
PARAMS="--host ... --secure --password ..."
if [ -x ./clickhouse ]
then
CLICKHOUSE_CLIENT="./clickhouse client"
elif command -v clickhouse-client >/dev/null 2>&1
then
CLICKHOUSE_CLIENT="clickhouse-client"
else
echo "clickhouse-client is not found"
exit 1
fi
QUERY_ID_PREFIX="benchmark_$RANDOM"
QUERY_NUM=1
cat "$QUERIES_FILE" | sed "s/{table}/${TABLE}/g" | while read query
do
for i in $(seq 1 $TRIES)
do
QUERY_ID="${QUERY_ID_PREFIX}_${QUERY_NUM}_${i}"
${CLICKHOUSE_CLIENT} ${PARAMS} --query_id "${QUERY_ID}" --format=Null --max_memory_usage=100G --query="$query"
echo -n '.'
done
QUERY_NUM=$((QUERY_NUM + 1))
echo
done
sleep 10
${CLICKHOUSE_CLIENT} ${PARAMS} --query "
WITH extractGroups(query_id, '(\d+)_(\d+)\$') AS num_run, num_run[1]::UInt8 AS num, num_run[2]::UInt8 AS run
SELECT groupArrayInsertAt(query_duration_ms / 1000, (run - 1)::UInt8)::String || ','
FROM clusterAllReplicas(default, system.query_log)
WHERE event_date >= yesterday() AND type = 2 AND query_id LIKE '${QUERY_ID_PREFIX}%'
GROUP BY num ORDER BY num FORMAT TSV
"

View File

@ -694,6 +694,49 @@ auto s = std::string{"Hello"};
**2.** Exception specifiers from C++03 are not used.
**3.** Constructs which have convenient syntactic sugar in modern C++, e.g.
```
// Traditional way without syntactic sugar
template <typename G, typename = std::enable_if_t<std::is_same<G, F>::value, void>> // SFINAE via std::enable_if, usage of ::value
std::pair<int, int> func(const E<G> & e) // explicitly specified return type
{
if (elements.count(e)) // .count() membership test
{
// ...
}
elements.erase(
std::remove_if(
elements.begin(), elements.end(),
[&](const auto x){
return x == 1;
}),
elements.end()); // remove-erase idiom
return std::make_pair(1, 2); // create pair via make_pair()
}
// With syntactic sugar (C++14/17/20)
template <typename G>
requires std::same_v<G, F> // SFINAE via C++20 concept, usage of C++14 template alias
auto func(const E<G> & e) // auto return type (C++14)
{
if (elements.contains(e)) // C++20 .contains membership test
{
// ...
}
elements.erase_if(
elements,
[&](const auto x){
return x == 1;
}); // C++20 std::erase_if
return {1, 2}; // or: return std::pair(1, 2); // create pair via initialization list or value initialization (C++17)
}
```
## Platform {#platform}
**1.** We write code for a specific platform.

View File

@ -426,7 +426,7 @@ Now `rule` can configure `method`, `headers`, `url`, `handler`:
- `status` — use with `static` type, response status code.
- `content_type` — use with `static` type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `content_type` — use with any type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `response_content` — use with `static` type, response content sent to client, when using the prefix file:// or config://, find the content from the file or configuration sends to client.

View File

@ -410,7 +410,7 @@ $ curl -v 'http://localhost:8123/predefined_query'
- `status` — используется с типом `static`, возвращает код состояния ответа.
- `content_type` — используется с типом `static`, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `content_type` — используется со всеми типами, возвращает [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type).
- `response_content` — используется с типом`static`, содержимое ответа, отправленное клиенту, при использовании префикса file:// or config://, находит содержимое из файла или конфигурации, отправленного клиенту.

View File

@ -157,6 +157,7 @@ def build(args):
if not args.skip_website:
website.process_benchmark_results(args)
website.minify_website(args)
redirects.build_static_redirects(args)

View File

@ -1,7 +1,10 @@
import hashlib
import json
import logging
import os
import shutil
import subprocess
import bs4
import util
@ -178,6 +181,59 @@ def build_website(args):
f.write(content.encode("utf-8"))
def get_css_in(args):
return [
f"'{args.website_dir}/css/bootstrap.css'",
f"'{args.website_dir}/css/docsearch.css'",
f"'{args.website_dir}/css/base.css'",
f"'{args.website_dir}/css/blog.css'",
f"'{args.website_dir}/css/docs.css'",
f"'{args.website_dir}/css/highlight.css'",
f"'{args.website_dir}/css/main.css'",
]
def get_js_in(args):
return [
f"'{args.website_dir}/js/jquery.js'",
f"'{args.website_dir}/js/popper.js'",
f"'{args.website_dir}/js/bootstrap.js'",
f"'{args.website_dir}/js/sentry.js'",
f"'{args.website_dir}/js/base.js'",
f"'{args.website_dir}/js/index.js'",
f"'{args.website_dir}/js/docsearch.js'",
f"'{args.website_dir}/js/docs.js'",
f"'{args.website_dir}/js/main.js'",
]
def minify_website(args):
css_in = " ".join(get_css_in(args))
css_out = f"{args.output_dir}/docs/css/base.css"
os.makedirs(f"{args.output_dir}/docs/css")
command = f"cat {css_in}"
output = subprocess.check_output(command, shell=True)
with open(css_out, "wb+") as f:
f.write(output)
with open(css_out, "rb") as f:
css_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8]
js_in = " ".join(get_js_in(args))
js_out = f"{args.output_dir}/docs/js/base.js"
os.makedirs(f"{args.output_dir}/docs/js")
command = f"cat {js_in}"
output = subprocess.check_output(command, shell=True)
with open(js_out, "wb+") as f:
f.write(output)
with open(js_out, "rb") as f:
js_digest = hashlib.sha3_224(f.read()).hexdigest()[0:8]
logging.info(js_digest)
def process_benchmark_results(args):
benchmark_root = os.path.join(args.website_dir, "benchmark")
required_keys = {

View File

@ -1294,17 +1294,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_INFO(log, "Listening for {}", server.getDescription());
}
auto & access_control = global_context->getAccessControl();
if (config().has("custom_settings_prefixes"))
access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes"));
access_control.setNoPasswordAllowed(config().getBool("allow_no_password", true));
access_control.setPlaintextPasswordAllowed(config().getBool("allow_plaintext_password", true));
/// Initialize access storages.
auto & access_control = global_context->getAccessControl();
try
{
access_control.addStoragesFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
access_control.setUpFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); });
}
catch (...)
{

View File

@ -545,6 +545,14 @@
-->
</user_directories>
<access_control_improvements>
<!-- Enables logic that users without permissive row policies can still read rows using a SELECT query.
For example, if there two users A, B and a row policy is defined only for A, then
if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
By default this setting is false for compatibility with earlier access configurations. -->
<users_without_row_policies_can_read_rows>false</users_without_row_policies_can_read_rows>
</access_control_improvements>
<!-- Default profile of settings. -->
<default_profile>default</default_profile>

View File

@ -149,6 +149,24 @@ AccessControl::AccessControl()
AccessControl::~AccessControl() = default;
void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_)
{
if (config_.has("custom_settings_prefixes"))
setCustomSettingsPrefixes(config_.getString("custom_settings_prefixes"));
setNoPasswordAllowed(config_.getBool("allow_no_password", true));
setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true));
setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool(
"access_control_improvements.users_without_row_policies_can_read_rows",
false /* false because we need to be compatible with earlier access configurations */));
addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_);
}
void AccessControl::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_)
{
auto storages = getStoragesPtr();
@ -170,11 +188,7 @@ void AccessControl::addUsersConfigStorage(const Poco::Util::AbstractConfiguratio
void AccessControl::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_)
{
auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); };
auto is_no_password_allowed_function = [this]() -> bool { return isNoPasswordAllowed(); };
auto is_plaintext_password_allowed_function = [this]() -> bool { return isPlaintextPasswordAllowed(); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function,
is_no_password_allowed_function, is_plaintext_password_allowed_function);
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, *this);
new_storage->setConfig(users_config_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}",
@ -207,11 +221,7 @@ void AccessControl::addUsersConfigStorage(
return;
}
}
auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); };
auto is_no_password_allowed_function = [this]() -> bool { return isNoPasswordAllowed(); };
auto is_plaintext_password_allowed_function = [this]() -> bool { return isPlaintextPasswordAllowed(); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function,
is_no_password_allowed_function, is_plaintext_password_allowed_function);
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, *this);
new_storage->load(users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath());

View File

@ -50,6 +50,9 @@ public:
AccessControl();
~AccessControl() override;
void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_,
const zkutil::GetZooKeeper & get_zookeeper_function_);
/// Parses access entities from a configuration loaded from users.xml.
/// This function add UsersConfigAccessStorage if it wasn't added before.
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_);
@ -122,6 +125,12 @@ public:
void setPlaintextPasswordAllowed(const bool allow_plaintext_password_);
bool isPlaintextPasswordAllowed() const;
/// Enables logic that users without permissive row policies can still read rows using a SELECT query.
/// For example, if there two users A, B and a row policy is defined only for A, then
/// if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
void setEnabledUsersWithoutRowPoliciesCanReadRows(bool enable) { users_without_row_policies_can_read_rows = enable; }
bool isEnabledUsersWithoutRowPoliciesCanReadRows() const { return users_without_row_policies_can_read_rows; }
UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const;
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);
@ -178,6 +187,7 @@ private:
std::unique_ptr<CustomSettingsPrefixes> custom_settings_prefixes;
std::atomic_bool allow_plaintext_password = true;
std::atomic_bool allow_no_password = true;
std::atomic_bool users_without_row_policies_can_read_rows = false;
};
}

View File

@ -28,17 +28,25 @@ namespace
permissions.push_back(filter);
}
ASTPtr getResult() &&
ASTPtr getResult(bool users_without_row_policies_can_read_rows) &&
{
/// Process permissive filters.
restrictions.push_back(makeASTForLogicalOr(std::move(permissions)));
if (!permissions.empty() || !users_without_row_policies_can_read_rows)
{
/// Process permissive filters.
restrictions.push_back(makeASTForLogicalOr(std::move(permissions)));
}
/// Process restrictive filters.
auto result = makeASTForLogicalAnd(std::move(restrictions));
ASTPtr result;
if (!restrictions.empty())
result = makeASTForLogicalAnd(std::move(restrictions));
bool value;
if (tryGetLiteralBool(result.get(), value) && value)
result = nullptr; /// The condition is always true, no need to check it.
if (result)
{
bool value;
if (tryGetLiteralBool(result.get(), value) && value)
result = nullptr; /// The condition is always true, no need to check it.
}
return result;
}
@ -234,7 +242,7 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled)
{
auto & mixed_filter = (*mixed_filters)[key];
mixed_filter.database_and_table_name = mixer.database_and_table_name;
mixed_filter.ast = std::move(mixer.mixer).getResult();
mixed_filter.ast = std::move(mixer.mixer).getResult(access_control.isEnabledUsersWithoutRowPoliciesCanReadRows());
}
enabled.mixed_filters.store(mixed_filters);

View File

@ -3,6 +3,7 @@
#include <Access/RowPolicy.h>
#include <Access/User.h>
#include <Access/SettingsProfile.h>
#include <Access/AccessControl.h>
#include <Dictionaries/IDictionary.h>
#include <Common/Config/ConfigReloader.h>
#include <Common/StringUtils/StringUtils.h>
@ -339,7 +340,7 @@ namespace
}
std::vector<AccessEntityPtr> parseRowPolicies(const Poco::Util::AbstractConfiguration & config)
std::vector<AccessEntityPtr> parseRowPolicies(const Poco::Util::AbstractConfiguration & config, bool users_without_row_policies_can_read_rows)
{
std::map<std::pair<String /* database */, String /* table */>, std::unordered_map<String /* user */, String /* filter */>> all_filters_map;
@ -395,8 +396,19 @@ namespace
const auto & [database, table_name] = database_and_table_name;
for (const String & user_name : user_names)
{
String filter;
auto it = user_to_filters.find(user_name);
String filter = (it != user_to_filters.end()) ? it->second : "1";
if (it != user_to_filters.end())
{
filter = it->second;
}
else
{
if (users_without_row_policies_can_read_rows)
continue;
else
filter = "1";
}
auto policy = std::make_shared<RowPolicy>();
policy->setFullName(user_name, database, table_name);
@ -411,7 +423,7 @@ namespace
SettingsProfileElements parseSettingsConstraints(const Poco::Util::AbstractConfiguration & config,
const String & path_to_constraints,
Fn<void(std::string_view)> auto && check_setting_name_function)
const AccessControl & access_control)
{
SettingsProfileElements profile_elements;
Poco::Util::AbstractConfiguration::Keys keys;
@ -419,8 +431,7 @@ namespace
for (const String & setting_name : keys)
{
if (check_setting_name_function)
check_setting_name_function(setting_name);
access_control.checkSettingNameIsAllowed(setting_name);
SettingsProfileElement profile_element;
profile_element.setting_name = setting_name;
@ -448,7 +459,7 @@ namespace
std::shared_ptr<SettingsProfile> parseSettingsProfile(
const Poco::Util::AbstractConfiguration & config,
const String & profile_name,
Fn<void(std::string_view)> auto && check_setting_name_function)
const AccessControl & access_control)
{
auto profile = std::make_shared<SettingsProfile>();
profile->setName(profile_name);
@ -470,13 +481,12 @@ namespace
if (key == "constraints" || key.starts_with("constraints["))
{
profile->elements.merge(parseSettingsConstraints(config, profile_config + "." + key, check_setting_name_function));
profile->elements.merge(parseSettingsConstraints(config, profile_config + "." + key, access_control));
continue;
}
const auto & setting_name = key;
if (check_setting_name_function)
check_setting_name_function(setting_name);
access_control.checkSettingNameIsAllowed(setting_name);
SettingsProfileElement profile_element;
profile_element.setting_name = setting_name;
@ -490,7 +500,7 @@ namespace
std::vector<AccessEntityPtr> parseSettingsProfiles(
const Poco::Util::AbstractConfiguration & config,
Fn<void(std::string_view)> auto && check_setting_name_function)
const AccessControl & access_control)
{
Poco::Util::AbstractConfiguration::Keys profile_names;
config.keys("profiles", profile_names);
@ -502,7 +512,7 @@ namespace
{
try
{
profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function));
profiles.push_back(parseSettingsProfile(config, profile_name, access_control));
}
catch (Exception & e)
{
@ -515,13 +525,8 @@ namespace
}
}
UsersConfigAccessStorage::UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_)
: UsersConfigAccessStorage(STORAGE_TYPE, check_setting_name_function_, is_no_password_allowed_function_, is_plaintext_password_allowed_function_)
{
}
UsersConfigAccessStorage::UsersConfigAccessStorage(const String & storage_name_, const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_)
: IAccessStorage(storage_name_), check_setting_name_function(check_setting_name_function_),is_no_password_allowed_function(is_no_password_allowed_function_), is_plaintext_password_allowed_function(is_plaintext_password_allowed_function_)
UsersConfigAccessStorage::UsersConfigAccessStorage(const String & storage_name_, const AccessControl & access_control_)
: IAccessStorage(storage_name_), access_control(access_control_)
{
}
@ -563,16 +568,16 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu
{
try
{
bool no_password_allowed = is_no_password_allowed_function();
bool plaintext_password_allowed = is_plaintext_password_allowed_function();
bool no_password_allowed = access_control.isNoPasswordAllowed();
bool plaintext_password_allowed = access_control.isPlaintextPasswordAllowed();
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
for (const auto & entity : parseUsers(config, no_password_allowed, plaintext_password_allowed))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseQuotas(config))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseRowPolicies(config))
for (const auto & entity : parseRowPolicies(config, access_control.isEnabledUsersWithoutRowPoliciesCanReadRows()))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function))
for (const auto & entity : parseSettingsProfiles(config, access_control))
all_entities.emplace_back(generateID(*entity), entity);
memory_storage.setAll(all_entities);
}

View File

@ -12,6 +12,7 @@ namespace Poco::Util
namespace DB
{
class AccessControl;
class ConfigReloader;
/// Implementation of IAccessStorage which loads all from users.xml periodically.
@ -20,13 +21,8 @@ class UsersConfigAccessStorage : public IAccessStorage
public:
static constexpr char STORAGE_TYPE[] = "users.xml";
using CheckSettingNameFunction = std::function<void(const std::string_view &)>;
using IsNoPasswordFunction = std::function<bool()>;
using IsPlaintextPasswordFunction = std::function<bool()>;
UsersConfigAccessStorage(const String & storage_name_ = STORAGE_TYPE, const CheckSettingNameFunction & check_setting_name_function_ = {}, const IsNoPasswordFunction & is_no_password_allowed_function_ ={}, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_ = {}); /// NOLINT
UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_); /// NOLINT
UsersConfigAccessStorage(const String & storage_name_, const AccessControl & access_control_);
~UsersConfigAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
@ -58,10 +54,8 @@ private:
scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override;
const AccessControl & access_control;
MemoryAccessStorage memory_storage;
CheckSettingNameFunction check_setting_name_function;
IsNoPasswordFunction is_no_password_allowed_function;
IsPlaintextPasswordFunction is_plaintext_password_allowed_function;
String path;
std::unique_ptr<ConfigReloader> config_reloader;
mutable std::mutex load_mutex;

View File

@ -83,7 +83,7 @@ public:
current_word = 0;
}
void update(const char * data, UInt64 size)
ALWAYS_INLINE void update(const char * data, UInt64 size)
{
const char * end = data + size;
@ -137,12 +137,12 @@ public:
}
template <typename T>
void update(const T & x)
ALWAYS_INLINE void update(const T & x)
{
update(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT
}
void update(const std::string & x)
ALWAYS_INLINE void update(const std::string & x)
{
update(x.data(), x.length());
}

View File

@ -3,6 +3,7 @@
#include <Common/logger_useful.h>
#include <Common/escapeForFileName.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/SeekAvoidingReadBuffer.h>
#include <IO/ReadHelpers.h>

View File

@ -55,7 +55,7 @@ public:
~BitReader() = default;
// reads bits_to_read high-bits from bits_buffer
inline UInt64 readBits(UInt8 bits_to_read)
ALWAYS_INLINE inline UInt64 readBits(UInt8 bits_to_read)
{
if (bits_to_read > bits_count)
fillBitBuffer();
@ -71,7 +71,7 @@ public:
return getBitsFromBitBuffer<PEEK>(8);
}
inline UInt8 readBit()
ALWAYS_INLINE inline UInt8 readBit()
{
return static_cast<UInt8>(readBits(1));
}
@ -122,7 +122,7 @@ private:
// Fills internal bits_buffer with data from source, reads at most 64 bits
size_t fillBitBuffer()
ALWAYS_INLINE size_t fillBitBuffer()
{
const size_t available = source_end - source_current;
const auto bytes_to_read = std::min<size_t>(64 / 8, available);

View File

@ -170,8 +170,6 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
}
else
{
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AggregatingTransform>(header, transform_params);

View File

@ -293,10 +293,11 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
}
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name)
HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_)
: server(server_)
, log(&Poco::Logger::get(name))
, default_settings(server.context()->getSettingsRef())
, content_type_override(content_type_override_)
{
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
}
@ -819,9 +820,9 @@ void HTTPHandler::processQuery(
customizeContext(request, context);
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
[&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone)
[&response, this] (const String & current_query_id, const String & content_type, const String & format, const String & timezone)
{
response.setContentType(content_type);
response.setContentType(content_type_override.value_or(content_type));
response.add("X-ClickHouse-Query-Id", current_query_id);
response.add("X-ClickHouse-Format", format);
response.add("X-ClickHouse-Timezone", timezone);
@ -991,8 +992,8 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
used_output.finalize();
}
DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_)
: HTTPHandler(server_, "DynamicQueryHandler"), param_name(param_name_)
DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & param_name_, const std::optional<String>& content_type_override_)
: HTTPHandler(server_, "DynamicQueryHandler", content_type_override_), param_name(param_name_)
{
}
@ -1052,8 +1053,9 @@ PredefinedQueryHandler::PredefinedQueryHandler(
const NameSet & receive_params_,
const std::string & predefined_query_,
const CompiledRegexPtr & url_regex_,
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_)
: HTTPHandler(server_, "PredefinedQueryHandler")
const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_,
const std::optional<String> & content_type_override_)
: HTTPHandler(server_, "PredefinedQueryHandler", content_type_override_)
, receive_params(receive_params_)
, predefined_query(predefined_query_)
, url_regex(url_regex_)
@ -1123,7 +1125,13 @@ std::string PredefinedQueryHandler::getQuery(HTTPServerRequest & request, HTMLFo
HTTPRequestHandlerFactoryPtr createDynamicHandlerFactory(IServer & server, const std::string & config_prefix)
{
auto query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query");
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(server, std::move(query_param_name));
std::optional<String> content_type_override;
if (server.config().has(config_prefix + ".handler.content_type"))
content_type_override = server.config().getString(config_prefix + ".handler.content_type");
auto factory = std::make_shared<HandlingRuleHTTPHandlerFactory<DynamicQueryHandler>>(
server, std::move(query_param_name), std::move(content_type_override));
factory->addFiltersFromConfig(server.config(), config_prefix);
@ -1180,6 +1188,10 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
headers_name_with_regex.emplace(std::make_pair(header_name, regex));
}
std::optional<String> content_type_override;
if (configuration.has(config_prefix + ".handler.content_type"))
content_type_override = configuration.getString(config_prefix + ".handler.content_type");
std::shared_ptr<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>> factory;
if (configuration.has(config_prefix + ".url"))
@ -1197,14 +1209,20 @@ HTTPRequestHandlerFactoryPtr createPredefinedHandlerFactory(IServer & server, co
std::move(analyze_receive_params),
std::move(predefined_query),
std::move(regex),
std::move(headers_name_with_regex));
std::move(headers_name_with_regex),
std::move(content_type_override));
factory->addFiltersFromConfig(configuration, config_prefix);
return factory;
}
}
factory = std::make_shared<HandlingRuleHTTPHandlerFactory<PredefinedQueryHandler>>(
server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{}, std::move(headers_name_with_regex));
server,
std::move(analyze_receive_params),
std::move(predefined_query),
CompiledRegexPtr{},
std::move(headers_name_with_regex),
std::move(content_type_override));
factory->addFiltersFromConfig(configuration, config_prefix);
return factory;

View File

@ -30,7 +30,7 @@ using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
class HTTPHandler : public HTTPRequestHandler
{
public:
HTTPHandler(IServer & server_, const std::string & name);
HTTPHandler(IServer & server_, const std::string & name, const std::optional<String> & content_type_override_);
virtual ~HTTPHandler() override;
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override;
@ -100,6 +100,9 @@ private:
/// See settings http_max_fields, http_max_field_name_size, http_max_field_value_size in HTMLForm.
const Settings & default_settings;
/// Overrides Content-Type provided by the format of the response.
std::optional<String> content_type_override;
// session is reset at the end of each request/response.
std::unique_ptr<Session> session;
@ -140,7 +143,7 @@ class DynamicQueryHandler : public HTTPHandler
private:
std::string param_name;
public:
explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query");
explicit DynamicQueryHandler(IServer & server_, const std::string & param_name_ = "query", const std::optional<String>& content_type_override_ = std::nullopt);
std::string getQuery(HTTPServerRequest & request, HTMLForm & params, ContextMutablePtr context) override;
@ -157,7 +160,8 @@ private:
public:
PredefinedQueryHandler(
IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_
, const CompiledRegexPtr & url_regex_, const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_);
, const CompiledRegexPtr & url_regex_, const std::unordered_map<String, CompiledRegexPtr> & header_name_with_regex_
, const std::optional<std::string> & content_type_override_);
virtual void customizeContext(HTTPServerRequest & request, ContextMutablePtr context) override;

View File

@ -415,6 +415,11 @@ public:
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
}
void onException() override
{
write_buf->finalize();
}
void onFinish() override
{
try

View File

@ -111,6 +111,13 @@ void PartitionedSink::consume(Chunk chunk)
}
}
void PartitionedSink::onException()
{
for (auto & [_, sink] : partition_id_to_sink)
{
sink->onException();
}
}
void PartitionedSink::onFinish()
{

View File

@ -22,6 +22,8 @@ public:
void consume(Chunk chunk) override;
void onException() override;
void onFinish() override;
virtual SinkPtr createSinkForPartition(const String & partition_id) = 0;

View File

@ -782,11 +782,25 @@ public:
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
}
void onException() override
{
write_buf->finalize();
}
void onFinish() override
{
writer->finalize();
writer->flush();
write_buf->finalize();
try
{
writer->finalize();
writer->flush();
write_buf->finalize();
}
catch (...)
{
/// Stop ParallelFormattingOutputFormat correctly.
writer.reset();
throw;
}
}
private:

View File

@ -469,6 +469,11 @@ public:
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
}
void onException() override
{
write_buf->finalize();
}
void onFinish() override
{
try

View File

@ -442,6 +442,11 @@ void StorageURLSink::consume(Chunk chunk)
writer->write(getHeader().cloneWithColumns(chunk.detachColumns()));
}
void StorageURLSink::onException()
{
write_buf->finalize();
}
void StorageURLSink::onFinish()
{
writer->finalize();

View File

@ -114,6 +114,7 @@ public:
std::string getName() const override { return "StorageURLSink"; }
void consume(Chunk chunk) override;
void onException() override;
void onFinish() override;
private:

View File

@ -220,7 +220,7 @@ class PRInfo:
else:
diff_object = PatchSet(response.text)
self.changed_files = {f.path for f in diff_object}
print("Fetched info about %d changed files", len(self.changed_files))
print("Fetched info about %d changed files" % len(self.changed_files))
def get_dict(self):
return {

View File

@ -0,0 +1,5 @@
<clickhouse>
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
</access_control_improvements>
</clickhouse>

View File

@ -19,6 +19,7 @@ ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/enable_access_control_improvements.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/disks.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/

View File

@ -18,4 +18,8 @@
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
<rotateOnOpen>true</rotateOnOpen>
</logger>
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
</access_control_improvements>
</clickhouse>

View File

@ -0,0 +1,23 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<default>
<!-- Allows all rows -->
<databases>
<mydb>
<filtered_table1>
<filter>1</filter>
</filtered_table1>
<filtered_table2>
<filter>1</filter>
</filtered_table2>
<filtered_table3>
<filter>1</filter>
</filtered_table3>
</mydb>
</databases>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,5 @@
<clickhouse>
<access_control_improvements>
<users_without_row_policies_can_read_rows remove="remove"/>
</access_control_improvements>
</clickhouse>

View File

@ -0,0 +1,18 @@
<?xml version="1.0"?>
<clickhouse>
<profiles>
<default>
<optimize_on_insert>0</optimize_on_insert>
</default>
</profiles>
<users>
<another>
<password/>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1 @@
../../normal_filters.xml

View File

@ -0,0 +1,24 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
<another>
<password/>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</another>
</users>
<quotas>
<default/>
</quotas>
</clickhouse>

View File

@ -0,0 +1,3 @@
<?xml version="1.0"?>
<clickhouse>
</clickhouse>

View File

@ -0,0 +1,23 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<default>
<!-- Allows no rows -->
<databases>
<mydb>
<filtered_table1>
<filter>NULL</filter>
</filtered_table1>
<filtered_table2>
<filter>NULL</filter>
</filtered_table2>
<filtered_table3>
<filter>NULL</filter>
</filtered_table3>
</mydb>
</databases>
</default>
</users>
</clickhouse>

View File

@ -0,0 +1,36 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<default>
<!-- For testing the table filters -->
<databases>
<mydb>
<!-- Simple expression filter -->
<filtered_table1>
<filter>a = 1</filter>
</filtered_table1>
<!-- Complex expression filter -->
<filtered_table2>
<filter>a + b &lt; 1 or c - d &gt; 5</filter>
</filtered_table2>
<!-- Filter with ALIAS column -->
<filtered_table3>
<filter>c = 1</filter>
</filtered_table3>
</mydb>
</databases>
</default>
<another>
<!-- For testing the table filters -->
<databases>
<mydb>
<local>
<filter>a = 1</filter>
</local>
</mydb>
</databases>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1,418 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
main_configs=["configs/config.d/disable_access_control_improvements.xml"],
user_configs=[
"configs/users.d/row_policy.xml",
"configs/users.d/another_user.xml",
],
)
def copy_policy_xml(local_file_name):
script_dir = os.path.dirname(os.path.realpath(__file__))
node.copy_file_to_container(
os.path.join(script_dir, local_file_name),
"/etc/clickhouse-server/users.d/row_policy.xml",
)
node.query("SYSTEM RELOAD CONFIG")
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
node.query(
"""
CREATE DATABASE mydb;
CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.table (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.table values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.filtered_table2 (a UInt8, b UInt8, c UInt8, d UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table2 values (0, 0, 0, 0), (1, 2, 3, 4), (4, 3, 2, 1), (0, 0, 6, 0);
CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.`.filtered_table4` (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1);
CREATE TABLE mydb.local (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
"""
)
node.query("INSERT INTO mydb.local values (2, 0), (2, 1), (1, 0), (1, 1)")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_policies():
try:
yield
finally:
copy_policy_xml("normal_filters.xml")
node.query("DROP POLICY IF EXISTS pA, pB ON mydb.filtered_table1")
def test_introspection():
policies = [
[
"another ON mydb.filtered_table1",
"another",
"mydb",
"filtered_table1",
"6068883a-0e9d-f802-7e22-0144f8e66d3c",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.filtered_table2",
"another",
"mydb",
"filtered_table2",
"c019e957-c60b-d54e-cc52-7c90dac5fb01",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.filtered_table3",
"another",
"mydb",
"filtered_table3",
"4cb080d0-44e8-dbef-6026-346655143628",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.local",
"another",
"mydb",
"local",
"5b23c389-7e18-06bf-a6bc-dd1afbbc0a97",
"users.xml",
"a = 1",
0,
0,
"['another']",
"[]",
],
[
"default ON mydb.filtered_table1",
"default",
"mydb",
"filtered_table1",
"9e8a8f62-4965-2b5e-8599-57c7b99b3549",
"users.xml",
"a = 1",
0,
0,
"['default']",
"[]",
],
[
"default ON mydb.filtered_table2",
"default",
"mydb",
"filtered_table2",
"cffae79d-b9bf-a2ef-b798-019c18470b25",
"users.xml",
"a + b < 1 or c - d > 5",
0,
0,
"['default']",
"[]",
],
[
"default ON mydb.filtered_table3",
"default",
"mydb",
"filtered_table3",
"12fc5cef-e3da-3940-ec79-d8be3911f42b",
"users.xml",
"c = 1",
0,
0,
"['default']",
"[]",
],
[
"default ON mydb.local",
"default",
"mydb",
"local",
"cdacaeb5-1d97-f99d-2bb0-4574f290629c",
"users.xml",
"1",
0,
0,
"['default']",
"[]",
],
]
assert node.query(
"SELECT * from system.row_policies ORDER BY short_name, database, table"
) == TSV(policies)
def test_dcl_introspection():
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"another ON mydb.local",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV(
["another", "default"]
)
assert node.query("SHOW POLICIES ON mydb.local") == TSV(["another", "default"])
assert node.query("SHOW POLICIES ON mydb.*") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"another ON mydb.local",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
assert node.query("SHOW POLICIES default") == TSV(
[
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table1")
== "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table2")
== "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table3")
== "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.local")
== "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
)
assert node.query("SHOW CREATE POLICY default") == TSV(
[
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
expected_access = (
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another\n"
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n"
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
)
assert expected_access in node.query("SHOW ACCESS")
copy_policy_xml("all_rows.xml")
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
]
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table1")
== "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING 1 TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table2")
== "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING 1 TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table3")
== "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING 1 TO default\n"
)
copy_policy_xml("no_rows.xml")
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
]
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table1")
== "CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING NULL TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table2")
== "CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING NULL TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.filtered_table3")
== "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING NULL TO default\n"
)
copy_policy_xml("no_filters.xml")
assert node.query("SHOW POLICIES") == ""
def test_dcl_management():
copy_policy_xml("no_filters.xml")
assert node.query("SHOW POLICIES") == ""
node.query("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a<b")
assert node.query("SELECT * FROM mydb.filtered_table1") == ""
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pA\n"
node.query("ALTER POLICY pA ON mydb.filtered_table1 TO default")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[0, 1]])
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pA\n"
node.query("ALTER POLICY pA ON mydb.filtered_table1 FOR SELECT USING a>b")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0]])
node.query("ALTER POLICY pA ON mydb.filtered_table1 RENAME TO pB")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0]])
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pB\n"
assert (
node.query("SHOW CREATE POLICY pB ON mydb.filtered_table1")
== "CREATE ROW POLICY pB ON mydb.filtered_table1 FOR SELECT USING a > b TO default\n"
)
node.query("DROP POLICY pB ON mydb.filtered_table1")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
assert node.query("SHOW POLICIES") == ""
def test_dcl_users_with_policies_from_users_xml():
node.query("CREATE USER X")
node.query("GRANT SELECT ON mydb.filtered_table1 TO X")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]])
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == ""
node.query("DROP USER X")
def test_some_users_without_policies():
copy_policy_xml("no_filters.xml")
assert node.query("SHOW POLICIES") == ""
node.query("CREATE USER X, Y")
node.query("GRANT SELECT ON mydb.filtered_table1 TO X, Y")
# permissive a >= b for X, none for Y
node.query(
"CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a >= b AS permissive TO X"
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == ""
# restrictive a >=b for X, none for Y
node.query("ALTER POLICY pA ON mydb.filtered_table1 AS restrictive")
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == ""
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == ""
# permissive a >= b for X, restrictive a <= b for X, none for Y
node.query("ALTER POLICY pA ON mydb.filtered_table1 AS permissive")
node.query(
"CREATE POLICY pB ON mydb.filtered_table1 FOR SELECT USING a <= b AS restrictive TO X"
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == ""
# permissive a >= b for X, restrictive a <= b for Y
node.query("ALTER POLICY pB ON mydb.filtered_table1 TO Y")
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == ""
node.query("DROP POLICY pA, pB ON mydb.filtered_table1")
node.query("DROP USER X, Y")

View File

@ -65,14 +65,28 @@ def test_dynamic_query_handler():
).status_code
)
res_default = cluster.instance.http_request(
"test_dynamic_handler_get?max_threads=1&get_dynamic_handler_query="
+ test_query,
method="GET",
headers={"XXX": "xxx"},
)
assert 200 == res_default.status_code
assert (
200
== cluster.instance.http_request(
"test_dynamic_handler_get?max_threads=1&get_dynamic_handler_query="
+ test_query,
method="GET",
headers={"XXX": "xxx"},
).status_code
"text/tab-separated-values; charset=UTF-8"
== res_default.headers["content-type"]
)
res_custom_ct = cluster.instance.http_request(
"test_dynamic_handler_get_custom_content_type?max_threads=1&get_dynamic_handler_query="
+ test_query,
method="GET",
headers={"XXX": "xxx"},
)
assert 200 == res_custom_ct.status_code
assert (
"application/whatever; charset=cp1337"
== res_custom_ct.headers["content-type"]
)
@ -116,22 +130,22 @@ def test_predefined_query_handler():
).status_code
)
res1 = cluster.instance.http_request(
"test_predefined_handler_get?max_threads=1&setting_name=max_threads",
method="GET",
headers={"XXX": "xxx"},
)
assert b"max_threads\t1\n" == res1.content
assert (
b"max_threads\t1\n"
== cluster.instance.http_request(
"test_predefined_handler_get?max_threads=1&setting_name=max_threads",
method="GET",
headers={"XXX": "xxx"},
).content
"text/tab-separated-values; charset=UTF-8" == res1.headers["content-type"]
)
assert (
b"max_final_threads\t1\nmax_threads\t1\n"
== cluster.instance.http_request(
"query_param_with_url/max_threads?max_threads=1&max_final_threads=1",
headers={"XXX": "max_final_threads"},
).content
res2 = cluster.instance.http_request(
"query_param_with_url/max_threads?max_threads=1&max_final_threads=1",
headers={"XXX": "max_final_threads"},
)
assert b"max_final_threads\t1\nmax_threads\t1\n" == res2.content
assert "application/generic+one" == res2.headers["content-type"]
def test_fixed_static_handler():

View File

@ -11,5 +11,15 @@
<query_param_name>get_dynamic_handler_query</query_param_name>
</handler>
</rule>
<rule>
<headers><XXX>xxx</XXX></headers>
<methods>GET</methods>
<url>/test_dynamic_handler_get_custom_content_type</url>
<handler>
<type>dynamic_query_handler</type>
<query_param_name>get_dynamic_handler_query</query_param_name>
<content_type>application/whatever; charset=cp1337</content_type>
</handler>
</rule>
</http_handlers>
</clickhouse>

View File

@ -19,6 +19,7 @@
<handler>
<type>predefined_query_handler</type>
<query>SELECT name, value FROM system.settings WHERE name = {setting_name_1:String} OR name = {setting_name_2:String}</query>
<content_type>application/generic+one</content_type>
</handler>
</rule>
</http_handlers>

View File

@ -409,45 +409,6 @@ def test_reload_users_xml_by_timer():
def test_introspection():
policies = [
[
"another ON mydb.filtered_table1",
"another",
"mydb",
"filtered_table1",
"6068883a-0e9d-f802-7e22-0144f8e66d3c",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.filtered_table2",
"another",
"mydb",
"filtered_table2",
"c019e957-c60b-d54e-cc52-7c90dac5fb01",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.filtered_table3",
"another",
"mydb",
"filtered_table3",
"4cb080d0-44e8-dbef-6026-346655143628",
"users.xml",
"1",
0,
0,
"['another']",
"[]",
],
[
"another ON mydb.local",
"another",
@ -500,19 +461,6 @@ def test_introspection():
"['default']",
"[]",
],
[
"default ON mydb.local",
"default",
"mydb",
"local",
"cdacaeb5-1d97-f99d-2bb0-4574f290629c",
"users.xml",
"1",
0,
0,
"['default']",
"[]",
],
]
assert node.query(
"SELECT * from system.row_policies ORDER BY short_name, database, table"
@ -522,31 +470,21 @@ def test_introspection():
def test_dcl_introspection():
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"another ON mydb.local",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV(
["another", "default"]
)
assert node.query("SHOW POLICIES ON mydb.local") == TSV(["another", "default"])
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == TSV(["default"])
assert node.query("SHOW POLICIES ON mydb.local") == TSV(["another"])
assert node.query("SHOW POLICIES ON mydb.*") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"another ON mydb.local",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
assert node.query("SHOW POLICIES default") == TSV(
@ -554,7 +492,6 @@ def test_dcl_introspection():
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
"default ON mydb.local",
]
)
@ -570,9 +507,8 @@ def test_dcl_introspection():
node.query("SHOW CREATE POLICY default ON mydb.filtered_table3")
== "CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
)
assert (
node.query("SHOW CREATE POLICY default ON mydb.local")
== "CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
assert "no row policy" in node.query_and_get_error(
"SHOW CREATE POLICY default ON mydb.local"
)
assert node.query("SHOW CREATE POLICY default") == TSV(
@ -580,58 +516,41 @@ def test_dcl_introspection():
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES ON mydb.filtered_table1") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES ON mydb.*") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
assert node.query("SHOW CREATE POLICIES") == TSV(
[
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another",
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another",
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default",
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default",
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default",
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default",
]
)
expected_access = (
"CREATE ROW POLICY another ON mydb.filtered_table1 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.filtered_table2 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.filtered_table3 FOR SELECT USING 1 TO another\n"
"CREATE ROW POLICY another ON mydb.local FOR SELECT USING a = 1 TO another\n"
"CREATE ROW POLICY default ON mydb.filtered_table1 FOR SELECT USING a = 1 TO default\n"
"CREATE ROW POLICY default ON mydb.filtered_table2 FOR SELECT USING ((a + b) < 1) OR ((c - d) > 5) TO default\n"
"CREATE ROW POLICY default ON mydb.filtered_table3 FOR SELECT USING c = 1 TO default\n"
"CREATE ROW POLICY default ON mydb.local FOR SELECT USING 1 TO default\n"
)
assert expected_access in node.query("SHOW ACCESS")
copy_policy_xml("all_rows.xml")
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
@ -653,9 +572,6 @@ def test_dcl_introspection():
copy_policy_xml("no_rows.xml")
assert node.query("SHOW POLICIES") == TSV(
[
"another ON mydb.filtered_table1",
"another ON mydb.filtered_table2",
"another ON mydb.filtered_table3",
"default ON mydb.filtered_table1",
"default ON mydb.filtered_table2",
"default ON mydb.filtered_table3",
@ -683,7 +599,9 @@ def test_dcl_management():
assert node.query("SHOW POLICIES") == ""
node.query("CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a<b")
assert node.query("SELECT * FROM mydb.filtered_table1") == ""
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
assert node.query("SHOW POLICIES ON mydb.filtered_table1") == "pA\n"
node.query("ALTER POLICY pA ON mydb.filtered_table1 TO default")
@ -787,6 +705,70 @@ def test_users_xml_is_readonly():
)
def test_dcl_users_with_policies_from_users_xml():
node.query("CREATE USER X")
node.query("GRANT SELECT ON mydb.filtered_table1 TO X")
assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]])
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
node.query("DROP USER X")
def test_some_users_without_policies():
copy_policy_xml("no_filters.xml")
assert node.query("SHOW POLICIES") == ""
node.query("CREATE USER X, Y")
node.query("GRANT SELECT ON mydb.filtered_table1 TO X, Y")
# permissive a >= b for X, none for Y
node.query(
"CREATE POLICY pA ON mydb.filtered_table1 FOR SELECT USING a >= b AS permissive TO X"
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
# restrictive a >=b for X, none for Y
node.query("ALTER POLICY pA ON mydb.filtered_table1 AS restrictive")
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
# permissive a >= b for X, restrictive a <= b for X, none for Y
node.query("ALTER POLICY pA ON mydb.filtered_table1 AS permissive")
node.query(
"CREATE POLICY pB ON mydb.filtered_table1 FOR SELECT USING a <= b AS restrictive TO X"
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == TSV(
[[0, 0], [0, 1], [1, 0], [1, 1]]
)
# permissive a >= b for X, restrictive a <= b for Y
node.query("ALTER POLICY pB ON mydb.filtered_table1 TO Y")
assert node.query("SELECT * FROM mydb.filtered_table1", user="X") == TSV(
[[0, 0], [1, 0], [1, 1]]
)
assert node.query("SELECT * FROM mydb.filtered_table1", user="Y") == TSV(
[[0, 0], [0, 1], [1, 1]]
)
node.query("DROP POLICY pA, pB ON mydb.filtered_table1")
node.query("DROP USER X, Y")
def test_tags_with_db_and_table_names():
copy_policy_xml("tags_with_db_and_table_names.xml")

View File

@ -17,5 +17,5 @@ truncate -s1025 "$tmp_path"
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM file('$tmp_path', 'LineAsString') FORMAT Null" |& grep -m1 -F -o -e OSReadChars
# NOTE: that OSCPUVirtualTimeMicroseconds is in microseconds, so 1e6 is not enough.
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM numbers(10e6) FORMAT Null" |& grep -m1 -F -o -e OSCPUVirtualTimeMicroseconds
$CLICKHOUSE_LOCAL --profile-events-delay-ms=-1 --print-profile-events -q "SELECT * FROM numbers(1e8) FORMAT Null" |& grep -m1 -F -o -e OSCPUVirtualTimeMicroseconds
exit 0

View File

@ -1,32 +0,0 @@
SET optimize_move_to_prewhere = 1;
DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column;
CREATE TABLE 02131_multiply_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x;
INSERT INTO 02131_multiply_row_policies_on_same_column VALUES (1), (2), (3), (4);
DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_multiply_row_policies_on_same_column;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
CREATE ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column USING x=1 TO ALL;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
CREATE ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column USING x=2 TO ALL;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
CREATE ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column USING x=3 TO ALL;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
CREATE ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column USING x<4 AS RESTRICTIVE TO ALL;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column;
DROP ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column;
SELECT count() FROM 02131_multiply_row_policies_on_same_column;
DROP TABLE 02131_multiply_row_policies_on_same_column;

View File

@ -0,0 +1,33 @@
None
1
2
3
4
R1: x == 1
1
R1, R2: (x == 1) OR (x == 2)
1
2
R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)
1
2
3
R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)
1
2
R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)
2
R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)
2
R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)
R4, R5: (x <= 2) AND (x >= 2)
2
R5: (x >= 2)
2
3
4
None
1
2
3
4

View File

@ -0,0 +1,54 @@
DROP TABLE IF EXISTS 02131_rptable;
CREATE TABLE 02131_rptable (x UInt8) ENGINE = MergeTree ORDER BY x;
INSERT INTO 02131_rptable VALUES (1), (2), (3), (4);
DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_rptable;
DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_rptable;
DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_rptable;
DROP ROW POLICY IF EXISTS 02131_filter_4 ON 02131_rptable;
DROP ROW POLICY IF EXISTS 02131_filter_5 ON 02131_rptable;
SELECT 'None';
SELECT * FROM 02131_rptable;
CREATE ROW POLICY 02131_filter_1 ON 02131_rptable USING x=1 AS permissive TO ALL;
SELECT 'R1: x == 1';
SELECT * FROM 02131_rptable;
CREATE ROW POLICY 02131_filter_2 ON 02131_rptable USING x=2 AS permissive TO ALL;
SELECT 'R1, R2: (x == 1) OR (x == 2)';
SELECT * FROM 02131_rptable;
CREATE ROW POLICY 02131_filter_3 ON 02131_rptable USING x=3 AS permissive TO ALL;
SELECT 'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)';
SELECT * FROM 02131_rptable;
CREATE ROW POLICY 02131_filter_4 ON 02131_rptable USING x<=2 AS restrictive TO ALL;
SELECT 'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)';
SELECT * FROM 02131_rptable;
CREATE ROW POLICY 02131_filter_5 ON 02131_rptable USING x>=2 AS restrictive TO ALL;
SELECT 'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
SELECT * FROM 02131_rptable;
DROP ROW POLICY 02131_filter_1 ON 02131_rptable;
SELECT 'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)';
SELECT * FROM 02131_rptable;
DROP ROW POLICY 02131_filter_2 ON 02131_rptable;
SELECT 'R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)';
SELECT * FROM 02131_rptable;
DROP ROW POLICY 02131_filter_3 ON 02131_rptable;
SELECT 'R4, R5: (x <= 2) AND (x >= 2)';
SELECT * FROM 02131_rptable;
DROP ROW POLICY 02131_filter_4 ON 02131_rptable;
SELECT 'R5: (x >= 2)';
SELECT * FROM 02131_rptable;
DROP ROW POLICY 02131_filter_5 ON 02131_rptable;
SELECT 'None';
SELECT * FROM 02131_rptable;
DROP TABLE 02131_rptable;