Improve isolation of query results in query cache

Fixes #58054
This commit is contained in:
Robert Schulze 2024-01-08 17:17:49 +00:00
parent 6665d23243
commit fabc06995e
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 168 additions and 14 deletions

View File

@ -29,10 +29,6 @@ Transactionally inconsistent caching is traditionally provided by client tools o
the same caching logic and configuration is often duplicated. With ClickHouse's query cache, the caching logic moves to the server side.
This reduces maintenance effort and avoids redundancy.
:::note
Security consideration: The cached query result is tied to the user executing it. Authorization checks are performed when the query is executed. This means that if there are any alterations to the user's role or permissions between the time the query is cached and when the cache is accessed, the result will not reflect these changes. We recommend using different users to distinguish between different levels of access, instead of actively toggling roles for a single user between queries, as this practice may lead to unexpected query results.
:::
## Configuration Settings and Usage
Setting [use_query_cache](settings/settings.md#use-query-cache) can be used to control whether a specific query or all queries of the

View File

@ -129,12 +129,15 @@ String queryStringFromAST(ASTPtr ast)
QueryCache::Key::Key(
ASTPtr ast_,
Block header_,
const String & user_name_, bool is_shared_,
const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_,
bool is_shared_,
std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed_)
: ast(removeQueryCacheSettings(ast_))
, header(header_)
, user_name(user_name_)
, user_id(user_id_)
, current_user_roles(current_user_roles_)
, is_shared(is_shared_)
, expires_at(expires_at_)
, is_compressed(is_compressed_)
@ -142,8 +145,8 @@ QueryCache::Key::Key(
{
}
QueryCache::Key::Key(ASTPtr ast_, const String & user_name_)
: QueryCache::Key(ast_, {}, user_name_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name
QueryCache::Key::Key(ASTPtr ast_, const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, {}, user_name_, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name
{
}
@ -401,7 +404,10 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
const auto & entry_key = entry->key;
const auto & entry_mapped = entry->mapped;
if (!entry_key.is_shared && entry_key.user_name != key.user_name)
const bool is_same_user_name = (entry_key.user_name == key.user_name);
const bool is_same_user_id = ((!entry_key.user_id.has_value() && !key.user_id.has_value()) || (entry_key.user_id.has_value() && key.user_id.has_value() && *entry_key.user_id == *key.user_id));
const bool is_same_current_user_roles = (entry_key.current_user_roles == key.current_user_roles);
if (!entry_key.is_shared && (!is_same_user_name || !is_same_user_id || !is_same_current_user_roles))
{
LOG_TRACE(logger, "Inaccessible query result found for query {}", doubleQuoteString(key.query_string));
return;

View File

@ -51,8 +51,16 @@ public:
/// Result metadata for constructing the pipe.
const Block header;
/// The user who executed the query.
/// The name, id and current roles of the user who executed the query.
/// These members are necessary to ensure that a (non-shared, see below) entry can only be written and read by the same user with
/// the same roles. Example attack scenarios:
/// - after DROP USER, it must not be possible to create a new user with with the dropped user name and access the dropped user's
/// query cache entries
/// - different roles of the same user may be tied to different row-level policies. It must not be possible to switch role and
/// access another role's cache entries
const String user_name;
std::optional<UUID> user_id;
std::vector<UUID> current_user_roles;
/// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different
/// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row
@ -74,12 +82,13 @@ public:
/// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_,
Block header_,
const String & user_name_, bool is_shared_,
const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_,
bool is_shared_,
std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed);
/// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name).
Key(ASTPtr ast_, const String & user_name_);
Key(ASTPtr ast_, const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
bool operator==(const Key & other) const;
};

View File

@ -1010,7 +1010,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (can_use_query_cache && settings.enable_reads_from_query_cache)
{
QueryCache::Key key(ast, context->getUserName());
QueryCache::Key key(ast, context->getUserName(), context->getUserID(), context->getCurrentRoles());
QueryCache::Reader reader = query_cache->createReader(key);
if (reader.hasCacheEntryForKey())
{
@ -1123,7 +1123,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), settings.query_cache_share_between_users,
context->getUserName(), context->getUserID(), context->getCurrentRoles(),
settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),
settings.query_cache_compress_entries);

View File

@ -37,11 +37,16 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr
std::vector<QueryCache::Cache::KeyMapped> content = query_cache->dump();
const String & user_name = context->getUserName();
std::optional<UUID> user_id = context->getUserID();
std::vector<UUID> current_user_roles = context->getCurrentRoles();
for (const auto & [key, query_result] : content)
{
/// Showing other user's queries is considered a security risk
if (!key.is_shared && key.user_name != user_name)
const bool is_same_user_name = (key.user_name == user_name);
const bool is_same_user_id = ((!key.user_id.has_value() && !user_id.has_value()) || (key.user_id.has_value() && user_id.has_value() && *key.user_id == *user_id));
const bool is_same_current_user_roles = (key.current_user_roles == current_user_roles);
if (!key.is_shared && (!is_same_user_name || !is_same_user_id || !is_same_current_user_roles))
continue;
res_columns[0]->insert(key.query_string); /// approximates the original query string

View File

@ -0,0 +1,28 @@
Attack 1
0
system.query_cache with old user 1
0
0 1
1 0
system.query_cache with new user 0
0
0 1
1 0
0 1
Attack 2
-- policy_1 test
1 1
3 1
6 1
-- policy_2 test
2 2
5 2
8 2
-- policy_1 with query cache test
1 1
3 1
6 1
-- policy_2 with query cache test
2 2
5 2
8 2

View File

@ -0,0 +1,109 @@
#!/usr/bin/env bash
# Tags: no-parallel, no-fasttest
# Tag no-parallel: Messes with internal cache
# no-fasttest: Produces wrong results in fasttest, unclear why, didn't reproduce locally.
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# -- Attack 1:
# - create a user,
# - run a query whose result is stored in the query cache,
# - drop the user, recreate it with the same name
# - test that the cache entry is inaccessible
${CLICKHOUSE_CLIENT} --query "SELECT 'Attack 1'"
rnd=`tr -dc 1-9 </dev/urandom | head -c 5` # disambiguates the specific query in system.query_log below
# echo $rnd
# Start with empty query cache (QC).
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS admin"
${CLICKHOUSE_CLIENT} --query "CREATE USER admin"
${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON *.* TO admin WITH GRANT OPTION"
# Insert cache entry
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT 0 == $rnd SETTINGS use_query_cache = 1"
# Check that the system view knows the new cache entry
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT 'system.query_cache with old user', count(*) FROM system.query_cache"
# Run query again. The 1st run must be a cache miss, the 2nd run a cache hit
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT 0 == $rnd SETTINGS use_query_cache = 1"
${CLICKHOUSE_CLIENT} --user "admin" --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' AND query = 'SELECT 0 == $rnd SETTINGS use_query_cache = 1' order by query_start_time"
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS admin"
${CLICKHOUSE_CLIENT} --query "CREATE USER admin"
${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON *.* TO admin WITH GRANT OPTION"
# Check that the system view reports the cache as empty
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT 'system.query_cache with new user', count(*) FROM system.query_cache"
# Run same query as old user. Expect a cache miss.
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT 0 == $rnd SETTINGS use_query_cache = 1"
${CLICKHOUSE_CLIENT} --user "admin" --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses'] FROM system.query_log WHERE type = 'QueryFinish' AND query = 'SELECT 0 == $rnd SETTINGS use_query_cache = 1' order by query_start_time"
# Cleanup
${CLICKHOUSE_CLIENT} --query "DROP USER admin"
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"
# -- Attack 2: (scenario from issue #58054)
# - create a user,
# - create two roles, each with different row policies
# - cached query result in the context of the 1st role must must not be visible in the context of the 2nd role
${CLICKHOUSE_CLIENT} --query "SELECT 'Attack 2'"
# Start with empty query cache (QC).
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"
${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS admin"
${CLICKHOUSE_CLIENT} --query "CREATE USER admin"
${CLICKHOUSE_CLIENT} --query "GRANT CURRENT GRANTS ON *.* TO admin WITH GRANT OPTION"
# Create table
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP TABLE IF EXISTS user_data"
${CLICKHOUSE_CLIENT} --user "admin" --query "CREATE TABLE user_data (ID UInt32, userID UInt32) ENGINE = MergeTree ORDER BY userID"
# Create roles with row-level security
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE IF EXISTS user_role_1"
# ${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE POLICY IF EXISTS user_policy_1"
${CLICKHOUSE_CLIENT} --user "admin" --query "CREATE ROLE user_role_1"
${CLICKHOUSE_CLIENT} --user "admin" --query "GRANT SELECT ON user_data TO user_role_1"
${CLICKHOUSE_CLIENT} --user "admin" --query "CREATE ROW POLICY user_policy_1 ON user_data FOR SELECT USING userID = 1 TO user_role_1"
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE IF EXISTS user_role_2"
# ${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE POLICY IF EXISTS user_policy_2"
${CLICKHOUSE_CLIENT} --user "admin" --query "CREATE ROLE user_role_2"
${CLICKHOUSE_CLIENT} --user "admin" --query "GRANT SELECT ON user_data TO user_role_2"
${CLICKHOUSE_CLIENT} --user "admin" --query "CREATE ROW POLICY user_policy_2 ON user_data FOR SELECT USING userID = 2 TO user_role_2"
# Grant roles to admin
${CLICKHOUSE_CLIENT} --user "admin" --query "GRANT user_role_1, user_role_2 TO admin"
${CLICKHOUSE_CLIENT} --user "admin" --query "INSERT INTO user_data (ID, userID) VALUES (1, 1), (2, 2), (3, 1), (4, 3), (5, 2), (6, 1), (7, 4), (8, 2)"
# Test...
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT '-- policy_1 test'"
${CLICKHOUSE_CLIENT} --user "admin" --multiquery "SET ROLE user_role_1; SELECT * FROM user_data" # should only return rows with userID = 1
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT '-- policy_2 test'"
${CLICKHOUSE_CLIENT} --user "admin" --multiquery "SET ROLE user_role_2; SELECT * FROM user_data" # should only return rows with userID = 2
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT '-- policy_1 with query cache test'"
${CLICKHOUSE_CLIENT} --user "admin" --multiquery "SET ROLE user_role_1; SELECT * FROM user_data SETTINGS use_query_cache = 1" # should only return rows with userID = 1
${CLICKHOUSE_CLIENT} --user "admin" --query "SELECT '-- policy_2 with query cache test'"
${CLICKHOUSE_CLIENT} --user "admin" --multiquery "SET ROLE user_role_2; SELECT * FROM user_data SETTINGS use_query_cache = 1" # should only return rows with userID = 2 (not userID = 1!)
# Cleanup
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE user_role_1"
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP ROLE user_role_2"
${CLICKHOUSE_CLIENT} --user "admin" --query "DROP TABLE user_data"
${CLICKHOUSE_CLIENT} --query "DROP USER admin"
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"