Merge branch 'master' into fix_database_replicated_enqueue_race

This commit is contained in:
Alexander Tokmakov 2021-06-01 11:49:29 +03:00
commit fe482109da
218 changed files with 1496 additions and 1358 deletions

View File

@ -10,12 +10,26 @@ assignees: ''
You have to provide the following information whenever possible.
**Describe the bug**
A clear and concise description of what works not as it is supposed to.
**Does it reproduce on recent release?**
[The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv)
**Enable crash reporting**
If possible, change "enabled" to true in "send_crash_reports" section in `config.xml`:
```
<send_crash_reports>
<!-- Changing <enabled> to true allows sending crash reports to -->
<!-- the ClickHouse core developers team via Sentry https://sentry.io -->
<enabled>false</enabled>
```
**How to reproduce**
* Which ClickHouse server version to use
* Which interface to use, if matters
* Non-default settings, if any
@ -24,10 +38,13 @@ A clear and concise description of what works not as it is supposed to.
* Queries to run that lead to unexpected result
**Expected behavior**
A clear and concise description of what you expected to happen.
**Error message and/or stacktrace**
If applicable, add screenshots to help explain your problem.
**Additional context**
Add any other context about the problem here.

View File

@ -26,8 +26,6 @@
#include <Poco/Observer.h>
#include <Poco/AutoPtr.h>
#include <Poco/PatternFormatter.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Message.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
@ -59,6 +57,7 @@
#include <Common/getExecutablePath.h>
#include <Common/getHashOfLoadedBinary.h>
#include <Common/Elf.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
# include <Common/config_version.h>
@ -70,6 +69,7 @@
#endif
#include <ucontext.h>
namespace fs = std::filesystem;
DB::PipeFDs signal_pipe;
@ -437,11 +437,11 @@ static void sanitizerDeathCallback()
static std::string createDirectory(const std::string & file)
{
auto path = Poco::Path(file).makeParent();
if (path.toString().empty())
fs::path path = fs::path(file).parent_path();
if (path.empty())
return "";
Poco::File(path).createDirectories();
return path.toString();
fs::create_directories(path);
return path;
};
@ -449,7 +449,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path
{
try
{
Poco::File(path).createDirectories();
fs::create_directories(path);
return true;
}
catch (...)
@ -470,7 +470,7 @@ void BaseDaemon::reloadConfiguration()
*/
config_path = config().getString("config-file", getDefaultConfigFileName());
DB::ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString());
config_processor.setConfigPath(fs::path(config_path).parent_path());
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (last_configuration != nullptr)
@ -524,18 +524,20 @@ std::string BaseDaemon::getDefaultConfigFileName() const
void BaseDaemon::closeFDs()
{
#if defined(OS_FREEBSD) || defined(OS_DARWIN)
Poco::File proc_path{"/dev/fd"};
fs::path proc_path{"/dev/fd"};
#else
Poco::File proc_path{"/proc/self/fd"};
fs::path proc_path{"/proc/self/fd"};
#endif
if (proc_path.isDirectory()) /// Hooray, proc exists
if (fs::is_directory(proc_path)) /// Hooray, proc exists
{
std::vector<std::string> fds;
/// in /proc/self/fd directory filenames are numeric file descriptors
proc_path.list(fds);
for (const auto & fd_str : fds)
/// in /proc/self/fd directory filenames are numeric file descriptors.
/// Iterate directory separately from closing fds to avoid closing iterated directory fd.
std::vector<int> fds;
for (const auto & path : fs::directory_iterator(proc_path))
fds.push_back(DB::parse<int>(path.path().filename()));
for (const auto & fd : fds)
{
int fd = DB::parse<int>(fd_str);
if (fd > 2 && fd != signal_pipe.fds_rw[0] && fd != signal_pipe.fds_rw[1])
::close(fd);
}
@ -597,7 +599,7 @@ void BaseDaemon::initialize(Application & self)
{
/** When creating pid file and looking for config, will search for paths relative to the working path of the program when started.
*/
std::string path = Poco::Path(config().getString("application.path")).setFileName("").toString();
std::string path = fs::path(config().getString("application.path")).replace_filename("");
if (0 != chdir(path.c_str()))
throw Poco::Exception("Cannot change directory to " + path);
}
@ -645,7 +647,7 @@ void BaseDaemon::initialize(Application & self)
std::string log_path = config().getString("logger.log", "");
if (!log_path.empty())
log_path = Poco::Path(log_path).setFileName("").toString();
log_path = fs::path(log_path).replace_filename("");
/** Redirect stdout, stderr to separate files in the log directory (or in the specified file).
* Some libraries write to stderr in case of errors in debug mode,
@ -708,8 +710,7 @@ void BaseDaemon::initialize(Application & self)
tryCreateDirectories(&logger(), core_path);
Poco::File cores = core_path;
if (!(cores.exists() && cores.isDirectory()))
if (!(fs::exists(core_path) && fs::is_directory(core_path)))
{
core_path = !log_path.empty() ? log_path : "/opt/";
tryCreateDirectories(&logger(), core_path);

View File

@ -1,6 +1,5 @@
#include <daemon/SentryWriter.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
@ -25,6 +24,7 @@
# include <stdio.h>
# include <filesystem>
namespace fs = std::filesystem;
namespace
{
@ -53,8 +53,7 @@ void setExtras()
sentry_set_extra("physical_cpu_cores", sentry_value_new_int32(getNumberOfPhysicalCPUCores()));
if (!server_data_path.empty())
sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(
Poco::File(server_data_path).freeSpace()).c_str()));
sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(fs::space(server_data_path).free).c_str()));
}
void sentry_logger(sentry_level_e level, const char * message, va_list args, void *)
@ -110,12 +109,12 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config)
if (enabled)
{
server_data_path = config.getString("path", "");
const std::filesystem::path & default_tmp_path = std::filesystem::path(config.getString("tmp_path", Poco::Path::temp())) / "sentry";
const std::filesystem::path & default_tmp_path = fs::path(config.getString("tmp_path", fs::temp_directory_path())) / "sentry";
const std::string & endpoint
= config.getString("send_crash_reports.endpoint");
const std::string & temp_folder_path
= config.getString("send_crash_reports.tmp_path", default_tmp_path);
Poco::File(temp_folder_path).createDirectories();
fs::create_directories(temp_folder_path);
sentry_options_t * options = sentry_options_new(); /// will be freed by sentry_init or sentry_shutdown
sentry_options_set_release(options, VERSION_STRING_SHORT);

View File

@ -6,10 +6,11 @@
#include "OwnFormattingChannel.h"
#include "OwnPatternFormatter.h"
#include <Poco/ConsoleChannel.h>
#include <Poco/File.h>
#include <Poco/Logger.h>
#include <Poco/Net/RemoteSyslogChannel.h>
#include <Poco/Path.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -20,11 +21,11 @@ namespace DB
// TODO: move to libcommon
static std::string createDirectory(const std::string & file)
{
auto path = Poco::Path(file).makeParent();
if (path.toString().empty())
auto path = fs::path(file).parent_path();
if (path.empty())
return "";
Poco::File(path).createDirectories();
return path.toString();
fs::create_directories(path);
return path;
};
void Loggers::setTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
@ -70,7 +71,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
// Set up two channel chains.
log_file = new Poco::FileChannel;
log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString());
log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(log_path));
log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));
@ -102,7 +103,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
std::cerr << "Logging errors to " << errorlog_path << std::endl;
error_log_file = new Poco::FileChannel;
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString());
error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(errorlog_path));
error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M"));
error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number");
error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true"));

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit a491c27b33109a842d577c0f7ac5f5f218859181
Subproject commit 53734f420f166e1ca2732dec8998469bfbb7731d

View File

@ -18,10 +18,10 @@ When using multiple `quantile*` functions with different levels in a query, the
**Syntax**
``` sql
quantileTDigest(level)(expr)
quantileTDigestWeighted(level)(expr, weight)
```
Alias: `medianTDigest`.
Alias: `medianTDigestWeighted`.
**Arguments**

View File

@ -8,7 +8,6 @@
#include <iomanip>
#include <random>
#include <pcg_random.hpp>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
@ -36,7 +35,9 @@
#include <Common/Config/configReadClient.h>
#include <Common/TerminalSize.h>
#include <Common/StudentTTest.h>
#include <filesystem>
namespace fs = std::filesystem;
/** A tool for evaluating ClickHouse performance.
* The tool emulates a case with fixed amount of simultaneously executing queries.
@ -119,8 +120,8 @@ public:
int main(const std::vector<std::string> &) override
{
if (!json_path.empty() && Poco::File(json_path).exists()) /// Clear file with previous results
Poco::File(json_path).remove();
if (!json_path.empty() && fs::exists(json_path)) /// Clear file with previous results
fs::remove(json_path);
readQueries();
runBenchmark();
@ -159,7 +160,7 @@ private:
bool print_stacktrace;
const Settings & settings;
SharedContextHolder shared_context;
ContextPtr global_context;
ContextMutablePtr global_context;
QueryProcessingStage::Enum query_processing_stage;
/// Don't execute new queries after timelimit or SIGINT or exception

View File

@ -25,7 +25,6 @@
#include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <common/find_symbols.h>
#include <common/LineReader.h>
@ -87,6 +86,8 @@
#include <Common/TerminalSize.h>
#include <Common/UTF8Helpers.h>
#include <Common/ProgressBar.h>
#include <filesystem>
#include <Common/filesystemHelpers.h>
#if !defined(ARCADIA_BUILD)
# include <Common/config_version.h>
@ -96,6 +97,7 @@
#pragma GCC optimize("-fno-var-tracking-assignments")
#endif
namespace fs = std::filesystem;
namespace DB
{
@ -181,7 +183,7 @@ private:
bool has_vertical_output_suffix = false; /// Is \G present at the end of the query string?
SharedContextHolder shared_context = Context::createShared();
ContextPtr context = Context::createGlobal(shared_context.get());
ContextMutablePtr context = Context::createGlobal(shared_context.get());
/// Buffer that reads from stdin in batch mode.
ReadBufferFromFileDescriptor std_in{STDIN_FILENO};
@ -276,7 +278,7 @@ private:
/// Set path for format schema files
if (config().has("format_schema_path"))
context->setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString());
context->setFormatSchemaPath(fs::weakly_canonical(config().getString("format_schema_path")));
/// Initialize query_id_formats if any
if (config().has("query_id_formats"))
@ -633,8 +635,8 @@ private:
history_file = home_path + "/.clickhouse-client-history";
}
if (!history_file.empty() && !Poco::File(history_file).exists())
Poco::File(history_file).createFile();
if (!history_file.empty() && !fs::exists(history_file))
FS::createFile(history_file);
LineReader::Patterns query_extenders = {"\\"};
LineReader::Patterns query_delimiters = {";", "\\G"};

View File

@ -12,14 +12,14 @@
namespace DB
{
class ClusterCopier : WithContext
class ClusterCopier : WithMutableContext
{
public:
ClusterCopier(const String & task_path_,
const String & host_id_,
const String & proxy_database_name_,
ContextPtr context_)
: WithContext(context_),
ContextMutablePtr context_)
: WithMutableContext(context_),
task_zookeeper_path(task_path_),
host_id(host_id_),
working_database_name(proxy_database_name_),

View File

@ -5,7 +5,9 @@
#include <Formats/registerFormats.h>
#include <ext/scope_guard_safe.h>
#include <unistd.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -26,7 +28,7 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0);
if (config().has("move-fault-probability"))
move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0);
base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current();
base_dir = (config().has("base-dir")) ? config().getString("base-dir") : fs::current_path().string();
if (config().has("experimental-use-sample-offset"))
@ -38,18 +40,18 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self)
process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid);
host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id;
process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString();
Poco::File(process_path).createDirectories();
process_path = fs::weakly_canonical(fs::path(base_dir) / ("clickhouse-copier_" + process_id));
fs::create_directories(process_path);
/// Override variables for BaseDaemon
if (config().has("log-level"))
config().setString("logger.level", config().getString("log-level"));
if (config().has("base-dir") || !config().has("logger.log"))
config().setString("logger.log", process_path + "/log.log");
config().setString("logger.log", fs::path(process_path) / "log.log");
if (config().has("base-dir") || !config().has("logger.errorlog"))
config().setString("logger.errorlog", process_path + "/log.err.log");
config().setString("logger.errorlog", fs::path(process_path) / "log.err.log");
Base::initialize(self);
}

View File

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

View File

@ -288,7 +288,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
bool is_symlink = fs::is_symlink(symlink_path);
fs::path points_to;
if (is_symlink)
points_to = fs::absolute(fs::read_symlink(symlink_path));
points_to = fs::weakly_canonical(fs::read_symlink(symlink_path));
if (is_symlink && points_to == main_bin_path)
{

View File

@ -32,7 +32,7 @@ public:
return BaseDaemon::logger();
}
ContextPtr context() const override
ContextMutablePtr context() const override
{
return global_context;
}
@ -58,7 +58,7 @@ protected:
std::string getDefaultConfigFileName() const override;
private:
ContextPtr global_context;
ContextMutablePtr global_context;
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;

View File

@ -42,9 +42,9 @@
#include <common/argsToConfig.h>
#include <Common/TerminalSize.h>
#include <Common/randomSeed.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -72,11 +72,11 @@ void LocalServer::initialize(Poco::Util::Application & self)
Poco::Util::Application::initialize(self);
/// Load config files if exists
if (config().has("config-file") || Poco::File("config.xml").exists())
if (config().has("config-file") || fs::exists("config.xml"))
{
const auto config_path = config().getString("config-file", "config.xml");
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(Poco::Path(config_path).makeParent().toString());
config_processor.setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config_processor.savePreprocessedConfig(loaded_config, loaded_config.configuration->getString("path", "."));
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
@ -100,7 +100,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
}
}
void LocalServer::applyCmdSettings(ContextPtr context)
void LocalServer::applyCmdSettings(ContextMutablePtr context)
{
context->applySettingsChanges(cmd_settings.changes());
}
@ -287,8 +287,8 @@ try
status.emplace(path + "status", StatusFile::write_full_info);
LOG_DEBUG(log, "Loading metadata from {}", path);
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
loadMetadataSystem(global_context);
attachSystemTables(global_context);
loadMetadata(global_context);
@ -479,7 +479,7 @@ void LocalServer::setupUsers()
{
ConfigurationPtr users_config;
if (config().has("users_config") || config().has("config-file") || Poco::File("config.xml").exists())
if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml"))
{
const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml"));
ConfigProcessor config_processor(users_config_path);
@ -645,7 +645,7 @@ void LocalServer::init(int argc, char ** argv)
argsToConfig(arguments, config(), 100);
}
void LocalServer::applyCmdOptions(ContextPtr context)
void LocalServer::applyCmdOptions(ContextMutablePtr context)
{
context->setDefaultFormat(config().getString("output-format", config().getString("format", "TSV")));
applyCmdSettings(context);

View File

@ -36,8 +36,8 @@ private:
std::string getInitialCreateTableQuery();
void tryInitPath();
void applyCmdOptions(ContextPtr context);
void applyCmdSettings(ContextPtr context);
void applyCmdOptions(ContextMutablePtr context);
void applyCmdSettings(ContextMutablePtr context);
void processQueries();
void setupUsers();
void cleanup();
@ -45,7 +45,7 @@ private:
protected:
SharedContextHolder shared_context;
ContextPtr global_context;
ContextMutablePtr global_context;
/// Settings specified via command line args
Settings cmd_settings;

View File

@ -1133,7 +1133,7 @@ try
}
SharedContextHolder shared_context = Context::createShared();
ContextPtr context = Context::createGlobal(shared_context.get());
auto context = Context::createGlobal(shared_context.get());
context->makeGlobalContext();
ReadBufferFromFileDescriptor file_in(STDIN_FILENO);

View File

@ -74,6 +74,7 @@
#include <Server/PostgreSQLHandlerFactory.h>
#include <Server/ProtocolServerAdapter.h>
#include <Server/HTTP/HTTPServer.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
@ -117,6 +118,8 @@ namespace CurrentMetrics
extern const Metric MaxDDLEntryID;
}
namespace fs = std::filesystem;
#if USE_JEMALLOC
static bool jemallocOptionEnabled(const char *name)
{
@ -183,19 +186,19 @@ void setupTmpPath(Poco::Logger * log, const std::string & path)
{
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
Poco::File(path).createDirectories();
fs::create_directories(path);
/// Clearing old temporary files.
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
fs::directory_iterator dir_end;
for (fs::directory_iterator it(path); it != dir_end; ++it)
{
if (it->isFile() && startsWith(it.name(), "tmp"))
if (it->is_regular_file() && startsWith(it->path().filename(), "tmp"))
{
LOG_DEBUG(log, "Removing old temporary file {}", it->path());
it->remove();
LOG_DEBUG(log, "Removing old temporary file {}", it->path().string());
fs::remove(it->path());
}
else
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path());
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path().string());
}
}
@ -678,37 +681,38 @@ int Server::main(const std::vector<std::string> & /*args*/)
* Examples: do repair of local data; clone all replicated tables from replica.
*/
{
Poco::File(path + "flags/").createDirectories();
global_context->setFlagsPath(path + "flags/");
auto flags_path = fs::path(path) / "flags/";
fs::create_directories(flags_path);
global_context->setFlagsPath(flags_path);
}
/** Directory with user provided files that are usable by 'file' table function.
*/
{
std::string user_files_path = config().getString("user_files_path", path + "user_files/");
std::string user_files_path = config().getString("user_files_path", fs::path(path) / "user_files/");
global_context->setUserFilesPath(user_files_path);
Poco::File(user_files_path).createDirectories();
fs::create_directories(user_files_path);
}
{
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", path + "dictionaries_lib/");
std::string dictionaries_lib_path = config().getString("dictionaries_lib_path", fs::path(path) / "dictionaries_lib/");
global_context->setDictionariesLibPath(dictionaries_lib_path);
Poco::File(dictionaries_lib_path).createDirectories();
fs::create_directories(dictionaries_lib_path);
}
/// top_level_domains_lists
{
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/") + "/";
TLDListsHolder::getInstance().parseConfig(top_level_domains_path, config());
const std::string & top_level_domains_path = config().getString("top_level_domains_path", fs::path(path) / "top_level_domains/");
TLDListsHolder::getInstance().parseConfig(fs::path(top_level_domains_path) / "", config());
}
{
Poco::File(path + "data/").createDirectories();
Poco::File(path + "metadata/").createDirectories();
fs::create_directories(fs::path(path) / "data/");
fs::create_directories(fs::path(path) / "metadata/");
/// Directory with metadata of tables, which was marked as dropped by Atomic database
Poco::File(path + "metadata_dropped/").createDirectories();
fs::create_directories(fs::path(path) / "metadata_dropped/");
}
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
@ -891,9 +895,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
#endif
/// Set path for format schema files
auto format_schema_path = Poco::File(config().getString("format_schema_path", path + "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path.path());
format_schema_path.createDirectories();
fs::path format_schema_path(config().getString("format_schema_path", fs::path(path) / "format_schemas/"));
global_context->setFormatSchemaPath(format_schema_path);
fs::create_directories(format_schema_path);
/// Check sanity of MergeTreeSettings on server startup
global_context->getMergeTreeSettings().sanityCheck(settings);

View File

@ -40,7 +40,7 @@ public:
return BaseDaemon::logger();
}
ContextPtr context() const override
ContextMutablePtr context() const override
{
return global_context;
}
@ -64,7 +64,7 @@ protected:
std::string getDefaultCorePath() const override;
private:
ContextPtr global_context;
ContextMutablePtr global_context;
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
using CreateServerFunc = std::function<void(UInt16)>;

View File

@ -7,7 +7,9 @@
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm_ext/erase.hpp>
#include <boost/algorithm/string/predicate.hpp>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -198,9 +200,9 @@ inline String AllowedClientHosts::IPSubnet::toString() const
if (isMaskAllBitsOne())
return prefix.toString();
else if (IPAddress{prefix_length, mask.family()} == mask)
return prefix.toString() + "/" + std::to_string(prefix_length);
return fs::path(prefix.toString()) / std::to_string(prefix_length);
else
return prefix.toString() + "/" + mask.toString();
return fs::path(prefix.toString()) / mask.toString();
}
inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const

View File

@ -51,7 +51,7 @@ private:
void setAllNoLock(const std::vector<std::pair<UUID, AccessEntityPtr>> & all_entities, Notifications & notifications);
void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const;
mutable std::mutex mutex;
mutable std::recursive_mutex mutex;
std::unordered_map<UUID, Entry> entries_by_id; /// We want to search entries both by ID and by the pair of name and type.
std::unordered_map<String, Entry *> entries_by_name_and_type[static_cast<size_t>(EntityType::MAX)];
mutable std::list<OnChangedHandler> handlers_by_type[static_cast<size_t>(EntityType::MAX)];

View File

@ -3,9 +3,10 @@
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ReadHelpers.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/URI.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -87,10 +88,10 @@ std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand() const
const auto & config = getConfig();
/// Path to executable folder
Poco::Path path{config.getString("application.dir", "/usr/bin")};
fs::path path(config.getString("application.dir", "/usr/bin"));
std::vector<std::string> cmd_args;
path.setFileName(serviceFileName());
path /= serviceFileName();
cmd_args.push_back("--http-port");
cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", getDefaultPort())));
@ -126,7 +127,7 @@ std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand() const
LOG_TRACE(getLog(), "Starting {}", serviceAlias());
return ShellCommand::executeDirect(path.toString(), cmd_args, ShellCommandDestructorStrategy(true));
return ShellCommand::executeDirect(path.string(), cmd_args, ShellCommandDestructorStrategy(true));
}
}

View File

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

View File

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

View File

@ -62,7 +62,7 @@ static std::string numberFromHost(const std::string & s)
bool ConfigProcessor::isPreprocessedFile(const std::string & path)
{
return endsWith(Poco::Path(path).getBaseName(), PREPROCESSED_SUFFIX);
return endsWith(fs::path(path).stem(), PREPROCESSED_SUFFIX);
}
@ -416,34 +416,32 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
{
Files files;
Poco::Path merge_dir_path(config_path);
fs::path merge_dir_path(config_path);
std::set<std::string> merge_dirs;
/// Add path_to_config/config_name.d dir
merge_dir_path.setExtension("d");
merge_dirs.insert(merge_dir_path.toString());
merge_dir_path.replace_extension("d");
merge_dirs.insert(merge_dir_path);
/// Add path_to_config/conf.d dir
merge_dir_path.setBaseName("conf");
merge_dirs.insert(merge_dir_path.toString());
merge_dir_path.replace_filename("conf.d");
merge_dirs.insert(merge_dir_path);
for (const std::string & merge_dir_name : merge_dirs)
{
Poco::File merge_dir(merge_dir_name);
if (!merge_dir.exists() || !merge_dir.isDirectory())
if (!fs::exists(merge_dir_name) || !fs::is_directory(merge_dir_name))
continue;
for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it)
for (fs::directory_iterator it(merge_dir_name); it != fs::directory_iterator(); ++it)
{
Poco::File & file = *it;
Poco::Path path(file.path());
std::string extension = path.getExtension();
std::string base_name = path.getBaseName();
fs::path path(it->path());
std::string extension = path.extension();
std::string base_name = path.stem();
// Skip non-config and temporary files
if (file.isFile() && (extension == "xml" || extension == "conf" || extension == "yaml" || extension == "yml") && !startsWith(base_name, "."))
{
files.push_back(file.path());
}
if (fs::is_regular_file(path)
&& (extension == ".xml" || extension == ".conf" || extension == ".yaml" || extension == ".yml")
&& !startsWith(base_name, "."))
files.push_back(it->path());
}
}
@ -548,7 +546,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
else
{
std::string default_path = "/etc/metrika.xml";
if (Poco::File(default_path).exists())
if (fs::exists(default_path))
include_from_path = default_path;
}
if (!include_from_path.empty())
@ -660,11 +658,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
if (!loaded_config.configuration->has("path"))
{
// Will use current directory
auto parent_path = Poco::Path(loaded_config.config_path).makeParent();
preprocessed_dir = parent_path.toString();
Poco::Path poco_new_path(new_path);
poco_new_path.setBaseName(poco_new_path.getBaseName() + PREPROCESSED_SUFFIX);
new_path = poco_new_path.toString();
fs::path parent_path = fs::path(loaded_config.config_path).parent_path();
preprocessed_dir = parent_path.string();
fs::path fs_new_path(new_path);
fs_new_path.replace_filename(fs_new_path.stem().string() + PREPROCESSED_SUFFIX + fs_new_path.extension().string());
new_path = fs_new_path.string();
}
else
{
@ -679,9 +677,9 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
}
preprocessed_path = (fs::path(preprocessed_dir) / fs::path(new_path)).string();
auto preprocessed_path_parent = Poco::Path(preprocessed_path).makeParent();
if (!preprocessed_path_parent.toString().empty())
Poco::File(preprocessed_path_parent).createDirectories();
auto preprocessed_path_parent = fs::path(preprocessed_path).parent_path();
if (!preprocessed_path_parent.empty())
fs::create_directories(preprocessed_path_parent);
}
DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml);
LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path);

View File

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

View File

@ -1,12 +1,15 @@
#include "ConfigReloader.h"
#include <Poco/Util/Application.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <Common/setThreadName.h>
#include "ConfigProcessor.h"
#include <filesystem>
#include <Common/filesystemHelpers.h>
namespace fs = std::filesystem;
namespace DB
{
@ -167,8 +170,8 @@ struct ConfigReloader::FileWithTimestamp
void ConfigReloader::FilesChangesTracker::addIfExists(const std::string & path_to_add)
{
if (!path_to_add.empty() && Poco::File(path_to_add).exists())
files.emplace(path_to_add, Poco::File(path_to_add).getLastModified().epochTime());
if (!path_to_add.empty() && fs::exists(path_to_add))
files.emplace(path_to_add, FS::getModificationTime(path_to_add));
}
bool ConfigReloader::FilesChangesTracker::isDifferOrNewerThan(const FilesChangesTracker & rhs)

View File

@ -1,8 +1,10 @@
#include "configReadClient.h"
#include <Poco/Util/LayeredConfiguration.h>
#include <Poco/File.h>
#include "ConfigProcessor.h"
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -11,11 +13,11 @@ bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::stri
std::string config_path;
if (config.has("config-file"))
config_path = config.getString("config-file");
else if (Poco::File("./clickhouse-client.xml").exists())
else if (fs::exists("./clickhouse-client.xml"))
config_path = "./clickhouse-client.xml";
else if (!home_path.empty() && Poco::File(home_path + "/.clickhouse-client/config.xml").exists())
else if (!home_path.empty() && fs::exists(home_path + "/.clickhouse-client/config.xml"))
config_path = home_path + "/.clickhouse-client/config.xml";
else if (Poco::File("/etc/clickhouse-client/config.xml").exists())
else if (fs::exists("/etc/clickhouse-client/config.xml"))
config_path = "/etc/clickhouse-client/config.xml";
if (!config_path.empty())

View File

@ -7,7 +7,6 @@
#include <iostream>
#include <mutex>
#include <Poco/File.h>
#include <Poco/Exception.h>
#include <IO/ReadBufferFromFileDescriptor.h>
@ -59,7 +58,7 @@ public:
Int64 res = -1;
bool file_doesnt_exists = !Poco::File(path).exists();
bool file_doesnt_exists = !fs::exists(path);
if (file_doesnt_exists && !create_if_need)
{
throw Poco::Exception("File " + path + " does not exist. "
@ -138,7 +137,7 @@ public:
// Not thread-safe and not synchronized between processes.
void fixIfBroken(UInt64 value)
{
bool file_exists = Poco::File(path).exists();
bool file_exists = fs::exists(path);
int fd = ::open(path.c_str(), O_RDWR | O_CREAT | O_CLOEXEC, 0666);
if (-1 == fd)

View File

@ -87,7 +87,7 @@ public:
/// Initializes query with current thread as master thread in constructor, and detaches it in destructor
struct QueryScope
{
explicit QueryScope(ContextPtr query_context);
explicit QueryScope(ContextMutablePtr query_context);
~QueryScope();
void logPeakMemoryUsage();

View File

@ -553,6 +553,7 @@
M(583, ILLEGAL_PROJECTION) \
M(584, PROJECTION_NOT_USED) \
M(585, CANNOT_PARSE_YAML) \
M(586, CANNOT_CREATE_FILE) \
\
M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \

View File

@ -21,6 +21,8 @@
# include <Common/config_version.h>
#endif
namespace fs = std::filesystem;
namespace DB
{
@ -177,7 +179,7 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
tryLogCurrentExceptionImpl(logger, start_of_message);
}
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & msg)
static void getNoSpaceLeftInfoMessage(std::filesystem::path path, String & msg)
{
path = std::filesystem::absolute(path);
/// It's possible to get ENOSPC for non existent file (e.g. if there are no free inodes and creat() fails)
@ -264,22 +266,12 @@ static std::string getExtraExceptionInfo(const std::exception & e)
String msg;
try
{
if (const auto * file_exception = dynamic_cast<const Poco::FileException *>(&e))
if (const auto * file_exception = dynamic_cast<const fs::filesystem_error *>(&e))
{
if (file_exception->code() == ENOSPC)
{
/// See Poco::FileImpl::handleLastErrorImpl(...)
constexpr const char * expected_error_message = "no space left on device: ";
if (startsWith(file_exception->message(), expected_error_message))
{
String path = file_exception->message().substr(strlen(expected_error_message));
getNoSpaceLeftInfoMessage(path, msg);
}
else
{
msg += "\nCannot print extra info for Poco::Exception";
}
}
if (file_exception->code() == std::errc::no_space_on_device)
getNoSpaceLeftInfoMessage(file_exception->path1(), msg);
else
msg += "\nCannot print extra info for Poco::Exception";
}
else if (const auto * errno_exception = dynamic_cast<const DB::ErrnoException *>(&e))
{

View File

@ -1,10 +1,11 @@
#pragma once
#include <Poco/File.h>
#include <Poco/Timestamp.h>
#include <string>
#include <filesystem>
#include <Common/filesystemHelpers.h>
namespace fs = std::filesystem;
class FileUpdatesTracker
{
@ -31,6 +32,6 @@ public:
private:
Poco::Timestamp getLastModificationTime() const
{
return Poco::File(path).getLastModified();
return FS::getModificationTimestamp(path);
}
};

View File

@ -4,7 +4,6 @@
#include <fcntl.h>
#include <errno.h>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <common/errnoToString.h>
#include <Common/ClickHouseRevision.h>
@ -14,7 +13,9 @@
#include <IO/LimitReadBuffer.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/Operators.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -45,7 +46,7 @@ StatusFile::StatusFile(std::string path_, FillFunction fill_)
: path(std::move(path_)), fill(std::move(fill_))
{
/// If file already exists. NOTE Minor race condition.
if (Poco::File(path).exists())
if (fs::exists(path))
{
std::string contents;
{

View File

@ -81,7 +81,7 @@ template <typename Thread>
template <typename ReturnType>
ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::optional<uint64_t> wait_microseconds)
{
auto on_error = [&]
auto on_error = [&](const std::string & reason)
{
if constexpr (std::is_same_v<ReturnType, void>)
{
@ -91,7 +91,9 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
std::swap(exception, first_exception);
std::rethrow_exception(exception);
}
throw DB::Exception("Cannot schedule a task", DB::ErrorCodes::CANNOT_SCHEDULE_TASK);
throw DB::Exception(DB::ErrorCodes::CANNOT_SCHEDULE_TASK,
"Cannot schedule a task: {} (threads={}, jobs={})", reason,
threads.size(), scheduled_jobs);
}
else
return false;
@ -105,13 +107,13 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
if (wait_microseconds) /// Check for optional. Condition is true if the optional is set and the value is zero.
{
if (!job_finished.wait_for(lock, std::chrono::microseconds(*wait_microseconds), pred))
return on_error();
return on_error(fmt::format("no free thread (timeout={})", *wait_microseconds));
}
else
job_finished.wait(lock, pred);
if (shutdown)
return on_error();
return on_error("shutdown");
/// We must not to allocate any memory after we emplaced a job in a queue.
/// Because if an exception would be thrown, we won't notify a thread about job occurrence.
@ -126,7 +128,7 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
catch (...)
{
/// Most likely this is a std::bad_alloc exception
return on_error();
return on_error("cannot allocate thread slot");
}
try
@ -136,7 +138,7 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
catch (...)
{
threads.pop_front();
return on_error();
return on_error("cannot allocate thread");
}
}

View File

@ -4,6 +4,7 @@
#include "TestKeeper.h"
#include <functional>
#include <filesystem>
#include <pcg-random/pcg_random.hpp>
#include <common/logger_useful.h>
@ -17,6 +18,7 @@
#define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000
namespace fs = std::filesystem;
namespace DB
{
@ -612,7 +614,7 @@ void ZooKeeper::removeChildren(const std::string & path)
Coordination::Requests ops;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1));
ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1));
children.pop_back();
}
multi(ops);
@ -628,9 +630,9 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path, const String &
Coordination::Requests ops;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
removeChildrenRecursive(path + "/" + children.back());
removeChildrenRecursive(fs::path(path) / children.back());
if (likely(keep_child_node.empty() || keep_child_node != children.back()))
ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1));
ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1));
children.pop_back();
}
multi(ops);
@ -648,7 +650,7 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, const Strin
Strings batch;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
String child_path = path + "/" + children.back();
String child_path = fs::path(path) / children.back();
tryRemoveChildrenRecursive(child_path);
if (likely(keep_child_node.empty() || keep_child_node != children.back()))
{

View File

@ -6,10 +6,15 @@
# include <mntent.h>
#endif
#include <cerrno>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Version.h>
#include <Poco/Timestamp.h>
#include <filesystem>
#include <fcntl.h>
#include <unistd.h>
#include <sys/types.h>
#include <utime.h>
namespace fs = std::filesystem;
namespace DB
{
@ -20,6 +25,8 @@ namespace ErrorCodes
extern const int SYSTEM_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_STATVFS;
extern const int PATH_ACCESS_DENIED;
extern const int CANNOT_CREATE_FILE;
}
@ -38,17 +45,13 @@ struct statvfs getStatVFS(const String & path)
bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t data_size [[maybe_unused]])
{
#if POCO_VERSION >= 0x01090000
auto free_space = Poco::File(path).freeSpace();
auto free_space = fs::space(path).free;
return data_size <= free_space;
#else
return true;
#endif
}
std::unique_ptr<TemporaryFile> createTemporaryFile(const std::string & path)
{
Poco::File(path).createDirectories();
fs::create_directories(path);
/// NOTE: std::make_shared cannot use protected constructors
return std::make_unique<TemporaryFile>(path);
@ -128,3 +131,73 @@ bool pathStartsWith(const String & path, const String & prefix_path)
}
}
/// Copied from Poco::File
namespace FS
{
bool createFile(const std::string & path)
{
int n = open(path.c_str(), O_WRONLY | O_CREAT | O_EXCL, S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP | S_IROTH | S_IWOTH);
if (n != -1)
{
close(n);
return true;
}
DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE);
}
bool canRead(const std::string & path)
{
struct stat st;
if (stat(path.c_str(), &st) == 0)
{
if (st.st_uid == geteuid())
return (st.st_mode & S_IRUSR) != 0;
else if (st.st_gid == getegid())
return (st.st_mode & S_IRGRP) != 0;
else
return (st.st_mode & S_IROTH) != 0 || geteuid() == 0;
}
DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED);
}
bool canWrite(const std::string & path)
{
struct stat st;
if (stat(path.c_str(), &st) == 0)
{
if (st.st_uid == geteuid())
return (st.st_mode & S_IWUSR) != 0;
else if (st.st_gid == getegid())
return (st.st_mode & S_IWGRP) != 0;
else
return (st.st_mode & S_IWOTH) != 0 || geteuid() == 0;
}
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

@ -36,3 +36,15 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p
bool pathStartsWith(const String & path, const String & prefix_path);
}
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

@ -1,6 +1,6 @@
#include <Common/renameat2.h>
#include <Common/Exception.h>
#include <Poco/File.h>
#include <filesystem>
#if defined(linux) || defined(__linux) || defined(__linux__)
#include <unistd.h>
@ -10,6 +10,8 @@
#include <sys/utsname.h>
#endif
namespace fs = std::filesystem;
namespace DB
{
@ -93,9 +95,9 @@ static bool renameat2(const std::string &, const std::string &, int)
static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path)
{
/// NOTE it's unsafe
if (Poco::File{new_path}.exists())
if (fs::exists(new_path))
throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS);
Poco::File{old_path}.renameTo(new_path);
fs::rename(old_path, new_path);
}
/// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places

View File

@ -5,7 +5,7 @@
struct ContextHolder
{
DB::SharedContextHolder shared_context;
DB::ContextPtr context;
DB::ContextMutablePtr context;
ContextHolder()
: shared_context(DB::Context::createShared())

View File

@ -80,10 +80,10 @@ public:
/// Parsing of external table used when sending tables via http
/// The `handlePart` function will be called for each table passed,
/// so it's also necessary to call `clean` at the end of the `handlePart`.
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithContext
class ExternalTablesHandler : public HTMLForm::PartHandler, BaseExternalTable, WithMutableContext
{
public:
ExternalTablesHandler(ContextPtr context_, const Poco::Net::NameValueCollection & params_) : WithContext(context_), params(params_) {}
ExternalTablesHandler(ContextMutablePtr context_, const Poco::Net::NameValueCollection & params_) : WithMutableContext(context_), params(params_) {}
void handlePart(const Poco::Net::MessageHeader & header, ReadBuffer & stream) override;

View File

@ -73,7 +73,7 @@ Native41::Native41(const String & password, const String & auth_plugin_data)
}
void Native41::authenticate(
const String & user_name, std::optional<String> auth_response, ContextPtr context,
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool, const Poco::Net::SocketAddress & address)
{
if (!auth_response)
@ -136,7 +136,7 @@ Sha256Password::Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logg
}
void Sha256Password::authenticate(
const String & user_name, std::optional<String> auth_response, ContextPtr context,
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address)
{
if (!auth_response)

View File

@ -32,7 +32,7 @@ public:
virtual String getAuthPluginData() = 0;
virtual void authenticate(
const String & user_name, std::optional<String> auth_response, ContextPtr context,
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) = 0;
};
@ -49,7 +49,7 @@ public:
String getAuthPluginData() override { return scramble; }
void authenticate(
const String & user_name, std::optional<String> auth_response, ContextPtr context,
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool /* is_secure_connection */, const Poco::Net::SocketAddress & address) override;
private:
@ -69,7 +69,7 @@ public:
String getAuthPluginData() override { return scramble; }
void authenticate(
const String & user_name, std::optional<String> auth_response, ContextPtr context,
const String & user_name, std::optional<String> auth_response, ContextMutablePtr context,
std::shared_ptr<PacketEndpoint> packet_endpoint, bool is_secure_connection, const Poco::Net::SocketAddress & address) override;
private:

View File

@ -802,7 +802,7 @@ protected:
static void setPassword(
const String & user_name,
const String & password,
ContextPtr context,
ContextMutablePtr context,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address)
{
@ -821,7 +821,7 @@ protected:
public:
virtual void authenticate(
const String & user_name,
ContextPtr context,
ContextMutablePtr context,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) = 0;
@ -835,7 +835,7 @@ class NoPasswordAuth : public AuthenticationMethod
public:
void authenticate(
const String & user_name,
ContextPtr context,
ContextMutablePtr context,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) override
{
@ -853,7 +853,7 @@ class CleartextPasswordAuth : public AuthenticationMethod
public:
void authenticate(
const String & user_name,
ContextPtr context,
ContextMutablePtr context,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address) override
{
@ -896,7 +896,7 @@ public:
void authenticate(
const String & user_name,
ContextPtr context,
ContextMutablePtr context,
Messaging::MessageTransport & mt,
const Poco::Net::SocketAddress & address)
{

View File

@ -415,7 +415,6 @@ class IColumn;
M(Bool, materialize_ttl_after_modify, true, "Apply TTL for old data, after ALTER MODIFY TTL query", 0) \
M(String, function_implementation, "", "Choose function implementation for specific target or variant (experimental). If empty enable all of them.", 0) \
M(Bool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \
M(Bool, allow_experimental_bigint_types, false, "Allow Int128, Int256, UInt256 and Decimal256 types", 0) \
M(Bool, data_type_default_nullable, false, "Data types without NULL or NOT NULL will make Nullable", 0) \
M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \
M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \
@ -458,13 +457,6 @@ class IColumn;
M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \
M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \
M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializeMySQL. If equal to 0, this setting is disabled", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \
M(Bool, enable_debug_queries, false, "Enabled debug queries, but now is obsolete", 0) \
M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing. Will be removed after 2021-02-12", 0) \
M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
@ -473,11 +465,20 @@ class IColumn;
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \
\
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing. Will be removed after 2021-09-08", 0) \
M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \
M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \
M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \
M(Bool, allow_experimental_funnel_functions, true, "Enable experimental functions for funnel analysis.", 0) \
\
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete setting, does nothing.", 0) \
M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing.", 0) \
M(Bool, enable_debug_queries, false, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_database_atomic, true, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_funnel_functions, true, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_bigint_types, true, "Obsolete setting, does nothing.", 0) \
M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \
/** The section above is for obsolete settings. Do not add anything there. */
// End of COMMON_SETTINGS

View File

@ -54,8 +54,8 @@ private:
};
std::vector<ViewInfo> views;
ContextPtr select_context;
ContextPtr insert_context;
ContextMutablePtr select_context;
ContextMutablePtr insert_context;
void process(const Block & block, ViewInfo & view);
};

View File

@ -341,8 +341,6 @@ struct WhichDataType
constexpr bool isNullable() const { return idx == TypeIndex::Nullable; }
constexpr bool isFunction() const { return idx == TypeIndex::Function; }
constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; }
constexpr bool IsBigIntOrDeimal() const { return isInt128() || isUInt128() || isInt256() || isUInt256() || isDecimal256(); }
};
/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)

View File

@ -1,7 +1,5 @@
#include <Databases/DatabaseAtomic.h>
#include <Databases/DatabaseOnDisk.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromFile.h>
@ -13,6 +11,8 @@
#include <filesystem>
#include <Interpreters/DDLTask.h>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
@ -37,12 +37,12 @@ public:
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, const String & logger_name, ContextPtr context_)
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", logger_name, context_)
, path_to_table_symlinks(getContext()->getPath() + "data/" + escapeForFileName(name_) + "/")
, path_to_metadata_symlink(getContext()->getPath() + "metadata/" + escapeForFileName(name_))
, path_to_table_symlinks(fs::path(getContext()->getPath()) / "data" / escapeForFileName(name_) / "")
, path_to_metadata_symlink(fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(name_))
, db_uuid(uuid)
{
assert(db_uuid != UUIDHelpers::Nil);
Poco::File(path_to_table_symlinks).createDirectories();
fs::create_directories(path_to_table_symlinks);
tryCreateMetadataSymlink();
}
@ -73,14 +73,14 @@ void DatabaseAtomic::drop(ContextPtr)
assert(tables.empty());
try
{
Poco::File(path_to_metadata_symlink).remove();
Poco::File(path_to_table_symlinks).remove(true);
fs::remove(path_to_metadata_symlink);
fs::remove_all(path_to_table_symlinks);
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
Poco::File(getMetadataPath()).remove(true);
fs::remove_all(getMetadataPath());
}
void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path)
@ -132,8 +132,8 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na
/// (it's more likely to lost connection, than to fail before applying local changes).
/// TODO better detection and recovery
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped
DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw
fs::rename(table_metadata_path, table_metadata_path_drop); /// Mark table as dropped
DatabaseOrdinary::detachTableUnlocked(table_name, lock); /// Should never throw
table_name_to_path.erase(table_name);
}
@ -316,7 +316,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
}
catch (...)
{
Poco::File(table_metadata_tmp_path).remove();
fs::remove(table_metadata_tmp_path);
if (locked_uuid)
DatabaseCatalog::instance().removeUUIDMappingFinally(query.uuid);
throw;
@ -416,11 +416,11 @@ UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
return UUIDHelpers::Nil;
}
void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
void DatabaseAtomic::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach)
{
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
if (has_force_restore_data_flag)
Poco::File(path_to_table_symlinks).remove(true);
fs::remove_all(path_to_table_symlinks);
DatabaseOrdinary::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach);
@ -432,7 +432,7 @@ void DatabaseAtomic::loadStoredObjects(ContextPtr local_context, bool has_force_
table_names = table_name_to_path;
}
Poco::File(path_to_table_symlinks).createDirectories();
fs::create_directories(path_to_table_symlinks);
for (const auto & table : table_names)
tryCreateSymlink(table.first, table.second, true);
}
@ -443,9 +443,9 @@ void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String &
try
{
String link = path_to_table_symlinks + escapeForFileName(table_name);
Poco::File data = Poco::Path(getContext()->getPath()).makeAbsolute().toString() + actual_data_path;
if (!if_data_path_exist || data.exists())
data.linkTo(link, Poco::File::LINK_SYMBOLIC);
fs::path data = fs::canonical(getContext()->getPath()) / actual_data_path;
if (!if_data_path_exist || fs::exists(data))
fs::create_directory_symlink(data, link);
}
catch (...)
{
@ -458,7 +458,7 @@ void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
try
{
String path = path_to_table_symlinks + escapeForFileName(table_name);
Poco::File{path}.remove();
fs::remove(path);
}
catch (...)
{
@ -471,17 +471,17 @@ void DatabaseAtomic::tryCreateMetadataSymlink()
/// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse,
/// it's needed only for convenient introspection.
assert(path_to_metadata_symlink != metadata_path);
Poco::File metadata_symlink(path_to_metadata_symlink);
if (metadata_symlink.exists())
fs::path metadata_symlink(path_to_metadata_symlink);
if (fs::exists(metadata_symlink))
{
if (!metadata_symlink.isLink())
if (!fs::is_symlink(metadata_symlink))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink);
}
else
{
try
{
Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC);
fs::create_directory_symlink(metadata_path, path_to_metadata_symlink);
}
catch (...)
{
@ -495,7 +495,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name)
/// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard
try
{
Poco::File(path_to_metadata_symlink).remove();
fs::remove(path_to_metadata_symlink);
}
catch (...)
{
@ -526,7 +526,7 @@ void DatabaseAtomic::renameDatabase(const String & new_name)
path_to_table_symlinks = getContext()->getPath() + "data/" + new_name_escaped + "/";
}
Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks);
fs::rename(old_path_to_table_symlinks, path_to_table_symlinks);
tryCreateMetadataSymlink();
}

View File

@ -47,7 +47,7 @@ public:
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
/// Atomic database cannot be detached if there is detached table which still in use
void assertCanBeDetached(bool cleanup) override;

View File

@ -11,10 +11,9 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/formatAST.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
@ -40,6 +39,8 @@
#include <Storages/PostgreSQL/PoolWithFailover.h>
#endif
namespace fs = std::filesystem;
namespace DB
{
@ -58,11 +59,12 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
try
{
/// Creates store/xxx/ for Atomic
Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories();
fs::create_directories(fs::path(metadata_path).parent_path());
/// Before 20.7 it's possible that .sql metadata file does not exist for some old database.
/// In this case Ordinary database is created on server startup if the corresponding metadata directory exists.
/// So we should remove metadata directory if database creation failed.
created = Poco::File(metadata_path).createDirectory();
created = fs::create_directory(metadata_path);
DatabasePtr impl = getImpl(create, metadata_path, context);
@ -74,11 +76,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m
}
catch (...)
{
Poco::File metadata_dir(metadata_path);
if (created && metadata_dir.exists())
metadata_dir.remove(true);
if (created && fs::exists(metadata_path))
fs::remove_all(metadata_path);
throw;
}
}

View File

@ -12,8 +12,9 @@
#include <common/logger_useful.h>
#include <ext/scope_guard_safe.h>
#include <iomanip>
#include <Poco/File.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -35,7 +36,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
void DatabaseLazy::loadStoredObjects(
ContextPtr local_context,
ContextMutablePtr local_context,
bool /* has_force_restore_data_flag */,
bool /*force_attach*/)
{
@ -43,8 +44,8 @@ void DatabaseLazy::loadStoredObjects(
{
const std::string table_name = file_name.substr(0, file_name.size() - 4);
auto detached_permanently_flag = Poco::File(getMetadataPath() + "/" + file_name + detached_suffix);
if (detached_permanently_flag.exists())
fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix);
if (fs::exists(detached_permanently_flag))
{
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
return;
@ -228,7 +229,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const
LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name));
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
const String table_metadata_path = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + ".sql");
try
{

View File

@ -27,7 +27,7 @@ public:
bool canContainDistributedTables() const override { return false; }
void loadStoredObjects(
ContextPtr context,
ContextMutablePtr context,
bool has_force_restore_data_flag, bool force_attach) override;
void createTable(

View File

@ -4,9 +4,9 @@
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/IStorage.h>
#include <Poco/File.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -42,9 +42,9 @@ void DatabaseMemory::dropTable(
try
{
table->drop();
Poco::File table_data_dir{getTableDataPath(table_name)};
if (table_data_dir.exists())
table_data_dir.remove(true);
fs::path table_data_dir{getTableDataPath(table_name)};
if (fs::exists(table_data_dir))
fs::remove_all(table_data_dir);
}
catch (...)
{

View File

@ -14,14 +14,14 @@
#include <Storages/StorageFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/escapeForFileName.h>
#include <common/logger_useful.h>
#include <Poco/DirectoryIterator.h>
#include <Databases/DatabaseOrdinary.h>
#include <Databases/DatabaseAtomic.h>
#include <Common/assert_cast.h>
#include <filesystem>
#include <Common/filesystemHelpers.h>
namespace fs = std::filesystem;
namespace DB
{
@ -45,7 +45,7 @@ std::pair<String, StoragePtr> createTableFromAST(
ASTCreateQuery ast_create_query,
const String & database_name,
const String & table_data_path_relative,
ContextPtr context,
ContextMutablePtr context,
bool has_force_restore_data_flag)
{
ast_create_query.attach = true;
@ -201,8 +201,8 @@ DatabaseOnDisk::DatabaseOnDisk(
, metadata_path(metadata_path_)
, data_path(data_path_)
{
Poco::File(local_context->getPath() + data_path).createDirectories();
Poco::File(metadata_path).createDirectories();
fs::create_directories(local_context->getPath() + data_path);
fs::create_directories(metadata_path);
}
@ -245,7 +245,7 @@ void DatabaseOnDisk::createTable(
if (!create.attach)
checkMetadataFilenameAvailability(table_name);
if (create.attach && Poco::File(table_metadata_path).exists())
if (create.attach && fs::exists(table_metadata_path))
{
ASTPtr ast_detached = parseQueryFromMetadata(log, local_context, table_metadata_path);
auto & create_detached = ast_detached->as<ASTCreateQuery &>();
@ -285,10 +285,10 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & ta
{
try
{
auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix);
fs::path detached_permanently_flag(table_metadata_path + detached_suffix);
if (detached_permanently_flag.exists())
detached_permanently_flag.remove();
if (fs::exists(detached_permanently_flag))
fs::remove(detached_permanently_flag);
}
catch (Exception & e)
{
@ -308,11 +308,11 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
/// If it was ATTACH query and file with table metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path);
fs::rename(table_metadata_tmp_path, table_metadata_path);
}
catch (...)
{
Poco::File(table_metadata_tmp_path).remove();
fs::remove(table_metadata_tmp_path);
throw;
}
}
@ -321,10 +321,10 @@ void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_nam
{
auto table = detachTable(table_name);
Poco::File detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix);
fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix);
try
{
detached_permanently_flag.createFile();
FS::createFile(detached_permanently_flag);
}
catch (Exception & e)
{
@ -350,25 +350,25 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na
bool renamed = false;
try
{
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop);
fs::rename(table_metadata_path, table_metadata_path_drop);
renamed = true;
table->drop();
table->is_dropped = true;
Poco::File table_data_dir{local_context->getPath() + table_data_path_relative};
if (table_data_dir.exists())
table_data_dir.remove(true);
fs::path table_data_dir(local_context->getPath() + table_data_path_relative);
if (fs::exists(table_data_dir))
fs::remove_all(table_data_dir);
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
attachTable(table_name, table, table_data_path_relative);
if (renamed)
Poco::File(table_metadata_path_drop).renameTo(table_metadata_path);
fs::rename(table_metadata_path_drop, table_metadata_path);
throw;
}
Poco::File(table_metadata_path_drop).remove();
fs::remove(table_metadata_path_drop);
}
void DatabaseOnDisk::checkMetadataFilenameAvailability(const String & to_table_name) const
@ -381,11 +381,11 @@ void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to
{
String table_metadata_path = getObjectMetadataPath(to_table_name);
if (Poco::File(table_metadata_path).exists())
if (fs::exists(table_metadata_path))
{
auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix);
fs::path detached_permanently_flag(table_metadata_path + detached_suffix);
if (detached_permanently_flag.exists())
if (fs::exists(detached_permanently_flag))
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(database_name), backQuote(to_table_name));
else
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(database_name), backQuote(to_table_name));
@ -463,7 +463,7 @@ void DatabaseOnDisk::renameTable(
/// Now table data are moved to new database, so we must add metadata and attach table to new database
to_database.createTable(local_context, to_table_name, table, attach_query);
Poco::File(table_metadata_path).remove();
fs::remove(table_metadata_path);
if (from_atomic_to_ordinary)
{
@ -528,8 +528,8 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
void DatabaseOnDisk::drop(ContextPtr local_context)
{
assert(tables.empty());
Poco::File(local_context->getPath() + getDataPath()).remove(false);
Poco::File(getMetadataPath()).remove(false);
fs::remove(local_context->getPath() + getDataPath());
fs::remove(getMetadataPath());
}
String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const
@ -540,10 +540,9 @@ String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const
time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_name) const
{
String table_metadata_path = getObjectMetadataPath(object_name);
Poco::File meta_file(table_metadata_path);
if (meta_file.exists())
return meta_file.getLastModified().epochTime();
if (fs::exists(table_metadata_path))
return FS::getModificationTime(table_metadata_path);
else
return static_cast<time_t>(0);
}
@ -555,56 +554,57 @@ void DatabaseOnDisk::iterateMetadataFiles(ContextPtr local_context, const Iterat
assert(getUUID() == UUIDHelpers::Nil);
static const char * tmp_drop_ext = ".sql.tmp_drop";
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
if (Poco::File(local_context->getPath() + getDataPath() + '/' + object_name).exists())
if (fs::exists(local_context->getPath() + getDataPath() + '/' + object_name))
{
Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql");
fs::rename(getMetadataPath() + file_name, getMetadataPath() + object_name + ".sql");
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
process_metadata_file(object_name + ".sql");
}
else
{
LOG_INFO(log, "Removing file {}", getMetadataPath() + file_name);
Poco::File(getMetadataPath() + file_name).remove();
fs::remove(getMetadataPath() + file_name);
}
};
/// Metadata files to load: name and flag for .tmp_drop files
std::set<std::pair<String, bool>> metadata_files;
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
fs::directory_iterator dir_end;
for (fs::directory_iterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
{
String file_name = dir_it->path().filename();
/// For '.svn', '.gitignore' directory and similar.
if (dir_it.name().at(0) == '.')
if (file_name.at(0) == '.')
continue;
/// There are .sql.bak files - skip them.
if (endsWith(dir_it.name(), ".sql.bak"))
if (endsWith(file_name, ".sql.bak"))
continue;
/// Permanently detached table flag
if (endsWith(dir_it.name(), ".sql.detached"))
if (endsWith(file_name, ".sql.detached"))
continue;
if (endsWith(dir_it.name(), ".sql.tmp_drop"))
if (endsWith(file_name, ".sql.tmp_drop"))
{
/// There are files that we tried to delete previously
metadata_files.emplace(dir_it.name(), false);
metadata_files.emplace(file_name, false);
}
else if (endsWith(dir_it.name(), ".sql.tmp"))
else if (endsWith(file_name, ".sql.tmp"))
{
/// There are files .sql.tmp - delete
LOG_INFO(log, "Removing file {}", dir_it->path());
Poco::File(dir_it->path()).remove();
LOG_INFO(log, "Removing file {}", dir_it->path().string());
fs::remove(dir_it->path());
}
else if (endsWith(dir_it.name(), ".sql"))
else if (endsWith(file_name, ".sql"))
{
/// The required files have names like `table_name.sql`
metadata_files.emplace(dir_it.name(), true);
metadata_files.emplace(file_name, true);
}
else
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(),
ErrorCodes::INCORRECT_FILE_NAME);
throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Incorrect file extension: {} in metadata directory {}", file_name, getMetadataPath());
}
/// Read and parse metadata in parallel
@ -651,7 +651,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
{
if (logger)
LOG_ERROR(logger, "File {} is empty. Removing.", metadata_file_path);
Poco::File(metadata_file_path).remove();
fs::remove(metadata_file_path);
return nullptr;
}
@ -670,8 +670,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(
auto & create = ast->as<ASTCreateQuery &>();
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
{
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
table_name = unescapeForFileName(table_name);
String table_name = unescapeForFileName(fs::path(metadata_file_path).stem());
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER && logger)
LOG_WARNING(

View File

@ -16,7 +16,7 @@ std::pair<String, StoragePtr> createTableFromAST(
ASTCreateQuery ast_create_query,
const String & database_name,
const String & table_data_path_relative,
ContextPtr context,
ContextMutablePtr context,
bool has_force_restore_data_flag);
/** Get the string with the table definition based on the CREATE query.

View File

@ -16,7 +16,6 @@
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Poco/DirectoryIterator.h>
#include <Common/Stopwatch.h>
#include <Common/ThreadPool.h>
#include <Common/escapeForFileName.h>
@ -35,7 +34,7 @@ static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
namespace
{
void tryAttachTable(
ContextPtr context,
ContextMutablePtr context,
const ASTCreateQuery & query,
DatabaseOrdinary & database,
const String & database_name,
@ -84,7 +83,7 @@ DatabaseOrdinary::DatabaseOrdinary(
{
}
void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/)
void DatabaseOrdinary::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool /*force_attach*/)
{
/** Tables load faster if they are loaded in sorted (by name) order.
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
@ -111,8 +110,7 @@ void DatabaseOrdinary::loadStoredObjects(ContextPtr local_context, bool has_forc
auto * create_query = ast->as<ASTCreateQuery>();
create_query->database = database_name;
auto detached_permanently_flag = Poco::File(full_path.string() + detached_suffix);
if (detached_permanently_flag.exists())
if (fs::exists(full_path.string() + detached_suffix))
{
/// FIXME: even if we don't load the table we can still mark the uuid of it as taken.
/// if (create_query->uuid != UUIDHelpers::Nil)
@ -281,11 +279,11 @@ void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_
try
{
/// rename atomically replaces the old file with the new one.
Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path);
fs::rename(table_metadata_tmp_path, table_metadata_path);
}
catch (...)
{
Poco::File(table_metadata_tmp_path).remove();
fs::remove(table_metadata_tmp_path);
throw;
}
}

View File

@ -20,7 +20,7 @@ public:
String getEngineName() const override { return "Ordinary"; }
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
void alterTable(
ContextPtr context,

View File

@ -212,7 +212,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach)
createDatabaseNodesInZooKeeper(current_zookeeper);
}
replica_path = zookeeper_path + "/replicas/" + getFullReplicaName();
replica_path = fs::path(zookeeper_path) / "replicas" / getFullReplicaName();
String replica_host_id;
if (current_zookeeper->tryGet(replica_path, replica_host_id))
@ -292,7 +292,7 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt
createEmptyLogEntry(current_zookeeper);
}
void DatabaseReplicated::loadStoredObjects(ContextPtr local_context, bool has_force_restore_data_flag, bool force_attach)
void DatabaseReplicated::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach)
{
tryConnectToZooKeeperAndInitDatabase(force_attach);

View File

@ -57,7 +57,7 @@ public:
void drop(ContextPtr /*context*/) override;
void loadStoredObjects(ContextPtr context, bool has_force_restore_data_flag, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr context, bool has_force_restore_data_flag, bool force_attach) override;
void shutdown() override;
friend struct DatabaseReplicatedTask;

View File

@ -2,6 +2,9 @@
#include <Databases/DatabaseReplicated.h>
#include <Interpreters/DDLTask.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -195,7 +198,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
}
}
UInt32 our_log_ptr = parse<UInt32>(current_zookeeper->get(database->replica_path + "/log_ptr"));
UInt32 our_log_ptr = parse<UInt32>(current_zookeeper->get(fs::path(database->replica_path) / "log_ptr"));
UInt32 entry_num = DatabaseReplicatedTask::getLogEntryNumber(entry_name);
if (entry_num <= our_log_ptr)
@ -204,13 +207,13 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
return {};
}
String entry_path = queue_dir + "/" + entry_name;
String entry_path = fs::path(queue_dir) / entry_name;
auto task = std::make_unique<DatabaseReplicatedTask>(entry_name, entry_path, database);
String initiator_name;
zkutil::EventPtr wait_committed_or_failed = std::make_shared<Poco::Event>();
String try_node_path = entry_path + "/try";
String try_node_path = fs::path(entry_path) / "try";
if (zookeeper->tryGet(try_node_path, initiator_name, nullptr, wait_committed_or_failed))
{
task->is_initial_query = initiator_name == task->host_id_str;
@ -242,7 +245,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
throw Coordination::Exception(code, try_node_path);
if (!zookeeper->exists(entry_path + "/committed"))
if (!zookeeper->exists(fs::path(entry_path) / "committed"))
{
out_reason = fmt::format("Entry {} was forcefully cancelled due to timeout", entry_name);
return {};
@ -251,7 +254,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
}
}
if (!zookeeper->exists(entry_path + "/committed"))
if (!zookeeper->exists(fs::path(entry_path) / "committed"))
{
out_reason = fmt::format("Entry {} hasn't been committed", entry_name);
return {};
@ -259,8 +262,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
if (task->is_initial_query)
{
assert(!zookeeper->exists(entry_path + "/try"));
assert(zookeeper->exists(entry_path + "/committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText()));
assert(!zookeeper->exists(fs::path(entry_path) / "try"));
assert(zookeeper->exists(fs::path(entry_path) / "committed") == (zookeeper->get(task->getFinishedNodePath()) == ExecutionStatus(0).serializeText()));
out_reason = fmt::format("Entry {} has been executed as initial query", entry_name);
return {};
}
@ -296,7 +299,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
bool DatabaseReplicatedDDLWorker::canRemoveQueueEntry(const String & entry_name, const Coordination::Stat &)
{
UInt32 entry_number = DDLTaskBase::getLogEntryNumber(entry_name);
UInt32 max_log_ptr = parse<UInt32>(getAndSetZooKeeper()->get(database->zookeeper_path + "/max_log_ptr"));
UInt32 max_log_ptr = parse<UInt32>(getAndSetZooKeeper()->get(fs::path(database->zookeeper_path) / "max_log_ptr"));
return entry_number + logs_to_keep < max_log_ptr;
}

View File

@ -122,7 +122,7 @@ public:
/// Load a set of existing tables.
/// You can call only once, right after the object is created.
virtual void loadStoredObjects(ContextPtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
virtual void loadStoredObjects(ContextMutablePtr /*context*/, bool /*has_force_restore_data_flag*/, bool /*force_attach*/ = false) {}
/// Check the existence of the table.
virtual bool isTableExist(const String & name, ContextPtr context) const = 0;

View File

@ -24,10 +24,10 @@
# include <Common/escapeForFileName.h>
# include <Common/parseAddress.h>
# include <Common/setThreadName.h>
# include <filesystem>
# include <Common/filesystemHelpers.h>
# include <Poco/DirectoryIterator.h>
# include <Poco/File.h>
namespace fs = std::filesystem;
namespace DB
{
@ -326,7 +326,7 @@ void DatabaseConnectionMySQL::shutdown()
void DatabaseConnectionMySQL::drop(ContextPtr /*context*/)
{
Poco::File(getMetadataPath()).remove(true);
fs::remove_all(getMetadataPath());
}
void DatabaseConnectionMySQL::cleanOutdatedTables()
@ -372,10 +372,10 @@ void DatabaseConnectionMySQL::attachTable(const String & table_name, const Stora
local_tables_cache[table_name].second = storage;
remove_or_detach_tables.erase(table_name);
Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);
if (remove_flag.exists())
remove_flag.remove();
if (fs::exists(remove_flag))
fs::remove(remove_flag);
}
StoragePtr DatabaseConnectionMySQL::detachTable(const String & table_name)
@ -399,17 +399,17 @@ String DatabaseConnectionMySQL::getMetadataPath() const
return metadata_path;
}
void DatabaseConnectionMySQL::loadStoredObjects(ContextPtr, bool, bool /*force_attach*/)
void DatabaseConnectionMySQL::loadStoredObjects(ContextMutablePtr, bool, bool /*force_attach*/)
{
std::lock_guard<std::mutex> lock{mutex};
Poco::DirectoryIterator iterator(getMetadataPath());
fs::directory_iterator iter(getMetadataPath());
for (Poco::DirectoryIterator end; iterator != end; ++iterator)
for (fs::directory_iterator end; iter != end; ++iter)
{
if (iterator->isFile() && endsWith(iterator.name(), suffix))
if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix))
{
const auto & filename = iterator.name();
const auto & filename = iter->path().filename().string();
const auto & table_name = unescapeForFileName(filename.substr(0, filename.size() - strlen(suffix)));
remove_or_detach_tables.emplace(table_name);
}
@ -420,27 +420,25 @@ void DatabaseConnectionMySQL::detachTablePermanently(ContextPtr, const String &
{
std::lock_guard<std::mutex> lock{mutex};
Poco::File remove_flag(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
fs::path remove_flag = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);
if (remove_or_detach_tables.count(table_name))
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " is dropped",
ErrorCodes::TABLE_IS_DROPPED);
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));
if (remove_flag.exists())
throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(database_name) +
"." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR);
if (fs::exists(remove_flag))
throw Exception(ErrorCodes::LOGICAL_ERROR, "The remove flag file already exists but the {}.{} does not exists remove tables, it is bug.",
backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));
auto table_iter = local_tables_cache.find(table_name);
if (table_iter == local_tables_cache.end())
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
ErrorCodes::UNKNOWN_TABLE);
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist", backQuoteIfNeed(database_name), backQuoteIfNeed(table_name));
remove_or_detach_tables.emplace(table_name);
try
{
table_iter->second.second->drop();
remove_flag.createFile();
FS::createFile(remove_flag);
}
catch (...)
{

View File

@ -74,7 +74,7 @@ public:
void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override;
void loadStoredObjects(ContextPtr, bool, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override;
StoragePtr detachTable(const String & table_name) override;

View File

@ -13,9 +13,11 @@
# include <Databases/MySQL/MaterializeMySQLSyncThread.h>
# include <Parsers/ASTCreateQuery.h>
# include <Storages/StorageMaterializeMySQL.h>
# include <Poco/File.h>
# include <Poco/Logger.h>
# include <Common/setThreadName.h>
# include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -92,7 +94,7 @@ void DatabaseMaterializeMySQL<Base>::setException(const std::exception_ptr & exc
}
template<typename Base>
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach)
void DatabaseMaterializeMySQL<Base>::loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach)
{
Base::loadStoredObjects(context_, has_force_restore_data_flag, force_attach);
if (!force_attach)
@ -158,10 +160,10 @@ template<typename Base>
void DatabaseMaterializeMySQL<Base>::drop(ContextPtr context_)
{
/// Remove metadata info
Poco::File metadata(Base::getMetadataPath() + "/.metadata");
fs::path metadata(Base::getMetadataPath() + "/.metadata");
if (metadata.exists())
metadata.remove(false);
if (fs::exists(metadata))
fs::remove(metadata);
Base::drop(context_);
}

View File

@ -43,7 +43,7 @@ protected:
public:
String getEngineName() const override { return "MaterializeMySQL"; }
void loadStoredObjects(ContextPtr context_, bool has_force_restore_data_flag, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr context_, bool has_force_restore_data_flag, bool force_attach) override;
void createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) override;

View File

@ -8,11 +8,13 @@
#include <Formats/MySQLBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <Poco/File.h>
#include <Common/quoteString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -193,12 +195,11 @@ void commitMetadata(const std::function<void()> & function, const String & persi
try
{
function();
Poco::File(persistent_tmp_path).renameTo(persistent_path);
fs::rename(persistent_tmp_path, persistent_path);
}
catch (...)
{
Poco::File(persistent_tmp_path).remove();
fs::remove(persistent_tmp_path);
throw;
}
}
@ -231,7 +232,7 @@ void MaterializeMetadata::transaction(const MySQLReplication::Position & positio
MaterializeMetadata::MaterializeMetadata(const String & path_, const Settings & settings_) : persistent_path(path_), settings(settings_)
{
if (Poco::File(persistent_path).exists())
if (fs::exists(persistent_path))
{
ReadBufferFromFile in(persistent_path, DBMS_DEFAULT_BUFFER_SIZE);
assertString("Version:\t" + toString(meta_version), in);

View File

@ -41,7 +41,7 @@ namespace ErrorCodes
static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync";
static ContextPtr createQueryContext(ContextPtr context)
static ContextMutablePtr createQueryContext(ContextPtr context)
{
Settings new_query_settings = context->getSettings();
new_query_settings.insert_allow_materialized_columns = true;
@ -59,7 +59,7 @@ static ContextPtr createQueryContext(ContextPtr context)
return query_context;
}
static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextPtr query_context, const String & database, const String & comment)
static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutablePtr query_context, const String & database, const String & comment)
{
try
{
@ -281,7 +281,7 @@ static inline void cleanOutdatedTables(const String & database_name, ContextPtr
}
static inline BlockOutputStreamPtr
getTableOutput(const String & database_name, const String & table_name, ContextPtr query_context, bool insert_materialized = false)
getTableOutput(const String & database_name, const String & table_name, ContextMutablePtr query_context, bool insert_materialized = false)
{
const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context);

View File

@ -12,11 +12,12 @@
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Common/escapeForFileName.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/File.h>
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
#include <Common/quoteString.h>
#include <Common/filesystemHelpers.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -209,9 +210,9 @@ void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr
detached_or_dropped.erase(table_name);
Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
if (table_marked_as_removed.exists())
table_marked_as_removed.remove();
fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);
if (fs::exists(table_marked_as_removed))
fs::remove(table_marked_as_removed);
}
@ -256,16 +257,8 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /
if (detached_or_dropped.count(table_name))
throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED);
Poco::File mark_table_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
try
{
mark_table_removed.createFile();
}
catch (...)
{
throw;
}
fs::path mark_table_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);
FS::createFile(mark_table_removed);
if (cache_tables)
cached_tables.erase(table_name);
@ -276,22 +269,22 @@ void DatabasePostgreSQL::dropTable(ContextPtr, const String & table_name, bool /
void DatabasePostgreSQL::drop(ContextPtr /*context*/)
{
Poco::File(getMetadataPath()).remove(true);
fs::remove_all(getMetadataPath());
}
void DatabasePostgreSQL::loadStoredObjects(ContextPtr /* context */, bool, bool /*force_attach*/)
void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, bool, bool /*force_attach*/)
{
{
std::lock_guard<std::mutex> lock{mutex};
Poco::DirectoryIterator iterator(getMetadataPath());
fs::directory_iterator iter(getMetadataPath());
/// Check for previously dropped tables
for (Poco::DirectoryIterator end; iterator != end; ++iterator)
for (fs::directory_iterator end; iter != end; ++iter)
{
if (iterator->isFile() && endsWith(iterator.name(), suffix))
if (fs::is_regular_file(iter->path()) && endsWith(iter->path().filename(), suffix))
{
const auto & file_name = iterator.name();
const auto & file_name = iter->path().filename().string();
const auto & table_name = unescapeForFileName(file_name.substr(0, file_name.size() - strlen(suffix)));
detached_or_dropped.emplace(table_name);
}
@ -325,9 +318,9 @@ void DatabasePostgreSQL::removeOutdatedTables()
{
auto table_name = *iter;
iter = detached_or_dropped.erase(iter);
Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix);
if (table_marked_as_removed.exists())
table_marked_as_removed.remove();
fs::path table_marked_as_removed = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + suffix);
if (fs::exists(table_marked_as_removed))
fs::remove(table_marked_as_removed);
}
else
++iter;

View File

@ -47,7 +47,7 @@ public:
bool empty() const override;
void loadStoredObjects(ContextPtr, bool, bool force_attach) override;
void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override;
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) override;

View File

@ -79,7 +79,7 @@ private:
mutable std::string invalidate_query_response;
ExternalQueryBuilder query_builder;
Block sample_block;
ContextPtr context;
ContextMutablePtr context;
ConnectionPoolWithFailoverPtr pool;
const std::string load_all_query;
Poco::Logger * log = &Poco::Logger::get("ClickHouseDictionarySource");

View File

@ -67,7 +67,7 @@ Block blockForKeys(
return block;
}
ContextPtr copyContextAndApplySettings(
ContextMutablePtr copyContextAndApplySettings(
const std::string & config_prefix,
ContextConstPtr context,
const Poco::Util::AbstractConfiguration & config)

View File

@ -3,14 +3,13 @@
#include <vector>
#include <common/types.h>
#include <Poco/File.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <DataStreams/IBlockInputStream.h>
#include <Columns/IColumn.h>
#include <Core/Block.h>
#include <Interpreters/Context_fwd.h>
namespace DB
{
@ -37,7 +36,7 @@ Block blockForKeys(
const std::vector<size_t> & requested_rows);
/// Used for applying settings to copied context in some register[...]Source functions
ContextPtr copyContextAndApplySettings(
ContextMutablePtr copyContextAndApplySettings(
const std::string & config_prefix,
ContextConstPtr context,
const Poco::Util::AbstractConfiguration & config);

View File

@ -5,7 +5,9 @@
#include <Poco/Exception.h>
#include <Poco/Util/Application.h>
#include "HierarchyFormatReader.h"
#include <filesystem>
namespace fs = std::filesystem;
bool RegionsHierarchyDataSource::isModified() const
{
@ -27,14 +29,13 @@ RegionsHierarchiesDataProvider::RegionsHierarchiesDataProvider(const std::string
void RegionsHierarchiesDataProvider::discoverFilesWithCustomHierarchies()
{
std::string basename = Poco::Path(path).getBaseName();
std::string basename = fs::path(path).stem();
fs::path dir_path = fs::canonical(path).parent_path();
Poco::Path dir_path = Poco::Path(path).absolute().parent();
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(dir_path); dir_it != dir_end; ++dir_it)
fs::directory_iterator dir_end;
for (fs::directory_iterator dir_it(dir_path); dir_it != dir_end; ++dir_it)
{
std::string candidate_basename = dir_it.path().getBaseName();
std::string candidate_basename = dir_it->path().stem();
if (candidate_basename.starts_with(basename)
&& (candidate_basename.size() > basename.size() + 1)

View File

@ -2,7 +2,9 @@
#include <IO/ReadBufferFromFile.h>
#include "NamesFormatReader.h"
#include <filesystem>
namespace fs = std::filesystem;
bool LanguageRegionsNamesDataSource::isModified() const
{
@ -11,7 +13,7 @@ bool LanguageRegionsNamesDataSource::isModified() const
size_t LanguageRegionsNamesDataSource::estimateTotalSize() const
{
return Poco::File(path).getSize();
return fs::file_size(path);
}
ILanguageRegionsNamesReaderPtr LanguageRegionsNamesDataSource::createReader()
@ -39,7 +41,7 @@ RegionsNamesDataProvider::RegionsNamesDataProvider(const std::string & directory
ILanguageRegionsNamesDataSourcePtr RegionsNamesDataProvider::getLanguageRegionsNamesSource(const std::string & language) const
{
const auto data_file = getDataFilePath(language);
if (Poco::File(data_file).exists())
if (fs::exists(data_file))
return std::make_unique<LanguageRegionsNamesDataSource>(data_file, language);
else
return {};

View File

@ -1,9 +1,4 @@
#include "FileDictionarySource.h"
#include <filesystem>
#include <Poco/File.h>
#include <common/logger_useful.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/filesystemHelpers.h>
@ -15,6 +10,7 @@
#include "registerDictionaries.h"
#include "DictionarySourceHelpers.h"
namespace DB
{
static const UInt64 max_block_size = 8192;
@ -68,9 +64,10 @@ std::string FileDictionarySource::toString() const
Poco::Timestamp FileDictionarySource::getLastModification() const
{
return Poco::File{filepath}.getLastModified();
return FS::getModificationTimestamp(filepath);
}
void registerDictionarySourceFile(DictionarySourceFactory & factory)
{
auto create_table_source = [=](const DictionaryStructure & dict_struct,

View File

@ -1,19 +1,19 @@
#include "LibraryDictionarySource.h"
#include <Poco/File.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <common/logger_useful.h>
#include <Common/filesystemHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/Context.h>
#include <filesystem>
#include <Dictionaries/DictionarySourceFactory.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/registerDictionaries.h>
namespace fs = std::filesystem;
namespace DB
{
@ -44,8 +44,8 @@ LibraryDictionarySource::LibraryDictionarySource(
if (created_from_ddl && !pathStartsWith(path, context->getDictionariesLibPath()))
throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, context->getDictionariesLibPath());
if (!Poco::File(path).exists())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", Poco::File(path).path());
if (!fs::exists(path))
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "LibraryDictionarySource: Can't load library {}: file doesn't exist", path);
description.init(sample_block);
bridge_helper = std::make_shared<LibraryBridgeHelper>(context, description.sample_block, dictionary_id);

View File

@ -6,9 +6,9 @@
#include <Interpreters/Context.h>
#include <Common/filesystemHelpers.h>
#include <Common/quoteString.h>
#include <IO/createReadBufferFromFileBase.h>
#include <fstream>
#include <unistd.h>
@ -60,27 +60,28 @@ class DiskLocalDirectoryIterator : public IDiskDirectoryIterator
{
public:
explicit DiskLocalDirectoryIterator(const String & disk_path_, const String & dir_path_)
: dir_path(dir_path_), iter(disk_path_ + dir_path_)
: dir_path(dir_path_), entry(fs::path(disk_path_) / dir_path_)
{
}
void next() override { ++iter; }
void next() override { ++entry; }
bool isValid() const override { return iter != Poco::DirectoryIterator(); }
bool isValid() const override { return entry != fs::directory_iterator(); }
String path() const override
{
if (iter->isDirectory())
return dir_path + iter.name() + '/';
if (entry->is_directory())
return dir_path / entry->path().filename() / "";
else
return dir_path + iter.name();
return dir_path / entry->path().filename();
}
String name() const override { return iter.name(); }
String name() const override { return entry->path().filename(); }
private:
String dir_path;
Poco::DirectoryIterator iter;
fs::path dir_path;
fs::directory_iterator entry;
};
@ -118,7 +119,7 @@ UInt64 DiskLocal::getTotalSpace() const
{
struct statvfs fs;
if (name == "default") /// for default disk we get space from path/data/
fs = getStatVFS(disk_path + "data/");
fs = getStatVFS((fs::path(disk_path) / "data/").string());
else
fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_blocks * fs.f_bsize;
@ -133,7 +134,7 @@ 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(disk_path + "data/");
fs = getStatVFS((fs::path(disk_path) / "data/").string());
else
fs = getStatVFS(disk_path);
UInt64 total_size = fs.f_bavail * fs.f_bsize;
@ -152,45 +153,43 @@ UInt64 DiskLocal::getUnreservedSpace() const
bool DiskLocal::exists(const String & path) const
{
return Poco::File(disk_path + path).exists();
return fs::exists(fs::path(disk_path) / path);
}
bool DiskLocal::isFile(const String & path) const
{
return Poco::File(disk_path + path).isFile();
return fs::is_regular_file(fs::path(disk_path) / path);
}
bool DiskLocal::isDirectory(const String & path) const
{
return Poco::File(disk_path + path).isDirectory();
return fs::is_directory(fs::path(disk_path) / path);
}
size_t DiskLocal::getFileSize(const String & path) const
{
return Poco::File(disk_path + path).getSize();
return fs::file_size(fs::path(disk_path) / path);
}
void DiskLocal::createDirectory(const String & path)
{
Poco::File(disk_path + path).createDirectory();
fs::create_directory(fs::path(disk_path) / path);
}
void DiskLocal::createDirectories(const String & path)
{
Poco::File(disk_path + path).createDirectories();
fs::create_directories(fs::path(disk_path) / path);
}
void DiskLocal::clearDirectory(const String & path)
{
std::vector<Poco::File> files;
Poco::File(disk_path + path).list(files);
for (auto & file : files)
file.remove();
for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path))
fs::remove(entry.path());
}
void DiskLocal::moveDirectory(const String & from_path, const String & to_path)
{
Poco::File(disk_path + from_path).renameTo(disk_path + to_path);
fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path);
}
DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path)
@ -200,99 +199,95 @@ DiskDirectoryIteratorPtr DiskLocal::iterateDirectory(const String & path)
void DiskLocal::moveFile(const String & from_path, const String & to_path)
{
Poco::File(disk_path + from_path).renameTo(disk_path + to_path);
fs::rename(fs::path(disk_path) / from_path, fs::path(disk_path) / to_path);
}
void DiskLocal::replaceFile(const String & from_path, const String & to_path)
{
Poco::File from_file(disk_path + from_path);
Poco::File to_file(disk_path + to_path);
if (to_file.exists())
{
Poco::File tmp_file(disk_path + to_path + ".old");
to_file.renameTo(tmp_file.path());
from_file.renameTo(disk_path + to_path);
tmp_file.remove();
}
else
from_file.renameTo(to_file.path());
fs::path from_file = fs::path(disk_path) / from_path;
fs::path to_file = fs::path(disk_path) / to_path;
fs::rename(from_file, to_file);
}
std::unique_ptr<ReadBufferFromFileBase>
DiskLocal::readFile(
const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, MMappedFileCache * mmap_cache) const
{
return createReadBufferFromFileBase(disk_path + path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size);
return createReadBufferFromFileBase(fs::path(disk_path) / path, estimated_size, aio_threshold, mmap_threshold, mmap_cache, buf_size);
}
std::unique_ptr<WriteBufferFromFileBase>
DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode)
{
int flags = (mode == WriteMode::Append) ? (O_APPEND | O_CREAT | O_WRONLY) : -1;
return std::make_unique<WriteBufferFromFile>(disk_path + path, buf_size, flags);
return std::make_unique<WriteBufferFromFile>(fs::path(disk_path) / path, buf_size, flags);
}
void DiskLocal::removeFile(const String & path)
{
auto fs_path = disk_path + path;
auto fs_path = fs::path(disk_path) / path;
if (0 != unlink(fs_path.c_str()))
throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK);
throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK);
}
void DiskLocal::removeFileIfExists(const String & path)
{
auto fs_path = disk_path + path;
auto fs_path = fs::path(disk_path) / path;
if (0 != unlink(fs_path.c_str()) && errno != ENOENT)
throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK);
throwFromErrnoWithPath("Cannot unlink file " + fs_path.string(), fs_path, ErrorCodes::CANNOT_UNLINK);
}
void DiskLocal::removeDirectory(const String & path)
{
auto fs_path = disk_path + path;
auto fs_path = fs::path(disk_path) / path;
if (0 != rmdir(fs_path.c_str()))
throwFromErrnoWithPath("Cannot rmdir " + fs_path, fs_path, ErrorCodes::CANNOT_RMDIR);
throwFromErrnoWithPath("Cannot rmdir " + fs_path.string(), fs_path, ErrorCodes::CANNOT_RMDIR);
}
void DiskLocal::removeRecursive(const String & path)
{
Poco::File(disk_path + path).remove(true);
fs::remove_all(fs::path(disk_path) / path);
}
void DiskLocal::listFiles(const String & path, std::vector<String> & file_names)
{
Poco::File(disk_path + path).list(file_names);
file_names.clear();
for (const auto & entry : fs::directory_iterator(fs::path(disk_path) / path))
file_names.emplace_back(entry.path().filename());
}
void DiskLocal::setLastModified(const String & path, const Poco::Timestamp & timestamp)
{
Poco::File(disk_path + path).setLastModified(timestamp);
FS::setModificationTime(fs::path(disk_path) / path, timestamp.epochTime());
}
Poco::Timestamp DiskLocal::getLastModified(const String & path)
{
return Poco::File(disk_path + path).getLastModified();
return FS::getModificationTimestamp(fs::path(disk_path) / path);
}
void DiskLocal::createHardLink(const String & src_path, const String & dst_path)
{
DB::createHardLink(disk_path + src_path, disk_path + dst_path);
DB::createHardLink(fs::path(disk_path) / src_path, fs::path(disk_path) / dst_path);
}
void DiskLocal::truncateFile(const String & path, size_t size)
{
int res = truncate((disk_path + path).c_str(), size);
int res = truncate((fs::path(disk_path) / path).string().data(), size);
if (-1 == res)
throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE);
}
void DiskLocal::createFile(const String & path)
{
Poco::File(disk_path + path).createFile();
FS::createFile(fs::path(disk_path) / path);
}
void DiskLocal::setReadOnly(const String & path)
{
Poco::File(disk_path + path).setReadOnly(true);
fs::permissions(fs::path(disk_path) / path,
fs::perms::owner_write | fs::perms::group_write | fs::perms::others_write,
fs::perm_options::remove);
}
bool inline isSameDiskType(const IDisk & one, const IDisk & another)
@ -303,14 +298,23 @@ bool inline isSameDiskType(const IDisk & one, const IDisk & another)
void DiskLocal::copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path)
{
if (isSameDiskType(*this, *to_disk))
Poco::File(disk_path + from_path).copyTo(to_disk->getPath() + to_path); /// Use more optimal way.
{
fs::path to = fs::path(to_disk->getPath()) / to_path;
fs::path from = fs::path(disk_path) / from_path;
if (from_path.ends_with('/'))
from = from.parent_path();
if (fs::is_directory(from))
to /= from.filename();
fs::copy(from, to, fs::copy_options::recursive | fs::copy_options::overwrite_existing); /// Use more optimal way.
}
else
IDisk::copy(from_path, to_disk, to_path); /// Copy files through buffers.
}
SyncGuardPtr DiskLocal::getDirectorySyncGuard(const String & path) const
{
return std::make_unique<LocalDirectorySyncGuard>(disk_path + path);
return std::make_unique<LocalDirectorySyncGuard>(fs::path(disk_path) / path);
}
DiskPtr DiskLocalReservation::getDisk(size_t i) const
@ -381,10 +385,8 @@ void registerDiskLocal(DiskFactory & factory)
throw Exception("Disk path must end with /. Disk " + name, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
if (Poco::File disk{path}; !disk.canRead() || !disk.canWrite())
{
if (!FS::canRead(path) || !FS::canWrite(path))
throw Exception("There is no RW access to the disk " + name + " (" + path + ")", ErrorCodes::PATH_ACCESS_DENIED);
}
bool has_space_ratio = config.has(config_prefix + ".keep_free_space_ratio");

View File

@ -6,8 +6,6 @@
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/File.h>
namespace DB
{
@ -27,7 +25,7 @@ public:
: name(name_), disk_path(path_), keep_free_space_bytes(keep_free_space_bytes_)
{
if (disk_path.back() != '/')
throw Exception("Disk path must ends with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Disk path must end with '/', but '" + disk_path + "' doesn't.", ErrorCodes::LOGICAL_ERROR);
}
const String & getName() const override { return name; }

View File

@ -6,7 +6,6 @@
#include <IO/WriteBufferFromFileBase.h>
#include <IO/WriteBufferFromString.h>
#include <Interpreters/Context.h>
#include <Poco/Path.h>
namespace DB
@ -24,7 +23,7 @@ namespace ErrorCodes
class DiskMemoryDirectoryIterator final : public IDiskDirectoryIterator
{
public:
explicit DiskMemoryDirectoryIterator(std::vector<Poco::Path> && dir_file_paths_)
explicit DiskMemoryDirectoryIterator(std::vector<fs::path> && dir_file_paths_)
: dir_file_paths(std::move(dir_file_paths_)), iter(dir_file_paths.begin())
{
}
@ -33,13 +32,13 @@ public:
bool isValid() const override { return iter != dir_file_paths.end(); }
String path() const override { return (*iter).toString(); }
String path() const override { return iter->string(); }
String name() const override { return (*iter).getFileName(); }
String name() const override { return iter->filename(); }
private:
std::vector<Poco::Path> dir_file_paths;
std::vector<Poco::Path>::iterator iter;
std::vector<fs::path> dir_file_paths;
std::vector<fs::path>::iterator iter;
};
@ -268,7 +267,7 @@ DiskDirectoryIteratorPtr DiskMemory::iterateDirectory(const String & path)
if (!path.empty() && files.find(path) == files.end())
throw Exception("Directory '" + path + "' does not exist", ErrorCodes::DIRECTORY_DOESNT_EXIST);
std::vector<Poco::Path> dir_file_paths;
std::vector<fs::path> dir_file_paths;
for (const auto & file : files)
if (parentPath(file.first) == path)
dir_file_paths.emplace_back(file.first);

View File

@ -172,8 +172,8 @@ void registerDiskHDFS(DiskFactory & factory)
const String & config_prefix,
ContextConstPtr context_) -> DiskPtr
{
Poco::File disk{context_->getPath() + "disks/" + name};
disk.createDirectories();
fs::path disk = fs::path(context_->getPath()) / "disks" / name;
fs::create_directories(disk);
String uri{config.getString(config_prefix + ".endpoint")};

View File

@ -22,7 +22,8 @@ bool IDisk::isDirectoryEmpty(const String & path)
void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path)
{
LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path);
LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} (path: {}) {} to {} (path: {}) {}.",
from_disk.getName(), from_disk.getPath(), from_path, to_disk.getName(), to_disk.getPath(), to_path);
auto in = from_disk.readFile(from_path);
auto out = to_disk.writeFile(to_path);
@ -41,16 +42,15 @@ void asyncCopy(IDisk & from_disk, String from_path, IDisk & to_disk, String to_p
[&from_disk, from_path, &to_disk, to_path]()
{
setThreadName("DiskCopier");
DB::copyFile(from_disk, from_path, to_disk, to_path + fileName(from_path));
DB::copyFile(from_disk, from_path, to_disk, fs::path(to_path) / fileName(from_path));
});
results.push_back(std::move(result));
}
else
{
Poco::Path path(from_path);
const String & dir_name = path.directory(path.depth() - 1);
const String dest = to_path + dir_name + "/";
fs::path dir_name = fs::path(from_path).parent_path().filename();
fs::path dest(fs::path(to_path) / dir_name);
to_disk.createDirectories(dest);
for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next())

View File

@ -7,16 +7,16 @@
#include <Common/Exception.h>
#include <Disks/Executor.h>
#include <Disks/DiskType.h>
#include "Disks/Executor.h"
#include <memory>
#include <mutex>
#include <utility>
#include <boost/noncopyable.hpp>
#include <Poco/Path.h>
#include <Poco/Timestamp.h>
#include <filesystem>
#include "Poco/Util/AbstractConfiguration.h"
namespace fs = std::filesystem;
namespace CurrentMetrics
{
@ -212,10 +212,10 @@ public:
virtual DiskType::Type getType() const = 0;
/// Invoked when Global Context is shutdown.
virtual void shutdown() { }
virtual void shutdown() {}
/// Performs action on disk startup.
virtual void startup() { }
virtual void startup() {}
/// Return some uniq string for file, overrode for S3
/// Required for distinguish different copies of the same part on S3
@ -233,7 +233,7 @@ public:
virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const;
/// Applies new settings for disk in runtime.
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) { }
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextConstPtr) {}
protected:
friend class DiskDecorator;
@ -294,25 +294,27 @@ public:
/// Return full path to a file on disk.
inline String fullPath(const DiskPtr & disk, const String & path)
{
return disk->getPath() + path;
return fs::path(disk->getPath()) / path;
}
/// Return parent path for the specified path.
inline String parentPath(const String & path)
{
return Poco::Path(path).parent().toString();
if (path.ends_with('/'))
return fs::path(path).parent_path().parent_path() / "";
return fs::path(path).parent_path() / "";
}
/// Return file name for the specified path.
inline String fileName(const String & path)
{
return Poco::Path(path).getFileName();
return fs::path(path).filename();
}
/// Return directory path for the specified path.
inline String directoryPath(const String & path)
{
return Poco::Path(path).setFileName("").toString();
return fs::path(path).parent_path() / "";
}
}

View File

@ -6,12 +6,12 @@
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/createHardLink.h>
#include <Common/quoteString.h>
#include <common/logger_useful.h>
#include <Common/checkStackSize.h>
#include <boost/algorithm/string.hpp>
#include <Common/filesystemHelpers.h>
namespace DB
@ -179,9 +179,9 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths
{
LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path));
Poco::File file(metadata_path + path);
fs::path file(metadata_path + path);
if (!file.isFile())
if (!fs::is_regular_file(file))
throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path);
try
@ -191,7 +191,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths
/// If there is no references - delete content from remote FS.
if (metadata.ref_count == 0)
{
file.remove();
fs::remove(file);
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path);
}
@ -199,7 +199,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths
{
--metadata.ref_count;
metadata.save();
file.remove();
fs::remove(file);
}
}
catch (const Exception & e)
@ -210,7 +210,7 @@ void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths
LOG_WARNING(log,
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
backQuote(path), e.nested() ? e.nested()->message() : e.message());
file.remove();
fs::remove(file);
}
else
throw;
@ -222,8 +222,8 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr
{
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
Poco::File file(metadata_path + path);
if (file.isFile())
fs::path file = fs::path(metadata_path) / path;
if (fs::is_regular_file(file))
{
removeMeta(path, fs_paths_keeper);
}
@ -231,7 +231,7 @@ void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr
{
for (auto it{iterateDirectory(path)}; it->isValid(); it->next())
removeMetaRecursive(it->path(), fs_paths_keeper);
file.remove();
fs::remove(file);
}
}
@ -296,13 +296,13 @@ IDiskRemote::IDiskRemote(
bool IDiskRemote::exists(const String & path) const
{
return Poco::File(metadata_path + path).exists();
return fs::exists(fs::path(metadata_path) / path);
}
bool IDiskRemote::isFile(const String & path) const
{
return Poco::File(metadata_path + path).isFile();
return fs::is_regular_file(fs::path(metadata_path) / path);
}
@ -326,7 +326,7 @@ void IDiskRemote::moveFile(const String & from_path, const String & to_path)
if (exists(to_path))
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path);
fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path);
}
@ -347,7 +347,7 @@ void IDiskRemote::replaceFile(const String & from_path, const String & to_path)
void IDiskRemote::removeFileIfExists(const String & path)
{
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
if (Poco::File(metadata_path + path).exists())
if (fs::exists(fs::path(metadata_path) / path))
{
removeMeta(path, fs_paths_keeper);
removeFromRemoteFS(fs_paths_keeper);
@ -385,19 +385,19 @@ void IDiskRemote::setReadOnly(const String & path)
bool IDiskRemote::isDirectory(const String & path) const
{
return Poco::File(metadata_path + path).isDirectory();
return fs::is_directory(fs::path(metadata_path) / path);
}
void IDiskRemote::createDirectory(const String & path)
{
Poco::File(metadata_path + path).createDirectory();
fs::create_directory(fs::path(metadata_path) / path);
}
void IDiskRemote::createDirectories(const String & path)
{
Poco::File(metadata_path + path).createDirectories();
fs::create_directories(fs::path(metadata_path) / path);
}
@ -411,7 +411,7 @@ void IDiskRemote::clearDirectory(const String & path)
void IDiskRemote::removeDirectory(const String & path)
{
Poco::File(metadata_path + path).remove();
fs::remove(fs::path(metadata_path) / path);
}
@ -430,13 +430,13 @@ void IDiskRemote::listFiles(const String & path, std::vector<String> & file_name
void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp)
{
Poco::File(metadata_path + path).setLastModified(timestamp);
FS::setModificationTime(fs::path(metadata_path) / path, timestamp.epochTime());
}
Poco::Timestamp IDiskRemote::getLastModified(const String & path)
{
return Poco::File(metadata_path + path).getLastModified();
return FS::getModificationTimestamp(fs::path(metadata_path) / path);
}

View File

@ -4,11 +4,12 @@
#include <atomic>
#include "Disks/DiskFactory.h"
#include "Disks/Executor.h"
#include <Poco/DirectoryIterator.h>
#include <utility>
#include <Common/MultiVersion.h>
#include <Common/ThreadPool.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -193,21 +194,21 @@ public:
void next() override { ++iter; }
bool isValid() const override { return iter != Poco::DirectoryIterator(); }
bool isValid() const override { return iter != fs::directory_iterator(); }
String path() const override
{
if (iter->isDirectory())
return folder_path + iter.name() + '/';
if (fs::is_directory(iter->path()))
return folder_path / iter->path().filename().string() / "";
else
return folder_path + iter.name();
return folder_path / iter->path().filename().string();
}
String name() const override { return iter.name(); }
String name() const override { return iter->path().filename(); }
private:
Poco::DirectoryIterator iter;
String folder_path;
fs::directory_iterator iter;
fs::path folder_path;
};

View File

@ -21,6 +21,7 @@ ReadIndirectBufferFromRemoteFS<T>::ReadIndirectBufferFromRemoteFS(
{
}
template<typename T>
off_t ReadIndirectBufferFromRemoteFS<T>::seek(off_t offset_, int whence)
{

View File

@ -14,7 +14,6 @@
#include <IO/SeekAvoidingReadBuffer.h>
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
#include <Poco/File.h>
#include <Common/createHardLink.h>
#include <Common/quoteString.h>
#include <Common/thread_local_rng.h>
@ -215,7 +214,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path, bool sen
createFileOperationObject("rename", revision, object_metadata);
}
Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path);
fs::rename(fs::path(metadata_path) / from_path, fs::path(metadata_path) / to_path);
}
std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, size_t buf_size, size_t, size_t, size_t, MMappedFileCache *) const
@ -675,8 +674,8 @@ void DiskS3::restore()
restoreFiles(information);
restoreFileOperations(information);
Poco::File restore_file(metadata_path + RESTORE_FILE_NAME);
restore_file.remove();
fs::path restore_file = fs::path(metadata_path) / RESTORE_FILE_NAME;
fs::remove(restore_file);
saveSchemaVersion(RESTORABLE_SCHEMA_VERSION);
@ -863,8 +862,9 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio
continue;
/// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them.
Poco::Path directory_path (path);
auto directory_name = directory_path.directory(directory_path.depth() - 1);
fs::path directory_path(path);
auto directory_name = directory_path.parent_path().filename().string();
auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); };
if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate))
continue;
@ -873,7 +873,14 @@ void DiskS3::restoreFileOperations(const RestoreInformation & restore_informatio
LOG_DEBUG(log, "Move directory to 'detached' {} -> {}", path, detached_path);
Poco::File(metadata_path + path).moveTo(metadata_path + detached_path);
fs::path from_path = fs::path(metadata_path) / path;
fs::path to_path = fs::path(metadata_path) / detached_path;
if (path.ends_with('/'))
to_path /= from_path.parent_path().filename();
else
to_path /= from_path.filename();
fs::copy(from_path, to_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing);
fs::remove_all(from_path);
}
}
@ -905,7 +912,9 @@ String DiskS3::revisionToString(UInt64 revision)
String DiskS3::pathToDetached(const String & source_path)
{
return Poco::Path(source_path).parent().append(Poco::Path("detached")).toString() + '/';
if (source_path.ends_with('/'))
return fs::path(source_path).parent_path().parent_path() / "detached/";
return fs::path(source_path).parent_path() / "detached/";
}
void DiskS3::onFreeze(const String & path)

View File

@ -174,7 +174,7 @@ void registerDiskS3(DiskFactory & factory)
throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS);
String metadata_path = config.getString(config_prefix + ".metadata_path", context->getPath() + "disks/" + name + "/");
Poco::File (metadata_path).createDirectories();
fs::create_directories(metadata_path);
std::shared_ptr<IDisk> s3disk = std::make_shared<DiskS3>(
name,

View File

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

View File

@ -8,7 +8,6 @@
namespace DB
{
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
template <typename T>
WriteIndirectBufferFromRemoteFS<T>::WriteIndirectBufferFromRemoteFS(
std::unique_ptr<T> impl_,

View File

@ -10,7 +10,7 @@
namespace DB
{
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
/// Stores data in S3/HDFS and adds the object path and object size to metadata file on local FS.
template <typename T>
class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator
{

View File

@ -2,6 +2,9 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include "gtest_disk.h"
#include <filesystem>
namespace fs = std::filesystem;
#if !defined(__clang__)
@ -22,7 +25,7 @@ DB::DiskPtr createDisk<DB::DiskMemory>()
template <>
DB::DiskPtr createDisk<DB::DiskLocal>()
{
Poco::File("tmp/").createDirectory();
fs::create_directory("tmp/");
return std::make_shared<DB::DiskLocal>("local_disk", "tmp/", 0);
}
@ -43,7 +46,7 @@ template <>
void destroyDisk<DB::DiskLocal>(DB::DiskPtr & disk)
{
disk.reset();
Poco::File("tmp/").remove(true);
fs::remove_all("tmp/");
}

View File

@ -1,7 +1,7 @@
#include <Formats/FormatSchemaInfo.h>
#include <Poco/Path.h>
#include <Interpreters/Context.h>
#include <Common/Exception.h>
#include <filesystem>
namespace DB
@ -11,6 +11,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace fs = std::filesystem;
namespace
{
@ -34,55 +35,66 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String &
String default_file_extension = getFormatSchemaDefaultFileExtension(format);
Poco::Path path;
fs::path path;
if (require_message)
{
size_t colon_pos = format_schema.find(':');
if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)
|| path.assign(format_schema.substr(0, colon_pos)).makeFile().getFileName().empty())
if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1))
{
throw Exception(
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
+ (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") +
". Got '" + format_schema
+ "'",
ErrorCodes::BAD_ARGUMENTS);
". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS);
}
else
{
path = fs::path(format_schema.substr(0, colon_pos));
String filename = path.has_filename() ? path.filename() : path.parent_path().filename();
if (filename.empty())
throw Exception(
"Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format"
+ (default_file_extension.empty() ? "" : ", e.g. 'schema." + default_file_extension + ":Message'") +
". Got '" + format_schema + "'", ErrorCodes::BAD_ARGUMENTS);
}
message_name = format_schema.substr(colon_pos + 1);
}
else
path.assign(format_schema).makeFile().getFileName();
{
path = fs::path(format_schema);
if (!path.has_filename())
path = path.parent_path() / "";
}
auto default_schema_directory = [&format_schema_path]()
{
static const String str = Poco::Path(format_schema_path).makeAbsolute().makeDirectory().toString();
static const String str = fs::canonical(format_schema_path) / "";
return str;
};
if (path.getExtension().empty() && !default_file_extension.empty())
path.setExtension(default_file_extension);
if (!path.has_extension() && !default_file_extension.empty())
path = path.parent_path() / (path.stem().string() + '.' + default_file_extension);
if (path.isAbsolute())
fs::path default_schema_directory_path(default_schema_directory());
if (path.is_absolute())
{
if (is_server)
throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS);
schema_path = path.getFileName();
schema_directory = path.makeParent().toString();
throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.string(), ErrorCodes::BAD_ARGUMENTS);
schema_path = path.filename();
schema_directory = path.parent_path() / "";
}
else if (path.depth() >= 1 && path.directory(0) == "..")
else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string()))
{
if (is_server)
throw Exception(
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(),
ErrorCodes::BAD_ARGUMENTS);
path = Poco::Path(default_schema_directory()).resolve(path).toString();
schema_path = path.getFileName();
schema_directory = path.makeParent().toString();
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})",
path.string());
path = default_schema_directory_path / path;
schema_path = path.filename();
schema_directory = path.parent_path() / "";
}
else
{
schema_path = path.toString();
schema_path = path;
schema_directory = default_schema_directory();
}
}

View File

@ -3,10 +3,11 @@
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeString.h>
#include <IO/ReadBufferFromFile.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Interpreters/Context.h>
#include <unistd.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
@ -68,21 +69,19 @@ public:
{
const char * filename = reinterpret_cast<const char *>(&chars[source_offset]);
const String user_files_path = getContext()->getUserFilesPath();
String user_files_absolute_path = Poco::Path(user_files_path).makeAbsolute().makeDirectory().toString();
Poco::Path poco_filepath = Poco::Path(filename);
if (poco_filepath.isRelative())
poco_filepath = Poco::Path(user_files_absolute_path, poco_filepath);
const String file_absolute_path = poco_filepath.absolute().toString();
checkReadIsAllowedOrThrow(user_files_absolute_path, file_absolute_path);
fs::path user_files_absolute_path = fs::canonical(fs::path(getContext()->getUserFilesPath()));
fs::path file_path(filename);
if (file_path.is_relative())
file_path = user_files_absolute_path / file_path;
fs::path file_absolute_path = fs::canonical(file_path);
checkReadIsAllowedOrThrow(user_files_absolute_path.string(), file_absolute_path);
checked_filenames[row] = file_absolute_path;
auto file = Poco::File(file_absolute_path);
checked_filenames[row] = file_absolute_path.string();
if (!file.exists())
throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path), ErrorCodes::FILE_DOESNT_EXIST);
if (!fs::exists(file_absolute_path))
throw Exception(fmt::format("File {} doesn't exist.", file_absolute_path.string()), ErrorCodes::FILE_DOESNT_EXIST);
const auto current_file_size = Poco::File(file_absolute_path).getSize();
const auto current_file_size = fs::file_size(file_absolute_path);
result_offset += current_file_size + 1;
res_offsets[row] = result_offset;
@ -117,8 +116,8 @@ private:
if (file_absolute_path.find(user_files_absolute_path) != 0)
throw Exception("File is not inside " + user_files_absolute_path, ErrorCodes::DATABASE_ACCESS_DENIED);
Poco::File path_poco_file = Poco::File(file_absolute_path);
if (path_poco_file.exists() && path_poco_file.isDirectory())
fs::path fs_path(file_absolute_path);
if (fs::exists(fs_path) && fs::is_directory(fs_path))
throw Exception("File can't be a directory", ErrorCodes::INCORRECT_FILE_NAME);
}
};

View File

@ -1,7 +1,6 @@
#include <gtest/gtest.h>
#include <stdexcept>
#include <Poco/File.h>
#include <IO/CascadeWriteBuffer.h>
#include <IO/MemoryReadWriteBuffer.h>
#include <IO/WriteBufferFromTemporaryFile.h>
@ -9,7 +8,9 @@
#include <IO/ConcatReadBuffer.h>
#include <IO/copyData.h>
#include <Common/typeid_cast.h>
#include <filesystem>
namespace fs = std::filesystem;
using namespace DB;
@ -235,7 +236,7 @@ try
buf.reset();
reread_buf.reset();
ASSERT_TRUE(!Poco::File(tmp_filename).exists());
ASSERT_TRUE(!fs::exists(tmp_filename));
}
}
catch (...)

View File

@ -24,7 +24,7 @@ namespace ErrorCodes
namespace ClusterProxy
{
ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log)
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log)
{
Settings new_settings = settings;
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);

View File

@ -31,7 +31,7 @@ class IStreamFactory;
/// - optimize_skip_unused_shards_nesting
///
/// @return new Context with adjusted settings
ContextPtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
ContextMutablePtr updateSettingsForCluster(const Cluster & cluster, ContextPtr context, const Settings & settings, Poco::Logger * log = nullptr);
/// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read.
/// `stream_factory` object encapsulates the logic of creating streams for a different type of query

View File

@ -74,8 +74,11 @@
#include <Interpreters/DatabaseCatalog.h>
#include <Storages/MergeTree/BackgroundJobsExecutor.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace ProfileEvents
{
extern const Event ContextLock;
@ -140,7 +143,7 @@ public:
/// Find existing session or create a new.
std::shared_ptr<NamedSession> acquireSession(
const String & session_id,
ContextPtr context,
ContextMutablePtr context,
std::chrono::steady_clock::duration timeout,
bool throw_if_not_found)
{
@ -513,7 +516,7 @@ SharedContextHolder::SharedContextHolder(std::unique_ptr<ContextSharedPart> shar
void SharedContextHolder::reset() { shared.reset(); }
ContextPtr Context::createGlobal(ContextSharedPart * shared)
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared)
{
auto res = std::shared_ptr<Context>(new Context);
res->shared = shared;
@ -530,19 +533,19 @@ SharedContextHolder Context::createShared()
return SharedContextHolder(std::make_unique<ContextSharedPart>());
}
ContextPtr Context::createCopy(const ContextConstPtr & other)
ContextMutablePtr Context::createCopy(const ContextPtr & other)
{
return std::shared_ptr<Context>(new Context(*other));
}
ContextPtr Context::createCopy(const ContextWeakConstPtr & other)
ContextMutablePtr Context::createCopy(const ContextWeakConstPtr & other)
{
auto ptr = other.lock();
if (!ptr) throw Exception("Can't copy an expired context", ErrorCodes::LOGICAL_ERROR);
return createCopy(ptr);
}
ContextPtr Context::createCopy(const ContextPtr & other)
ContextMutablePtr Context::createCopy(const ContextMutablePtr & other)
{
return createCopy(std::const_pointer_cast<const Context>(other));
}
@ -1079,7 +1082,7 @@ void Context::addViewSource(const StoragePtr & storage)
}
StoragePtr Context::getViewSource()
StoragePtr Context::getViewSource() const
{
return view_source;
}
@ -1313,7 +1316,7 @@ void Context::setMacros(std::unique_ptr<Macros> && macros)
shared->macros.set(std::move(macros));
}
ContextPtr Context::getQueryContext() const
ContextMutablePtr Context::getQueryContext() const
{
auto ptr = query_context.lock();
if (!ptr) throw Exception("There is no query or query context has expired", ErrorCodes::THERE_IS_NO_QUERY);
@ -1326,21 +1329,21 @@ bool Context::isInternalSubquery() const
return ptr && ptr.get() != this;
}
ContextPtr Context::getSessionContext() const
ContextMutablePtr Context::getSessionContext() const
{
auto ptr = session_context.lock();
if (!ptr) throw Exception("There is no session or session context has expired", ErrorCodes::THERE_IS_NO_SESSION);
return ptr;
}
ContextPtr Context::getGlobalContext() const
ContextMutablePtr Context::getGlobalContext() const
{
auto ptr = global_context.lock();
if (!ptr) throw Exception("There is no global context or global context has expired", ErrorCodes::LOGICAL_ERROR);
return ptr;
}
ContextPtr Context::getBufferContext() const
ContextMutablePtr Context::getBufferContext() const
{
if (!buffer_context) throw Exception("There is no buffer context", ErrorCodes::LOGICAL_ERROR);
return buffer_context;
@ -1871,7 +1874,7 @@ std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name
}
void Context::reloadClusterConfig()
void Context::reloadClusterConfig() const
{
while (true)
{
@ -1958,7 +1961,7 @@ bool Context::hasTraceCollector() const
}
std::shared_ptr<QueryLog> Context::getQueryLog()
std::shared_ptr<QueryLog> Context::getQueryLog() const
{
auto lock = getLock();
@ -1969,7 +1972,7 @@ std::shared_ptr<QueryLog> Context::getQueryLog()
}
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog()
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog() const
{
auto lock = getLock();
@ -1980,7 +1983,7 @@ std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog()
}
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database) const
{
auto lock = getLock();
@ -1998,7 +2001,7 @@ std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
}
std::shared_ptr<TraceLog> Context::getTraceLog()
std::shared_ptr<TraceLog> Context::getTraceLog() const
{
auto lock = getLock();
@ -2009,7 +2012,7 @@ std::shared_ptr<TraceLog> Context::getTraceLog()
}
std::shared_ptr<TextLog> Context::getTextLog()
std::shared_ptr<TextLog> Context::getTextLog() const
{
auto lock = getLock();
@ -2020,7 +2023,7 @@ std::shared_ptr<TextLog> Context::getTextLog()
}
std::shared_ptr<MetricLog> Context::getMetricLog()
std::shared_ptr<MetricLog> Context::getMetricLog() const
{
auto lock = getLock();
@ -2042,7 +2045,7 @@ std::shared_ptr<AsynchronousMetricLog> Context::getAsynchronousMetricLog() const
}
std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog()
std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog() const
{
auto lock = getLock();
@ -2212,14 +2215,14 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
if (!max_size_to_drop || size <= max_size_to_drop)
return;
Poco::File force_file(getFlagsPath() + "force_drop_table");
bool force_file_exists = force_file.exists();
fs::path force_file(getFlagsPath() + "force_drop_table");
bool force_file_exists = fs::exists(force_file);
if (force_file_exists)
{
try
{
force_file.remove();
fs::remove(force_file);
return;
}
catch (...)
@ -2241,9 +2244,9 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
"Example:\nsudo touch '{}' && sudo chmod 666 '{}'",
backQuoteIfNeed(database), backQuoteIfNeed(table),
size_str, max_size_to_drop_str,
force_file.path(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist",
force_file.path(),
force_file.path(), force_file.path());
force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist",
force_file.string(),
force_file.string(), force_file.string());
}
@ -2640,11 +2643,14 @@ ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const
return metadata_transaction;
}
PartUUIDsPtr Context::getPartUUIDs()
PartUUIDsPtr Context::getPartUUIDs() const
{
auto lock = getLock();
if (!part_uuids)
part_uuids = std::make_shared<PartUUIDs>();
/// For context itself, only this initialization is not const.
/// We could have done in constructor.
/// TODO: probably, remove this from Context.
const_cast<PartUUIDsPtr &>(part_uuids) = std::make_shared<PartUUIDs>();
return part_uuids;
}
@ -2663,11 +2669,11 @@ void Context::setReadTaskCallback(ReadTaskCallback && callback)
next_task_callback = callback;
}
PartUUIDsPtr Context::getIgnoredPartUUIDs()
PartUUIDsPtr Context::getIgnoredPartUUIDs() const
{
auto lock = getLock();
if (!ignored_part_uuids)
ignored_part_uuids = std::make_shared<PartUUIDs>();
const_cast<PartUUIDsPtr &>(ignored_part_uuids) = std::make_shared<PartUUIDs>();
return ignored_part_uuids;
}

View File

@ -252,12 +252,12 @@ private:
StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
ContextWeakPtr query_context;
ContextWeakPtr session_context; /// Session context or nullptr. Could be equal to this.
ContextWeakPtr global_context; /// Global context. Could be equal to this.
ContextWeakMutablePtr query_context;
ContextWeakMutablePtr session_context; /// Session context or nullptr. Could be equal to this.
ContextWeakMutablePtr global_context; /// Global context. Could be equal to this.
/// XXX: move this stuff to shared part instead.
ContextPtr buffer_context; /// Buffer context. Could be equal to this.
ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this.
public:
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
@ -293,10 +293,10 @@ private:
public:
/// Create initial Context with ContextShared and etc.
static ContextPtr createGlobal(ContextSharedPart * shared);
static ContextPtr createCopy(const ContextWeakConstPtr & other);
static ContextPtr createCopy(const ContextConstPtr & other);
static ContextPtr createCopy(const ContextPtr & other);
static ContextMutablePtr createGlobal(ContextSharedPart * shared);
static ContextMutablePtr createCopy(const ContextWeakConstPtr & other);
static ContextMutablePtr createCopy(const ContextMutablePtr & other);
static ContextMutablePtr createCopy(const ContextPtr & other);
static SharedContextHolder createShared();
void copyFrom(const ContextPtr & other);
@ -459,7 +459,7 @@ public:
StoragePtr executeTableFunction(const ASTPtr & table_expression);
void addViewSource(const StoragePtr & storage);
StoragePtr getViewSource();
StoragePtr getViewSource() const;
String getCurrentDatabase() const;
String getCurrentQueryId() const { return client_info.current_query_id; }
@ -555,14 +555,14 @@ public:
/// For methods below you may need to acquire the context lock by yourself.
ContextPtr getQueryContext() const;
ContextMutablePtr getQueryContext() const;
bool hasQueryContext() const { return !query_context.expired(); }
bool isInternalSubquery() const;
ContextPtr getSessionContext() const;
ContextMutablePtr getSessionContext() const;
bool hasSessionContext() const { return !session_context.expired(); }
ContextPtr getGlobalContext() const;
ContextMutablePtr getGlobalContext() const;
bool hasGlobalContext() const { return !global_context.expired(); }
bool isGlobalContext() const
{
@ -570,10 +570,10 @@ public:
return ptr && ptr.get() == this;
}
ContextPtr getBufferContext() const;
ContextMutablePtr getBufferContext() const;
void setQueryContext(ContextPtr context_) { query_context = context_; }
void setSessionContext(ContextPtr context_) { session_context = context_; }
void setQueryContext(ContextMutablePtr context_) { query_context = context_; }
void setSessionContext(ContextMutablePtr context_) { session_context = context_; }
void makeQueryContext() { query_context = shared_from_this(); }
void makeSessionContext() { session_context = shared_from_this(); }
@ -668,7 +668,7 @@ public:
void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers");
/// Sets custom cluster, but doesn't update configuration
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
void reloadClusterConfig();
void reloadClusterConfig() const;
Compiler & getCompiler();
@ -681,17 +681,17 @@ public:
bool hasTraceCollector() const;
/// Nullptr if the query log is not ready for this moment.
std::shared_ptr<QueryLog> getQueryLog();
std::shared_ptr<QueryThreadLog> getQueryThreadLog();
std::shared_ptr<TraceLog> getTraceLog();
std::shared_ptr<TextLog> getTextLog();
std::shared_ptr<MetricLog> getMetricLog();
std::shared_ptr<QueryLog> getQueryLog() const;
std::shared_ptr<QueryThreadLog> getQueryThreadLog() const;
std::shared_ptr<TraceLog> getTraceLog() const;
std::shared_ptr<TextLog> getTextLog() const;
std::shared_ptr<MetricLog> getMetricLog() const;
std::shared_ptr<AsynchronousMetricLog> getAsynchronousMetricLog() const;
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog();
std::shared_ptr<OpenTelemetrySpanLog> getOpenTelemetrySpanLog() const;
/// Returns an object used to log operations with parts if it possible.
/// Provide table name to make required checks.
std::shared_ptr<PartLog> getPartLog(const String & part_database);
std::shared_ptr<PartLog> getPartLog(const String & part_database) const;
const MergeTreeSettings & getMergeTreeSettings() const;
const MergeTreeSettings & getReplicatedMergeTreeSettings() const;
@ -778,8 +778,8 @@ public:
MySQLWireContext mysql;
PartUUIDsPtr getPartUUIDs();
PartUUIDsPtr getIgnoredPartUUIDs();
PartUUIDsPtr getPartUUIDs() const;
PartUUIDsPtr getIgnoredPartUUIDs() const;
ReadTaskCallback getReadTaskCallback() const;
void setReadTaskCallback(ReadTaskCallback && callback);
@ -820,7 +820,7 @@ struct NamedSession
{
NamedSessionKey key;
UInt64 close_cycle = 0;
ContextPtr context;
ContextMutablePtr context;
std::chrono::steady_clock::duration timeout;
NamedSessions & parent;

View File

@ -22,10 +22,12 @@ class Context;
/// Most used types have shorter names
/// TODO: in the first part of refactoring all the context pointers are non-const.
using ContextPtr = std::shared_ptr<Context>;
using ContextPtr = std::shared_ptr<const Context>;
using ContextConstPtr = std::shared_ptr<const Context>;
using ContextWeakPtr = std::weak_ptr<Context>;
using ContextMutablePtr = std::shared_ptr<Context>;
using ContextWeakPtr = std::weak_ptr<const Context>;
using ContextWeakConstPtr = std::weak_ptr<const Context>;
using ContextWeakMutablePtr = std::weak_ptr<Context>;
template <class Shared = ContextPtr>
struct WithContextImpl
@ -50,5 +52,6 @@ protected:
using WithContext = WithContextImpl<>;
using WithConstContext = WithContextImpl<ContextConstPtr>;
using WithMutableContext = WithContextImpl<ContextMutablePtr>;
}

View File

@ -145,7 +145,7 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context)
query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth);
}
ContextPtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/)
{
auto query_context = Context::createCopy(from_context);
query_context->makeQueryContext();
@ -355,7 +355,7 @@ void DatabaseReplicatedTask::parseQueryFromEntry(ContextPtr context)
}
}
ContextPtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper)
ContextMutablePtr DatabaseReplicatedTask::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper)
{
auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper);
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;

View File

@ -3,6 +3,7 @@
#include <Core/Types.h>
#include <Interpreters/Cluster.h>
#include <Common/ZooKeeper/Types.h>
#include <filesystem>
namespace Poco
{
@ -14,6 +15,8 @@ namespace zkutil
class ZooKeeper;
}
namespace fs = std::filesystem;
namespace DB
{
@ -98,11 +101,11 @@ struct DDLTaskBase
virtual String getShardID() const = 0;
virtual ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper);
virtual ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper);
inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; }
inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; }
inline String getShardNodePath() const { return entry_path + "/shards/" + getShardID(); }
inline String getActiveNodePath() const { return fs::path(entry_path) / "active" / host_id_str; }
inline String getFinishedNodePath() const { return fs::path(entry_path) / "finished" / host_id_str; }
inline String getShardNodePath() const { return fs::path(entry_path) / "shards" / getShardID(); }
static String getLogEntryName(UInt32 log_entry_number);
static UInt32 getLogEntryNumber(const String & log_entry_name);
@ -136,7 +139,7 @@ struct DatabaseReplicatedTask : public DDLTaskBase
String getShardID() const override;
void parseQueryFromEntry(ContextPtr context) override;
ContextPtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override;
ContextMutablePtr makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & zookeeper) override;
DatabaseReplicated * database;
};

Some files were not shown because too many files have changed in this diff Show More