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 <Common/logger_useful.h>
#include <Common/formatReadable.h>
#include <Core/UUID.h>
#include <base/getMemoryAmount.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/String.h>
@ -35,7 +36,6 @@
#include <Loggers/OwnPatternFormatter.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h>
#include <IO/SharedThreadPools.h>
#include <Parsers/ASTInsertQuery.h>
@ -48,7 +48,6 @@
#include <Dictionaries/registerDictionaries.h>
#include <Disks/registerDisks.h>
#include <Formats/registerFormats.h>
#include <Formats/FormatFactory.h>
#include <boost/algorithm/string/replace.hpp>
#include <boost/program_options/options_description.hpp>
#include <base/argsToConfig.h>
@ -211,7 +210,7 @@ void LocalServer::tryInitPath()
else
{
// 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();
std::filesystem::path parent_folder;
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
/// 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))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Unsuccessful attempt to create working directory: {} already exists.", default_path.string());
if (fs::exists(default_path))
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;
path = default_path.string();
LOG_DEBUG(log, "Working directory created: {}", path);
}
if (path.back() != '/')
path += '/';
global_context->setPath(fs::path(path) / "");
fs::create_directories(fs::path(path) / "user_defined/");
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
fs::create_directories(fs::path(path) / "metadata_dropped/");
global_context->setTemporaryStoragePath(fs::path(path) / "tmp" / "", 0);
global_context->setFlagsPath(fs::path(path) / "flags" / "");
global_context->setPath(path);
global_context->setTemporaryStoragePath(path + "tmp/", 0);
global_context->setFlagsPath(path + "flags");
global_context->setUserFilesPath(""); // user's files are everywhere
global_context->setUserFilesPath(""); /// user's files are everywhere
std::string user_scripts_path = config().getString("user_scripts_path", fs::path(path) / "user_scripts/");
global_context->setUserScriptsPath(user_scripts_path);
fs::create_directories(user_scripts_path);
/// 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())
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
@ -299,10 +289,9 @@ void LocalServer::cleanup()
// Delete the temporary directory if needed.
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();
LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string());
remove_all(dir);
}
}
catch (...)
@ -481,6 +470,9 @@ try
registerFormats();
processConfig();
SCOPE_EXIT({ cleanup(); });
initTTYBuffer(toProgressOption(config().getString("progress", "default")));
ASTAlterCommand::setFormatAlterCommandsWithParentheses(true);
@ -530,16 +522,12 @@ try
}
catch (const DB::Exception & e)
{
cleanup();
bool need_print_stack_trace = config().getBool("stacktrace", false);
std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl;
return e.code() ? e.code() : -1;
}
catch (...)
{
cleanup();
std::cerr << getCurrentExceptionMessage(false) << std::endl;
return getCurrentExceptionCode();
}
@ -705,6 +693,7 @@ void LocalServer::processConfig()
if (config().has("path"))
{
String path = global_context->getPath();
fs::create_directories(fs::path(path));
/// Lock path directory before read
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 / "metadata/");
fs::create_directories(path / "data");
fs::create_directories(path / "metadata");
/// 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"))

View File

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

View File

@ -56,7 +56,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path);
#endif
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
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);
table = getTableUnlocked(table_name);
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();
if (txn && !local_context->isInternalSubquery())
txn->commit(); /// Commit point (a sort of) for Replicated database

View File

@ -18,7 +18,6 @@
#include <fcntl.h>
#include <sys/stat.h>
#include <Disks/DiskFactory.h>
#include <Disks/IO/WriteBufferFromTemporaryFile.h>
#include <Common/randomSeed.h>
@ -224,7 +223,7 @@ static UInt64 getTotalSpaceByName(const String & name, const String & disk_path,
{
struct statvfs fs;
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
fs = getStatVFS(disk_path);
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
struct statvfs fs;
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
fs = getStatVFS(disk_path);
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_,
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
DiskLocal(
const String & name_,
const String & path_,

View File

@ -35,8 +35,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
if (!disk)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk is not specified");
if (fs::path prefix_path(prefix); prefix_path.has_parent_path())
disk->createDirectories(prefix_path.parent_path());
disk->createDirectories((fs::path("") / prefix).parent_path());
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));
}
String default_path = fs::path{global_context->getPath()} / "user_defined/";
String path = config.getString(disk_path_key, default_path);
return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path);
else
{
String default_path = fs::path{global_context->getPath()} / "user_defined" / "";
String path = config.getString(disk_path_key, default_path);
return std::make_unique<UserDefinedSQLObjectsDiskStorage>(global_context, path);
}
}
}

View File

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

View File

@ -1044,29 +1044,30 @@ try
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
fs::create_directories(path);
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
if (fs::exists(path))
{
if (it->is_regular_file())
/// Clearing old temporary files.
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (startsWith(it->path().filename(), "tmp"))
if (it->is_regular_file())
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
if (startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
}
else
LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string());
/// 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 (...)
{
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));
}
@ -1091,9 +1092,7 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size)
VolumePtr volume = createLocalSingleDiskVolume(shared->tmp_path, shared->getConfigRefWithLock(lock));
for (const auto & disk : volume->getDisks())
{
setupTmpPath(shared->log, disk->getPath());
}
TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
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
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))
{
@ -1043,10 +1043,11 @@ void DatabaseCatalog::loadMarkedAsDroppedTables()
String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const
{
return getContext()->getPath() + "metadata_dropped/" +
escapeForFileName(table_id.getDatabaseName()) + "." +
escapeForFileName(table_id.getTableName()) + "." +
toString(table_id.uuid) + ".sql";
return std::filesystem::path(getContext()->getPath()) / "metadata_dropped" /
fmt::format("{}.{}.{}.sql",
escapeForFileName(table_id.getDatabaseName()),
escapeForFileName(table_id.getTableName()),
toString(table_id.uuid));
}
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.
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_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");
}
size_t new_consumprion = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && settings.max_size_on_disk && new_consumprion > settings.max_size_on_disk)
size_t new_consumption = stat.compressed_size + compressed_delta;
if (compressed_delta > 0 && settings.max_size_on_disk && new_consumption > settings.max_size_on_disk)
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.uncompressed_size += uncompressed_delta;