diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md
new file mode 100644
index 00000000000..aabf31ef561
--- /dev/null
+++ b/docs/en/operations/query-cache.md
@@ -0,0 +1,112 @@
+---
+slug: /en/operations/query-cache
+sidebar_position: 65
+sidebar_label: Query Cache [experimental]
+---
+
+# Query Cache [experimental]
+
+The query cache allows to compute `SELECT` queries just once and to serve further executions of the same query directly from the cache.
+Depending on the type of the queries, this can dramatically reduce latency and resource consumption of the ClickHouse server.
+
+## Background, Design and Limitations
+
+Query caches can generally be viewed as transactionally consistent or inconsistent.
+
+- In transactionally consistent caches, the database invalidates (discards) cached query results if the result of the `SELECT` query changes
+ or potentially changes. In ClickHouse, operations which change the data include inserts/updates/deletes in/of/from tables or collapsing
+ merges. Transactionally consistent caching is especially suitable for OLTP databases, for example
+ [MySQL](https://dev.mysql.com/doc/refman/5.6/en/query-cache.html) (which removed query cache after v8.0) and
+ [Oracle](https://docs.oracle.com/database/121/TGDBA/tune_result_cache.htm).
+- In transactionally inconsistent caches, slight inaccuracies in query results are accepted under the assumption that all cache entries are
+ assigned a validity period after which they expire (e.g. 1 minute) and that the underlying data changes only little during this period.
+ This approach is overall more suitable for OLAP databases. As an example where transactionally inconsistent caching is sufficient,
+ consider an hourly sales report in a reporting tool which is simultaneously accessed by multiple users. Sales data changes typically
+ slowly enough that the database only needs to compute the report once (represented by the first `SELECT` query). Further queries can be
+ served directly from the query cache. In this example, a reasonable validity period could be 30 min.
+
+Transactionally inconsistent caching is traditionally provided by client tools or proxy packages interacting with the database. As a result,
+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.
+
+:::warning
+The query cache is an experimental feature that should not be used in production. There are known cases (e.g. in distributed query
+processing) where wrong results are returned.
+:::
+
+## Configuration Settings and Usage
+
+As long as the result cache is experimental it must be activated using the following configuration setting:
+
+```sql
+SET allow_experimental_query_cache = true;
+```
+
+Afterwards, setting [use_query_cache](settings/settings.md#use-query-cache) can be used to control whether a specific query or all queries
+of the current session should utilize the query cache. For example, the first execution of query
+
+```sql
+SELECT some_expensive_calculation(column_1, column_2)
+FROM table
+SETTINGS use_query_cache = true;
+```
+
+will store the query result in the query cache. Subsequent executions of the same query (also with parameter `use_query_cache = true`) will
+read the computed result from the cache and return it immediately.
+
+The way the cache is utilized can be configured in more detail using settings [enable_writes_to_query_cache](settings/settings.md#enable-writes-to-query-cache)
+and [enable_reads_from_query_cache](settings/settings.md#enable-reads-from-query-cache) (both `true` by default). The former setting
+controls whether query results are stored in the cache, whereas the latter setting determines if the database should try to retrieve query
+results from the cache. For example, the following query will use the cache only passively, i.e. attempt to read from it but not store its
+result in it:
+
+```sql
+SELECT some_expensive_calculation(column_1, column_2)
+FROM table
+SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;
+```
+
+For maximum control, it is generally recommended to provide settings "use_query_cache", "enable_writes_to_query_cache" and
+"enable_reads_from_query_cache" only with specific queries. It is also possible to enable caching at user or profile level (e.g. via `SET
+use_query_cache = true`) but one should keep in mind that all `SELECT` queries including monitoring or debugging queries to system tables
+may return cached results then.
+
+The query cache can be cleared using statement `SYSTEM DROP QUERY CACHE`. The content of the query cache is displayed in system table
+`SYSTEM.QUERY_CACHE`. The number of query cache hits and misses are shown as events "QueryCacheHits" and "QueryCacheMisses" in system table
+`SYSTEM.EVENTS`. Both counters are only updated for `SELECT` queries which run with setting "use_query_cache = true". Other queries do not
+affect the cache miss counter.
+
+The query cache exists once per ClickHouse server process. However, cache results are by default not shared between users. This can be
+changed (see below) but doing so is not recommended for security reasons.
+
+Query results are referenced in the query cache by the [Abstract Syntax Tree (AST)](https://en.wikipedia.org/wiki/Abstract_syntax_tree) of
+their query. This means that caching is agnostic to upper/lowercase, for example `SELECT 1` and `select 1` are treated as the same query. To
+make the matching more natural, all query-level settings related to the query cache are removed from the AST.
+
+If the query was aborted due to an exception or user cancellation, no entry is written into the query cache.
+
+The size of the query cache, the maximum number of cache entries and the maximum size of cache entries (in bytes and in records) can
+be configured using different [server configuration options](server-configuration-parameters/settings.md#server_configuration_parameters_query-cache).
+
+To define how long a query must run at least such that its result can be cached, you can use setting
+[query_cache_min_query_duration](settings/settings.md#query-cache-min-query-duration). For example, the result of query
+
+``` sql
+SELECT some_expensive_calculation(column_1, column_2)
+FROM table
+SETTINGS use_query_cache = true, query_cache_min_query_duration = 5000;
+```
+
+is only cached if the query runs longer than 5 seconds. It is also possible to specify how often a query needs to run until its result is
+cached - for that use setting [query_cache_min_query_runs](settings/settings.md#query-cache-min-query-runs).
+
+Entries in the query cache become stale after a certain time period (time-to-live). By default, this period is 60 seconds but a different
+value can be specified at session, profile or query level using setting [query_cache_ttl](settings/settings.md#query-cache-ttl).
+
+Also, results of queries with non-deterministic functions such as `rand()` and `now()` are not cached. This can be overruled using
+setting [query_cache_store_results_of_queries_with_nondeterministic_functions](settings/settings.md#query-cache-store-results-of-queries-with-nondeterministic-functions).
+
+Finally, entries in the query cache are not shared between users due to security reasons. For example, user A must not be able to bypass a
+row policy on a table by running the same query as another user B for whom no such policy exists. However, if necessary, cache entries can
+be marked accessible by other users (i.e. shared) by supplying setting
+[query_cache_share_between_users](settings/settings.md#query-cache-share-between-users).
diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md
index 9a67edd75ca..05b96a3b526 100644
--- a/docs/en/operations/server-configuration-parameters/settings.md
+++ b/docs/en/operations/server-configuration-parameters/settings.md
@@ -1270,30 +1270,30 @@ If the table does not exist, ClickHouse will create it. If the structure of the
```
-## query_result_cache {#server_configuration_parameters_query-result-cache}
+## query_cache {#server_configuration_parameters_query-cache}
-[Query result cache](../query-result-cache.md) configuration.
+[Query result cache](../query-cache.md) configuration.
The following settings are available:
-- `size`: The maximum cache size in bytes. 0 means the query result cache is disabled. Default value: `1073741824` (1 GiB).
-- `max_entries`: The maximum number of SELECT query results stored in the cache. Default value: `1024`.
-- `max_entry_size`: The maximum size in bytes SELECT query results may have to be saved in the cache. Default value: `1048576` (1 MiB).
-- `max_entry_records`: The maximum number of records SELECT query results may have to be saved in the cache. Default value: `30000000` (30 mil).
+- `size`: The maximum cache size in bytes. 0 means the query cache is disabled. Default value: `1073741824` (1 GiB).
+- `max_entries`: The maximum number of `SELECT` query results stored in the cache. Default value: `1024`.
+- `max_entry_size`: The maximum size in bytes `SELECT` query results may have to be saved in the cache. Default value: `1048576` (1 MiB).
+- `max_entry_records`: The maximum number of records `SELECT` query results may have to be saved in the cache. Default value: `30000000` (30 mil).
:::warning
-Data for the query result cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query result cache altogether.
+Data for the query cache is allocated in DRAM. If memory is scarce, make sure to set a small value for `size` or disable the query cache altogether.
:::
**Example**
```xml
-
+
1073741824
1024
1048576
30000000
-
+
```
## query_thread_log {#server_configuration_parameters-query_thread_log}
diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md
index 1407971c4f2..559241fbf39 100644
--- a/docs/en/operations/settings/settings.md
+++ b/docs/en/operations/settings/settings.md
@@ -1301,10 +1301,10 @@ Possible values:
Default value: `3`.
-## use_query_result_cache {#use-query-result-cache}
+## use_query_cache {#use-query-cache}
-If turned on, SELECT queries may utilize the [query result cache](../query-result-cache.md). Parameters [enable_reads_from_query_result_cache](#enable-reads-from-query-result-cache)
-and [enable_writes_to_query_result_cache](#enable-writes-to-query-result-cache) control in more detail how the cache is used.
+If turned on, `SELECT` queries may utilize the [query cache](../query-cache.md). Parameters [enable_reads_from_query_cache](#enable-readsfrom-query-cache)
+and [enable_writes_to_query_cache](#enable-writes-to-query-cache) control in more detail how the cache is used.
Possible values:
@@ -1313,9 +1313,9 @@ Possible values:
Default value: `0`.
-## enable_reads_from_query_result_cache {#enable-reads-from-query-result-cache}
+## enable_reads_from_query_cache {#enable-reads-from-query-cache}
-If turned on, results of SELECT queries are retrieved from the [query result cache](../query-result-cache.md).
+If turned on, results of `SELECT` queries are retrieved from the [query cache](../query-cache.md).
Possible values:
@@ -1324,9 +1324,9 @@ Possible values:
Default value: `1`.
-## enable_writes_to_query_result_cache {#enable-writes-to-query-result-cache}
+## enable_writes_to_query_cache {#enable-writes-to-query-cache}
-If turned on, results of SELECT queries are stored in the [query result cache](../query-result-cache.md).
+If turned on, results of `SELECT` queries are stored in the [query cache](../query-.md).
Possible values:
@@ -1335,9 +1335,9 @@ Possible values:
Default value: `1`.
-## query_result_cache_store_results_of_queries_with_nondeterministic_functions {#query-result-cache-store-results-of-queries-with-nondeterministic-functions}
+## query_cache_store_results_of_queries_with_nondeterministic_functions {#query--store-results-of-queries-with-nondeterministic-functions}
-If turned on, then results of SELECT queries with non-deterministic functions (e.g. `rand()`, `now()`) can be cached in the [query result cache](../query-result-cache.md).
+If turned on, then results of `SELECT` queries with non-deterministic functions (e.g. `rand()`, `now()`) can be cached in the [query cache](../query-cache.md).
Possible values:
@@ -1346,9 +1346,9 @@ Possible values:
Default value: `0`.
-## query_result_cache_min_query_runs {#query-result-cache-min-query-runs}
+## query_cache_min_query_runs {#query-cache-min-query-runs}
-Minimum number of times a SELECT query must run before its result is stored in the [query result cache](../query-result-cache.md).
+Minimum number of times a `SELECT` query must run before its result is stored in the [query cache](../query-cache.md).
Possible values:
@@ -1356,9 +1356,9 @@ Possible values:
Default value: `0`
-## query_result_cache_min_query_duration {#query-result-cache-min-query-duration}
+## query_cache_min_query_duration {#query-cache-min-query-duration}
-Minimum duration in milliseconds a query needs to run for its result to be stored in the [query result cache](../query-result-cache.md).
+Minimum duration in milliseconds a query needs to run for its result to be stored in the [query cache](../query-cache.md).
Possible values:
@@ -1366,9 +1366,9 @@ Possible values:
Default value: `0`
-## query_result_cache_ttl {#query-result-cache-ttl}
+## query_cache_ttl {#query-cache-ttl}
-After this time in seconds entries in the [query result cache](../query-result-cache.md) become stale.
+After this time in seconds entries in the [query cache](../query-cache.md) become stale.
Possible values:
@@ -1376,9 +1376,9 @@ Possible values:
Default value: `60`
-## query_result_cache_share_between_users {#query-result-cache-share-between-users}
+## query_cache_share_between_users {#query-cache-share-between-users}
-If turned on, the result of SELECT queries cached in the [query result cache](../query-result-cache.md) can be read by other users.
+If turned on, the result of `SELECT` queries cached in the [query cache](../query-cache.md) can be read by other users.
It is not recommended to enable this setting due to security reasons.
Possible values:
diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md
index 7ac0ce84e5b..300205a7ef4 100644
--- a/docs/en/sql-reference/statements/system.md
+++ b/docs/en/sql-reference/statements/system.md
@@ -103,9 +103,9 @@ Its size can be configured using the server-level setting [uncompressed_cache_si
Reset the compiled expression cache.
The compiled expression cache is enabled/disabled with the query/user/profile-level setting [compile_expressions](../../operations/settings/settings.md#compile-expressions).
-## DROP QUERY RESULT CACHE
+## DROP QUERY CACHE
-Resets the [query result cache](../../operations/query-result-cache.md).
+Resets the [query cache](../../operations/query-cache.md).
## FLUSH LOGS
diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp
index 8b6c43b450e..7b1ab1b8180 100644
--- a/programs/server/Server.cpp
+++ b/programs/server/Server.cpp
@@ -1517,13 +1517,13 @@ try
global_context->setMMappedFileCache(mmap_cache_size);
/// A cache for query results.
- size_t query_result_cache_size = config().getUInt64("query_result_cache.size", 1_GiB);
- if (query_result_cache_size)
- global_context->setQueryResultCache(
- query_result_cache_size,
- config().getUInt64("query_result_cache.max_entries", 1024),
- config().getUInt64("query_result_cache.max_entry_size", 1_MiB),
- config().getUInt64("query_result_cache.max_entry_records", 30'000'000));
+ size_t query_cache_size = config().getUInt64("query_cache.size", 1_GiB);
+ if (query_cache_size)
+ global_context->setQueryCache(
+ query_cache_size,
+ config().getUInt64("query_cache.max_entries", 1024),
+ config().getUInt64("query_cache.max_entry_size", 1_MiB),
+ config().getUInt64("query_cache.max_entry_records", 30'000'000));
#if USE_EMBEDDED_COMPILER
/// 128 MB
diff --git a/programs/server/config.xml b/programs/server/config.xml
index 0ed8ec48e83..bd46263f851 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -1466,13 +1466,13 @@
-->
-
-
+
+
-
+
don't replace it
}
-void QueryResultCache::Writer::buffer(Chunk && partial_query_result)
+void QueryCache::Writer::buffer(Chunk && partial_query_result)
{
if (skip_insert)
return;
@@ -208,7 +208,7 @@ void QueryResultCache::Writer::buffer(Chunk && partial_query_result)
}
}
-void QueryResultCache::Writer::finalizeWrite()
+void QueryCache::Writer::finalizeWrite()
{
if (skip_insert)
return;
@@ -239,7 +239,7 @@ void QueryResultCache::Writer::finalizeWrite()
}
else
++it;
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Removed {} stale entries", removed_items);
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "Removed {} stale entries", removed_items);
}
/// Insert or replace if enough space
@@ -250,23 +250,23 @@ void QueryResultCache::Writer::finalizeWrite()
cache_size_in_bytes -= it->second.sizeInBytes(); // key replacement
cache[key] = std::move(query_result);
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Stored result of query {}", key.queryStringFromAst());
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stored result of query {}", key.queryStringFromAst());
}
}
-QueryResultCache::Reader::Reader(const Cache & cache_, const Key & key, size_t & cache_size_in_bytes_, const std::lock_guard &)
+QueryCache::Reader::Reader(const Cache & cache_, const Key & key, size_t & cache_size_in_bytes_, const std::lock_guard &)
{
auto it = cache_.find(key);
if (it == cache_.end())
{
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "No entry found for query {}", key.queryStringFromAst());
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "No entry found for query {}", key.queryStringFromAst());
return;
}
if (it->first.username.has_value() && it->first.username != key.username)
{
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Inaccessible entry found for query {}", key.queryStringFromAst());
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst());
return;
}
@@ -274,33 +274,33 @@ QueryResultCache::Reader::Reader(const Cache & cache_, const Key & key, size_t &
{
cache_size_in_bytes_ -= it->second.sizeInBytes();
const_cast(cache_).erase(it);
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Stale entry found and removed for query {}", key.queryStringFromAst());
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found and removed for query {}", key.queryStringFromAst());
return;
}
pipe = Pipe(std::make_shared(it->first.header, it->second.chunks));
- LOG_TRACE(&Poco::Logger::get("QueryResultCache"), "Entry found for query {}", key.queryStringFromAst());
+ LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst());
}
-bool QueryResultCache::Reader::hasCacheEntryForKey() const
+bool QueryCache::Reader::hasCacheEntryForKey() const
{
bool res = !pipe.empty();
if (res)
- ProfileEvents::increment(ProfileEvents::QueryResultCacheHits);
+ ProfileEvents::increment(ProfileEvents::QueryCacheHits);
else
- ProfileEvents::increment(ProfileEvents::QueryResultCacheMisses);
+ ProfileEvents::increment(ProfileEvents::QueryCacheMisses);
return res;
}
-Pipe && QueryResultCache::Reader::getPipe()
+Pipe && QueryCache::Reader::getPipe()
{
chassert(!pipe.empty()); // cf. hasCacheEntryForKey()
return std::move(pipe);
}
-QueryResultCache::QueryResultCache(size_t max_cache_size_in_bytes_, size_t max_cache_entries_, size_t max_cache_entry_size_in_bytes_, size_t max_cache_entry_size_in_rows_)
+QueryCache::QueryCache(size_t max_cache_size_in_bytes_, size_t max_cache_entries_, size_t max_cache_entry_size_in_bytes_, size_t max_cache_entry_size_in_rows_)
: max_cache_size_in_bytes(max_cache_size_in_bytes_)
, max_cache_entries(max_cache_entries_)
, max_cache_entry_size_in_bytes(max_cache_entry_size_in_bytes_)
@@ -308,19 +308,19 @@ QueryResultCache::QueryResultCache(size_t max_cache_size_in_bytes_, size_t max_c
{
}
-QueryResultCache::Reader QueryResultCache::createReader(const Key & key)
+QueryCache::Reader QueryCache::createReader(const Key & key)
{
std::lock_guard lock(mutex);
return Reader(cache, key, cache_size_in_bytes, lock);
}
-QueryResultCache::Writer QueryResultCache::createWriter(const Key & key, std::chrono::milliseconds min_query_runtime)
+QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::milliseconds min_query_runtime)
{
std::lock_guard lock(mutex);
return Writer(mutex, cache, key, cache_size_in_bytes, max_cache_size_in_bytes, max_cache_entries, max_cache_entry_size_in_bytes, max_cache_entry_size_in_rows, min_query_runtime);
}
-void QueryResultCache::reset()
+void QueryCache::reset()
{
std::lock_guard lock(mutex);
cache.clear();
@@ -328,7 +328,7 @@ void QueryResultCache::reset()
cache_size_in_bytes = 0;
}
-size_t QueryResultCache::recordQueryRun(const Key & key)
+size_t QueryCache::recordQueryRun(const Key & key)
{
static constexpr size_t TIMES_EXECUTED_MAX_SIZE = 10'000;
diff --git a/src/Interpreters/Cache/QueryResultCache.h b/src/Interpreters/Cache/QueryCache.h
similarity index 89%
rename from src/Interpreters/Cache/QueryResultCache.h
rename to src/Interpreters/Cache/QueryCache.h
index 65cab854a45..45f48c7a558 100644
--- a/src/Interpreters/Cache/QueryResultCache.h
+++ b/src/Interpreters/Cache/QueryCache.h
@@ -18,7 +18,7 @@ bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context);
/// returned. In order to still obtain sufficiently up-to-date query results, a expiry time (TTL) must be specified for each cache entry
/// after which it becomes stale and is ignored. Stale entries are removed opportunistically from the cache, they are only evicted when a
/// new entry is inserted and the cache has insufficient capacity.
-class QueryResultCache
+class QueryCache
{
public:
/// Represents a query result in the cache.
@@ -82,9 +82,9 @@ public:
/// Buffers multiple partial query result chunks (buffer()) and eventually stores them as cache entry (finalizeWrite()).
///
/// Implementation note: Queries may throw exceptions during runtime, e.g. out-of-memory errors. In this case, no query result must be
- /// written into the query result cache. Unfortunately, neither the Writer nor the special transform added on top of the query pipeline
- /// which holds the Writer know whether they are destroyed because the query ended successfully or because of an exception (otherwise,
- /// we could simply implement a check in their destructors). To handle exceptions correctly nevertheless, we do the actual insert in
+ /// written into the query cache. Unfortunately, neither the Writer nor the special transform added on top of the query pipeline which
+ /// holds the Writer know whether they are destroyed because the query ended successfully or because of an exception (otherwise, we
+ /// could simply implement a check in their destructors). To handle exceptions correctly nevertheless, we do the actual insert in
/// finalizeWrite() as opposed to the Writer destructor. This function is then called only for successful queries in finish_callback()
/// which runs before the transform and the Writer are destroyed, whereas for unsuccessful queries we do nothing (the Writer is
/// destroyed w/o inserting anything).
@@ -117,7 +117,7 @@ public:
size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_,
std::chrono::milliseconds min_query_runtime_);
- friend class QueryResultCache; /// for createWriter()
+ friend class QueryCache; /// for createWriter()
};
/// Looks up a query result for a key in the cache and (if found) constructs a pipe with the query result chunks as source.
@@ -129,10 +129,10 @@ public:
private:
Reader(const Cache & cache_, const Key & key, size_t & cache_size_in_bytes_, const std::lock_guard &);
Pipe pipe;
- friend class QueryResultCache; /// for createReader()
+ friend class QueryCache; /// for createReader()
};
- QueryResultCache(size_t max_cache_size_in_bytes_, size_t max_cache_entries_, size_t max_cache_entry_size_in_bytes_, size_t max_cache_entry_size_in_rows_);
+ QueryCache(size_t max_cache_size_in_bytes_, size_t max_cache_entries_, size_t max_cache_entry_size_in_bytes_, size_t max_cache_entry_size_in_rows_);
Reader createReader(const Key & key);
Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime);
@@ -160,9 +160,9 @@ private:
const size_t max_cache_entry_size_in_bytes;
const size_t max_cache_entry_size_in_rows;
- friend class StorageSystemQueryResultCache;
+ friend class StorageSystemQueryCache;
};
-using QueryResultCachePtr = std::shared_ptr;
+using QueryCachePtr = std::shared_ptr;
}
diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp
index 2a8734596c5..d9a7aa2e677 100644
--- a/src/Interpreters/Context.cpp
+++ b/src/Interpreters/Context.cpp
@@ -40,7 +40,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
@@ -236,7 +236,7 @@ struct ContextSharedPart : boost::noncopyable
mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache.
mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices.
mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices.
- mutable QueryResultCachePtr query_result_cache; /// Cache of query results.
+ mutable QueryCachePtr query_cache; /// Cache of query results.
mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
ProcessList process_list; /// Executing queries at the moment.
GlobalOvercommitTracker global_overcommit_tracker;
@@ -2041,27 +2041,27 @@ void Context::dropIndexMarkCache() const
shared->index_mark_cache->reset();
}
-void Context::setQueryResultCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_records)
+void Context::setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_records)
{
auto lock = getLock();
- if (shared->query_result_cache)
- throw Exception(ErrorCodes::LOGICAL_ERROR, "Query result cache has been already created.");
+ if (shared->query_cache)
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Query cache has been already created.");
- shared->query_result_cache = std::make_shared(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_records);
+ shared->query_cache = std::make_shared(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_records);
}
-QueryResultCachePtr Context::getQueryResultCache() const
+QueryCachePtr Context::getQueryCache() const
{
auto lock = getLock();
- return shared->query_result_cache;
+ return shared->query_cache;
}
-void Context::dropQueryResultCache() const
+void Context::dropQueryCache() const
{
auto lock = getLock();
- if (shared->query_result_cache)
- shared->query_result_cache->reset();
+ if (shared->query_cache)
+ shared->query_cache->reset();
}
void Context::setMMappedFileCache(size_t cache_size_in_num_entries)
@@ -2104,8 +2104,8 @@ void Context::dropCaches() const
if (shared->index_mark_cache)
shared->index_mark_cache->reset();
- if (shared->query_result_cache)
- shared->query_result_cache->reset();
+ if (shared->query_cache)
+ shared->query_cache->reset();
if (shared->mmap_cache)
shared->mmap_cache->reset();
diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h
index 10983b15d7f..00dc4204496 100644
--- a/src/Interpreters/Context.h
+++ b/src/Interpreters/Context.h
@@ -81,8 +81,8 @@ class Macros;
struct Progress;
struct FileProgress;
class Clusters;
+class QueryCache;
class QueryLog;
-class QueryResultCache;
class QueryThreadLog;
class QueryViewsLog;
class PartLog;
@@ -861,9 +861,9 @@ public:
void dropMMappedFileCache() const;
/// Create a cache of query results for statements which run repeatedly.
- void setQueryResultCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_records);
- std::shared_ptr getQueryResultCache() const;
- void dropQueryResultCache() const;
+ void setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_records);
+ std::shared_ptr getQueryCache() const;
+ void dropQueryCache() const;
/** Clear the caches of the uncompressed blocks and marks.
* This is usually done when renaming tables, changing the type of columns, deleting a table.
diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp
index 19b31a858f4..abd0ecd6ea1 100644
--- a/src/Interpreters/InterpreterSystemQuery.cpp
+++ b/src/Interpreters/InterpreterSystemQuery.cpp
@@ -327,9 +327,9 @@ BlockIO InterpreterSystemQuery::execute()
getContext()->checkAccess(AccessType::SYSTEM_DROP_MMAP_CACHE);
system_context->dropMMappedFileCache();
break;
- case Type::DROP_QUERY_RESULT_CACHE:
- getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_RESULT_CACHE);
- getContext()->dropQueryResultCache();
+ case Type::DROP_QUERY_CACHE:
+ getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE);
+ getContext()->dropQueryCache();
break;
#if USE_EMBEDDED_COMPILER
case Type::DROP_COMPILED_EXPRESSION_CACHE:
@@ -969,7 +969,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
case Type::DROP_DNS_CACHE:
case Type::DROP_MARK_CACHE:
case Type::DROP_MMAP_CACHE:
- case Type::DROP_QUERY_RESULT_CACHE:
+ case Type::DROP_QUERY_CACHE:
#if USE_EMBEDDED_COMPILER
case Type::DROP_COMPILED_EXPRESSION_CACHE:
#endif
diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp
index 646f1e89fc3..f46adf91ee0 100644
--- a/src/Interpreters/executeQuery.cpp
+++ b/src/Interpreters/executeQuery.cpp
@@ -14,7 +14,7 @@
#include
#include
#include
-#include
+#include
#include
#include
@@ -716,48 +716,48 @@ static std::tuple executeQueryImpl(
/// If
/// - it is a SELECT query,
- /// - passive (read) use of the query result cache is enabled, and
- /// - the query result cache knows the query result
- /// then replace the pipeline by a new pipeline with a single source that is populated from the query result cache
- auto query_result_cache = context->getQueryResultCache();
- bool read_result_from_query_result_cache = false; /// a query must not read from *and* write to the query result cache at the same time
- if (query_result_cache != nullptr
- && (settings.allow_experimental_query_result_cache && settings.use_query_result_cache && settings.enable_reads_from_query_result_cache)
+ /// - passive (read) use of the query cache is enabled, and
+ /// - the query cache knows the query result
+ /// then replace the pipeline by a new pipeline with a single source that is populated from the query cache
+ auto query_cache = context->getQueryCache();
+ bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time
+ if (query_cache != nullptr
+ && (settings.allow_experimental_query_cache && settings.use_query_cache && settings.enable_reads_from_query_cache)
&& res.pipeline.pulling())
{
- QueryResultCache::Key key(
+ QueryCache::Key key(
ast, res.pipeline.getHeader(),
std::make_optional(context->getUserName()),
- std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_ttl));
- QueryResultCache::Reader reader = query_result_cache->createReader(key);
+ std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl));
+ QueryCache::Reader reader = query_cache->createReader(key);
if (reader.hasCacheEntryForKey())
{
res.pipeline = QueryPipeline(reader.getPipe());
- read_result_from_query_result_cache = true;
+ read_result_from_query_cache = true;
}
}
/// If
/// - it is a SELECT query, and
- /// - active (write) use of the query result cache is enabled
- /// then add a processor on top of the pipeline which stores the result in the query result cache.
- if (!read_result_from_query_result_cache
- && query_result_cache != nullptr
- && settings.allow_experimental_query_result_cache && settings.use_query_result_cache && settings.enable_writes_to_query_result_cache
+ /// - active (write) use of the query cache is enabled
+ /// then add a processor on top of the pipeline which stores the result in the query cache.
+ if (!read_result_from_query_cache
+ && query_cache != nullptr
+ && settings.allow_experimental_query_cache && settings.use_query_cache && settings.enable_writes_to_query_cache
&& res.pipeline.pulling()
- && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_result_cache_store_results_of_queries_with_nondeterministic_functions))
+ && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
{
- QueryResultCache::Key key(
+ QueryCache::Key key(
ast, res.pipeline.getHeader(),
- settings.query_result_cache_share_between_users ? std::nullopt : std::make_optional(context->getUserName()),
- std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_ttl));
+ settings.query_cache_share_between_users ? std::nullopt : std::make_optional(context->getUserName()),
+ std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl));
- const size_t num_query_runs = query_result_cache->recordQueryRun(key);
- if (num_query_runs > settings.query_result_cache_min_query_runs)
+ const size_t num_query_runs = query_cache->recordQueryRun(key);
+ if (num_query_runs > settings.query_cache_min_query_runs)
{
- auto stream_in_query_result_cache_transform = std::make_shared(res.pipeline.getHeader(), query_result_cache, key,
- std::chrono::milliseconds(context->getSettings().query_result_cache_min_query_duration.totalMilliseconds()));
- res.pipeline.streamIntoQueryResultCache(stream_in_query_result_cache_transform);
+ auto stream_in_query_cache_transform = std::make_shared(res.pipeline.getHeader(), query_cache, key,
+ std::chrono::milliseconds(context->getSettings().query_cache_min_query_duration.totalMilliseconds()));
+ res.pipeline.streamIntoQueryCache(stream_in_query_cache_transform);
}
}
@@ -908,10 +908,10 @@ static std::tuple executeQueryImpl(
auto finish_callback = [elem,
context,
ast,
- allow_experimental_query_result_cache = settings.allow_experimental_query_result_cache,
- use_query_result_cache = settings.use_query_result_cache,
- enable_writes_to_query_result_cache = settings.enable_writes_to_query_result_cache,
- query_result_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_result_cache_store_results_of_queries_with_nondeterministic_functions,
+ allow_experimental_query_cache = settings.allow_experimental_query_cache,
+ use_query_cache = settings.use_query_cache,
+ enable_writes_to_query_cache = settings.enable_writes_to_query_cache,
+ query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions,
log_queries,
log_queries_min_type = settings.log_queries_min_type,
log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(),
@@ -921,15 +921,15 @@ static std::tuple executeQueryImpl(
pulling_pipeline = pipeline.pulling(),
query_span](QueryPipeline & query_pipeline) mutable
{
- /// If active (write) use of the query result cache is enabled and the query is eligible for result caching, then store the
- /// query result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache.
- auto query_result_cache = context->getQueryResultCache();
- if (query_result_cache != nullptr
+ /// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query
+ /// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache.
+ auto query_cache = context->getQueryCache();
+ if (query_cache != nullptr
&& pulling_pipeline
- && allow_experimental_query_result_cache && use_query_result_cache && enable_writes_to_query_result_cache
- && (!astContainsNonDeterministicFunctions(ast, context) || query_result_cache_store_results_of_queries_with_nondeterministic_functions))
+ && allow_experimental_query_cache && use_query_cache && enable_writes_to_query_cache
+ && (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions))
{
- query_pipeline.finalizeWriteInQueryResultCache();
+ query_pipeline.finalizeWriteInQueryCache();
}
QueryStatusPtr process_list_elem = context->getProcessListElement();
diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h
index 4290f4e2a94..02ddbc7dcd2 100644
--- a/src/Parsers/ASTSystemQuery.h
+++ b/src/Parsers/ASTSystemQuery.h
@@ -25,7 +25,7 @@ public:
DROP_INDEX_MARK_CACHE,
DROP_INDEX_UNCOMPRESSED_CACHE,
DROP_MMAP_CACHE,
- DROP_QUERY_RESULT_CACHE,
+ DROP_QUERY_CACHE,
#if USE_EMBEDDED_COMPILER
DROP_COMPILED_EXPRESSION_CACHE,
#endif
diff --git a/src/Processors/Transforms/StreamInQueryCacheTransform.cpp b/src/Processors/Transforms/StreamInQueryCacheTransform.cpp
new file mode 100644
index 00000000000..1ba57ea8ed2
--- /dev/null
+++ b/src/Processors/Transforms/StreamInQueryCacheTransform.cpp
@@ -0,0 +1,24 @@
+#include
+
+namespace DB
+{
+
+StreamInQueryCacheTransform::StreamInQueryCacheTransform(
+ const Block & header_, QueryCachePtr cache, const QueryCache::Key & cache_key, std::chrono::milliseconds min_query_duration)
+ : ISimpleTransform(header_, header_, false)
+ , cache_writer(cache->createWriter(cache_key, min_query_duration))
+{
+}
+
+void StreamInQueryCacheTransform::transform(Chunk & chunk)
+{
+ cache_writer.buffer(chunk.clone());
+}
+
+void StreamInQueryCacheTransform::finalizeWriteInQueryCache()
+{
+ if (!isCancelled())
+ cache_writer.finalizeWrite();
+}
+
+};
diff --git a/src/Processors/Transforms/StreamInQueryCacheTransform.h b/src/Processors/Transforms/StreamInQueryCacheTransform.h
new file mode 100644
index 00000000000..15d977cd445
--- /dev/null
+++ b/src/Processors/Transforms/StreamInQueryCacheTransform.h
@@ -0,0 +1,26 @@
+#pragma once
+
+#include
+#include
+
+namespace DB
+{
+
+class StreamInQueryCacheTransform : public ISimpleTransform
+{
+public:
+ StreamInQueryCacheTransform(
+ const Block & header_, QueryCachePtr cache, const QueryCache::Key & cache_key, std::chrono::milliseconds min_query_duration);
+
+protected:
+ void transform(Chunk & chunk) override;
+
+public:
+ void finalizeWriteInQueryCache();
+ String getName() const override { return "StreamInQueryCacheTransform"; }
+
+private:
+ QueryCache::Writer cache_writer;
+};
+
+}
diff --git a/src/Processors/Transforms/StreamInQueryResultCacheTransform.cpp b/src/Processors/Transforms/StreamInQueryResultCacheTransform.cpp
deleted file mode 100644
index 841fcfdf8b5..00000000000
--- a/src/Processors/Transforms/StreamInQueryResultCacheTransform.cpp
+++ /dev/null
@@ -1,24 +0,0 @@
-#include
-
-namespace DB
-{
-
-StreamInQueryResultCacheTransform::StreamInQueryResultCacheTransform(
- const Block & header_, QueryResultCachePtr cache, const QueryResultCache::Key & cache_key, std::chrono::milliseconds min_query_duration)
- : ISimpleTransform(header_, header_, false)
- , cache_writer(cache->createWriter(cache_key, min_query_duration))
-{
-}
-
-void StreamInQueryResultCacheTransform::transform(Chunk & chunk)
-{
- cache_writer.buffer(chunk.clone());
-}
-
-void StreamInQueryResultCacheTransform::finalizeWriteInQueryResultCache()
-{
- if (!isCancelled())
- cache_writer.finalizeWrite();
-}
-
-};
diff --git a/src/Processors/Transforms/StreamInQueryResultCacheTransform.h b/src/Processors/Transforms/StreamInQueryResultCacheTransform.h
deleted file mode 100644
index a90d33a0681..00000000000
--- a/src/Processors/Transforms/StreamInQueryResultCacheTransform.h
+++ /dev/null
@@ -1,26 +0,0 @@
-#pragma once
-
-#include
-#include
-
-namespace DB
-{
-
-class StreamInQueryResultCacheTransform : public ISimpleTransform
-{
-public:
- StreamInQueryResultCacheTransform(
- const Block & header_, QueryResultCachePtr cache, const QueryResultCache::Key & cache_key, std::chrono::milliseconds min_query_duration);
-
-protected:
- void transform(Chunk & chunk) override;
-
-public:
- void finalizeWriteInQueryResultCache();
- String getName() const override { return "StreamInQueryResultCacheTransform"; }
-
-private:
- QueryResultCache::Writer cache_writer;
-};
-
-}
diff --git a/src/QueryPipeline/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp
index aa01801b1ec..b7b18014f1f 100644
--- a/src/QueryPipeline/QueryPipeline.cpp
+++ b/src/QueryPipeline/QueryPipeline.cpp
@@ -18,7 +18,7 @@
#include
#include
#include
-#include
+#include
#include
#include
@@ -525,7 +525,7 @@ bool QueryPipeline::tryGetResultRowsAndBytes(UInt64 & result_rows, UInt64 & resu
return true;
}
-void QueryPipeline::streamIntoQueryResultCache(std::shared_ptr transform)
+void QueryPipeline::streamIntoQueryCache(std::shared_ptr transform)
{
assert(pulling());
@@ -534,16 +534,16 @@ void QueryPipeline::streamIntoQueryResultCache(std::shared_ptremplace_back(transform);
}
-void QueryPipeline::finalizeWriteInQueryResultCache()
+void QueryPipeline::finalizeWriteInQueryCache()
{
auto it = std::find_if(
processors->begin(), processors->end(),
- [](ProcessorPtr processor){ return dynamic_cast(&*processor); });
+ [](ProcessorPtr processor){ return dynamic_cast(&*processor); });
- /// the pipeline should theoretically contain just one StreamInQueryResultCacheTransform
+ /// the pipeline should theoretically contain just one StreamInQueryCacheTransform
if (it != processors->end())
- dynamic_cast(**it).finalizeWriteInQueryResultCache();
+ dynamic_cast(**it).finalizeWriteInQueryCache();
}
void QueryPipeline::addStorageHolder(StoragePtr storage)
diff --git a/src/QueryPipeline/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h
index da43aa035f3..55c78ca78ed 100644
--- a/src/QueryPipeline/QueryPipeline.h
+++ b/src/QueryPipeline/QueryPipeline.h
@@ -31,7 +31,7 @@ class SinkToStorage;
class ISource;
class ISink;
class ReadProgressCallback;
-class StreamInQueryResultCacheTransform;
+class StreamInQueryCacheTransform;
struct ColumnWithTypeAndName;
using ColumnsWithTypeAndName = std::vector;
@@ -105,8 +105,8 @@ public:
void setLimitsAndQuota(const StreamLocalLimits & limits, std::shared_ptr quota_);
bool tryGetResultRowsAndBytes(UInt64 & result_rows, UInt64 & result_bytes) const;
- void streamIntoQueryResultCache(std::shared_ptr transform);
- void finalizeWriteInQueryResultCache();
+ void streamIntoQueryCache(std::shared_ptr transform);
+ void finalizeWriteInQueryCache();
void setQuota(std::shared_ptr quota_);
diff --git a/src/Storages/System/StorageSystemQueryResultCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp
similarity index 68%
rename from src/Storages/System/StorageSystemQueryResultCache.cpp
rename to src/Storages/System/StorageSystemQueryCache.cpp
index cb6349b6d47..2de8e4594b9 100644
--- a/src/Storages/System/StorageSystemQueryResultCache.cpp
+++ b/src/Storages/System/StorageSystemQueryCache.cpp
@@ -1,15 +1,15 @@
-#include "StorageSystemQueryResultCache.h"
+#include "StorageSystemQueryCache.h"
#include
#include
#include
-#include
+#include
#include
namespace DB
{
-NamesAndTypesList StorageSystemQueryResultCache::getNamesAndTypes()
+NamesAndTypesList StorageSystemQueryCache::getNamesAndTypes()
{
return {
{"query", std::make_shared()},
@@ -21,23 +21,23 @@ NamesAndTypesList StorageSystemQueryResultCache::getNamesAndTypes()
};
}
-StorageSystemQueryResultCache::StorageSystemQueryResultCache(const StorageID & table_id_)
+StorageSystemQueryCache::StorageSystemQueryCache(const StorageID & table_id_)
: IStorageSystemOneBlock(table_id_)
{
}
-void StorageSystemQueryResultCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
+void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
- auto query_result_cache = context->getQueryResultCache();
+ auto query_cache = context->getQueryCache();
- if (!query_result_cache)
+ if (!query_cache)
return;
const String & username = context->getUserName();
- std::lock_guard lock(query_result_cache->mutex);
+ std::lock_guard lock(query_cache->mutex);
- for (const auto & [key, result] : query_result_cache->cache)
+ for (const auto & [key, result] : query_cache->cache)
{
/// Showing other user's queries is considered a security risk
if (key.username.has_value() && key.username != username)
diff --git a/src/Storages/System/StorageSystemQueryResultCache.h b/src/Storages/System/StorageSystemQueryCache.h
similarity index 52%
rename from src/Storages/System/StorageSystemQueryResultCache.h
rename to src/Storages/System/StorageSystemQueryCache.h
index 4862878a31a..5ff5f0a0454 100644
--- a/src/Storages/System/StorageSystemQueryResultCache.h
+++ b/src/Storages/System/StorageSystemQueryCache.h
@@ -5,12 +5,12 @@
namespace DB
{
-class StorageSystemQueryResultCache final : public IStorageSystemOneBlock
+class StorageSystemQueryCache final : public IStorageSystemOneBlock
{
public:
- explicit StorageSystemQueryResultCache(const StorageID & table_id_);
+ explicit StorageSystemQueryCache(const StorageID & table_id_);
- std::string getName() const override { return "SystemQueryResultCache"; }
+ std::string getName() const override { return "SystemQueryCache"; }
static NamesAndTypesList getNamesAndTypes();
diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp
index eeb08d7e3d4..07db151069f 100644
--- a/src/Storages/System/attachSystemTables.cpp
+++ b/src/Storages/System/attachSystemTables.cpp
@@ -73,7 +73,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
@@ -176,7 +176,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach(context, system_database, "part_moves_between_shards");
attach(context, system_database, "asynchronous_inserts");
attach(context, system_database, "filesystem_cache");
- attach(context, system_database, "query_result_cache");
+ attach(context, system_database, "query_cache");
attach(context, system_database, "remote_data_paths");
attach(context, system_database, "certificates");
attach(context, system_database, "named_collections");
diff --git a/tests/queries/0_stateless/02494_query_result_cache_case_agnostic_matching.reference b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_case_agnostic_matching.reference
rename to tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql
new file mode 100644
index 00000000000..9440a1fd9c0
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_case_agnostic_matching.sql
@@ -0,0 +1,29 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+-- Start with empty query cache (QC) and query log
+SYSTEM DROP QUERY CACHE;
+DROP TABLE system.query_log SYNC;
+
+-- Insert an entry into the query cache.
+SELECT 1 SETTINGS use_query_cache = true;
+-- Check that entry in QC exists
+SELECT COUNT(*) FROM system.query_cache;
+
+-- Run the same SELECT but with different case (--> select). We want its result to be served from the QC.
+SELECT '---';
+select 1 SETTINGS use_query_cache = true;
+
+-- There should still be just one entry in the QC
+SELECT COUNT(*) FROM system.query_cache;
+
+-- The second query should cause a QC hit.
+SYSTEM FLUSH LOGS;
+SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses']
+FROM system.query_log
+WHERE type = 'QueryFinish'
+ AND query = 'select 1 SETTINGS use_query_cache = true;';
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_drop_cache.reference b/tests/queries/0_stateless/02494_query_cache_drop_cache.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_drop_cache.reference
rename to tests/queries/0_stateless/02494_query_cache_drop_cache.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql
new file mode 100644
index 00000000000..1f61472fcb0
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_drop_cache.sql
@@ -0,0 +1,12 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+-- Cache query result in query cache
+SELECT 1 SETTINGS use_query_cache = true;
+SELECT count(*) FROM system.query_cache;
+
+-- No query results are cached after DROP
+SYSTEM DROP QUERY CACHE;
+SELECT count(*) FROM system.query_cache;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_eligible_queries.reference b/tests/queries/0_stateless/02494_query_cache_eligible_queries.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_eligible_queries.reference
rename to tests/queries/0_stateless/02494_query_cache_eligible_queries.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql
new file mode 100644
index 00000000000..b4bc9e2c258
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_eligible_queries.sql
@@ -0,0 +1,68 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+DROP TABLE IF EXISTS eligible_test;
+DROP TABLE IF EXISTS eligible_test2;
+
+-- enable query cache session-wide but also force it individually in each of below statements
+SET use_query_cache = true;
+
+-- check that SELECT statements create entries in the query cache ...
+SELECT 1 SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
+
+-- ... and all other statements also should not create entries:
+
+-- CREATE
+CREATE TABLE eligible_test (a String) ENGINE=MergeTree ORDER BY a; -- SETTINGS use_query_cache = true; -- SETTINGS rejected as unknown
+SELECT COUNT(*) FROM system.query_cache;
+
+-- ALTER
+ALTER TABLE eligible_test ADD COLUMN b String SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- INSERT
+INSERT INTO eligible_test VALUES('a', 'b'); -- SETTINGS use_query_cache = true; -- SETTINGS rejected as unknown
+SELECT COUNT(*) FROM system.query_cache;
+INSERT INTO eligible_test SELECT * FROM eligible_test SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- SHOW
+SHOW TABLES SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- CHECK
+CHECK TABLE eligible_test SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- DESCRIBE
+DESCRIBE TABLE eligible_test SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- EXISTS
+EXISTS TABLE eligible_test SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- KILL
+KILL QUERY WHERE query_id='3-857d-4a57-9ee0-3c7da5d60a90' SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- OPTIMIZE
+OPTIMIZE TABLE eligible_test FINAL SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- TRUNCATE
+TRUNCATE TABLE eligible_test SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+-- RENAME
+RENAME TABLE eligible_test TO eligible_test2 SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
+DROP TABLE eligible_test2;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_events.reference b/tests/queries/0_stateless/02494_query_cache_events.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_events.reference
rename to tests/queries/0_stateless/02494_query_cache_events.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_events.sql b/tests/queries/0_stateless/02494_query_cache_events.sql
new file mode 100644
index 00000000000..d775467d525
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_events.sql
@@ -0,0 +1,32 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+-- Start with empty query cache QC and query log
+SYSTEM DROP QUERY CACHE;
+DROP TABLE system.query_log SYNC;
+
+-- Run a query with QC on. The first execution is a QC miss.
+SELECT '---';
+SELECT 1 SETTINGS use_query_cache = true;
+
+SYSTEM FLUSH LOGS;
+SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses']
+FROM system.query_log
+WHERE type = 'QueryFinish'
+ AND query = 'SELECT 1 SETTINGS use_query_cache = true;';
+
+
+-- Run previous query again with query cache on
+SELECT '---';
+SELECT 1 SETTINGS use_query_cache = true;
+
+DROP TABLE system.query_log SYNC;
+SYSTEM FLUSH LOGS;
+SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses']
+FROM system.query_log
+WHERE type = 'QueryFinish'
+ AND query = 'SELECT 1 SETTINGS use_query_cache = true;';
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_exception_handling.reference b/tests/queries/0_stateless/02494_query_cache_exception_handling.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_exception_handling.reference
rename to tests/queries/0_stateless/02494_query_cache_exception_handling.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_exception_handling.sql b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql
new file mode 100644
index 00000000000..4d686d81ed3
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_exception_handling.sql
@@ -0,0 +1,12 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- If an exception is thrown during query execution, no entry must be created in the query cache
+SELECT throwIf(1) SETTINGS use_query_cache = true; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_explain.reference b/tests/queries/0_stateless/02494_query_cache_explain.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_explain.reference
rename to tests/queries/0_stateless/02494_query_cache_explain.reference
diff --git a/tests/queries/0_stateless/02494_query_result_cache_explain.sql b/tests/queries/0_stateless/02494_query_cache_explain.sql
similarity index 54%
rename from tests/queries/0_stateless/02494_query_result_cache_explain.sql
rename to tests/queries/0_stateless/02494_query_cache_explain.sql
index 0daed9df151..67717efde13 100644
--- a/tests/queries/0_stateless/02494_query_result_cache_explain.sql
+++ b/tests/queries/0_stateless/02494_query_cache_explain.sql
@@ -1,23 +1,23 @@
-- Tags: no-parallel
-- Tag no-parallel: Messes with internal cache
-SET allow_experimental_query_result_cache = true;
+SET allow_experimental_query_cache = true;
-SYSTEM DROP QUERY RESULT CACHE;
+SYSTEM DROP QUERY CACHE;
--- Run a silly query with a non-trivial plan and put the result into the query result cache (QRC)
-SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_result_cache = true;
-SELECT count(*) FROM system.query_result_cache;
+-- Run a silly query with a non-trivial plan and put the result into the query cache QC
+SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_cache = true;
+SELECT count(*) FROM system.query_cache;
--- EXPLAIN PLAN should show the same regardless if the result is calculated or read from the QRC
+-- EXPLAIN PLAN should show the same regardless if the result is calculated or read from the QC
EXPLAIN PLAN SELECT 1 + number from system.numbers LIMIT 1;
-EXPLAIN PLAN SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_result_cache = true; -- (*)
+EXPLAIN PLAN SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_cache = true; -- (*)
--- EXPLAIN PIPELINE should show the same regardless if the result is calculated or read from the QRC
+-- EXPLAIN PIPELINE should show the same regardless if the result is calculated or read from the QC
EXPLAIN PIPELINE SELECT 1 + number from system.numbers LIMIT 1;
-EXPLAIN PIPELINE SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_result_cache = true; -- (*)
+EXPLAIN PIPELINE SELECT 1 + number from system.numbers LIMIT 1 SETTINGS use_query_cache = true; -- (*)
--- Statements (*) must not cache their results into the QRC
-SELECT count(*) FROM system.query_result_cache;
+-- Statements (*) must not cache their results into the QC
+SELECT count(*) FROM system.query_cache;
-SYSTEM DROP QUERY RESULT CACHE;
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_min_query_duration.reference b/tests/queries/0_stateless/02494_query_cache_min_query_duration.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_min_query_duration.reference
rename to tests/queries/0_stateless/02494_query_cache_min_query_duration.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql
new file mode 100644
index 00000000000..7d759c86130
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_min_query_duration.sql
@@ -0,0 +1,20 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- This creates an entry in the query cache ...
+SELECT 1 SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
+
+SELECT '---';
+
+-- ... but this does not because the query executes much faster than the specified minumum query duration for caching the result
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_duration = 10000;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_min_query_runs.reference b/tests/queries/0_stateless/02494_query_cache_min_query_runs.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_min_query_runs.reference
rename to tests/queries/0_stateless/02494_query_cache_min_query_runs.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql
new file mode 100644
index 00000000000..2401beeab13
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_min_query_runs.sql
@@ -0,0 +1,34 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- Cache the query after the 1st query invocation
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 0;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '---';
+
+SYSTEM DROP QUERY CACHE;
+
+-- Cache the query result after the 2nd query invocation
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 1;
+SELECT COUNT(*) FROM system.query_cache;
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 1;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '---';
+
+SYSTEM DROP QUERY CACHE;
+
+-- Cache the query result after the 3rd query invocation
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 2;
+SELECT COUNT(*) FROM system.query_cache;
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 2;
+SELECT COUNT(*) FROM system.query_cache;
+SELECT 1 SETTINGS use_query_cache = true, query_cache_min_query_runs = 2;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_nondeterministic_functions.reference b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_nondeterministic_functions.reference
rename to tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql
new file mode 100644
index 00000000000..534d63aa427
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_nondeterministic_functions.sql
@@ -0,0 +1,18 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- rand() is non-deterministic, with default settings no entry in the query cache should be created
+SELECT COUNT(rand(1)) SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '---';
+
+-- But an entry can be forced using a setting
+SELECT COUNT(RAND(1)) SETTINGS use_query_cache = true, query_cache_store_results_of_queries_with_nondeterministic_functions = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_normalize_ast.reference b/tests/queries/0_stateless/02494_query_cache_normalize_ast.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_normalize_ast.reference
rename to tests/queries/0_stateless/02494_query_cache_normalize_ast.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql
new file mode 100644
index 00000000000..7e3cd273312
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_normalize_ast.sql
@@ -0,0 +1,31 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+-- Start with empty query cache (QC) and query log.
+SYSTEM DROP QUERY CACHE;
+DROP TABLE system.query_log SYNC;
+
+-- Run query whose result gets cached in the query cache.
+-- Besides "use_query_cache", pass two more knobs (one QC-specific knob and one non-QC-specific knob). We just care
+-- *that* they are passed and not about their effect.
+SELECT 1 SETTINGS use_query_cache = true, query_cache_store_results_of_queries_with_nondeterministic_functions = true, max_threads = 16;
+
+-- Check that entry in QC exists
+SELECT COUNT(*) FROM system.query_cache;
+
+-- Run the same SELECT but with different SETTINGS. We want its result to be served from the QC (--> passive mode, achieve it by
+-- disabling active mode)
+SELECT '---';
+SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false, max_threads = 16;
+
+-- Technically, both SELECT queries have different ASTs, leading to different QC keys. QC does some AST normalization (erase all
+-- QC-related settings) such that the keys match regardless. Verify by checking that the second query caused a QC hit.
+SYSTEM FLUSH LOGS;
+SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses']
+FROM system.query_log
+WHERE type = 'QueryFinish'
+ AND query = 'SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false, max_threads = 16;';
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_passive_usage.reference b/tests/queries/0_stateless/02494_query_cache_passive_usage.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_passive_usage.reference
rename to tests/queries/0_stateless/02494_query_cache_passive_usage.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql
new file mode 100644
index 00000000000..8f1e3972b6d
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_passive_usage.sql
@@ -0,0 +1,41 @@
+-- Tags: no-parallel
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+-- Start with empty query cache (QC).
+SYSTEM DROP QUERY CACHE;
+
+-- By default, don't write query result into QC.
+SELECT 1;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '-----';
+
+-- Try to retrieve query from empty QC using the passive mode. Do this by disabling the active mode. The cache should still be empty (no insert).
+SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '-----';
+
+-- Put query into cache.
+SELECT 1 SETTINGS use_query_cache = true;
+SELECT COUNT(*) FROM system.query_cache;
+
+SELECT '-----';
+
+-- Run same query with passive mode again. There must still be one entry in the QC and we must have a QC hit.
+
+-- Get rid of log of previous SELECT
+DROP TABLE system.query_log SYNC;
+
+SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;
+SELECT COUNT(*) FROM system.query_cache;
+
+SYSTEM FLUSH LOGS;
+SELECT ProfileEvents['QueryCacheHits'], ProfileEvents['QueryCacheMisses']
+FROM system.query_log
+WHERE type = 'QueryFinish'
+ AND query = 'SELECT 1 SETTINGS use_query_cache = true, enable_writes_to_query_cache = false;';
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_secrets.reference b/tests/queries/0_stateless/02494_query_cache_secrets.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_secrets.reference
rename to tests/queries/0_stateless/02494_query_cache_secrets.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_secrets.sql b/tests/queries/0_stateless/02494_query_cache_secrets.sql
new file mode 100644
index 00000000000..99a972b003c
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_secrets.sql
@@ -0,0 +1,15 @@
+-- Tags: no-parallel, no-fasttest
+-- Tag no-fasttest: Depends on OpenSSL
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- Cache a result of a query with secret in the query cache
+SELECT hex(encrypt('aes-128-ecb', 'plaintext', 'passwordpassword')) SETTINGS use_query_cache = true;
+
+-- The secret should not be revealed in system.query_cache
+SELECT query FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_ttl_long.reference b/tests/queries/0_stateless/02494_query_cache_ttl_long.reference
similarity index 100%
rename from tests/queries/0_stateless/02494_query_result_cache_ttl_long.reference
rename to tests/queries/0_stateless/02494_query_cache_ttl_long.reference
diff --git a/tests/queries/0_stateless/02494_query_cache_ttl_long.sql b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql
new file mode 100644
index 00000000000..135ddf2195c
--- /dev/null
+++ b/tests/queries/0_stateless/02494_query_cache_ttl_long.sql
@@ -0,0 +1,31 @@
+-- Tags: no-fasttest, no-parallel, long
+-- Tag no-fasttest: Test runtime is > 6 sec
+-- Tag long: Test runtime is > 6 sec
+-- Tag no-parallel: Messes with internal cache
+
+SET allow_experimental_query_cache = true;
+
+SYSTEM DROP QUERY CACHE;
+
+-- Cache query result into query cache with a TTL of 3 sec
+SELECT 1 SETTINGS use_query_cache = true, query_cache_ttl = 3;
+
+-- Expect one non-stale cache entry
+SELECT COUNT(*) FROM system.query_cache;
+SELECT stale FROM system.query_cache;
+
+-- Wait until entry is expired
+SELECT sleep(3);
+SELECT sleep(3);
+SELECT stale FROM system.query_cache;
+
+SELECT '---';
+
+-- Run same query as before
+SELECT 1 SETTINGS use_query_cache = true, query_cache_ttl = 3;
+
+-- The entry should have been refreshed (non-stale)
+SELECT COUNT(*) FROM system.query_cache;
+SELECT stale FROM system.query_cache;
+
+SYSTEM DROP QUERY CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_case_agnostic_matching.sql b/tests/queries/0_stateless/02494_query_result_cache_case_agnostic_matching.sql
deleted file mode 100644
index e37c0a9cb3f..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_case_agnostic_matching.sql
+++ /dev/null
@@ -1,29 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
--- Start with empty query result cache (QRC) and query log
-SYSTEM DROP QUERY RESULT CACHE;
-DROP TABLE system.query_log SYNC;
-
--- Insert an entry into the query result cache.
-SELECT 1 SETTINGS use_query_result_cache = true;
--- Check that entry in QRC exists
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- Run the same SELECT but with different case (--> select). We want its result to be served from the QRC.
-SELECT '---';
-select 1 SETTINGS use_query_result_cache = true;
-
--- There should still be just one entry in the QRC
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- The second query should cause a QRC hit.
-SYSTEM FLUSH LOGS;
-SELECT ProfileEvents['QueryResultCacheHits'], ProfileEvents['QueryResultCacheMisses']
-FROM system.query_log
-WHERE type = 'QueryFinish'
- AND query = 'select 1 SETTINGS use_query_result_cache = true;';
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_drop_cache.sql b/tests/queries/0_stateless/02494_query_result_cache_drop_cache.sql
deleted file mode 100644
index 49e4298bd76..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_drop_cache.sql
+++ /dev/null
@@ -1,12 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
--- Cache query result in query result cache
-SELECT 1 SETTINGS use_query_result_cache = true;
-SELECT count(*) FROM system.query_result_cache;
-
--- No query results are cached after DROP
-SYSTEM DROP QUERY RESULT CACHE;
-SELECT count(*) FROM system.query_result_cache;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_eligible_queries.sql b/tests/queries/0_stateless/02494_query_result_cache_eligible_queries.sql
deleted file mode 100644
index 23a869f9df7..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_eligible_queries.sql
+++ /dev/null
@@ -1,68 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-DROP TABLE IF EXISTS eligible_test;
-DROP TABLE IF EXISTS eligible_test2;
-
--- enable query result cache session-wide but also force it individually in each of below statements
-SET use_query_result_cache = true;
-
--- check that SELECT statements create entries in the query result cache ...
-SELECT 1 SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- ... and all other statements also should not create entries:
-
--- CREATE
-CREATE TABLE eligible_test (a String) ENGINE=MergeTree ORDER BY a; -- SETTINGS use_query_result_cache = true; -- SETTINGS rejected as unknown
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- ALTER
-ALTER TABLE eligible_test ADD COLUMN b String SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- INSERT
-INSERT INTO eligible_test VALUES('a', 'b'); -- SETTINGS use_query_result_cache = true; -- SETTINGS rejected as unknown
-SELECT COUNT(*) FROM system.query_result_cache;
-INSERT INTO eligible_test SELECT * FROM eligible_test SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- SHOW
-SHOW TABLES SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- CHECK
-CHECK TABLE eligible_test SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- DESCRIBE
-DESCRIBE TABLE eligible_test SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- EXISTS
-EXISTS TABLE eligible_test SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- KILL
-KILL QUERY WHERE query_id='3-857d-4a57-9ee0-3c7da5d60a90' SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- OPTIMIZE
-OPTIMIZE TABLE eligible_test FINAL SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- TRUNCATE
-TRUNCATE TABLE eligible_test SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- RENAME
-RENAME TABLE eligible_test TO eligible_test2 SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
-DROP TABLE eligible_test2;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_events.sql b/tests/queries/0_stateless/02494_query_result_cache_events.sql
deleted file mode 100644
index 73f95ef8f36..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_events.sql
+++ /dev/null
@@ -1,32 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
--- Start with empty query result cache (QRC) and query log
-SYSTEM DROP QUERY RESULT CACHE;
-DROP TABLE system.query_log SYNC;
-
--- Run a query with QRC on. The first execution is a QRC miss.
-SELECT '---';
-SELECT 1 SETTINGS use_query_result_cache = true;
-
-SYSTEM FLUSH LOGS;
-SELECT ProfileEvents['QueryResultCacheHits'], ProfileEvents['QueryResultCacheMisses']
-FROM system.query_log
-WHERE type = 'QueryFinish'
- AND query = 'SELECT 1 SETTINGS use_query_result_cache = true;';
-
-
--- Run previous query again with query result cache on
-SELECT '---';
-SELECT 1 SETTINGS use_query_result_cache = true;
-
-DROP TABLE system.query_log SYNC;
-SYSTEM FLUSH LOGS;
-SELECT ProfileEvents['QueryResultCacheHits'], ProfileEvents['QueryResultCacheMisses']
-FROM system.query_log
-WHERE type = 'QueryFinish'
- AND query = 'SELECT 1 SETTINGS use_query_result_cache = true;';
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_exception_handling.sql b/tests/queries/0_stateless/02494_query_result_cache_exception_handling.sql
deleted file mode 100644
index 4ba3b73ad2f..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_exception_handling.sql
+++ /dev/null
@@ -1,12 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- If an exception is thrown during query execution, no entry must be created in the query result cache
-SELECT throwIf(1) SETTINGS use_query_result_cache = true; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO }
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_min_query_duration.sql b/tests/queries/0_stateless/02494_query_result_cache_min_query_duration.sql
deleted file mode 100644
index 37cd4590396..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_min_query_duration.sql
+++ /dev/null
@@ -1,20 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- This creates an entry in the query result cache ...
-SELECT 1 SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
-SELECT '---';
-
--- ... but this does not because the query executes much faster than the specified minumum query duration for caching the result
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_duration = 10000;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_min_query_runs.sql b/tests/queries/0_stateless/02494_query_result_cache_min_query_runs.sql
deleted file mode 100644
index 4a93ee507ab..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_min_query_runs.sql
+++ /dev/null
@@ -1,34 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- Cache the query result after the 1st query invocation
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 0;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '---';
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- Cache the query result after the 2nd query invocation
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 1;
-SELECT COUNT(*) FROM system.query_result_cache;
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 1;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '---';
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- Cache the query result after the 3rd query invocation
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 2;
-SELECT COUNT(*) FROM system.query_result_cache;
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 2;
-SELECT COUNT(*) FROM system.query_result_cache;
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_min_query_runs = 2;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_nondeterministic_functions.sql b/tests/queries/0_stateless/02494_query_result_cache_nondeterministic_functions.sql
deleted file mode 100644
index 87dc04c9919..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_nondeterministic_functions.sql
+++ /dev/null
@@ -1,18 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- rand() is non-deterministic, with default settings no entry in the query result cache should be created
-SELECT COUNT(rand(1)) SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '---';
-
--- But an entry can be forced using a setting
-SELECT COUNT(RAND(1)) SETTINGS use_query_result_cache = true, query_result_cache_store_results_of_queries_with_nondeterministic_functions = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_normalize_ast.sql b/tests/queries/0_stateless/02494_query_result_cache_normalize_ast.sql
deleted file mode 100644
index 15bab7e5584..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_normalize_ast.sql
+++ /dev/null
@@ -1,31 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
--- Start with empty query result cache (QRC) and query log.
-SYSTEM DROP QUERY RESULT CACHE;
-DROP TABLE system.query_log SYNC;
-
--- Run query whose result gets cached in the query result cache.
--- Besides "use_query_result_cache", pass two more knobs (one QRC-specific knob and one non-QRC-specific knob). We just care
--- *that* they are passed and not about their effect.
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_store_results_of_queries_with_nondeterministic_functions = true, max_threads = 16;
-
--- Check that entry in QRC exists
-SELECT COUNT(*) FROM system.query_result_cache;
-
--- Run the same SELECT but with different SETTINGS. We want its result to be served from the QRC (--> passive mode, achieve it by
--- disabling active mode)
-SELECT '---';
-SELECT 1 SETTINGS use_query_result_cache = true, enable_writes_to_query_result_cache = false, max_threads = 16;
-
--- Technically, both SELECT queries have different ASTs, leading to different QRC keys. QRC does some AST normalization (erase all
--- QRC-related settings) such that the keys match regardless. Verify by checking that the second query caused a QRC hit.
-SYSTEM FLUSH LOGS;
-SELECT ProfileEvents['QueryResultCacheHits'], ProfileEvents['QueryResultCacheMisses']
-FROM system.query_log
-WHERE type = 'QueryFinish'
- AND query = 'SELECT 1 SETTINGS use_query_result_cache = true, enable_writes_to_query_result_cache = false, max_threads = 16;';
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_passive_usage.sql b/tests/queries/0_stateless/02494_query_result_cache_passive_usage.sql
deleted file mode 100644
index 86c06461463..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_passive_usage.sql
+++ /dev/null
@@ -1,41 +0,0 @@
--- Tags: no-parallel
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
--- Start with empty query result cache (QRC).
-SYSTEM DROP QUERY RESULT CACHE;
-
--- By default, don't write query result into query result cache (QRC).
-SELECT 1;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '-----';
-
--- Try to retrieve query result from empty QRC using the passive mode. Do this by disabling the active mode. The cache should still be empty (no insert).
-SELECT 1 SETTINGS use_query_result_cache = true, enable_writes_to_query_result_cache = false;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '-----';
-
--- Put query result into cache.
-SELECT 1 SETTINGS use_query_result_cache = true;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SELECT '-----';
-
--- Run same query with passive mode again. There must still be one entry in the QRC and we must have a QRC hit.
-
--- Get rid of log of previous SELECT
-DROP TABLE system.query_log SYNC;
-
-SELECT 1 SETTINGS use_query_result_cache = true, enable_writes_to_query_result_cache = false;
-SELECT COUNT(*) FROM system.query_result_cache;
-
-SYSTEM FLUSH LOGS;
-SELECT ProfileEvents['QueryResultCacheHits'], ProfileEvents['QueryResultCacheMisses']
-FROM system.query_log
-WHERE type = 'QueryFinish'
- AND query = 'SELECT 1 SETTINGS use_query_result_cache = true, enable_writes_to_query_result_cache = false;';
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_secrets.sql b/tests/queries/0_stateless/02494_query_result_cache_secrets.sql
deleted file mode 100644
index b45db639efb..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_secrets.sql
+++ /dev/null
@@ -1,15 +0,0 @@
--- Tags: no-parallel, no-fasttest
--- Tag no-fasttest: Depends on OpenSSL
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- Cache a result of a query with secret in the query result cache
-SELECT hex(encrypt('aes-128-ecb', 'plaintext', 'passwordpassword')) SETTINGS use_query_result_cache = true;
-
--- The secret should not be revealed in system.query_result_cache
-SELECT query FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;
diff --git a/tests/queries/0_stateless/02494_query_result_cache_ttl_long.sql b/tests/queries/0_stateless/02494_query_result_cache_ttl_long.sql
deleted file mode 100644
index 7acee9b2a5b..00000000000
--- a/tests/queries/0_stateless/02494_query_result_cache_ttl_long.sql
+++ /dev/null
@@ -1,31 +0,0 @@
--- Tags: no-fasttest, no-parallel, long
--- Tag no-fasttest: Test runtime is > 6 sec
--- Tag long: Test runtime is > 6 sec
--- Tag no-parallel: Messes with internal cache
-
-SET allow_experimental_query_result_cache = true;
-
-SYSTEM DROP QUERY RESULT CACHE;
-
--- Cache query result into query result cache with a TTL of 3 sec
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_ttl = 3;
-
--- Expect one non-stale cache entry
-SELECT COUNT(*) FROM system.query_result_cache;
-SELECT stale FROM system.query_result_cache;
-
--- Wait until entry is expired
-SELECT sleep(3);
-SELECT sleep(3);
-SELECT stale FROM system.query_result_cache;
-
-SELECT '---';
-
--- Run same query as before
-SELECT 1 SETTINGS use_query_result_cache = true, query_result_cache_ttl = 3;
-
--- The entry should have been refreshed (non-stale)
-SELECT COUNT(*) FROM system.query_result_cache;
-SELECT stale FROM system.query_result_cache;
-
-SYSTEM DROP QUERY RESULT CACHE;