mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #36639 from kssenii/better-drop-cache
Improve drop cache query
This commit is contained in:
commit
81db081b88
@ -555,7 +555,7 @@ void LRUFileCache::remove(const Key & key)
|
||||
fs::remove(key_path);
|
||||
}
|
||||
|
||||
void LRUFileCache::tryRemoveAll()
|
||||
void LRUFileCache::remove(bool force_remove_unreleasable)
|
||||
{
|
||||
/// Try remove all cached files by cache_base_path.
|
||||
/// Only releasable file segments are evicted.
|
||||
@ -567,12 +567,13 @@ void LRUFileCache::tryRemoveAll()
|
||||
auto & [key, offset] = *it++;
|
||||
|
||||
auto * cell = getCell(key, offset, cache_lock);
|
||||
if (cell->releasable())
|
||||
if (cell->releasable() || force_remove_unreleasable)
|
||||
{
|
||||
auto file_segment = cell->file_segment;
|
||||
if (file_segment)
|
||||
{
|
||||
std::lock_guard<std::mutex> segment_lock(file_segment->mutex);
|
||||
file_segment->detached = true;
|
||||
remove(file_segment->key(), file_segment->offset(), cache_lock, segment_lock);
|
||||
}
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
|
||||
virtual void remove(const Key & key) = 0;
|
||||
|
||||
virtual void tryRemoveAll() = 0;
|
||||
virtual void remove(bool force_remove_unreleasable) = 0;
|
||||
|
||||
static bool isReadOnly();
|
||||
|
||||
@ -145,7 +145,7 @@ public:
|
||||
|
||||
void remove(const Key & key) override;
|
||||
|
||||
void tryRemoveAll() override;
|
||||
void remove(bool force_remove_unreleasable) override;
|
||||
|
||||
std::vector<String> tryGetCachePaths(const Key & key) override;
|
||||
|
||||
|
@ -455,6 +455,8 @@ void FileSegment::complete(State state)
|
||||
std::lock_guard cache_lock(cache->mutex);
|
||||
std::lock_guard segment_lock(mutex);
|
||||
|
||||
assertNotDetached();
|
||||
|
||||
bool is_downloader = isDownloaderImpl(segment_lock);
|
||||
if (!is_downloader)
|
||||
{
|
||||
@ -477,8 +479,6 @@ void FileSegment::complete(State state)
|
||||
|
||||
download_state = state;
|
||||
|
||||
assertNotDetached();
|
||||
|
||||
try
|
||||
{
|
||||
completeImpl(cache_lock, segment_lock);
|
||||
|
@ -306,12 +306,12 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
{
|
||||
auto caches = FileCacheFactory::instance().getAll();
|
||||
for (const auto & [_, cache_data] : caches)
|
||||
cache_data.cache->tryRemoveAll();
|
||||
cache_data.cache->remove(query.force_removal);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto cache = FileCacheFactory::instance().get(query.filesystem_cache_path);
|
||||
cache->tryRemoveAll();
|
||||
cache->remove(query.force_removal);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -192,6 +192,13 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|
||||
<< (settings.hilite ? hilite_keyword : "") << " SECOND"
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
else if (type == Type::DROP_FILESYSTEM_CACHE)
|
||||
{
|
||||
if (!filesystem_cache_path.empty())
|
||||
settings.ostr << (settings.hilite ? hilite_none : "") << filesystem_cache_path;
|
||||
if (force_removal)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FORCE";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -89,7 +89,10 @@ public:
|
||||
String volume;
|
||||
String disk;
|
||||
UInt64 seconds{};
|
||||
|
||||
/// Values for `drop filesystem cache` system query.
|
||||
String filesystem_cache_path;
|
||||
bool force_removal = false;
|
||||
|
||||
String getID(char) const override { return "SYSTEM query"; }
|
||||
|
||||
|
@ -346,6 +346,16 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
res->seconds = seconds->as<ASTLiteral>()->value.get<UInt64>();
|
||||
break;
|
||||
}
|
||||
case Type::DROP_FILESYSTEM_CACHE:
|
||||
{
|
||||
ParserLiteral path_parser;
|
||||
ASTPtr ast;
|
||||
if (path_parser.parse(pos, ast, expected))
|
||||
res->filesystem_cache_path = ast->as<ASTLiteral>()->value.safeGet<String>();
|
||||
if (ParserKeyword{"FORCE"}.ignore(pos, expected))
|
||||
res->force_removal = true;
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
{
|
||||
|
@ -10,7 +10,18 @@
|
||||
<cache_enabled>0</cache_enabled>
|
||||
<data_cache_max_size>22548578304</data_cache_max_size>
|
||||
<cache_on_write_operations>1</cache_on_write_operations>
|
||||
<data_cache_path>./s3_cache/</data_cache_path>
|
||||
</s3_cache>
|
||||
<s3_cache_2>
|
||||
<type>s3</type>
|
||||
<endpoint>http://localhost:11111/test/00170_test/</endpoint>
|
||||
<access_key_id>clickhouse</access_key_id>
|
||||
<secret_access_key>clickhouse</secret_access_key>
|
||||
<data_cache_enabled>1</data_cache_enabled>
|
||||
<cache_enabled>0</cache_enabled>
|
||||
<data_cache_max_size>22548578304</data_cache_max_size>
|
||||
<cache_on_write_operations>0</cache_on_write_operations>
|
||||
</s3_cache_2>
|
||||
</disks>
|
||||
<policies>
|
||||
<s3_cache>
|
||||
@ -20,6 +31,13 @@
|
||||
</main>
|
||||
</volumes>
|
||||
</s3_cache>
|
||||
<s3_cache_2>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>s3_cache_2</disk>
|
||||
</main>
|
||||
</volumes>
|
||||
</s3_cache_2>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
</clickhouse>
|
||||
|
@ -0,0 +1,30 @@
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
0
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
2
|
||||
SYSTEM DROP FILESYSTEM CACHE FORCE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
0
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
1
|
||||
SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 }
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
1
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
3
|
||||
SYSTEM DROP FILESYSTEM CACHE './s3_cache/';
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
2
|
34
tests/queries/0_stateless/02286_drop_filesystem_cache.sql
Normal file
34
tests/queries/0_stateless/02286_drop_filesystem_cache.sql
Normal file
@ -0,0 +1,34 @@
|
||||
-- Tags: no-parallel, no-fasttest, no-s3-storage
|
||||
|
||||
-- { echo }
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE FORCE;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SELECT * FROM test FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE './data'; -- { serverError 36 }
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
DROP TABLE IF EXISTS test2;
|
||||
CREATE TABLE test2 (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache_2', min_bytes_for_wide_part = 10485760;
|
||||
INSERT INTO test2 SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT * FROM test2 FORMAT Null;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM DROP FILESYSTEM CACHE './s3_cache/';
|
||||
SELECT count() FROM system.filesystem_cache;
|
Loading…
Reference in New Issue
Block a user