Final fixes

This commit is contained in:
kssenii 2021-05-28 21:17:16 +03:00
parent 0d393c0006
commit 2a631aaf08
47 changed files with 74 additions and 115 deletions

View File

@ -57,7 +57,6 @@
#include <Common/getExecutablePath.h> #include <Common/getExecutablePath.h>
#include <Common/getHashOfLoadedBinary.h> #include <Common/getHashOfLoadedBinary.h>
#include <Common/Elf.h> #include <Common/Elf.h>
#include <Poco/File.h>
#include <filesystem> #include <filesystem>
#if !defined(ARCADIA_BUILD) #if !defined(ARCADIA_BUILD)

View File

@ -1,6 +1,5 @@
#include <daemon/SentryWriter.h> #include <daemon/SentryWriter.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h> #include <Poco/Util/LayeredConfiguration.h>

View File

@ -25,7 +25,6 @@
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp> #include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h> #include <Poco/String.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <common/find_symbols.h> #include <common/find_symbols.h>
#include <common/LineReader.h> #include <common/LineReader.h>
@ -88,7 +87,7 @@
#include <Common/UTF8Helpers.h> #include <Common/UTF8Helpers.h>
#include <Common/ProgressBar.h> #include <Common/ProgressBar.h>
#include <filesystem> #include <filesystem>
#include <Common/createFile.h> #include <Common/FileSystemHelpers.h>
#if !defined(ARCADIA_BUILD) #if !defined(ARCADIA_BUILD)
# include <Common/config_version.h> # include <Common/config_version.h>

View File

@ -8,7 +8,6 @@
#include <Poco/FormattingChannel.h> #include <Poco/FormattingChannel.h>
#include <Poco/PatternFormatter.h> #include <Poco/PatternFormatter.h>
#include <Poco/UUIDGenerator.h> #include <Poco/UUIDGenerator.h>
#include <Poco/File.h>
#include <Poco/Process.h> #include <Poco/Process.h>
#include <Poco/FileChannel.h> #include <Poco/FileChannel.h>
#include <Poco/SplitterChannel.h> #include <Poco/SplitterChannel.h>

View File

@ -3,7 +3,6 @@
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPRequest.h> #include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h> #include <Poco/URI.h>
#include <filesystem> #include <filesystem>

View File

@ -8,7 +8,6 @@
#include <IO/WriteBufferFromOStream.h> #include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Poco/Path.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <Common/ShellCommand.h> #include <Common/ShellCommand.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>

View File

@ -5,10 +5,8 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Access/AccessType.h> #include <Access/AccessType.h>
#include <Parsers/IdentifierQuotingStyle.h> #include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/File.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <Poco/Net/HTTPRequest.h> #include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h> #include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <Common/ShellCommand.h> #include <Common/ShellCommand.h>

View File

@ -15,12 +15,9 @@
#include <Poco/DOM/NodeList.h> #include <Poco/DOM/NodeList.h>
#include <Poco/DOM/NamedNodeMap.h> #include <Poco/DOM/NamedNodeMap.h>
#include <Poco/AutoPtr.h> #include <Poco/AutoPtr.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>
#include <Poco/ConsoleChannel.h> #include <Poco/ConsoleChannel.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>

View File

@ -1,11 +1,12 @@
#include "ConfigReloader.h" #include "ConfigReloader.h"
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Poco/File.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include "ConfigProcessor.h" #include "ConfigProcessor.h"
#include <filesystem> #include <filesystem>
#include <Common/FileSystemHelpers.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -170,10 +171,7 @@ struct ConfigReloader::FileWithTimestamp
void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add) void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add)
{ {
if (!path_to_add.empty() && fs::exists(path_to_add)) if (!path_to_add.empty() && fs::exists(path_to_add))
{ files.emplace(path_to_add, FS::getModificationTime(path_to_add));
fs::file_time_type fs_time = fs::last_write_time(path_to_add);
files.emplace(path_to_add, fs::file_time_type::clock::to_time_t(fs_time));
}
} }
bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs) bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs)

View File

@ -1,10 +1,12 @@
#include "createFile.h" #include <fcntl.h>
#include <unistd.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <utime.h>
#include <Common/ErrorCodes.h> #include <Common/ErrorCodes.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <unistd.h> #include <Poco/Timestamp.h>
#include <sys/types.h> #include <Common/FileSystemHelpers.h>
#include <sys/stat.h>
#include <fcntl.h>
namespace DB namespace DB
{ {
@ -61,4 +63,25 @@ bool canWrite(const std::string & path)
DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
} }
time_t getModificationTime(const std::string & path)
{
struct stat st;
if (stat(path.c_str(), &st) == 0)
return st.st_mtime;
DB::throwFromErrnoWithPath("Cannot check modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
}
Poco::Timestamp getModificationTimestamp(const std::string & path)
{
return Poco::Timestamp::fromEpochTime(getModificationTime(path));
}
void setModificationTime(const std::string & path, time_t time)
{
struct utimbuf tb;
tb.actime = time;
tb.modtime = time;
if (utime(path.c_str(), &tb) != 0)
DB::throwFromErrnoWithPath("Cannot set modification time for file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
}
} }

View File

@ -0,0 +1,14 @@
#pragma once
#include <string>
namespace FS
{
bool createFile(const std::string & path);
bool canRead(const std::string & path);
bool canWrite(const std::string & path);
time_t getModificationTime(const std::string & path);
Poco::Timestamp getModificationTimestamp(const std::string & path);
void setModificationTime(const std::string & path, time_t time);
}

View File

@ -3,6 +3,7 @@
#include <Poco/Timestamp.h> #include <Poco/Timestamp.h>
#include <string> #include <string>
#include <filesystem> #include <filesystem>
#include <Common/FileSystemHelpers.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -31,8 +32,6 @@ public:
private: private:
Poco::Timestamp getLastModificationTime() const Poco::Timestamp getLastModificationTime() const
{ {
fs::file_time_type fs_time = fs::last_write_time(path); return FS::getModificationTimestamp(path);
auto micro_sec = std::chrono::duration_cast<std::chrono::microseconds>(fs_time.time_since_epoch());
return Poco::Timestamp(micro_sec.count());
} }
}; };

View File

@ -1,9 +0,0 @@
#pragma once
#include <string>
namespace FS
{
bool createFile(const std::string & path);
bool canRead(const std::string & path);
bool canWrite(const std::string & path);
}

View File

@ -6,8 +6,6 @@
# include <mntent.h> # include <mntent.h>
#endif #endif
#include <cerrno> #include <cerrno>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Version.h> #include <Poco/Version.h>
#include <filesystem> #include <filesystem>

View File

@ -11,8 +11,6 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/formatAST.h> #include <Parsers/formatAST.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <filesystem> #include <filesystem>

View File

@ -19,7 +19,7 @@
#include <Databases/DatabaseAtomic.h> #include <Databases/DatabaseAtomic.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <filesystem> #include <filesystem>
#include <Common/createFile.h> #include <Common/FileSystemHelpers.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -542,10 +542,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
String table_metadata_path = getObjectMetadataPath(object_name); String table_metadata_path = getObjectMetadataPath(object_name);
if (fs::exists(table_metadata_path)) if (fs::exists(table_metadata_path))
{ return FS::getModificationTime(table_metadata_path);
fs::file_time_type fs_time = fs::last_write_time(table_metadata_path);
return fs::file_time_type::clock::to_time_t(fs_time);
}
else else
return static_cast<time_t>(0); return static_cast<time_t>(0);
} }

View File

@ -25,7 +25,7 @@
# include <Common/parseAddress.h> # include <Common/parseAddress.h>
# include <Common/setThreadName.h> # include <Common/setThreadName.h>
# include <filesystem> # include <filesystem>
# include <Common/createFile.h> # include <Common/FileSystemHelpers.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;

View File

@ -14,7 +14,7 @@
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h> #include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/createFile.h> #include <Common/FileSystemHelpers.h>
#include <filesystem> #include <filesystem>
namespace fs = std::filesystem; namespace fs = std::filesystem;

View File

@ -3,14 +3,13 @@
#include <vector> #include <vector>
#include <common/types.h> #include <common/types.h>
#include <Poco/File.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <Columns/IColumn.h> #include <Columns/IColumn.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
namespace DB namespace DB
{ {

View File

@ -1,9 +1,5 @@
#include "FileDictionarySource.h" #include "FileDictionarySource.h"
#include <filesystem> #include <filesystem>
#include <Poco/File.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
@ -14,6 +10,8 @@
#include "DictionaryStructure.h" #include "DictionaryStructure.h"
#include "registerDictionaries.h" #include "registerDictionaries.h"
#include "DictionarySourceHelpers.h" #include "DictionarySourceHelpers.h"
#include <Common/FileSystemHelpers.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;
@ -70,9 +68,7 @@ std::string FileDictionarySource::toString() const
Poco::Timestamp FileDictionarySource::getLastModification() const Poco::Timestamp FileDictionarySource::getLastModification() const
{ {
fs::file_time_type fs_time = fs::last_write_time(filepath); return FS::getModificationTimestamp(filepath);
auto micro_sec = std::chrono::duration_cast<std::chrono::microseconds>(fs_time.time_since_epoch());
return Poco::Timestamp(micro_sec.count());
} }

View File

@ -7,8 +7,7 @@
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <IO/createReadBufferFromFileBase.h> #include <IO/createReadBufferFromFileBase.h>
#include <Poco/File.h> #include <Common/FileSystemHelpers.h>
#include <Common/createFile.h>
#include <fstream> #include <fstream>
#include <unistd.h> #include <unistd.h>
@ -260,14 +259,12 @@ void DiskLocal::listFiles(const String & path, std::vector<String> & file_names)
void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp) void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp)
{ {
fs::last_write_time(fs::path(disk_path) / path, static_cast<fs::file_time_type>(std::chrono::microseconds(timestamp.epochMicroseconds()))); FS::setModificationTime(fs::path(disk_path) / path, timestamp.epochTime());
} }
Poco::Timestamp DiskLocal::getLastModified(const String & path) Poco::Timestamp DiskLocal::getLastModified(const String & path)
{ {
fs::file_time_type fs_time = fs::last_write_time(fs::path(disk_path) / path); return FS::getModificationTimestamp(fs::path(disk_path) / path);
auto micro_sec = std::chrono::duration_cast<std::chrono::microseconds>(fs_time.time_since_epoch());
return Poco::Timestamp(micro_sec.count());
} }
void DiskLocal::createHardLink(const String & src_path, const String & dst_path) void DiskLocal::createHardLink(const String & src_path, const String & dst_path)

View File

@ -11,6 +11,7 @@
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/checkStackSize.h> #include <Common/checkStackSize.h>
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
#include <Common/FileSystemHelpers.h>
namespace DB namespace DB
@ -429,16 +430,13 @@ void IDiskRemote::listFiles(const String & path, std::vector<String> & file_name
void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp) void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp)
{ {
fs::last_write_time(fs::path(metadata_path) / path, FS::setModificationTime(fs::path(metadata_path) / path, timestamp.epochTime());
static_cast<fs::file_time_type>(std::chrono::microseconds(timestamp.epochMicroseconds())));
} }
Poco::Timestamp IDiskRemote::getLastModified(const String & path) Poco::Timestamp IDiskRemote::getLastModified(const String & path)
{ {
fs::file_time_type fs_time = fs::last_write_time(fs::path(metadata_path) / path); return FS::getModificationTimestamp(fs::path(metadata_path) / path);
auto micro_sec = std::chrono::duration_cast<std::chrono::microseconds>(fs_time.time_since_epoch());
return Poco::Timestamp(micro_sec.count());
} }

View File

@ -8,8 +8,6 @@
#include <set> #include <set>
#include <Poco/File.h>
namespace namespace
{ {

View File

@ -74,14 +74,6 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String &
if (!path.has_extension() && !default_file_extension.empty()) if (!path.has_extension() && !default_file_extension.empty())
path = path.parent_path() / (path.stem().string() + '.' + default_file_extension); path = path.parent_path() / (path.stem().string() + '.' + default_file_extension);
fs::path default_schema_directory_path(default_schema_directory());
auto path_is_subdirectory_of = [](fs::path inner, const fs::path & outer) -> bool
{
while (inner != outer && inner != "/")
inner = inner.parent_path();
return inner == outer;
};
if (path.is_absolute()) if (path.is_absolute())
{ {
if (is_server) if (is_server)
@ -89,12 +81,13 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String &
schema_path = path.filename(); schema_path = path.filename();
schema_directory = path.parent_path() / ""; schema_directory = path.parent_path() / "";
} }
else if (path.has_parent_path() && !path_is_subdirectory_of(path, default_schema_directory_path)) else if (!fs::weakly_canonical(path).string().starts_with(fs::weakly_canonical(default_schema_directory()).string()))
{ {
if (is_server) if (is_server)
throw Exception( throw Exception(
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(), "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.string(),
ErrorCodes::BAD_ARGUMENTS); ErrorCodes::BAD_ARGUMENTS);
fs::path default_schema_directory_path(default_schema_directory());
if (default_schema_directory_path.is_absolute()) if (default_schema_directory_path.is_absolute())
path = default_schema_directory_path; path = default_schema_directory_path;
else else

View File

@ -5,7 +5,6 @@
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Databases/DatabaseMemory.h> #include <Databases/DatabaseMemory.h>
#include <Databases/DatabaseOnDisk.h> #include <Databases/DatabaseOnDisk.h>
#include <Poco/File.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Storages/StorageMemory.h> #include <Storages/StorageMemory.h>
#include <Storages/LiveView/TemporaryLiveViewCleaner.h> #include <Storages/LiveView/TemporaryLiveViewCleaner.h>
@ -18,6 +17,7 @@
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <filesystem> #include <filesystem>
#include <Common/FileSystemHelpers.h>
#if !defined(ARCADIA_BUILD) #if !defined(ARCADIA_BUILD)
# include "config_core.h" # include "config_core.h"
@ -782,8 +782,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
} }
addUUIDMapping(table_id.uuid); addUUIDMapping(table_id.uuid);
fs::file_time_type fs_time = fs::last_write_time(dropped_metadata_path); drop_time = FS::getModificationTime(dropped_metadata_path);
drop_time = fs::file_time_type::clock::to_time_t(fs_time);
} }
std::lock_guard lock(tables_marked_dropped_mutex); std::lock_guard lock(tables_marked_dropped_mutex);

View File

@ -3,12 +3,11 @@
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/Config/ConfigProcessor.h> #include <Common/Config/ConfigProcessor.h>
#include <Common/getMultipleKeysFromConfig.h> #include <Common/getMultipleKeysFromConfig.h>
#include <Poco/Glob.h> #include <Poco/Glob.h>
#include <Poco/File.h> #include <Common/FileSystemHelpers.h>
#include <Poco/Path.h>
#include <filesystem> #include <filesystem>
namespace fs = std::filesystem; namespace fs = std::filesystem;
namespace DB namespace DB
@ -21,9 +20,7 @@ ExternalLoaderXMLConfigRepository::ExternalLoaderXMLConfigRepository(
Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name) Poco::Timestamp ExternalLoaderXMLConfigRepository::getUpdateTime(const std::string & definition_entity_name)
{ {
fs::file_time_type fs_time = fs::last_write_time(definition_entity_name); return FS::getModificationTimestamp(definition_entity_name);
auto micro_sec = std::chrono::duration_cast<std::chrono::microseconds>(fs_time.time_since_epoch());
return Poco::Timestamp(micro_sec.count());
} }
std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames() std::set<std::string> ExternalLoaderXMLConfigRepository::getAllLoadablesDefinitionNames()

View File

@ -1,5 +1,3 @@
#include <Poco/File.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h> #include <Interpreters/executeDDLQueryOnCluster.h>

View File

@ -1,11 +1,8 @@
#include <Server/HTTP/HTTPServerResponse.h> #include <Server/HTTP/HTTPServerResponse.h>
#include <Server/HTTP/HTTPServerRequest.h> #include <Server/HTTP/HTTPServerRequest.h>
#include <Poco/CountingStream.h> #include <Poco/CountingStream.h>
#include <Poco/DateTimeFormat.h> #include <Poco/DateTimeFormat.h>
#include <Poco/DateTimeFormatter.h> #include <Poco/DateTimeFormatter.h>
#include <Poco/File.h>
#include <Poco/FileStream.h> #include <Poco/FileStream.h>
#include <Poco/Net/HTTPChunkedStream.h> #include <Poco/Net/HTTPChunkedStream.h>
#include <Poco/Net/HTTPFixedLengthStream.h> #include <Poco/Net/HTTPFixedLengthStream.h>
@ -13,6 +10,7 @@
#include <Poco/Net/HTTPStream.h> #include <Poco/Net/HTTPStream.h>
#include <Poco/StreamCopier.h> #include <Poco/StreamCopier.h>
namespace DB namespace DB
{ {

View File

@ -39,7 +39,6 @@
#include <Poco/Base64Decoder.h> #include <Poco/Base64Decoder.h>
#include <Poco/Base64Encoder.h> #include <Poco/Base64Encoder.h>
#include <Poco/File.h>
#include <Poco/Net/HTTPBasicCredentials.h> #include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/Net/HTTPStream.h> #include <Poco/Net/HTTPStream.h>
#include <Poco/Net/NetException.h> #include <Poco/Net/NetException.h>

View File

@ -138,11 +138,10 @@ void StaticRequestHandler::writeResponse(WriteBuffer & out)
if (startsWith(response_expression, file_prefix)) if (startsWith(response_expression, file_prefix))
{ {
const auto & file_name = response_expression.substr(file_prefix.size(), response_expression.size() - file_prefix.size()); const auto & file_name = response_expression.substr(file_prefix.size() + 1, response_expression.size() - file_prefix.size());
fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath())); fs::path user_files_absolute_path = fs::canonical(fs::path(server.context()->getUserFilesPath()));
/// Fixme: it does not work with fs::path(user_files_absolute_path) / file_name String file_path = fs::weakly_canonical(user_files_absolute_path / file_name);
String file_path = fs::canonical(user_files_absolute_path.string() + "/" + file_name);
if (!fs::exists(file_path)) if (!fs::exists(file_path))
throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME); throw Exception("Invalid file name " + file_path + " for static HTTPHandler. ", ErrorCodes::INCORRECT_FILE_NAME);

View File

@ -17,8 +17,6 @@
#include <Storages/MergeTree/MergeTreeIOSettings.h> #include <Storages/MergeTree/MergeTreeIOSettings.h>
#include <Storages/MergeTree/KeyCondition.h> #include <Storages/MergeTree/KeyCondition.h>
#include <Poco/Path.h>
#include <shared_mutex> #include <shared_mutex>
namespace zkutil namespace zkutil

View File

@ -6,7 +6,6 @@
#include <Interpreters/inplaceBlockConversions.h> #include <Interpreters/inplaceBlockConversions.h>
#include <Storages/MergeTree/IMergeTreeReader.h> #include <Storages/MergeTree/IMergeTreeReader.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Poco/File.h>
namespace DB namespace DB

View File

@ -3326,7 +3326,7 @@ MergeTreeData::getDetachedParts() const
for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) for (const auto & [path, disk] : getRelativeDataPathsWithDisks())
{ {
for (auto it = disk->iterateDirectory(path + MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next()) for (auto it = disk->iterateDirectory(fs::path(path) / MergeTreeData::DETACHED_DIR_NAME); it->isValid(); it->next())
{ {
res.emplace_back(); res.emplace_back();
auto & part = res.back(); auto & part = res.back();

View File

@ -7,7 +7,6 @@
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <Compression/CompressedReadBuffer.h> #include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h> #include <Compression/CompressedWriteBuffer.h>
#include <Poco/File.h>
namespace DB namespace DB

View File

@ -2,7 +2,6 @@
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <Storages/MergeTree/MergeTreeReaderCompact.h> #include <Storages/MergeTree/MergeTreeReaderCompact.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h> #include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
#include <Poco/File.h>
namespace DB namespace DB

View File

@ -5,7 +5,6 @@
#include <Storages/MergeTree/IMergeTreeReader.h> #include <Storages/MergeTree/IMergeTreeReader.h>
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Poco/File.h>
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>

View File

@ -1,5 +1,4 @@
#include "MergeTreeDataPartWide.h" #include "MergeTreeDataPartWide.h"
#include <Poco/File.h>
#include <Storages/MergeTree/MergeTreeReaderWide.h> #include <Storages/MergeTree/MergeTreeReaderWide.h>
#include <Storages/MergeTree/MergeTreeDataPartWriterWide.h> #include <Storages/MergeTree/MergeTreeDataPartWriterWide.h>
#include <Storages/MergeTree/IMergeTreeDataPartWriter.h> #include <Storages/MergeTree/IMergeTreeDataPartWriter.h>

View File

@ -3,7 +3,6 @@
#include <optional> #include <optional>
#include <unordered_set> #include <unordered_set>
#include <Poco/File.h>
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h> #include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <Storages/MergeTree/MergeTreeSelectProcessor.h> #include <Storages/MergeTree/MergeTreeSelectProcessor.h>
#include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h> #include <Storages/MergeTree/MergeTreeReverseSelectProcessor.h>

View File

@ -11,7 +11,6 @@
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <DataStreams/ITTLAlgorithm.h> #include <DataStreams/ITTLAlgorithm.h>
#include <DataStreams/OneBlockInputStream.h> #include <DataStreams/OneBlockInputStream.h>

View File

@ -1,7 +1,6 @@
#include <Storages/MergeTree/MergeTreeMarksLoader.h> #include <Storages/MergeTree/MergeTreeMarksLoader.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Poco/File.h>
#include <utility> #include <utility>

View File

@ -4,9 +4,6 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <utility> #include <utility>

View File

@ -2,7 +2,6 @@
#include <Storages/MergeTree/MergeTreeDataPartCompact.h> #include <Storages/MergeTree/MergeTreeDataPartCompact.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <Poco/File.h>
namespace DB namespace DB
{ {

View File

@ -1,6 +1,5 @@
#include <Storages/MergeTree/MergedBlockOutputStream.h> #include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Poco/File.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>

View File

@ -1,7 +1,6 @@
#include <algorithm> #include <algorithm>
#include <optional> #include <optional>
#include <Poco/File.h>
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>
#include <Storages/MergeTree/MergeTreeIndexGranularity.h> #include <Storages/MergeTree/MergeTreeIndexGranularity.h>

View File

@ -14,9 +14,7 @@
#include <Interpreters/castColumn.h> #include <Interpreters/castColumn.h>
#include <Common/assert_cast.h> #include <Common/assert_cast.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Poco/String.h> /// toLower #include <Poco/String.h> /// toLower
#include <Poco/File.h>
#include <Processors/Sources/SourceWithProgress.h> #include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Pipe.h> #include <Processors/Pipe.h>

View File

@ -8,7 +8,6 @@
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Poco/Net/HTTPRequest.h> #include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Processors/Pipe.h> #include <Processors/Pipe.h>
#include <Storages/StorageFactory.h> #include <Storages/StorageFactory.h>
#include <Storages/StorageURL.h> #include <Storages/StorageURL.h>

View File

@ -1,7 +1,7 @@
#include <unistd.h> #include <unistd.h>
#include <iostream> #include <iostream>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/createFile.h> #include <Common/FileSystemHelpers.h>
#include <filesystem> #include <filesystem>
namespace fs = std::filesystem; namespace fs = std::filesystem;