mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge pull request #53124 from kssenii/allowed-caches-dir-for-dynamic-disks
Allow to restrict fs caches paths (mainly useful for dynamic disks)
This commit is contained in:
commit
3d2fb2eced
@ -211,6 +211,11 @@ mv /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml.tmp /etc/cli
|
||||
sudo chown clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml
|
||||
sudo chgrp clickhouse /etc/clickhouse-server/config.d/s3_storage_policy_by_default.xml
|
||||
|
||||
sudo cat /etc/clickhouse-server/config.d/logger_trace.xml \
|
||||
| sed "s|<level>trace</level>|<level>test</level>|" \
|
||||
> /etc/clickhouse-server/config.d/logger_trace.xml.tmp
|
||||
mv /etc/clickhouse-server/config.d/logger_trace.xml.tmp /etc/clickhouse-server/config.d/logger_trace.xml
|
||||
|
||||
start
|
||||
|
||||
stress --hung-check --drop-databases --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" --global-time-limit 1200 \
|
||||
|
@ -1572,6 +1572,11 @@ try
|
||||
global_context->setFormatSchemaPath(format_schema_path);
|
||||
fs::create_directories(format_schema_path);
|
||||
|
||||
/// Set path for filesystem caches
|
||||
fs::path filesystem_caches_path(config().getString("filesystem_caches_path", ""));
|
||||
if (!filesystem_caches_path.empty())
|
||||
global_context->setFilesystemCachesPath(filesystem_caches_path);
|
||||
|
||||
/// Check sanity of MergeTreeSettings on server startup
|
||||
{
|
||||
size_t background_pool_tasks = global_context->getMergeMutateExecutor()->getMaxTasksCount();
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
@ -40,10 +41,24 @@ void registerDiskCache(DiskFactory & factory, bool /* global_skip_access_check *
|
||||
FileCacheSettings file_cache_settings;
|
||||
file_cache_settings.loadFromConfig(config, config_prefix);
|
||||
|
||||
if (file_cache_settings.base_path.empty())
|
||||
file_cache_settings.base_path = fs::path(context->getPath()) / "disks" / name / "cache/";
|
||||
else if (fs::path(file_cache_settings.base_path).is_relative())
|
||||
file_cache_settings.base_path = fs::path(context->getPath()) / "caches" / file_cache_settings.base_path;
|
||||
auto config_fs_caches_dir = context->getFilesystemCachesPath();
|
||||
if (config_fs_caches_dir.empty())
|
||||
{
|
||||
if (fs::path(file_cache_settings.base_path).is_relative())
|
||||
file_cache_settings.base_path = fs::path(context->getPath()) / "caches" / file_cache_settings.base_path;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (fs::path(file_cache_settings.base_path).is_relative())
|
||||
file_cache_settings.base_path = fs::path(config_fs_caches_dir) / file_cache_settings.base_path;
|
||||
|
||||
if (!pathStartsWith(file_cache_settings.base_path, config_fs_caches_dir))
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Filesystem cache path {} must lie inside default filesystem cache path `{}`",
|
||||
file_cache_settings.base_path, config_fs_caches_dir);
|
||||
}
|
||||
}
|
||||
|
||||
auto cache = FileCacheFactory::instance().getOrCreate(name, file_cache_settings);
|
||||
auto disk = disk_it->second;
|
||||
|
@ -51,13 +51,13 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
FileCache::FileCache(const FileCacheSettings & settings)
|
||||
FileCache::FileCache(const std::string & cache_name, const FileCacheSettings & settings)
|
||||
: max_file_segment_size(settings.max_file_segment_size)
|
||||
, bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0)
|
||||
, delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms)
|
||||
, boundary_alignment(settings.boundary_alignment)
|
||||
, background_download_threads(settings.background_download_threads)
|
||||
, log(&Poco::Logger::get("FileCache"))
|
||||
, log(&Poco::Logger::get("FileCache(" + cache_name + ")"))
|
||||
, metadata(settings.base_path)
|
||||
{
|
||||
main_priority = std::make_unique<LRUFileCachePriority>(settings.max_size, settings.max_elements);
|
||||
@ -990,6 +990,7 @@ void FileCache::loadMetadata()
|
||||
fs::remove(offset_it->path());
|
||||
continue;
|
||||
}
|
||||
LOG_TEST(log, "Added file segment {}:{} (size: {}) with path: {}", key, offset, size, offset_it->path().string());
|
||||
|
||||
const auto & file_segment_metadata = file_segment_metadata_it->second;
|
||||
chassert(file_segment_metadata->file_segment->assertCorrectness());
|
||||
|
@ -58,7 +58,7 @@ public:
|
||||
using PriorityIterator = IFileCachePriority::Iterator;
|
||||
using PriorityIterationResult = IFileCachePriority::IterationResult;
|
||||
|
||||
explicit FileCache(const FileCacheSettings & settings);
|
||||
FileCache(const std::string & cache_name, const FileCacheSettings & settings);
|
||||
|
||||
~FileCache();
|
||||
|
||||
|
@ -29,7 +29,7 @@ FileCachePtr FileCacheFactory::getOrCreate(
|
||||
auto it = caches_by_name.find(cache_name);
|
||||
if (it == caches_by_name.end())
|
||||
{
|
||||
auto cache = std::make_shared<FileCache>(file_cache_settings);
|
||||
auto cache = std::make_shared<FileCache>(cache_name, file_cache_settings);
|
||||
it = caches_by_name.emplace(
|
||||
cache_name, std::make_unique<FileCacheData>(cache, file_cache_settings)).first;
|
||||
}
|
||||
|
@ -215,6 +215,7 @@ struct ContextSharedPart : boost::noncopyable
|
||||
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
|
||||
String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries.
|
||||
String user_scripts_path; /// Path to the directory with user provided scripts.
|
||||
String filesystem_caches_path; /// Path to the directory with filesystem caches.
|
||||
ConfigurationPtr config; /// Global configuration settings.
|
||||
|
||||
String tmp_path; /// Path to the temporary files that occur when processing the request.
|
||||
@ -771,6 +772,12 @@ String Context::getUserScriptsPath() const
|
||||
return shared->user_scripts_path;
|
||||
}
|
||||
|
||||
String Context::getFilesystemCachesPath() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
return shared->filesystem_caches_path;
|
||||
}
|
||||
|
||||
Strings Context::getWarnings() const
|
||||
{
|
||||
Strings common_warnings;
|
||||
@ -862,6 +869,16 @@ void Context::setPath(const String & path)
|
||||
shared->user_scripts_path = shared->path + "user_scripts/";
|
||||
}
|
||||
|
||||
void Context::setFilesystemCachesPath(const String & path)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!fs::path(path).is_absolute())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem caches path must be absolute: {}", path);
|
||||
|
||||
shared->filesystem_caches_path = path;
|
||||
}
|
||||
|
||||
static void setupTmpPath(Poco::Logger * log, const std::string & path)
|
||||
try
|
||||
{
|
||||
|
@ -491,6 +491,7 @@ public:
|
||||
String getUserFilesPath() const;
|
||||
String getDictionariesLibPath() const;
|
||||
String getUserScriptsPath() const;
|
||||
String getFilesystemCachesPath() const;
|
||||
|
||||
/// A list of warnings about server configuration to place in `system.warnings` table.
|
||||
Strings getWarnings() const;
|
||||
@ -501,6 +502,8 @@ public:
|
||||
TemporaryDataOnDiskScopePtr getSharedTempDataOnDisk() const;
|
||||
void setTempDataOnDisk(TemporaryDataOnDiskScopePtr temp_data_on_disk_);
|
||||
|
||||
void setFilesystemCachesPath(const String & path);
|
||||
|
||||
void setPath(const String & path);
|
||||
void setFlagsPath(const String & path);
|
||||
void setUserFilesPath(const String & path);
|
||||
|
@ -209,7 +209,7 @@ TEST_F(FileCacheTest, get)
|
||||
|
||||
{
|
||||
std::cerr << "Step 1\n";
|
||||
auto cache = DB::FileCache(settings);
|
||||
auto cache = DB::FileCache("1", settings);
|
||||
cache.initialize();
|
||||
auto key = cache.createKeyForPath("key1");
|
||||
|
||||
@ -568,7 +568,7 @@ TEST_F(FileCacheTest, get)
|
||||
{
|
||||
/// Test LRUCache::restore().
|
||||
|
||||
auto cache2 = DB::FileCache(settings);
|
||||
auto cache2 = DB::FileCache("2", settings);
|
||||
cache2.initialize();
|
||||
auto key = cache2.createKeyForPath("key1");
|
||||
|
||||
@ -587,7 +587,7 @@ TEST_F(FileCacheTest, get)
|
||||
settings2.max_file_segment_size = 10;
|
||||
settings2.base_path = caches_dir / "cache2";
|
||||
fs::create_directories(settings2.base_path);
|
||||
auto cache2 = DB::FileCache(settings2);
|
||||
auto cache2 = DB::FileCache("3", settings2);
|
||||
cache2.initialize();
|
||||
auto key = cache2.createKeyForPath("key1");
|
||||
|
||||
@ -602,7 +602,7 @@ TEST_F(FileCacheTest, get)
|
||||
{
|
||||
/// Test delated cleanup
|
||||
|
||||
auto cache = FileCache(settings);
|
||||
auto cache = FileCache("4", settings);
|
||||
cache.initialize();
|
||||
cache.cleanup();
|
||||
const auto key = cache.createKeyForPath("key10");
|
||||
@ -633,7 +633,7 @@ TEST_F(FileCacheTest, get)
|
||||
|
||||
auto settings2{settings};
|
||||
settings2.delayed_cleanup_interval_ms = 0;
|
||||
auto cache = DB::FileCache(settings2);
|
||||
auto cache = DB::FileCache("5", settings2);
|
||||
cache.initialize();
|
||||
const auto key = cache.createKeyForPath("key10");
|
||||
const auto key_path = cache.getPathInLocalCache(key);
|
||||
@ -662,7 +662,7 @@ TEST_F(FileCacheTest, writeBuffer)
|
||||
settings.max_file_segment_size = 5;
|
||||
settings.base_path = cache_base_path;
|
||||
|
||||
FileCache cache(settings);
|
||||
FileCache cache("6", settings);
|
||||
cache.initialize();
|
||||
|
||||
auto write_to_cache = [&cache](const String & key, const Strings & data, bool flush)
|
||||
@ -767,7 +767,7 @@ TEST_F(FileCacheTest, temporaryData)
|
||||
settings.max_file_segment_size = 1_KiB;
|
||||
settings.base_path = cache_base_path;
|
||||
|
||||
DB::FileCache file_cache(settings);
|
||||
DB::FileCache file_cache("7", settings);
|
||||
file_cache.initialize();
|
||||
|
||||
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(nullptr, &file_cache, 0);
|
||||
@ -908,7 +908,7 @@ TEST_F(FileCacheTest, CachedReadBuffer)
|
||||
wb->next();
|
||||
wb->finalize();
|
||||
|
||||
auto cache = std::make_shared<DB::FileCache>(settings);
|
||||
auto cache = std::make_shared<DB::FileCache>("8", settings);
|
||||
cache->initialize();
|
||||
auto key = cache->createKeyForPath(file_path);
|
||||
|
||||
|
@ -10,7 +10,7 @@
|
||||
<cached_s3>
|
||||
<type>cache</type>
|
||||
<max_size>1Gi</max_size>
|
||||
<path>/var/lib/clickhouse/s3_cache/</path>
|
||||
<path>cached_s3/</path>
|
||||
<disk>s3</disk>
|
||||
</cached_s3>
|
||||
</disks>
|
||||
|
@ -1,4 +1,5 @@
|
||||
<clickhouse>
|
||||
<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches/</filesystem_caches_path>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<s3_disk>
|
||||
|
@ -12,7 +12,7 @@ SETTINGS min_bytes_for_wide_part = 10485760,
|
||||
disk = disk(
|
||||
type = cache,
|
||||
max_size = '128Mi',
|
||||
path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
enable_bypass_cache_with_threashold = 1,
|
||||
bypass_cache_threashold = 100,
|
||||
delayed_cleanup_interval_ms = 100,
|
||||
|
@ -15,7 +15,7 @@ SETTINGS min_bytes_for_wide_part = 10485760,
|
||||
disk = disk(
|
||||
type = cache,
|
||||
max_size = '128Mi',
|
||||
path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
enable_bypass_cache_with_threashold = 1,
|
||||
bypass_cache_threashold = 100,
|
||||
delayed_cleanup_interval_ms = 100,
|
||||
|
@ -14,7 +14,7 @@ SETTINGS min_bytes_for_wide_part = 10485760,
|
||||
disk = disk(
|
||||
type = cache,
|
||||
max_size = '128Mi',
|
||||
path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
cache_on_write_operations= 1,
|
||||
enable_filesystem_query_cache_limit = 1,
|
||||
delayed_cleanup_interval_ms = 100,
|
||||
|
@ -17,7 +17,7 @@ SETTINGS min_bytes_for_wide_part = 10485760,
|
||||
disk = disk(
|
||||
type = cache,
|
||||
max_size = '128Mi',
|
||||
path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
cache_on_write_operations= 1,
|
||||
enable_filesystem_query_cache_limit = 1,
|
||||
delayed_cleanup_interval_ms = 100,
|
||||
|
@ -1 +1 @@
|
||||
134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/caches/s3_cache/ 100 2 0
|
||||
134217728 10000000 33554432 4194304 1 0 0 0 /var/lib/clickhouse/filesystem_caches/s3_cache/ 100 2 0
|
||||
|
@ -22,7 +22,7 @@ SETTINGS min_bytes_for_wide_part = 0,
|
||||
type = cache,
|
||||
max_size = '128Mi',
|
||||
max_file_segment_size = '10Ki',
|
||||
path = '/var/lib/clickhouse/${CLICKHOUSE_TEST_UNIQUE_NAME}_cache',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}',
|
||||
cache_on_write_operations = 1,
|
||||
enable_filesystem_query_cache_limit = 1,
|
||||
delayed_cleanup_interval_ms = 100,
|
||||
|
@ -7,7 +7,7 @@ CREATE TABLE test (a Int32, b String)
|
||||
ENGINE = MergeTree() ORDER BY tuple()
|
||||
SETTINGS disk = disk(
|
||||
type = 'local_blob_storage',
|
||||
path = '/var/lib/clickhouse/disks/${CLICKHOUSE_TEST_UNIQUE_NAME}/');
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}/');
|
||||
|
||||
INSERT INTO test SELECT 1, 'test';
|
||||
SELECT * FROM test;
|
||||
@ -19,7 +19,7 @@ ENGINE = MergeTree() ORDER BY tuple()
|
||||
SETTINGS disk = disk(
|
||||
type = 'cache',
|
||||
max_size = '10Mi',
|
||||
path = '/var/lib/clickhouse/caches/${CLICKHOUSE_TEST_UNIQUE_NAME}/',
|
||||
path = '${CLICKHOUSE_TEST_UNIQUE_NAME}/',
|
||||
disk = disk(type='local_blob_storage', path='/var/lib/clickhouse/disks/${CLICKHOUSE_TEST_UNIQUE_NAME}/'));
|
||||
|
||||
INSERT INTO test SELECT 1, 'test';
|
||||
|
@ -0,0 +1,45 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
DROP TABLE IF EXISTS test_1;
|
||||
DROP TABLE IF EXISTS test_2;
|
||||
|
||||
CREATE TABLE test (a Int32)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS disk = disk(type = cache,
|
||||
max_size = '1Mi',
|
||||
path = '/kek',
|
||||
disk = 'local_disk'); -- {serverError BAD_ARGUMENTS}
|
||||
|
||||
CREATE TABLE test (a Int32)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS disk = disk(type = cache,
|
||||
max_size = '1Mi',
|
||||
path = '/var/lib/clickhouse/filesystem_caches/../kek',
|
||||
disk = 'local_disk'); -- {serverError BAD_ARGUMENTS}
|
||||
|
||||
CREATE TABLE test (a Int32)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS disk = disk(type = cache,
|
||||
max_size = '1Mi',
|
||||
path = '../kek',
|
||||
disk = 'local_disk'); -- {serverError BAD_ARGUMENTS}
|
||||
|
||||
CREATE TABLE test_1 (a Int32)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS disk = disk(type = cache,
|
||||
max_size = '1Mi',
|
||||
path = '/var/lib/clickhouse/filesystem_caches/kek',
|
||||
disk = 'local_disk');
|
||||
|
||||
CREATE TABLE test_2 (a Int32)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY tuple()
|
||||
SETTINGS disk = disk(type = cache,
|
||||
max_size = '1Mi',
|
||||
path = 'kek',
|
||||
disk = 'local_disk');
|
Loading…
Reference in New Issue
Block a user