mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Merge pull request #63074 from ClickHouse/local-less-directories
Less directories in clickhouse-local
This commit is contained in:
commit
7ac5821a23
@ -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);
|
||||
|
@ -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"))
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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_,
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IUserDefinedSQLObjectsStorage;
|
||||
|
||||
std::unique_ptr<IUserDefinedSQLObjectsStorage> createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context);
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user