Merge pull request #63074 from ClickHouse/local-less-directories

Less directories in clickhouse-local
This commit is contained in:
Alexey Milovidov 2024-05-05 00:19:31 +02:00 committed by GitHub
commit 7ac5821a23
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 81 additions and 75 deletions

View File

@ -3,6 +3,7 @@
#include <sys/resource.h> #include <sys/resource.h>
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
#include <Core/UUID.h>
#include <base/getMemoryAmount.h> #include <base/getMemoryAmount.h>
#include <Poco/Util/XMLConfiguration.h> #include <Poco/Util/XMLConfiguration.h>
#include <Poco/String.h> #include <Poco/String.h>
@ -35,7 +36,6 @@
#include <Loggers/OwnPatternFormatter.h> #include <Loggers/OwnPatternFormatter.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h> #include <IO/UseSSL.h>
#include <IO/SharedThreadPools.h> #include <IO/SharedThreadPools.h>
#include <Parsers/ASTInsertQuery.h> #include <Parsers/ASTInsertQuery.h>
@ -48,7 +48,6 @@
#include <Dictionaries/registerDictionaries.h> #include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h> #include <Disks/registerDisks.h>
#include <Formats/registerFormats.h> #include <Formats/registerFormats.h>
#include <Formats/FormatFactory.h>
#include <boost/algorithm/string/replace.hpp> #include <boost/algorithm/string/replace.hpp>
#include <boost/program_options/options_description.hpp> #include <boost/program_options/options_description.hpp>
#include <base/argsToConfig.h> #include <base/argsToConfig.h>
@ -211,7 +210,7 @@ void LocalServer::tryInitPath()
else else
{ {
// The path is not provided explicitly - use a unique path in the system temporary directory // The path is not provided explicitly - use a unique path in the system temporary directory
// (or in the current dir if temporary don't exist) // (or in the current dir if a temporary doesn't exist)
LoggerRawPtr log = &logger(); LoggerRawPtr log = &logger();
std::filesystem::path parent_folder; std::filesystem::path parent_folder;
std::filesystem::path default_path; std::filesystem::path default_path;
@ -237,39 +236,30 @@ void LocalServer::tryInitPath()
/// as we can't accurately distinguish those situations we don't touch any existent folders /// as we can't accurately distinguish those situations we don't touch any existent folders
/// we just try to pick some free name for our working folder /// we just try to pick some free name for our working folder
default_path = parent_folder / fmt::format("clickhouse-local-{}-{}-{}", getpid(), time(nullptr), randomSeed()); default_path = parent_folder / fmt::format("clickhouse-local-{}", UUIDHelpers::generateV4());
if (exists(default_path)) if (fs::exists(default_path))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} already exists.", default_path.string()); throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to set up the working directory: {} already exists.", default_path.string());
create_directory(default_path); /// The directory can be created lazily during the runtime.
temporary_directory_to_delete = default_path; temporary_directory_to_delete = default_path;
path = default_path.string(); path = default_path.string();
LOG_DEBUG(log, "Working directory created: {}", path); LOG_DEBUG(log, "Working directory created: {}", path);
} }
if (path.back() != '/') global_context->setPath(fs::path(path) / "");
path += '/';
fs::create_directories(fs::path(path) / "user_defined/"); global_context->setTemporaryStoragePath(fs::path(path) / "tmp" / "", 0);
fs::create_directories(fs::path(path) / "data/"); global_context->setFlagsPath(fs::path(path) / "flags" / "");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(fs::path(path) / "metadata_dropped/");
global_context->setPath(path); global_context->setUserFilesPath(""); /// user's files are everywhere
global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere
std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/"); std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/");
global_context->setUserScriptsPath(user_scripts_path); global_context->setUserScriptsPath(user_scripts_path);
fs::create_directories(user_scripts_path);
/// top_level_domains_lists /// top_level_domains_lists
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/"); const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
if (!top_level_domains_path.empty()) if (!top_level_domains_path.empty())
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config()); TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
} }
@ -299,10 +289,9 @@ void LocalServer::cleanup()
// Delete the temporary directory if needed. // Delete the temporary directory if needed.
if (temporary_directory_to_delete) if (temporary_directory_to_delete)
{ {
const auto dir = *temporary_directory_to_delete; LOG_DEBUG(&logger(), "Removing temporary directory: {}", temporary_directory_to_delete->string());
fs::remove_all(*temporary_directory_to_delete);
temporary_directory_to_delete.reset(); temporary_directory_to_delete.reset();
LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string());
remove_all(dir);
} }
} }
catch (...) catch (...)
@ -481,6 +470,9 @@ try
registerFormats(); registerFormats();
processConfig(); processConfig();
SCOPE_EXIT({ cleanup(); });
initTTYBuffer(toProgressOption(config().getString("progress", "default"))); initTTYBuffer(toProgressOption(config().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true); ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
@ -530,16 +522,12 @@ try
} }
catch (const DB::Exception & e) catch (const DB::Exception & e)
{ {
cleanup();
bool need_print_stack_trace = config().getBool("stacktrace", false); bool need_print_stack_trace = config().getBool("stacktrace", false);
std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl; std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl;
return e.code() ? e.code() : -1; return e.code() ? e.code() : -1;
} }
catch (...) catch (...)
{ {
cleanup();
std::cerr << getCurrentExceptionMessage(false) << std::endl; std::cerr << getCurrentExceptionMessage(false) << std::endl;
return getCurrentExceptionCode(); return getCurrentExceptionCode();
} }
@ -705,6 +693,7 @@ void LocalServer::processConfig()
if (config().has("path")) if (config().has("path"))
{ {
String path = global_context->getPath(); String path = global_context->getPath();
fs::create_directories(fs::path(path));
/// Lock path directory before read /// Lock path directory before read
status.emplace(fs::path(path) / "status", StatusFile::write_full_info); status.emplace(fs::path(path) / "status", StatusFile::write_full_info);

View File

@ -1205,11 +1205,11 @@ try
} }
{ {
fs::create_directories(path / "data/"); fs::create_directories(path / "data");
fs::create_directories(path / "metadata/"); fs::create_directories(path / "metadata");
/// Directory with metadata of tables, which was marked as dropped by Atomic database /// Directory with metadata of tables, which was marked as dropped by Atomic database
fs::create_directories(path / "metadata_dropped/"); fs::create_directories(path / "metadata_dropped");
} }
if (config().has("interserver_http_port") && config().has("interserver_https_port")) if (config().has("interserver_http_port") && config().has("interserver_https_port"))

View File

@ -113,17 +113,17 @@ private:
public: public:
explicit LoadFromSQL(ContextPtr context_) explicit LoadFromSQL(ContextPtr context_)
: WithContext(context_) : WithContext(context_)
, metadata_path( , metadata_path(fs::weakly_canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY)
{ {
if (fs::exists(metadata_path)) if (fs::exists(metadata_path))
cleanUp(); cleanup();
else
fs::create_directories(metadata_path);
} }
std::vector<std::string> listCollections() const std::vector<std::string> listCollections() const
{ {
if (!fs::exists(metadata_path))
return {};
std::vector<std::string> collection_names; std::vector<std::string> collection_names;
fs::directory_iterator it{metadata_path}; fs::directory_iterator it{metadata_path};
for (; it != fs::directory_iterator{}; ++it) for (; it != fs::directory_iterator{}; ++it)
@ -280,7 +280,7 @@ private:
/// Delete .tmp files. They could be left undeleted in case of /// Delete .tmp files. They could be left undeleted in case of
/// some exception or abrupt server restart. /// some exception or abrupt server restart.
void cleanUp() void cleanup()
{ {
fs::directory_iterator it{metadata_path}; fs::directory_iterator it{metadata_path};
std::vector<std::string> files_to_remove; std::vector<std::string> files_to_remove;
@ -308,11 +308,11 @@ private:
return create_query; return create_query;
} }
static void writeCreateQueryToMetadata( void writeCreateQueryToMetadata(
const ASTCreateNamedCollectionQuery & query, const ASTCreateNamedCollectionQuery & query,
const std::string & path, const std::string & path,
const Settings & settings, const Settings & settings,
bool replace = false) bool replace = false) const
{ {
if (!replace && fs::exists(path)) if (!replace && fs::exists(path))
{ {
@ -322,6 +322,8 @@ private:
path); path);
} }
fs::create_directories(metadata_path);
auto tmp_path = path + ".tmp"; auto tmp_path = path + ".tmp";
String formatted_query = serializeAST(query); String formatted_query = serializeAST(query);
WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL); WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL);

View File

@ -42,14 +42,23 @@ namespace ErrorCodes
extern const int CANNOT_CREATE_FILE; extern const int CANNOT_CREATE_FILE;
} }
struct statvfs getStatVFS(const String & path) struct statvfs getStatVFS(String path)
{ {
struct statvfs fs; struct statvfs fs;
while (statvfs(path.c_str(), &fs) != 0) while (statvfs(path.c_str(), &fs) != 0)
{ {
if (errno == EINTR) if (errno == EINTR)
continue; continue;
DB::ErrnoException::throwFromPath(DB::ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)");
/// Sometimes we create directories lazily, so we can request free space in a directory that yet to be created.
auto fs_path = std::filesystem::path(path);
if (errno == ENOENT && fs_path.has_parent_path())
{
path = fs_path.parent_path();
continue;
}
ErrnoException::throwFromPath(ErrorCodes::CANNOT_STATVFS, path, "Could not calculate available disk space (statvfs)");
} }
return fs; return fs;
} }

View File

@ -56,7 +56,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path);
#endif #endif
String getFilesystemName([[maybe_unused]] const String & mount_point); String getFilesystemName([[maybe_unused]] const String & mount_point);
struct statvfs getStatVFS(const String & path); struct statvfs getStatVFS(String path);
/// Returns true if path starts with prefix path /// Returns true if path starts with prefix path
bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path); bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path);

View File

@ -137,6 +137,9 @@ void DatabaseAtomic::dropTableImpl(ContextPtr local_context, const String & tabl
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
table = getTableUnlocked(table_name); table = getTableUnlocked(table_name);
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID()); table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
fs::create_directory(fs::path(table_metadata_path_drop).parent_path());
auto txn = local_context->getZooKeeperMetadataTransaction(); auto txn = local_context->getZooKeeperMetadataTransaction();
if (txn && !local_context->isInternalSubquery()) if (txn && !local_context->isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database txn->commit(); /// Commit point (a sort of) for Replicated database

View File

@ -18,7 +18,6 @@
#include <fcntl.h> #include <fcntl.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <Disks/DiskFactory.h>
#include <Disks/IO/WriteBufferFromTemporaryFile.h> #include <Disks/IO/WriteBufferFromTemporaryFile.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
@ -224,7 +223,7 @@ static UInt64 getTotalSpaceByName(const String & name, const String & disk_path,
{ {
struct statvfs fs; struct statvfs fs;
if (name == "default") /// for default disk we get space from path/data/ if (name == "default") /// for default disk we get space from path/data/
fs = getStatVFS((fs::path(disk_path) / "data/").string()); fs = getStatVFS((fs::path(disk_path) / "data" / "").string());
else else
fs = getStatVFS(disk_path); fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_blocks * fs.f_frsize; UInt64 total_size = fs.f_blocks * fs.f_frsize;
@ -248,7 +247,7 @@ std::optional<UInt64> DiskLocal::getAvailableSpace() const
/// available for superuser only and for system purposes /// available for superuser only and for system purposes
struct statvfs fs; struct statvfs fs;
if (name == "default") /// for default disk we get space from path/data/ if (name == "default") /// for default disk we get space from path/data/
fs = getStatVFS((fs::path(disk_path) / "data/").string()); fs = getStatVFS((fs::path(disk_path) / "data" / "").string());
else else
fs = getStatVFS(disk_path); fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_bavail * fs.f_frsize; UInt64 total_size = fs.f_bavail * fs.f_frsize;

View File

@ -21,6 +21,7 @@ public:
DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_, DiskLocal(const String & name_, const String & path_, UInt64 keep_free_space_bytes_,
const Poco::Util::AbstractConfiguration & config, const String & config_prefix); const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
DiskLocal( DiskLocal(
const String & name_, const String & name_,
const String & path_, const String & path_,

View File

@ -35,8 +35,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
if (!disk) if (!disk)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk is not specified"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk is not specified");
if (fs::path prefix_path(prefix); prefix_path.has_parent_path()) disk->createDirectories((fs::path("") / prefix).parent_path());
disk->createDirectories(prefix_path.parent_path());
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);

View File

@ -35,10 +35,12 @@ std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorag
} }
return std::make_unique<UserDefinedSQLObjectsZooKeeperStorage>(global_context, config.getString(zookeeper_path_key)); return std::make_unique<UserDefinedSQLObjectsZooKeeperStorage>(global_context, config.getString(zookeeper_path_key));
} }
else
String default_path = fs::path{global_context->getPath()} / "user_defined/"; {
String default_path = fs::path{global_context->getPath()} / "user_defined" / "";
String path = config.getString(disk_path_key, default_path); String path = config.getString(disk_path_key, default_path);
return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path); return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path);
} }
}
} }

View File

@ -5,6 +5,7 @@
namespace DB namespace DB
{ {
class IUserDefinedSQLObjectsStorage; class IUserDefinedSQLObjectsStorage;
std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context); std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context);

View File

@ -1044,8 +1044,8 @@ try
{ {
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path); if (fs::exists(path))
{
/// Clearing old temporary files. /// Clearing old temporary files.
fs::directory_iterator dir_end; fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it) for (fs::directory_iterator it(path); it != dir_end; ++it)
@ -1063,10 +1063,11 @@ try
/// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types. /// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types.
} }
} }
}
catch (...) catch (...)
{ {
DB::tryLogCurrentException(log, fmt::format( DB::tryLogCurrentException(log, fmt::format(
"Caught exception while setup temporary path: {}. " "Caught exception while setting up temporary path: {}. "
"It is ok to skip this exception as cleaning old temporary files is not necessary", path)); "It is ok to skip this exception as cleaning old temporary files is not necessary", path));
} }
@ -1091,9 +1092,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size)
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock)); VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock));
for (const auto & disk : volume->getDisks()) for (const auto & disk : volume->getDisks())
{
setupTmpPath(shared->log, disk->getPath()); setupTmpPath(shared->log, disk->getPath());
}
TemporaryDataOnDiskSettings temporary_data_on_disk_settings; TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
temporary_data_on_disk_settings.max_size_on_disk = max_size; temporary_data_on_disk_settings.max_size_on_disk = max_size;

View File

@ -988,7 +988,7 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
/// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper /// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper
std::map<String, StorageID> dropped_metadata; std::map<String, StorageID> dropped_metadata;
String path = getContext()->getPath() + "metadata_dropped/"; String path = std::filesystem::path(getContext()->getPath()) / "metadata_dropped" / "";
if (!std::filesystem::exists(path)) if (!std::filesystem::exists(path))
{ {
@ -1043,10 +1043,11 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const
{ {
return getContext()->getPath() + "metadata_dropped/" + return std::filesystem::path(getContext()->getPath()) / "metadata_dropped" /
escapeForFileName(table_id.getDatabaseName()) + "." + fmt::format("{}.{}.{}.sql",
escapeForFileName(table_id.getTableName()) + "." + escapeForFileName(table_id.getDatabaseName()),
toString(table_id.uuid) + ".sql"; escapeForFileName(table_id.getTableName()),
toString(table_id.uuid));
} }
String DatabaseCatalog::getPathForMetadata(const StorageID & table_id) const String DatabaseCatalog::getPathForMetadata(const StorageID & table_id) const

View File

@ -140,7 +140,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
/// Will write file with database metadata, if needed. /// Will write file with database metadata, if needed.
String database_name_escaped = escapeForFileName(database_name); String database_name_escaped = escapeForFileName(database_name);
fs::path metadata_path = fs::canonical(getContext()->getPath()); fs::path metadata_path = fs::weakly_canonical(getContext()->getPath());
fs::create_directories(metadata_path / "metadata");
fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp"); fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp");
fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql"); fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql");

View File

@ -43,10 +43,10 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz
throw Exception(ErrorCodes::LOGICAL_ERROR, "Negative temporary data size"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Negative temporary data size");
} }
size_t new_consumprion = stat.compressed_size + compressed_delta; size_t new_consumption = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && settings.max_size_on_disk && new_consumprion > settings.max_size_on_disk) if (compressed_delta > 0 && settings.max_size_on_disk && new_consumption > settings.max_size_on_disk)
throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES, throw Exception(ErrorCodes::TOO_MANY_ROWS_OR_BYTES,
"Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumprion, settings.max_size_on_disk); "Limit for temporary files size exceeded (would consume {} / {} bytes)", new_consumption, settings.max_size_on_disk);
stat.compressed_size += compressed_delta; stat.compressed_size += compressed_delta;
stat.uncompressed_size += uncompressed_delta; stat.uncompressed_size += uncompressed_delta;