Merge branch 'master' of github.com:ClickHouse/ClickHouse into dt64_timeslots

This commit is contained in:
zvonand 2022-07-17 22:59:18 +02:00
commit 18827ab312
68 changed files with 929 additions and 180 deletions

View File

@ -106,7 +106,9 @@ void Client::processError(const String & query) const
std::vector<String> Client::loadWarningMessages()
{
std::vector<String> messages;
connection->sendQuery(connection_parameters.timeouts, "SELECT message FROM system.warnings", "" /* query_id */,
connection->sendQuery(connection_parameters.timeouts,
"SELECT * FROM viewIfPermitted(SELECT message FROM system.warnings ELSE null('message String'))",
"" /* query_id */,
QueryProcessingStage::Complete,
&global_context->getSettingsRef(),
&global_context->getClientInfo(), false, {});

View File

@ -604,9 +604,23 @@
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>
<!-- By default, for backward compatibility ON CLUSTER queries ignore CLUSTER grant,
however you can change this behaviour by setting this to true -->
<on_cluster_queries_require_cluster_grant>false</on_cluster_queries_require_cluster_grant>
<!-- By default, for backward compatibility "SELECT * FROM system.<table>" doesn't require any grants and can be executed
by any user. You can change this behaviour by setting this to true.
If it's set to true then this query requires "GRANT SELECT ON system.<table>" just like as for non-system tables.
Exceptions: a few system tables ("tables", "columns", "databases", and some constant tables like "one", "contributors")
are still accessible for everyone; and if there is a SHOW privilege (e.g. "SHOW USERS") granted the corresponding system
table (i.e. "system.users") will be accessible. -->
<select_from_system_db_requires_grant>false</select_from_system_db_requires_grant>
<!-- By default, for backward compatibility "SELECT * FROM information_schema.<table>" doesn't require any grants and can be
executed by any user. You can change this behaviour by setting this to true.
If it's set to true then this query requires "GRANT SELECT ON information_schema.<table>" just like as for ordinary tables. -->
<select_from_information_schema_requires_grant>false</select_from_information_schema_requires_grant>
</access_control_improvements>
<!-- Default profile of settings. -->

View File

@ -165,13 +165,12 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
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 */));
setOnClusterQueriesRequireClusterGrant(config_.getBool(
"access_control_improvements.on_cluster_queries_require_cluster_grant",
false /* false because we need to be compatible with earlier access configurations */));
/// Optional improvements in access control system.
/// The default values are false because we need to be compatible with earlier access configurations
setEnabledUsersWithoutRowPoliciesCanReadRows(config_.getBool("access_control_improvements.users_without_row_policies_can_read_rows", false));
setOnClusterQueriesRequireClusterGrant(config_.getBool("access_control_improvements.on_cluster_queries_require_cluster_grant", false));
setSelectFromSystemDatabaseRequiresGrant(config_.getBool("access_control_improvements.select_from_system_db_requires_grant", false));
setSelectFromInformationSchemaRequiresGrant(config_.getBool("access_control_improvements.select_from_information_schema_requires_grant", false));
addStoragesFromMainConfig(config_, config_path_, get_zookeeper_function_);
}

View File

@ -152,6 +152,12 @@ public:
void setOnClusterQueriesRequireClusterGrant(bool enable) { on_cluster_queries_require_cluster_grant = enable; }
bool doesOnClusterQueriesRequireClusterGrant() const { return on_cluster_queries_require_cluster_grant; }
void setSelectFromSystemDatabaseRequiresGrant(bool enable) { select_from_system_db_requires_grant = enable; }
bool doesSelectFromSystemDatabaseRequireGrant() const { return select_from_system_db_requires_grant; }
void setSelectFromInformationSchemaRequiresGrant(bool enable) { select_from_information_schema_requires_grant = enable; }
bool doesSelectFromInformationSchemaRequireGrant() const { return select_from_information_schema_requires_grant; }
std::shared_ptr<const ContextAccess> getContextAccess(
const UUID & user_id,
const std::vector<UUID> & current_roles,
@ -215,6 +221,8 @@ private:
std::atomic_bool allow_no_password = true;
std::atomic_bool users_without_row_policies_can_read_rows = false;
std::atomic_bool on_cluster_queries_require_cluster_grant = false;
std::atomic_bool select_from_system_db_requires_grant = false;
std::atomic_bool select_from_information_schema_requires_grant = false;
};
}

View File

@ -388,11 +388,11 @@ public:
return res;
}
void modifyFlags(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed)
void modifyFlags(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed)
{
flags_added = false;
flags_removed = false;
modifyFlagsRec(function, flags_added, flags_removed);
modifyFlagsRec(function, grant_option, flags_added, flags_removed);
if (flags_added || flags_removed)
optimizeTree();
}
@ -669,11 +669,11 @@ private:
}
template <typename ... ParentNames>
void modifyFlagsRec(const ModifyFlagsFunction & function, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names)
void modifyFlagsRec(const ModifyFlagsFunction & function, bool grant_option, bool & flags_added, bool & flags_removed, const ParentNames & ... parent_names)
{
auto invoke = [&function](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags
auto invoke = [function, grant_option](const AccessFlags & flags_, const AccessFlags & min_flags_with_children_, const AccessFlags & max_flags_with_children_, std::string_view database_ = {}, std::string_view table_ = {}, std::string_view column_ = {}) -> AccessFlags
{
return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_);
return function(flags_, min_flags_with_children_, max_flags_with_children_, database_, table_, column_, grant_option);
};
if constexpr (sizeof...(ParentNames) < 3)
@ -683,7 +683,7 @@ private:
for (auto & child : *children | boost::adaptors::map_values)
{
const String & child_name = *child.node_name;
child.modifyFlagsRec(function, flags_added, flags_removed, parent_names..., child_name);
child.modifyFlagsRec(function, grant_option, flags_added, flags_removed, parent_names..., child_name);
}
}
}
@ -1062,24 +1062,21 @@ void AccessRights::modifyFlags(const ModifyFlagsFunction & function)
{
if (!root)
return;
bool flags_added, flags_removed;
root->modifyFlags(function, flags_added, flags_removed);
root->modifyFlags(function, false, flags_added, flags_removed);
if (flags_removed && root_with_grant_option)
root_with_grant_option->makeIntersection(*root);
}
void AccessRights::modifyFlagsWithGrantOption(const ModifyFlagsFunction & function)
{
if (!root_with_grant_option)
return;
bool flags_added, flags_removed;
root_with_grant_option->modifyFlags(function, flags_added, flags_removed);
if (flags_added)
if (root_with_grant_option)
{
if (!root)
root = std::make_unique<Node>();
root->makeUnion(*root_with_grant_option);
root_with_grant_option->modifyFlags(function, true, flags_added, flags_removed);
if (flags_added)
{
if (!root)
root = std::make_unique<Node>();
root->makeUnion(*root_with_grant_option);
}
}
}

View File

@ -109,9 +109,9 @@ public:
const AccessFlags & max_flags_with_children,
std::string_view database,
std::string_view table,
std::string_view column)>;
std::string_view column,
bool grant_option)>;
void modifyFlags(const ModifyFlagsFunction & function);
void modifyFlagsWithGrantOption(const ModifyFlagsFunction & function);
friend bool operator ==(const AccessRights & left, const AccessRights & right);
friend bool operator !=(const AccessRights & left, const AccessRights & right) { return !(left == right); }

View File

@ -44,9 +44,17 @@ namespace
}
AccessRights addImplicitAccessRights(const AccessRights & access)
AccessRights addImplicitAccessRights(const AccessRights & access, const AccessControl & access_control)
{
auto modifier = [&](const AccessFlags & flags, const AccessFlags & min_flags_with_children, const AccessFlags & max_flags_with_children, std::string_view database, std::string_view table, std::string_view column) -> AccessFlags
AccessFlags max_flags;
auto modifier = [&](const AccessFlags & flags,
const AccessFlags & min_flags_with_children,
const AccessFlags & max_flags_with_children,
std::string_view database,
std::string_view table,
std::string_view column,
bool /* grant_option */) -> AccessFlags
{
size_t level = !database.empty() + !table.empty() + !column.empty();
AccessFlags res = flags;
@ -115,17 +123,80 @@ namespace
res |= show_databases;
}
max_flags |= res;
return res;
};
AccessRights res = access;
res.modifyFlags(modifier);
res.modifyFlagsWithGrantOption(modifier);
/// Anyone has access to the "system" and "information_schema" database.
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
/// If "select_from_system_db_requires_grant" is enabled we provide implicit grants only for a few tables in the system database.
if (access_control.doesSelectFromSystemDatabaseRequireGrant())
{
const char * always_accessible_tables[] = {
/// Constant tables
"one",
/// "numbers", "numbers_mt", "zeros", "zeros_mt" were excluded because they can generate lots of values and
/// that can decrease performance in some cases.
"contributors",
"licenses",
"time_zones",
"collations",
"formats",
"privileges",
"data_type_families",
"table_engines",
"table_functions",
"aggregate_function_combinators",
"functions", /// Can contain user-defined functions
/// The following tables hide some rows if the current user doesn't have corresponding SHOW privileges.
"databases",
"tables",
"columns",
/// Specific to the current session
"settings",
"current_roles",
"enabled_roles",
"quota_usage"
};
for (const auto * table_name : always_accessible_tables)
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, table_name);
if (max_flags.contains(AccessType::SHOW_USERS))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "users");
if (max_flags.contains(AccessType::SHOW_ROLES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "roles");
if (max_flags.contains(AccessType::SHOW_ROW_POLICIES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "row_policies");
if (max_flags.contains(AccessType::SHOW_SETTINGS_PROFILES))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "settings_profiles");
if (max_flags.contains(AccessType::SHOW_QUOTAS))
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE, "quotas");
}
else
{
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
}
/// If "select_from_information_schema_requires_grant" is enabled we don't provide implicit grants for the information_schema database.
if (!access_control.doesSelectFromInformationSchemaRequireGrant())
{
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
}
return res;
}
@ -247,7 +318,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr<const EnabledRolesInfo> &
void ContextAccess::calculateAccessRights() const
{
access = std::make_shared<AccessRights>(mixAccessRightsFromUserAndRoles(*user, *roles_info));
access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*access));
access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*access, *access_control));
if (trace_log)
{
@ -342,7 +413,7 @@ std::shared_ptr<const ContextAccess> ContextAccess::getFullAccess()
auto full_access = std::shared_ptr<ContextAccess>(new ContextAccess);
full_access->is_full_access = true;
full_access->access = std::make_shared<AccessRights>(AccessRights::getFullAccess());
full_access->access_with_implicit = std::make_shared<AccessRights>(addImplicitAccessRights(*full_access->access));
full_access->access_with_implicit = full_access->access;
return full_access;
}();
return res;
@ -413,7 +484,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
};
if (is_full_access)
return access_granted();
return true;
if (user_was_dropped)
return access_denied("User has been dropped", ErrorCodes::UNKNOWN_USER);
@ -422,7 +493,7 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg
flags &= ~AccessType::CLUSTER;
if (!flags)
return access_granted();
return true;
/// Access to temporary tables is controlled in an unusual way, not like normal tables.
/// Creating of temporary tables is controlled by AccessType::CREATE_TEMPORARY_TABLES grant,

View File

@ -43,7 +43,7 @@ public:
~ThetaSketchData() = default;
/// Insert original value without hash, as `datasketches::update_theta_sketch.update` will do the hash internal.
void insertOriginal(const StringRef & value)
void insertOriginal(StringRef value)
{
getSkUpdate()->update(value.data, value.size);
}

View File

@ -50,52 +50,58 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti
{
/// NOTE: Once you will update the completion list,
/// do not forget to update 01676_clickhouse_client_autocomplete.sh
WriteBufferFromOwnString query;
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
"SELECT name FROM system.functions"
" UNION ALL "
"SELECT name FROM system.table_engines"
" UNION ALL "
"SELECT name FROM system.formats"
" UNION ALL "
"SELECT name FROM system.table_functions"
" UNION ALL "
"SELECT name FROM system.data_type_families"
" UNION ALL "
"SELECT name FROM system.merge_tree_settings"
" UNION ALL "
"SELECT name FROM system.settings"
" UNION ALL ";
String query;
auto add_subquery = [&](std::string_view select, std::string_view result_column_name)
{
if (!query.empty())
query += " UNION ALL ";
query += fmt::format("SELECT * FROM viewIfPermitted({} ELSE null('{} String'))", select, result_column_name);
};
auto add_column = [&](std::string_view column_name, std::string_view table_name, bool distinct, std::optional<Int64> limit)
{
add_subquery(
fmt::format(
"SELECT {}{} FROM system.{}{}",
(distinct ? "DISTINCT " : ""),
column_name,
table_name,
(limit ? (" LIMIT " + std::to_string(*limit)) : "")),
column_name);
};
add_column("name", "functions", false, {});
add_column("name", "table_engines", false, {});
add_column("name", "formats", false, {});
add_column("name", "table_functions", false, {});
add_column("name", "data_type_families", false, {});
add_column("name", "merge_tree_settings", false, {});
add_column("name", "settings", false, {});
if (!basic_suggestion)
{
query << "SELECT cluster FROM system.clusters"
" UNION ALL "
"SELECT macro FROM system.macros"
" UNION ALL "
"SELECT policy_name FROM system.storage_policies"
" UNION ALL ";
add_column("cluster", "clusters", false, {});
add_column("macro", "macros", false, {});
add_column("policy_name", "storage_policies", false, {});
}
query << "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate";
add_subquery("SELECT concat(func.name, comb.name) AS x FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate", "x");
/// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero.
if (suggestion_limit > 0)
{
String limit_str = toString(suggestion_limit);
query << " UNION ALL "
"SELECT name FROM system.databases LIMIT " << limit_str
<< " UNION ALL "
"SELECT DISTINCT name FROM system.tables LIMIT " << limit_str
<< " UNION ALL ";
add_column("name", "databases", false, suggestion_limit);
add_column("name", "tables", true, suggestion_limit);
if (!basic_suggestion)
{
query << "SELECT DISTINCT name FROM system.dictionaries LIMIT " << limit_str
<< " UNION ALL ";
add_column("name", "dictionaries", true, suggestion_limit);
}
query << "SELECT DISTINCT name FROM system.columns LIMIT " << limit_str;
add_column("name", "columns", true, suggestion_limit);
}
query << ") WHERE notEmpty(res)";
return query.str();
query = "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" + query + ") WHERE notEmpty(res)";
return query;
}
template <typename ConnectionType>

View File

@ -509,7 +509,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
if (secondary_index)
next_position += secondary_index->size();
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
auto insert_key = [&](StringRef ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
{
auto inserted_pos = cur_index.insert(ref);
positions[num_added_rows] = inserted_pos;

View File

@ -92,7 +92,7 @@ struct ReverseIndexHashTableCell
/// Special case when we want to compare with something not in index_column.
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
bool keyEquals(StringRef object, size_t hash_ [[maybe_unused]], const State & state) const
{
auto index = key;
if constexpr (has_base_index)
@ -322,7 +322,7 @@ public:
static constexpr bool is_numeric_column = isNumericColumn(static_cast<ColumnType *>(nullptr));
static constexpr bool use_saved_hash = !is_numeric_column;
UInt64 insert(const StringRef & data);
UInt64 insert(StringRef data);
/// Returns the found data's index in the dictionary. If index is not built, builds it.
UInt64 getInsertionPoint(StringRef data)
@ -383,7 +383,7 @@ private:
void buildIndex();
UInt64 getHash(const StringRef & ref) const
UInt64 getHash(StringRef ref) const
{
if constexpr (is_numeric_column)
{
@ -478,7 +478,7 @@ ColumnUInt64::MutablePtr ReverseIndex<IndexType, ColumnType>::calcHashes() const
}
template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
UInt64 ReverseIndex<IndexType, ColumnType>::insert(StringRef data)
{
if (!index)
buildIndex();

View File

@ -12,7 +12,7 @@ struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash,
using Base::Base;
static constexpr bool need_zero_value_storage = false;
// external
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const Key & getKey(const value_type & value_) { return value_.first; }
};
@ -32,7 +32,7 @@ struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16,
void setZero() { this->value.first.items[1] = 0; }
// external
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const StringKey16 & getKey(const value_type & value_) { return value_.first; }
};
@ -53,7 +53,7 @@ struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24,
void setZero() { this->value.first.c = 0; }
// external
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const StringKey24 & getKey(const value_type & value_) { return value_.first; }
};

View File

@ -49,12 +49,12 @@ struct SpaceSavingArena
template <>
struct SpaceSavingArena<StringRef>
{
StringRef emplace(const StringRef & key)
StringRef emplace(StringRef key)
{
return copyStringInArena(arena, key);
}
void free(const StringRef & key)
void free(StringRef key)
{
if (key.data)
arena.free(const_cast<char *>(key.data), key.size);

View File

@ -20,13 +20,13 @@ TLDList::TLDList(size_t size)
: tld_container(size)
, pool(std::make_unique<Arena>(10 << 20))
{}
bool TLDList::insert(const StringRef & host)
bool TLDList::insert(StringRef host)
{
bool inserted;
tld_container.emplace(DB::ArenaKeyHolder{host, *pool}, inserted);
return inserted;
}
bool TLDList::has(const StringRef & host) const
bool TLDList::has(StringRef host) const
{
return tld_container.has(host);
}

View File

@ -23,9 +23,9 @@ public:
explicit TLDList(size_t size);
/// Return true if the tld_container does not contains such element.
bool insert(const StringRef & host);
bool insert(StringRef host);
/// Check is there such TLD
bool has(const StringRef & host) const;
bool has(StringRef host) const;
size_t size() const { return tld_container.size(); }
private:

View File

@ -14,7 +14,7 @@ String quoteString(std::string_view x)
}
String doubleQuoteString(const StringRef & x)
String doubleQuoteString(StringRef x)
{
String res(x.size, '\0');
WriteBufferFromString wb(res);
@ -23,7 +23,7 @@ String doubleQuoteString(const StringRef & x)
}
String backQuote(const StringRef & x)
String backQuote(StringRef x)
{
String res(x.size, '\0');
{
@ -34,7 +34,7 @@ String backQuote(const StringRef & x)
}
String backQuoteIfNeed(const StringRef & x)
String backQuoteIfNeed(StringRef x)
{
String res(x.size, '\0');
{

View File

@ -16,12 +16,12 @@ namespace DB
}
/// Double quote the string.
String doubleQuoteString(const StringRef & x);
String doubleQuoteString(StringRef x);
/// Quote the identifier with backquotes.
String backQuote(const StringRef & x);
String backQuote(StringRef x);
/// Quote the identifier with backquotes, if required.
String backQuoteIfNeed(const StringRef & x);
String backQuoteIfNeed(StringRef x);
}

View File

@ -105,7 +105,7 @@ ColumnPtr FlatDictionary::getColumn(
getItemsImpl<ValueType, true>(
attribute,
ids,
[&](size_t row, const StringRef value, bool is_null)
[&](size_t row, StringRef value, bool is_null)
{
(*vec_null_map_to)[row] = is_null;
out->insertData(value.data, value.size);
@ -115,7 +115,7 @@ ColumnPtr FlatDictionary::getColumn(
getItemsImpl<ValueType, false>(
attribute,
ids,
[&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); },
[&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); },
default_value_extractor);
}
else

View File

@ -585,7 +585,7 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type>::getAttributeColumn(
getItemsImpl<ValueType, true>(
attribute,
keys_object,
[&](size_t row, const StringRef value, bool is_null)
[&](size_t row, StringRef value, bool is_null)
{
(*vec_null_map_to)[row] = is_null;
out->insertData(value.data, value.size);
@ -595,7 +595,7 @@ ColumnPtr HashedArrayDictionary<dictionary_key_type>::getAttributeColumn(
getItemsImpl<ValueType, false>(
attribute,
keys_object,
[&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); },
[&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); },
default_value_extractor);
}
else

View File

@ -117,7 +117,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
getItemsImpl<ValueType, true>(
attribute,
extractor,
[&](size_t row, const StringRef value, bool is_null)
[&](size_t row, StringRef value, bool is_null)
{
(*vec_null_map_to)[row] = is_null;
out->insertData(value.data, value.size);
@ -127,7 +127,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getColumn(
getItemsImpl<ValueType, false>(
attribute,
extractor,
[&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); },
[&](size_t, StringRef value, bool) { out->insertData(value.data, value.size); },
default_value_extractor);
}
else

View File

@ -261,7 +261,7 @@ ColumnPtr IPAddressDictionary::getColumn(
getItemsImpl<ValueType>(
attribute,
key_columns,
[&](const size_t, const StringRef value) { out->insertData(value.data, value.size); },
[&](const size_t, StringRef value) { out->insertData(value.data, value.size); },
default_value_extractor);
}
else

View File

@ -151,7 +151,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
getItemsImpl<ValueType, true>(
attribute,
modified_key_columns,
[&](size_t row, const StringRef value, bool is_null)
[&](size_t row, StringRef value, bool is_null)
{
(*vec_null_map_to)[row] = is_null;
out->insertData(value.data, value.size);
@ -161,7 +161,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumn(
getItemsImpl<ValueType, false>(
attribute,
modified_key_columns,
[&](size_t, const StringRef value, bool)
[&](size_t, StringRef value, bool)
{
out->insertData(value.data, value.size);
},
@ -255,7 +255,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumnInternal(
getItemsInternalImpl<ValueType, true>(
attribute,
key_to_index,
[&](size_t row, const StringRef value, bool is_null)
[&](size_t row, StringRef value, bool is_null)
{
(*vec_null_map_to)[row] = is_null;
out->insertData(value.data, value.size);
@ -264,7 +264,7 @@ ColumnPtr RangeHashedDictionary<dictionary_key_type>::getColumnInternal(
getItemsInternalImpl<ValueType, false>(
attribute,
key_to_index,
[&](size_t, const StringRef value, bool)
[&](size_t, StringRef value, bool)
{
out->insertData(value.data, value.size);
});

View File

@ -25,7 +25,7 @@ void onError(std::string error_message)
throw DB::Exception(error_message, DB::ErrorCodes::OPENSSL_ERROR);
}
StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const StringRef & key, std::array<char, EVP_MAX_KEY_LENGTH> & folded_key)
StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array<char, EVP_MAX_KEY_LENGTH> & folded_key)
{
assert(cipher_key_size <= EVP_MAX_KEY_LENGTH);
memcpy(folded_key.data(), key.data, cipher_key_size);
@ -38,7 +38,7 @@ StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const
return StringRef(folded_key.data(), cipher_key_size);
}
const EVP_CIPHER * getCipherByName(const StringRef & cipher_name)
const EVP_CIPHER * getCipherByName(StringRef cipher_name)
{
// NOTE: cipher obtained not via EVP_CIPHER_fetch() would cause extra work on each context reset
// with EVP_CIPHER_CTX_reset() or EVP_EncryptInit_ex(), but using EVP_CIPHER_fetch()

View File

@ -32,9 +32,9 @@ namespace ErrorCodes
namespace OpenSSLDetails
{
[[noreturn]] void onError(std::string error_message);
StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, const StringRef & key, std::array<char, EVP_MAX_KEY_LENGTH> & folded_key);
StringRef foldEncryptionKeyInMySQLCompatitableMode(size_t cipher_key_size, StringRef key, std::array<char, EVP_MAX_KEY_LENGTH> & folded_key);
const EVP_CIPHER * getCipherByName(const StringRef & name);
const EVP_CIPHER * getCipherByName(StringRef name);
enum class CompatibilityMode
{
@ -53,7 +53,7 @@ enum class CipherMode
template <CipherMode mode>
struct KeyHolder
{
inline StringRef setKey(size_t cipher_key_size, const StringRef & key) const
inline StringRef setKey(size_t cipher_key_size, StringRef key) const
{
if (key.size != cipher_key_size)
throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size),
@ -66,7 +66,7 @@ struct KeyHolder
template <>
struct KeyHolder<CipherMode::MySQLCompatibility>
{
inline StringRef setKey(size_t cipher_key_size, const StringRef & key)
inline StringRef setKey(size_t cipher_key_size, StringRef key)
{
if (key.size < cipher_key_size)
throw DB::Exception(fmt::format("Invalid key size: {} expected {}", key.size, cipher_key_size),
@ -120,7 +120,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher)
}
template <CipherMode mode>
inline void validateIV(const StringRef & iv_value, const size_t cipher_iv_size)
inline void validateIV(StringRef iv_value, const size_t cipher_iv_size)
{
// In MySQL mode we don't care if IV is longer than expected, only if shorter.
if ((mode == CipherMode::MySQLCompatibility && iv_value.size != 0 && iv_value.size < cipher_iv_size)

View File

@ -27,7 +27,7 @@ class IPAddressVariant
{
public:
explicit IPAddressVariant(const StringRef & address_str)
explicit IPAddressVariant(StringRef address_str)
{
/// IP address parser functions require that the input is
/// NULL-terminated so we need to copy it.
@ -75,7 +75,7 @@ struct IPAddressCIDR
UInt8 prefix;
};
IPAddressCIDR parseIPWithCIDR(const StringRef cidr_str)
IPAddressCIDR parseIPWithCIDR(StringRef cidr_str)
{
std::string_view cidr_str_view(cidr_str);
size_t pos_slash = cidr_str_view.find('/');

View File

@ -1053,7 +1053,7 @@ template void readDateTimeTextFallback<void>(time_t &, ReadBuffer &, const DateL
template bool readDateTimeTextFallback<bool>(time_t &, ReadBuffer &, const DateLUTImpl &);
void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field)
void skipJSONField(ReadBuffer & buf, StringRef name_of_field)
{
if (buf.eof())
throw Exception("Unexpected EOF for key '" + name_of_field.toString() + "'", ErrorCodes::INCORRECT_DATA);

View File

@ -1238,7 +1238,7 @@ inline void skipWhitespaceIfAny(ReadBuffer & buf, bool one_line = false)
}
/// Skips json value.
void skipJSONField(ReadBuffer & buf, const StringRef & name_of_field);
void skipJSONField(ReadBuffer & buf, StringRef name_of_field);
/** Read serialized exception.

View File

@ -66,7 +66,7 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac
/// The same, but quotes apply only if there are characters that do not match the identifier without quotes
template <typename F>
static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffer & buf, F && write_quoted_string)
static inline void writeProbablyQuotedStringImpl(StringRef s, WriteBuffer & buf, F && write_quoted_string)
{
if (isValidIdentifier(std::string_view{s})
/// This are valid identifiers but are problematic if present unquoted in SQL query.
@ -79,19 +79,19 @@ static inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffe
write_quoted_string(s, buf);
}
void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf)
void writeProbablyBackQuotedString(StringRef s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); });
}
void writeProbablyDoubleQuotedString(const StringRef & s, WriteBuffer & buf)
void writeProbablyDoubleQuotedString(StringRef s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
}
void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
void writeProbablyBackQuotedStringMySQL(StringRef s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](StringRef s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); });
}
void writePointerHex(const void * ptr, WriteBuffer & buf)

View File

@ -102,7 +102,7 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf)
buf.write(s.data(), s.size());
}
inline void writeStringBinary(const StringRef & s, WriteBuffer & buf)
inline void writeStringBinary(StringRef s, WriteBuffer & buf)
{
writeVarUInt(s.size, buf);
buf.write(s.data, s.size);
@ -360,7 +360,7 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b
}
inline void writeJSONString(const StringRef & s, WriteBuffer & buf, const FormatSettings & settings)
inline void writeJSONString(StringRef s, WriteBuffer & buf, const FormatSettings & settings)
{
writeJSONString(s.data, s.data + s.size, buf, settings);
}
@ -435,7 +435,7 @@ inline void writeEscapedString(const String & s, WriteBuffer & buf)
}
inline void writeEscapedString(const StringRef & ref, WriteBuffer & buf)
inline void writeEscapedString(StringRef ref, WriteBuffer & buf)
{
writeEscapedString(ref.data, ref.size, buf);
}
@ -462,7 +462,7 @@ void writeAnyQuotedString(const String & s, WriteBuffer & buf)
template <char quote_character>
void writeAnyQuotedString(const StringRef & ref, WriteBuffer & buf)
void writeAnyQuotedString(StringRef ref, WriteBuffer & buf)
{
writeAnyQuotedString<quote_character>(ref.data, ref.data + ref.size, buf);
}
@ -473,7 +473,7 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf)
writeAnyQuotedString<'\''>(s, buf);
}
inline void writeQuotedString(const StringRef & ref, WriteBuffer & buf)
inline void writeQuotedString(StringRef ref, WriteBuffer & buf)
{
writeAnyQuotedString<'\''>(ref, buf);
}
@ -488,7 +488,7 @@ inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf)
writeAnyQuotedString<'"'>(s, buf);
}
inline void writeDoubleQuotedString(const StringRef & s, WriteBuffer & buf)
inline void writeDoubleQuotedString(StringRef s, WriteBuffer & buf)
{
writeAnyQuotedString<'"'>(s, buf);
}
@ -499,13 +499,13 @@ inline void writeDoubleQuotedString(std::string_view s, WriteBuffer & buf)
}
/// Outputs a string in backquotes.
inline void writeBackQuotedString(const StringRef & s, WriteBuffer & buf)
inline void writeBackQuotedString(StringRef s, WriteBuffer & buf)
{
writeAnyQuotedString<'`'>(s, buf);
}
/// Outputs a string in backquotes for MySQL.
inline void writeBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
inline void writeBackQuotedStringMySQL(StringRef s, WriteBuffer & buf)
{
writeChar('`', buf);
writeAnyEscapedString<'`', true>(s.data, s.data + s.size, buf);
@ -514,9 +514,9 @@ inline void writeBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
/// Write quoted if the string doesn't look like and identifier.
void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf);
void writeProbablyDoubleQuotedString(const StringRef & s, WriteBuffer & buf);
void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf);
void writeProbablyBackQuotedString(StringRef s, WriteBuffer & buf);
void writeProbablyDoubleQuotedString(StringRef s, WriteBuffer & buf);
void writeProbablyBackQuotedStringMySQL(StringRef s, WriteBuffer & buf);
/** Outputs the string in for the CSV format.
@ -559,7 +559,7 @@ void writeCSVString(const String & s, WriteBuffer & buf)
}
template <char quote = '"'>
void writeCSVString(const StringRef & s, WriteBuffer & buf)
void writeCSVString(StringRef s, WriteBuffer & buf)
{
writeCSVString<quote>(s.data, s.data + s.size, buf);
}
@ -616,7 +616,7 @@ inline void writeXMLStringForTextElementOrAttributeValue(const String & s, Write
writeXMLStringForTextElementOrAttributeValue(s.data(), s.data() + s.size(), buf);
}
inline void writeXMLStringForTextElementOrAttributeValue(const StringRef & s, WriteBuffer & buf)
inline void writeXMLStringForTextElementOrAttributeValue(StringRef s, WriteBuffer & buf)
{
writeXMLStringForTextElementOrAttributeValue(s.data, s.data + s.size, buf);
}
@ -657,7 +657,7 @@ inline void writeXMLStringForTextElement(const String & s, WriteBuffer & buf)
writeXMLStringForTextElement(s.data(), s.data() + s.size(), buf);
}
inline void writeXMLStringForTextElement(const StringRef & s, WriteBuffer & buf)
inline void writeXMLStringForTextElement(StringRef s, WriteBuffer & buf)
{
writeXMLStringForTextElement(s.data, s.data + s.size, buf);
}
@ -890,7 +890,7 @@ requires is_arithmetic_v<T>
inline void writeBinary(const T & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const String & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(const StringRef & x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(StringRef x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(std::string_view x, WriteBuffer & buf) { writeStringBinary(x, buf); }
inline void writeBinary(const Decimal32 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
inline void writeBinary(const Decimal64 & x, WriteBuffer & buf) { writePODBinary(x, buf); }
@ -1017,7 +1017,7 @@ inline void writeQuoted(const String & x, WriteBuffer & buf) { writeQuotedString
inline void writeQuoted(std::string_view x, WriteBuffer & buf) { writeQuotedString(x, buf); }
inline void writeQuoted(const StringRef & x, WriteBuffer & buf) { writeQuotedString(x, buf); }
inline void writeQuoted(StringRef x, WriteBuffer & buf) { writeQuotedString(x, buf); }
inline void writeQuoted(const LocalDate & x, WriteBuffer & buf)
{
@ -1050,7 +1050,7 @@ inline void writeDoubleQuoted(const String & x, WriteBuffer & buf) { writeDouble
inline void writeDoubleQuoted(std::string_view x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); }
inline void writeDoubleQuoted(const StringRef & x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); }
inline void writeDoubleQuoted(StringRef x, WriteBuffer & buf) { writeDoubleQuotedString(x, buf); }
inline void writeDoubleQuoted(const LocalDate & x, WriteBuffer & buf)
{

View File

@ -238,7 +238,7 @@ struct AggregationMethodString
std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
static void insertKeyIntoColumns(const StringRef & key, std::vector<IColumn *> & key_columns, const Sizes &)
static void insertKeyIntoColumns(StringRef key, std::vector<IColumn *> & key_columns, const Sizes &)
{
static_cast<ColumnString *>(key_columns[0])->insertData(key.data, key.size);
}
@ -270,7 +270,7 @@ struct AggregationMethodStringNoCache
std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
static void insertKeyIntoColumns(const StringRef & key, std::vector<IColumn *> & key_columns, const Sizes &)
static void insertKeyIntoColumns(StringRef key, std::vector<IColumn *> & key_columns, const Sizes &)
{
static_cast<ColumnString *>(key_columns[0])->insertData(key.data, key.size);
}
@ -302,7 +302,7 @@ struct AggregationMethodFixedString
std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
static void insertKeyIntoColumns(const StringRef & key, std::vector<IColumn *> & key_columns, const Sizes &)
static void insertKeyIntoColumns(StringRef key, std::vector<IColumn *> & key_columns, const Sizes &)
{
static_cast<ColumnFixedString *>(key_columns[0])->insertData(key.data, key.size);
}
@ -333,7 +333,7 @@ struct AggregationMethodFixedStringNoCache
std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
static void insertKeyIntoColumns(const StringRef & key, std::vector<IColumn *> & key_columns, const Sizes &)
static void insertKeyIntoColumns(StringRef key, std::vector<IColumn *> & key_columns, const Sizes &)
{
static_cast<ColumnFixedString *>(key_columns[0])->insertData(key.data, key.size);
}
@ -501,7 +501,7 @@ struct AggregationMethodSerialized
std::optional<Sizes> shuffleKeyColumns(std::vector<IColumn *> &, const Sizes &) { return {}; }
static void insertKeyIntoColumns(const StringRef & key, std::vector<IColumn *> & key_columns, const Sizes &)
static void insertKeyIntoColumns(StringRef key, std::vector<IColumn *> & key_columns, const Sizes &)
{
const auto * pos = key.data;
for (auto & column : key_columns)

View File

@ -1191,7 +1191,7 @@ void Context::setSettings(const Settings & settings_)
}
void Context::setSetting(const StringRef & name, const String & value)
void Context::setSetting(StringRef name, const String & value)
{
auto lock = getLock();
if (name == "profile")
@ -1206,7 +1206,7 @@ void Context::setSetting(const StringRef & name, const String & value)
}
void Context::setSetting(const StringRef & name, const Field & value)
void Context::setSetting(StringRef name, const Field & value)
{
auto lock = getLock();
if (name == "profile")

View File

@ -607,8 +607,8 @@ public:
void setSettings(const Settings & settings_);
/// Set settings by name.
void setSetting(const StringRef & name, const String & value);
void setSetting(const StringRef & name, const Field & value);
void setSetting(StringRef name, const String & value);
void setSetting(StringRef name, const Field & value);
void applySettingChange(const SettingChange & change);
void applySettingsChanges(const SettingsChanges & changes);

View File

@ -509,6 +509,25 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << ')';
written = true;
}
if (!written && name == "viewIfPermitted"sv)
{
/// viewIfPermitted() needs special formatting: ELSE instead of comma between arguments, and better indents too.
const auto * nl_or_nothing = settings.one_line ? "" : "\n";
auto indent0 = settings.one_line ? "" : String(4u * frame.indent, ' ');
auto indent1 = settings.one_line ? "" : String(4u * (frame.indent + 1), ' ');
auto indent2 = settings.one_line ? "" : String(4u * (frame.indent + 2), ' ');
settings.ostr << (settings.hilite ? hilite_function : "") << name << "(" << (settings.hilite ? hilite_none : "") << nl_or_nothing;
FormatStateStacked frame_nested = frame;
frame_nested.need_parens = false;
frame_nested.indent += 2;
arguments->children[0]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent1 << (settings.hilite ? hilite_keyword : "") << (settings.one_line ? " " : "")
<< "ELSE " << (settings.hilite ? hilite_none : "") << nl_or_nothing << indent2;
arguments->children[1]->formatImpl(settings, state, frame_nested);
settings.ostr << nl_or_nothing << indent0 << ")";
return;
}
}
if (!written && arguments->children.size() >= 2)

View File

@ -1068,13 +1068,16 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserIdentifier id_parser;
ParserKeyword view("VIEW");
ParserSelectWithUnionQuery select;
ASTPtr identifier;
ASTPtr query;
if (!view.ignore(pos, expected))
bool if_permitted = false;
if (ParserKeyword{"VIEWIFPERMITTED"}.ignore(pos, expected))
if_permitted = true;
else if (!ParserKeyword{"VIEW"}.ignore(pos, expected))
return false;
if (pos->type != TokenType::OpeningRoundBracket)
@ -1094,15 +1097,30 @@ bool ParserTableFunctionView::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
return false;
}
ASTPtr else_ast;
if (if_permitted)
{
if (!ParserKeyword{"ELSE"}.ignore(pos, expected))
return false;
if (!ParserWithOptionalAlias{std::make_unique<ParserFunction>(true, true), true}.parse(pos, else_ast, expected))
return false;
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
auto expr_list = std::make_shared<ASTExpressionList>();
expr_list->children.push_back(query);
if (if_permitted)
expr_list->children.push_back(else_ast);
auto function_node = std::make_shared<ASTFunction>();
tryGetIdentifierNameInto(identifier, function_node->name);
auto expr_list_with_single_query = std::make_shared<ASTExpressionList>();
expr_list_with_single_query->children.push_back(query);
function_node->name = "view";
function_node->arguments = expr_list_with_single_query;
function_node->name = if_permitted ? "viewIfPermitted" : "view";
function_node->arguments = expr_list;
function_node->children.push_back(function_node->arguments);
node = function_node;
return true;
@ -1971,6 +1989,7 @@ const char * ParserAlias::restricted_keywords[] =
"WITH",
"INTERSECT",
"EXCEPT",
"ELSE",
nullptr
};

View File

@ -162,7 +162,7 @@ protected:
bool is_table_function;
};
// A special function parser for view table function.
// A special function parser for view and viewIfPermitted table functions.
// It parses an SELECT query as its argument and doesn't support getColumnName().
class ParserTableFunctionView : public IParserBase
{

View File

@ -62,7 +62,7 @@ const String & JSONEachRowRowInputFormat::columnName(size_t i) const
return getPort().getHeader().getByPosition(i).name;
}
inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index)
inline size_t JSONEachRowRowInputFormat::columnIndex(StringRef name, size_t key_index)
{
/// Optimization by caching the order of fields (which is almost always the same)
/// and a quick check to match the next expected field, instead of searching the hash table.
@ -124,7 +124,7 @@ static inline void skipColonDelimeter(ReadBuffer & istr)
skipWhitespaceIfAny(istr);
}
void JSONEachRowRowInputFormat::skipUnknownField(const StringRef & name_ref)
void JSONEachRowRowInputFormat::skipUnknownField(StringRef name_ref)
{
if (!format_settings.skip_unknown_fields)
throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);

View File

@ -40,9 +40,9 @@ private:
void syncAfterError() override;
const String & columnName(size_t i) const;
size_t columnIndex(const StringRef & name, size_t key_index);
size_t columnIndex(StringRef name, size_t key_index);
bool advanceToNextKey(size_t key_index);
void skipUnknownField(const StringRef & name_ref);
void skipUnknownField(StringRef name_ref);
StringRef readColumnName(ReadBuffer & buf);
void readField(size_t index, MutableColumns & columns);
void readJSONObject(MutableColumns & columns);

View File

@ -71,7 +71,7 @@ static const Graphite::Pattern undef_pattern =
.type = undef_pattern.TypeUndef,
};
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, const StringRef path)
inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, StringRef path)
{
if (params.patterns_typed)
{

View File

@ -23,7 +23,7 @@ namespace ErrorCodes
using CompiledRegexPtr = std::shared_ptr<const re2::RE2>;
static inline bool checkRegexExpression(const StringRef & match_str, const CompiledRegexPtr & compiled_regex)
static inline bool checkRegexExpression(StringRef match_str, const CompiledRegexPtr & compiled_regex)
{
int num_captures = compiled_regex->NumberOfCapturingGroups() + 1;
@ -32,7 +32,7 @@ static inline bool checkRegexExpression(const StringRef & match_str, const Compi
return compiled_regex->Match(match_input, 0, match_str.size, re2::RE2::Anchor::ANCHOR_BOTH, matches, num_captures);
}
static inline bool checkExpression(const StringRef & match_str, const std::pair<String, CompiledRegexPtr> & expression)
static inline bool checkExpression(StringRef match_str, const std::pair<String, CompiledRegexPtr> & expression)
{
if (expression.second)
return checkRegexExpression(match_str, expression.second);

View File

@ -180,9 +180,13 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
if (!table_expression->database_and_table_name)
{
// If it's a view table function, add a fake db.table name.
if (table_expression->table_function && table_expression->table_function->as<ASTFunction>()->name == "view")
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
else
if (table_expression->table_function)
{
auto table_function_name = table_expression->table_function->as<ASTFunction>()->name;
if ((table_function_name == "view") || (table_function_name == "viewIfPermitted"))
table_expression->database_and_table_name = std::make_shared<ASTTableIdentifier>("__view");
}
if (!table_expression->database_and_table_name)
throw Exception("Logical error: incorrect table expression", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -36,8 +36,11 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes()
void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -66,8 +66,11 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes()
void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -78,7 +78,11 @@ NamesAndTypesList StorageSystemQuotaUsage::getNamesAndTypesImpl(bool add_column_
void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
auto usage = context->getQuotaUsage();
if (!usage)
return;

View File

@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes()
void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
std::vector<UUID> ids = access_control.findAll<Quota>();
size_t column_index = 0;

View File

@ -15,7 +15,11 @@ NamesAndTypesList StorageSystemQuotasUsage::getNamesAndTypes()
void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_QUOTAS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_QUOTAS);
auto all_quotas_usage = context->getAccessControl().getAllQuotasUsage();
StorageSystemQuotaUsage::fillDataImpl(res_columns, context, /* add_column_is_current = */ true, all_quotas_usage);
}

View File

@ -31,8 +31,11 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes()
void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());

View File

@ -27,8 +27,11 @@ NamesAndTypesList StorageSystemRoles::getNamesAndTypes()
void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROLES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_ROLES);
std::vector<UUID> ids = access_control.findAll<Role>();
size_t column_index = 0;

View File

@ -53,8 +53,11 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes()
void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_ROW_POLICIES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_ROW_POLICIES);
std::vector<UUID> ids = access_control.findAll<RowPolicy>();
size_t column_index = 0;

View File

@ -37,8 +37,11 @@ NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes()
void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
std::vector<UUID> ids = access_control.findAll<User>();
boost::range::push_back(ids, access_control.findAll<Role>());
boost::range::push_back(ids, access_control.findAll<SettingsProfile>());

View File

@ -34,8 +34,11 @@ NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes()
void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES);
std::vector<UUID> ids = access_control.findAll<SettingsProfile>();
size_t column_index = 0;

View File

@ -60,8 +60,11 @@ NamesAndTypesList StorageSystemUsers::getNamesAndTypes()
void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
context->checkAccess(AccessType::SHOW_USERS);
/// If "select_from_system_db_requires_grant" is enabled the access rights were already checked in InterpreterSelectQuery.
const auto & access_control = context->getAccessControl();
if (!access_control.doesSelectFromSystemDatabaseRequireGrant())
context->checkAccess(AccessType::SHOW_USERS);
std::vector<UUID> ids = access_control.findAll<User>();
size_t column_index = 0;

View File

@ -23,7 +23,12 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte
ColumnsDescription cached_columns, bool use_global_context) const
{
ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
context->checkAccess(AccessType::CREATE_TEMPORARY_TABLE | getSourceAccessType());
AccessFlags required_access = getSourceAccessType();
String function_name = getName();
if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted"))
required_access |= AccessType::CREATE_TEMPORARY_TABLE;
context->checkAccess(required_access);
auto context_to_use = use_global_context ? context->getGlobalContext() : context;

View File

@ -0,0 +1,113 @@
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Storages/StorageNull.h>
#include <Storages/StorageView.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionViewIfPermitted.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Interpreters/evaluateConstantExpression.h>
#include "registerTableFunctions.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int ACCESS_DENIED;
}
const ASTSelectWithUnionQuery & TableFunctionViewIfPermitted::getSelectQuery() const
{
return *create.select;
}
void TableFunctionViewIfPermitted::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto * function = ast_function->as<ASTFunction>();
if (!function || !function->arguments || (function->arguments->children.size() != 2))
throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Table function '{}' requires two arguments: a SELECT query and a table function",
getName());
const auto & arguments = function->arguments->children;
auto * select = arguments[0]->as<ASTSelectWithUnionQuery>();
if (!select)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a SELECT query as its first argument", getName());
create.set(create.select, select->clone());
else_ast = arguments[1];
if (!else_ast->as<ASTFunction>())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table function '{}' requires a table function as its second argument", getName());
else_table_function = TableFunctionFactory::instance().get(else_ast, context);
}
ColumnsDescription TableFunctionViewIfPermitted::getActualTableStructure(ContextPtr context) const
{
return else_table_function->getActualTableStructure(context);
}
StoragePtr TableFunctionViewIfPermitted::executeImpl(
const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, ColumnsDescription /* cached_columns */) const
{
StoragePtr storage;
auto columns = getActualTableStructure(context);
if (isPermitted(context, columns))
{
storage = std::make_shared<StorageView>(StorageID(getDatabaseName(), table_name), create, columns, "");
}
else
{
storage = else_table_function->execute(else_ast, context, table_name);
}
storage->startup();
return storage;
}
bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const
{
Block sample_block;
try
{
/// Will throw ACCESS_DENIED if the current user is not allowed to execute the SELECT query.
sample_block = InterpreterSelectWithUnionQuery::getSampleBlock(create.children[0], context);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ACCESS_DENIED)
return false;
throw;
}
/// We check that columns match only if permitted (otherwise we could reveal the structure to an user who must not know it).
ColumnsDescription columns{sample_block.getNamesAndTypesList()};
if (columns != else_columns)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Table function '{}' requires a SELECT query with the result columns matching a table function after 'ELSE'. "
"Currently the result columns of the SELECT query are {}, and the table function after 'ELSE' gives {}",
getName(),
columns.toString(),
else_columns.toString());
}
return true;
}
void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionViewIfPermitted>();
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <TableFunctions/ITableFunction.h>
#include <Parsers/ASTCreateQuery.h>
#include <base/types.h>
namespace DB
{
/* viewIfPermitted(query ELSE null('structure'))
* Works as "view(query)" if the current user has the permissions required to execute "query"; works as "null('structure')" otherwise.
*/
class TableFunctionViewIfPermitted : public ITableFunction
{
public:
static constexpr auto name = "viewIfPermitted";
std::string getName() const override { return name; }
const ASTSelectWithUnionQuery & getSelectQuery() const;
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const String & table_name, ColumnsDescription cached_columns) const override;
const char * getStorageTypeName() const override { return "ViewIfPermitted"; }
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
bool isPermitted(const ContextPtr & context, const ColumnsDescription & else_columns) const;
ASTCreateQuery create;
ASTPtr else_ast;
TableFunctionPtr else_table_function;
};
}

View File

@ -42,6 +42,7 @@ void registerTableFunctions()
registerTableFunctionJDBC(factory);
registerTableFunctionView(factory);
registerTableFunctionViewIfPermitted(factory);
#if USE_MYSQL
registerTableFunctionMySQL(factory);

View File

@ -40,6 +40,7 @@ void registerTableFunctionODBC(TableFunctionFactory & factory);
void registerTableFunctionJDBC(TableFunctionFactory & factory);
void registerTableFunctionView(TableFunctionFactory & factory);
void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory);
#if USE_MYSQL
void registerTableFunctionMySQL(TableFunctionFactory & factory);

View File

@ -2,5 +2,7 @@
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
<on_cluster_queries_require_cluster_grant>true</on_cluster_queries_require_cluster_grant>
<select_from_system_db_requires_grant>true</select_from_system_db_requires_grant>
<select_from_information_schema_requires_grant>true</select_from_information_schema_requires_grant>
</access_control_improvements>
</clickhouse>

View File

@ -21,5 +21,8 @@
<access_control_improvements>
<users_without_row_policies_can_read_rows>true</users_without_row_policies_can_read_rows>
<on_cluster_queries_require_cluster_grant>true</on_cluster_queries_require_cluster_grant>
<select_from_system_db_requires_grant>true</select_from_system_db_requires_grant>
<select_from_information_schema_requires_grant>true</select_from_information_schema_requires_grant>
</access_control_improvements>
</clickhouse>

View File

@ -434,6 +434,7 @@ def test_required_privileges():
node1.query("INSERT INTO tbl VALUES (100)")
node1.query("CREATE USER u1")
node1.query("GRANT CLUSTER ON *.* TO u1")
backup_name = new_backup_name()
expected_error = "necessary to have grant BACKUP ON default.tbl"
@ -478,6 +479,7 @@ def test_system_users():
backup_name = new_backup_name()
node1.query("CREATE USER u2 SETTINGS allow_backup=false")
node1.query("GRANT CLUSTER ON *.* TO u2")
expected_error = "necessary to have grant BACKUP ON system.users"
assert expected_error in node1.query_and_get_error(

View File

@ -19,6 +19,7 @@
</networks>
<allow_databases>
<database>default</database>
<database>system</database>
</allow_databases>
</test_allow>
</users>

View File

@ -1,5 +1,7 @@
<clickhouse>
<access_control_improvements>
<users_without_row_policies_can_read_rows remove="remove"/>
<select_from_system_db_requires_grant remove="remove"/>
<select_from_information_schema_requires_grant remove="remove"/>
</access_control_improvements>
</clickhouse>

View File

@ -13,6 +13,9 @@
</networks>
<profile>default</profile>
<quota>default</quota>
<allow_databases>
<database>mydb</database>
</allow_databases>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1,162 @@
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/another_user.xml",
],
)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
node.query("CREATE DATABASE mydb")
node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log")
node.query("CREATE TABLE table2(x UInt32) ENGINE=Log")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_after_test():
try:
node.query("CREATE USER OR REPLACE sqluser")
yield
finally:
pass
def test_system_db():
assert node.query("SELECT count()>0 FROM system.settings") == "1\n"
assert node.query("SELECT count()>0 FROM system.users") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n"
assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n"
expected_error = "necessary to have grant SHOW USERS ON *.*"
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="another"
)
assert node.query("SELECT count()>0 FROM system.clusters", user="another") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="another"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="another"
)
== "0\n"
)
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
expected_error = "necessary to have grant SHOW USERS ON *.*"
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="sqluser"
)
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "0\n"
)
node.query("GRANT SHOW USERS ON *.* TO sqluser")
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "1\n"
)
def test_information_schema():
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="another",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="another",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "0\n"
)
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "1\n"
)

View File

@ -0,0 +1,16 @@
<?xml version="1.0"?>
<clickhouse>
<users>
<another>
<password/>
<networks>
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
<allow_databases>
<database>mydb</database>
</allow_databases>
</another>
</users>
</clickhouse>

View File

@ -0,0 +1,192 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node",
user_configs=[
"configs/another_user.xml",
],
)
@pytest.fixture(scope="module", autouse=True)
def started_cluster():
try:
cluster.start()
node.query("CREATE DATABASE mydb")
node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log")
node.query("CREATE TABLE table2(x UInt32) ENGINE=Log")
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(autouse=True)
def reset_after_test():
try:
node.query("CREATE USER OR REPLACE sqluser")
yield
finally:
pass
def test_system_db():
assert node.query("SELECT count()>0 FROM system.settings") == "1\n"
assert node.query("SELECT count()>0 FROM system.users") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table1'") == "1\n"
assert node.query("SELECT count() FROM system.tables WHERE name='table2'") == "1\n"
assert node.query("SELECT count()>0 FROM system.settings", user="another") == "1\n"
expected_error = (
"necessary to have grant SELECT for at least one column on system.users"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="another"
)
expected_error = (
"necessary to have grant SELECT for at least one column on system.clusters"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.clusters", user="another"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="another"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="another"
)
== "0\n"
)
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
expected_error = (
"necessary to have grant SELECT for at least one column on system.users"
)
assert expected_error in node.query_and_get_error(
"SELECT count()>0 FROM system.users", user="sqluser"
)
expected_error = (
"necessary to have grant SELECT for at least one column on system.clusters"
)
assert node.query_and_get_error(
"SELECT count()>0 FROM system.clusters", user="sqluser"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "0\n"
)
node.query("GRANT SELECT ON system.users TO sqluser")
node.query("GRANT SELECT ON system.clusters TO sqluser")
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert node.query("SELECT count()>0 FROM system.settings", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
assert node.query("SELECT count()>0 FROM system.clusters", user="sqluser") == "1\n"
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table1'", user="sqluser"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM system.tables WHERE name='table2'", user="sqluser"
)
== "1\n"
)
node.query("REVOKE ALL ON *.* FROM sqluser")
node.query("GRANT SHOW USERS ON *.* TO sqluser")
assert node.query("SELECT count()>0 FROM system.users", user="sqluser") == "1\n"
def test_information_schema():
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'"
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'"
)
== "1\n"
)
expected_error = (
"necessary to have grant SELECT(table_name) ON information_schema.tables"
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="another",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="another",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
assert expected_error in node.query_and_get_error(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
node.query("GRANT SELECT ON information_schema.* TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "0\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "0\n"
)
node.query("GRANT SHOW ON mydb.table1 TO sqluser")
node.query("GRANT SHOW ON table2 TO sqluser")
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table1'",
user="sqluser",
)
== "1\n"
)
assert (
node.query(
"SELECT count() FROM information_schema.tables WHERE table_name='table2'",
user="sqluser",
)
== "1\n"
)

View File

@ -65,3 +65,38 @@ def test_merge():
"it's necessary to have grant SELECT ON default.table2"
in instance.query_and_get_error(select_query, user="A")
)
def test_view_if_permitted():
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))"
)
== "1\n"
)
expected_error = "requires a SELECT query with the result columns matching a table function after 'ELSE'"
assert expected_error in instance.query_and_get_error(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x Int32'))"
)
assert expected_error in instance.query_and_get_error(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('y UInt32'))"
)
instance.query("CREATE USER A")
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))",
user="A",
)
== ""
)
instance.query("GRANT SELECT ON table1 TO A")
assert (
instance.query(
"SELECT * FROM viewIfPermitted(SELECT * FROM table1 ELSE null('x UInt32'))",
user="A",
)
== "1\n"
)

View File

@ -9,6 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CLIENT} -q "drop user if exists u_00600"
${CLICKHOUSE_CLIENT} -q "create user u_00600 settings max_execution_time=60, readonly=1"
${CLICKHOUSE_CLIENT} -q "grant select on system.numbers to u_00600"
function wait_for_query_to_start()
{

View File

@ -10,6 +10,7 @@ set -e
user=user_$CLICKHOUSE_TEST_UNIQUE_NAME
$CLICKHOUSE_CLIENT --query "DROP USER IF EXISTS $user"
$CLICKHOUSE_CLIENT --query "CREATE USER $user IDENTIFIED WITH PLAINTEXT_PASSWORD BY 'hello'"
$CLICKHOUSE_CLIENT --query "GRANT SELECT ON system.numbers TO $user"
trap '$CLICKHOUSE_CLIENT --query "DROP USER $user"' EXIT
# Wait for query to start executing. At that time, the password should be cleared.