diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5f2a51406e1..553b48584bc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,6 @@ #include #include #include -#include #include #include #include @@ -48,7 +48,6 @@ #include #include #include -#include #include #include #include @@ -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); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 326f632d61d..bcfe3799818 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -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")) diff --git a/src/Common/NamedCollections/NamedCollectionUtils.cpp b/src/Common/NamedCollections/NamedCollectionUtils.cpp index 317ca3c84ac..e9b319f02a5 100644 --- a/src/Common/NamedCollections/NamedCollectionUtils.cpp +++ b/src/Common/NamedCollections/NamedCollectionUtils.cpp @@ -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 listCollections() const { + if (!fs::exists(metadata_path)) + return {}; + std::vector 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 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); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 0d3b5cb83c8..2d053c615d9 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -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; } diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 8591cd6cf92..a4eb212455d 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -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); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index c2d0fbe1c00..ada06e3b035 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -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 diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 33f7ca1ec19..e5cde775f79 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -18,7 +18,6 @@ #include #include -#include #include #include @@ -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 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; diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index b9703019c19..e6088e21a3b 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -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_, diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index 92219a7f25f..91eb214d941 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -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); diff --git a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp index f8847024508..0b1a74884cf 100644 --- a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp +++ b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.cpp @@ -35,10 +35,12 @@ std::unique_ptr createUserDefinedSQLObjectsStorag } return std::make_unique(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(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(global_context, path); + } } } diff --git a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h index 01659372dec..fc1d79adcf7 100644 --- a/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h +++ b/src/Functions/UserDefined/createUserDefinedSQLObjectsStorage.h @@ -5,6 +5,7 @@ namespace DB { + class IUserDefinedSQLObjectsStorage; std::unique_ptr createUserDefinedSQLObjectsStorage(const ContextMutablePtr & global_context); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 726b480930a..44d36e94441 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -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; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 0caca88c283..a9b0e7e7a98 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -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 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 diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 29c6c101910..82d2c3fd5f7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -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"); diff --git a/src/Interpreters/TemporaryDataOnDisk.cpp b/src/Interpreters/TemporaryDataOnDisk.cpp index 26a78d53aab..9a237738b3e 100644 --- a/src/Interpreters/TemporaryDataOnDisk.cpp +++ b/src/Interpreters/TemporaryDataOnDisk.cpp @@ -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;