mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove namespace pollution
This commit is contained in:
parent
5aff138956
commit
25f941020b
@ -28,7 +28,7 @@ public:
|
||||
void exception() override { logException(); }
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("ServerErrorHandler");
|
||||
Poco::Logger * log = &Poco::Logger::get("ServerErrorHandler");
|
||||
|
||||
void logException()
|
||||
{
|
||||
|
@ -9,13 +9,6 @@
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
|
||||
/// TODO Remove this.
|
||||
using Poco::Logger;
|
||||
using Poco::Message;
|
||||
using DB::LogsLevel;
|
||||
using DB::CurrentThread;
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename... Ts> constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); }
|
||||
@ -31,8 +24,8 @@ namespace
|
||||
|
||||
#define LOG_IMPL(logger, priority, PRIORITY, ...) do \
|
||||
{ \
|
||||
const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \
|
||||
(CurrentThread::getGroup()->client_logs_level >= (priority)); \
|
||||
const bool is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \
|
||||
(DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \
|
||||
if ((logger)->is((PRIORITY)) || is_clients_log) \
|
||||
{ \
|
||||
std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \
|
||||
@ -42,7 +35,7 @@ namespace
|
||||
file_function += __FILE__; \
|
||||
file_function += "; "; \
|
||||
file_function += __PRETTY_FUNCTION__; \
|
||||
Message poco_message((logger)->name(), formatted_message, \
|
||||
Poco::Message poco_message((logger)->name(), formatted_message, \
|
||||
(PRIORITY), file_function.c_str(), __LINE__); \
|
||||
channel->log(poco_message); \
|
||||
} \
|
||||
@ -50,9 +43,9 @@ namespace
|
||||
} while (false)
|
||||
|
||||
|
||||
#define LOG_TRACE(logger, ...) LOG_IMPL(logger, LogsLevel::trace, Message::PRIO_TRACE, __VA_ARGS__)
|
||||
#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, LogsLevel::debug, Message::PRIO_DEBUG, __VA_ARGS__)
|
||||
#define LOG_INFO(logger, ...) LOG_IMPL(logger, LogsLevel::information, Message::PRIO_INFORMATION, __VA_ARGS__)
|
||||
#define LOG_WARNING(logger, ...) LOG_IMPL(logger, LogsLevel::warning, Message::PRIO_WARNING, __VA_ARGS__)
|
||||
#define LOG_ERROR(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_ERROR, __VA_ARGS__)
|
||||
#define LOG_FATAL(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_FATAL, __VA_ARGS__)
|
||||
#define LOG_TRACE(logger, ...) LOG_IMPL(logger, DB::LogsLevel::trace, Poco::Message::PRIO_TRACE, __VA_ARGS__)
|
||||
#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG, __VA_ARGS__)
|
||||
#define LOG_INFO(logger, ...) LOG_IMPL(logger, DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION, __VA_ARGS__)
|
||||
#define LOG_WARNING(logger, ...) LOG_IMPL(logger, DB::LogsLevel::warning, Poco::Message::PRIO_WARNING, __VA_ARGS__)
|
||||
#define LOG_ERROR(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_ERROR, __VA_ARGS__)
|
||||
#define LOG_FATAL(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_FATAL, __VA_ARGS__)
|
||||
|
@ -124,7 +124,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context)
|
||||
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
|
||||
const StackTrace stack_trace(signal_context);
|
||||
|
||||
StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe.
|
||||
StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe.
|
||||
query_id.size = std::min(query_id.size, max_query_id_size);
|
||||
|
||||
DB::writeBinary(sig, out);
|
||||
@ -162,7 +162,7 @@ public:
|
||||
};
|
||||
|
||||
explicit SignalListener(BaseDaemon & daemon_)
|
||||
: log(&Logger::get("BaseDaemon"))
|
||||
: log(&Poco::Logger::get("BaseDaemon"))
|
||||
, daemon(daemon_)
|
||||
{
|
||||
}
|
||||
@ -231,7 +231,7 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
BaseDaemon & daemon;
|
||||
|
||||
void onTerminate(const std::string & message, UInt32 thread_num) const
|
||||
@ -715,7 +715,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
|
||||
|
||||
void BaseDaemon::logRevision() const
|
||||
{
|
||||
Logger::root().information("Starting " + std::string{VERSION_FULL}
|
||||
Poco::Logger::root().information("Starting " + std::string{VERSION_FULL}
|
||||
+ " with revision " + std::to_string(ClickHouseRevision::get())
|
||||
+ ", PID " + std::to_string(getpid()));
|
||||
}
|
||||
@ -732,7 +732,7 @@ void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn)
|
||||
{
|
||||
task_failed = true;
|
||||
Poco::AutoPtr<Poco::TaskFailedNotification> fn(_tfn);
|
||||
Logger *lg = &(logger());
|
||||
Poco::Logger * lg = &(logger());
|
||||
LOG_ERROR(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText());
|
||||
ServerApplication::terminate();
|
||||
}
|
||||
|
@ -135,7 +135,7 @@ static void attachSystemTables(const Context & context)
|
||||
int LocalServer::main(const std::vector<std::string> & /*args*/)
|
||||
try
|
||||
{
|
||||
Logger * log = &logger();
|
||||
Poco::Logger * log = &logger();
|
||||
ThreadStatus thread_status;
|
||||
UseSSL use_ssl;
|
||||
|
||||
|
@ -25,7 +25,7 @@ ODBCBlockInputStream::ODBCBlockInputStream(
|
||||
, result{statement}
|
||||
, iterator{result.begin()}
|
||||
, max_block_size{max_block_size_}
|
||||
, log(&Logger::get("ODBCBlockInputStream"))
|
||||
, log(&Poco::Logger::get("ODBCBlockInputStream"))
|
||||
{
|
||||
if (sample_block.columns() != result.columnCount())
|
||||
throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " + toString(sample_block.columns())
|
||||
|
@ -94,7 +94,7 @@ ODBCBlockOutputStream::ODBCBlockOutputStream(Poco::Data::Session && session_,
|
||||
, table_name(remote_table_name_)
|
||||
, sample_block(sample_block_)
|
||||
, quoting(quoting_)
|
||||
, log(&Logger::get("ODBCBlockOutputStream"))
|
||||
, log(&Poco::Logger::get("ODBCBlockOutputStream"))
|
||||
{
|
||||
description.init(sample_block);
|
||||
}
|
||||
|
@ -89,7 +89,7 @@ namespace CurrentMetrics
|
||||
namespace
|
||||
{
|
||||
|
||||
void setupTmpPath(Logger * log, const std::string & path)
|
||||
void setupTmpPath(Poco::Logger * log, const std::string & path)
|
||||
{
|
||||
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
|
||||
|
||||
@ -212,7 +212,7 @@ void Server::defineOptions(Poco::Util::OptionSet & options)
|
||||
|
||||
int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
Logger * log = &logger();
|
||||
Poco::Logger * log = &logger();
|
||||
UseSSL use_ssl;
|
||||
|
||||
ThreadStatus thread_status;
|
||||
|
@ -309,7 +309,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
throw;
|
||||
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
|
||||
LOG_WARNING(
|
||||
&Logger::get("AddressPatterns"),
|
||||
&Poco::Logger::get("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
@ -342,7 +342,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
throw;
|
||||
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
|
||||
LOG_WARNING(
|
||||
&Logger::get("AddressPatterns"),
|
||||
&Poco::Logger::get("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
|
@ -249,16 +249,16 @@ private:
|
||||
{
|
||||
}
|
||||
|
||||
Logger * get()
|
||||
Poco::Logger * get()
|
||||
{
|
||||
if (!log)
|
||||
log = &Logger::get("Connection (" + parent.getDescription() + ")");
|
||||
log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")");
|
||||
|
||||
return log;
|
||||
}
|
||||
|
||||
private:
|
||||
std::atomic<Logger *> log;
|
||||
std::atomic<Poco::Logger *> log;
|
||||
Connection & parent;
|
||||
};
|
||||
|
||||
|
@ -56,7 +56,7 @@ public:
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable)
|
||||
: Base(max_connections_,
|
||||
&Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
&Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_),
|
||||
port(port_),
|
||||
default_database(default_database_),
|
||||
|
@ -35,7 +35,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover(
|
||||
LoadBalancing load_balancing,
|
||||
time_t decrease_error_period_,
|
||||
size_t max_error_cap_)
|
||||
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Logger::get("ConnectionPoolWithFailover"))
|
||||
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover"))
|
||||
, default_load_balancing(load_balancing)
|
||||
{
|
||||
const std::string & local_hostname = getFQDNOrHostName();
|
||||
|
@ -35,7 +35,7 @@ TimeoutSetter::~TimeoutSetter()
|
||||
catch (std::exception & e)
|
||||
{
|
||||
// Sometimes catched on macos
|
||||
LOG_ERROR(&Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what());
|
||||
LOG_ERROR(&Poco::Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -66,21 +66,21 @@ ConfigProcessor::ConfigProcessor(
|
||||
, name_pool(new Poco::XML::NamePool(65521))
|
||||
, dom_parser(name_pool)
|
||||
{
|
||||
if (log_to_console && !Logger::has("ConfigProcessor"))
|
||||
if (log_to_console && !Poco::Logger::has("ConfigProcessor"))
|
||||
{
|
||||
channel_ptr = new Poco::ConsoleChannel;
|
||||
log = &Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
|
||||
log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
|
||||
}
|
||||
else
|
||||
{
|
||||
log = &Logger::get("ConfigProcessor");
|
||||
log = &Poco::Logger::get("ConfigProcessor");
|
||||
}
|
||||
}
|
||||
|
||||
ConfigProcessor::~ConfigProcessor()
|
||||
{
|
||||
if (channel_ptr) /// This means we have created a new console logger in the constructor.
|
||||
Logger::destroy("ConfigProcessor");
|
||||
Poco::Logger::destroy("ConfigProcessor");
|
||||
}
|
||||
|
||||
|
||||
|
@ -116,7 +116,7 @@ private:
|
||||
|
||||
bool throw_on_bad_incl;
|
||||
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
Poco::AutoPtr<Poco::Channel> channel_ptr;
|
||||
|
||||
Substitutions substitutions;
|
||||
|
@ -69,7 +69,7 @@ private:
|
||||
|
||||
static constexpr auto reload_interval = std::chrono::seconds(2);
|
||||
|
||||
Poco::Logger * log = &Logger::get("ConfigReloader");
|
||||
Poco::Logger * log = &Poco::Logger::get("ConfigReloader");
|
||||
|
||||
std::string path;
|
||||
std::string include_from_path;
|
||||
|
@ -202,7 +202,7 @@ bool DNSResolver::updateCache()
|
||||
}
|
||||
|
||||
if (!lost_hosts.empty())
|
||||
LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts);
|
||||
LOG_INFO(&Poco::Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts);
|
||||
|
||||
return updated;
|
||||
}
|
||||
|
@ -118,7 +118,7 @@ void throwFromErrnoWithPath(const std::string & s, const std::string & path, int
|
||||
|
||||
void tryLogCurrentException(const char * log_name, const std::string & start_of_message)
|
||||
{
|
||||
tryLogCurrentException(&Logger::get(log_name), start_of_message);
|
||||
tryLogCurrentException(&Poco::Logger::get(log_name), start_of_message);
|
||||
}
|
||||
|
||||
void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message)
|
||||
|
@ -37,7 +37,7 @@ private:
|
||||
Map map;
|
||||
bool initialized = false;
|
||||
|
||||
Logger * log = &Logger::get("FileChecker");
|
||||
Poco::Logger * log = &Poco::Logger::get("FileChecker");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -306,7 +306,7 @@ private:
|
||||
auto it = cells.find(key);
|
||||
if (it == cells.end())
|
||||
{
|
||||
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
|
||||
@ -324,7 +324,7 @@ private:
|
||||
|
||||
if (current_size > (1ull << 63))
|
||||
{
|
||||
LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it.");
|
||||
abort();
|
||||
}
|
||||
}
|
||||
|
@ -50,13 +50,13 @@ MemoryTracker::~MemoryTracker()
|
||||
void MemoryTracker::logPeakMemoryUsage() const
|
||||
{
|
||||
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak));
|
||||
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak));
|
||||
}
|
||||
|
||||
void MemoryTracker::logMemoryUsage(Int64 current) const
|
||||
{
|
||||
const auto * description = description_ptr.load(std::memory_order_relaxed);
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current));
|
||||
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current));
|
||||
}
|
||||
|
||||
|
||||
|
@ -152,9 +152,9 @@ private:
|
||||
|
||||
protected:
|
||||
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
PoolBase(unsigned max_items_, Logger * log_)
|
||||
PoolBase(unsigned max_items_, Poco::Logger * log_)
|
||||
: max_items(max_items_), log(log_)
|
||||
{
|
||||
items.reserve(max_items);
|
||||
|
@ -57,7 +57,7 @@ public:
|
||||
NestedPools nested_pools_,
|
||||
time_t decrease_error_period_,
|
||||
size_t max_error_cap_,
|
||||
Logger * log_)
|
||||
Poco::Logger * log_)
|
||||
: nested_pools(std::move(nested_pools_))
|
||||
, decrease_error_period(decrease_error_period_)
|
||||
, max_error_cap(max_error_cap_)
|
||||
@ -134,7 +134,7 @@ protected:
|
||||
/// The time when error counts were last decreased.
|
||||
time_t last_error_decrease_time = 0;
|
||||
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
template <typename TNestedPool>
|
||||
|
@ -79,7 +79,7 @@ namespace ErrorCodes
|
||||
|
||||
template <typename ProfilerImpl>
|
||||
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const UInt64 thread_id, const int clock_type, UInt32 period, const int pause_signal_)
|
||||
: log(&Logger::get("QueryProfiler"))
|
||||
: log(&Poco::Logger::get("QueryProfiler"))
|
||||
, pause_signal(pause_signal_)
|
||||
{
|
||||
#if USE_UNWIND
|
||||
|
@ -102,7 +102,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(config_prefix, keys);
|
||||
Logger * logger = &Logger::get("SensitiveDataMaskerConfigRead");
|
||||
Poco::Logger * logger = &Poco::Logger::get("SensitiveDataMaskerConfigRead");
|
||||
|
||||
std::set<std::string> used_names;
|
||||
|
||||
|
@ -43,9 +43,9 @@ StatusFile::StatusFile(const std::string & path_)
|
||||
}
|
||||
|
||||
if (!contents.empty())
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents);
|
||||
LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents);
|
||||
else
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path);
|
||||
LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path);
|
||||
}
|
||||
|
||||
fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666);
|
||||
@ -90,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_)
|
||||
StatusFile::~StatusFile()
|
||||
{
|
||||
if (0 != close(fd))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
|
||||
if (0 != unlink(path.c_str()))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
private:
|
||||
zkutil::ZooKeeperHolderPtr zookeeper_holder;
|
||||
std::string path;
|
||||
Logger * log = &Logger::get("zkutil::Increment");
|
||||
Poco::Logger * log = &Poco::Logger::get("zkutil::Increment");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
LeaderElection(DB::BackgroundSchedulePool & pool_, const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "")
|
||||
: pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_)
|
||||
, log_name("LeaderElection (" + path + ")")
|
||||
, log(&Logger::get(log_name))
|
||||
, log(&Poco::Logger::get(log_name))
|
||||
{
|
||||
task = pool.createTask(log_name, [this] { threadFunction(); });
|
||||
createNode();
|
||||
@ -67,7 +67,7 @@ private:
|
||||
LeadershipHandler handler;
|
||||
std::string identifier;
|
||||
std::string log_name;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
EphemeralNodeHolderPtr node;
|
||||
std::string node_name;
|
||||
|
@ -21,7 +21,7 @@ namespace zkutil
|
||||
zookeeper_holder(zookeeper_holder_),
|
||||
lock_path(lock_prefix_ + "/" + lock_name_),
|
||||
lock_message(lock_message_),
|
||||
log(&Logger::get("zkutil::Lock"))
|
||||
log(&Poco::Logger::get("zkutil::Lock"))
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
if (create_parent_path_)
|
||||
@ -72,7 +72,7 @@ namespace zkutil
|
||||
|
||||
std::string lock_path;
|
||||
std::string lock_message;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
};
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ static void check(int32_t code, const std::string & path)
|
||||
void ZooKeeper::init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_,
|
||||
int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_)
|
||||
{
|
||||
log = &Logger::get("ZooKeeper");
|
||||
log = &Poco::Logger::get("ZooKeeper");
|
||||
hosts = hosts_;
|
||||
identity = identity_;
|
||||
session_timeout_ms = session_timeout_ms_;
|
||||
|
@ -269,7 +269,7 @@ private:
|
||||
|
||||
std::mutex mutex;
|
||||
|
||||
Logger * log = nullptr;
|
||||
Poco::Logger * log = nullptr;
|
||||
};
|
||||
|
||||
|
||||
|
@ -70,7 +70,7 @@ private:
|
||||
mutable std::mutex mutex;
|
||||
ZooKeeper::Ptr ptr;
|
||||
|
||||
Logger * log = &Logger::get("ZooKeeperHolder");
|
||||
Poco::Logger * log = &Poco::Logger::get("ZooKeeperHolder");
|
||||
|
||||
static std::string nullptr_exception_message;
|
||||
};
|
||||
|
@ -20,8 +20,8 @@ int main(int argc, char ** argv)
|
||||
}
|
||||
|
||||
Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
|
||||
Logger::root().setChannel(channel);
|
||||
Logger::root().setLevel("trace");
|
||||
Poco::Logger::root().setChannel(channel);
|
||||
Poco::Logger::root().setLevel("trace");
|
||||
|
||||
zkutil::ZooKeeper zk(argv[1]);
|
||||
std::string unused;
|
||||
|
@ -12,7 +12,7 @@ TEST(Logger, Log)
|
||||
{
|
||||
Poco::Logger::root().setLevel("none");
|
||||
Poco::Logger::root().setChannel(Poco::AutoPtr<Poco::NullChannel>(new Poco::NullChannel()));
|
||||
Logger * log = &Logger::get("Log");
|
||||
Poco::Logger * log = &Poco::Logger::get("Log");
|
||||
|
||||
/// This test checks that we don't pass this string to fmtlib, because it is the only argument.
|
||||
EXPECT_NO_THROW(LOG_INFO(log, "Hello {} World"));
|
||||
|
@ -111,7 +111,7 @@ void BackgroundSchedulePoolTaskInfo::execute()
|
||||
static const int32_t slow_execution_threshold_ms = 200;
|
||||
|
||||
if (milliseconds >= slow_execution_threshold_ms)
|
||||
LOG_TRACE(&Logger::get(log_name), "Execution took {} ms.", milliseconds);
|
||||
LOG_TRACE(&Poco::Logger::get(log_name), "Execution took {} ms.", milliseconds);
|
||||
|
||||
{
|
||||
std::lock_guard lock_schedule(schedule_mutex);
|
||||
@ -156,7 +156,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met
|
||||
, memory_metric(memory_metric_)
|
||||
, thread_name(thread_name_)
|
||||
{
|
||||
LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size);
|
||||
LOG_INFO(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size);
|
||||
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
@ -179,7 +179,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool()
|
||||
queue.wakeUpAll();
|
||||
delayed_thread.join();
|
||||
|
||||
LOG_TRACE(&Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish.");
|
||||
LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish.");
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
}
|
||||
|
@ -994,7 +994,7 @@ private:
|
||||
class Sha256Password : public IPlugin
|
||||
{
|
||||
public:
|
||||
Sha256Password(RSA & public_key_, RSA & private_key_, Logger * log_)
|
||||
Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logger * log_)
|
||||
: public_key(public_key_)
|
||||
, private_key(private_key_)
|
||||
, log(log_)
|
||||
@ -1130,7 +1130,7 @@ public:
|
||||
private:
|
||||
RSA & public_key;
|
||||
RSA & private_key;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
String scramble;
|
||||
};
|
||||
#endif
|
||||
|
@ -598,7 +598,7 @@ namespace details
|
||||
|
||||
void SettingsCollectionUtils::warningNameNotFound(const StringRef & name)
|
||||
{
|
||||
static auto * log = &Logger::get("Settings");
|
||||
static auto * log = &Poco::Logger::get("Settings");
|
||||
LOG_WARNING(log, "Unknown setting {}, skipping", name);
|
||||
}
|
||||
|
||||
|
@ -47,7 +47,7 @@ protected:
|
||||
/** From here we will get the completed blocks after the aggregation. */
|
||||
std::unique_ptr<IBlockInputStream> impl;
|
||||
|
||||
Logger * log = &Logger::get("AggregatingBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("AggregatingBlockInputStream");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -168,7 +168,7 @@ private:
|
||||
const SortDescription description;
|
||||
String sign_column_name;
|
||||
|
||||
Logger * log = &Logger::get("CollapsingFinalBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("CollapsingFinalBlockInputStream");
|
||||
|
||||
bool first = true;
|
||||
|
||||
|
@ -21,7 +21,7 @@ ColumnGathererStream::ColumnGathererStream(
|
||||
const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_,
|
||||
size_t block_preferred_size_)
|
||||
: column_name(column_name_), sources(source_streams.size()), row_sources_buf(row_sources_buf_)
|
||||
, block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream"))
|
||||
, block_preferred_size(block_preferred_size_), log(&Poco::Logger::get("ColumnGathererStream"))
|
||||
{
|
||||
if (source_streams.empty())
|
||||
throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
@ -44,7 +44,7 @@ private:
|
||||
size_t bytes_to_transfer = 0;
|
||||
|
||||
using Logger = Poco::Logger;
|
||||
Logger * log = &Logger::get("CreatingSetsBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream");
|
||||
|
||||
void createAll();
|
||||
void createOne(SubqueryForSet & subquery);
|
||||
|
@ -104,7 +104,7 @@ private:
|
||||
String codec;
|
||||
size_t min_free_disk_space;
|
||||
|
||||
Logger * log = &Logger::get("MergeSortingBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("MergeSortingBlockInputStream");
|
||||
|
||||
Blocks blocks;
|
||||
size_t sum_rows_in_blocks = 0;
|
||||
|
@ -555,7 +555,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate
|
||||
/// Not yet partitioned (splitted to buckets) block. Will partition it and place result to 'splitted_blocks'.
|
||||
if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty())
|
||||
{
|
||||
LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
|
||||
LOG_TRACE(&Poco::Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split.");
|
||||
|
||||
input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block);
|
||||
input.block = Block();
|
||||
|
@ -96,7 +96,7 @@ private:
|
||||
std::atomic<bool> has_overflows {false};
|
||||
int current_bucket_num = -1;
|
||||
|
||||
Logger * log = &Logger::get("MergingAggregatedMemoryEfficientBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("MergingAggregatedMemoryEfficientBlockInputStream");
|
||||
|
||||
|
||||
struct Input
|
||||
|
@ -23,7 +23,7 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream(
|
||||
: description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_)
|
||||
, source_blocks(inputs_.size())
|
||||
, cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_)
|
||||
, log(&Logger::get("MergingSortedBlockInputStream"))
|
||||
, log(&Poco::Logger::get("MergingSortedBlockInputStream"))
|
||||
{
|
||||
children.insert(children.end(), inputs_.begin(), inputs_.end());
|
||||
header = children.at(0)->getHeader();
|
||||
|
@ -60,7 +60,7 @@ private:
|
||||
std::atomic<bool> executed {false};
|
||||
std::vector<std::unique_ptr<TemporaryFileStream>> temporary_inputs;
|
||||
|
||||
Logger * log = &Logger::get("ParallelAggregatingBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("ParallelAggregatingBlockInputStream");
|
||||
|
||||
|
||||
ManyAggregatedDataVariants many_data;
|
||||
|
@ -359,7 +359,7 @@ private:
|
||||
/// Wait for the completion of all threads.
|
||||
std::atomic<bool> joined_threads { false };
|
||||
|
||||
Logger * log = &Logger::get("ParallelInputsProcessor");
|
||||
Poco::Logger * log = &Poco::Logger::get("ParallelInputsProcessor");
|
||||
};
|
||||
|
||||
|
||||
|
@ -151,7 +151,7 @@ private:
|
||||
PoolMode pool_mode = PoolMode::GET_MANY;
|
||||
StorageID main_table = StorageID::createEmpty();
|
||||
|
||||
Logger * log = &Logger::get("RemoteBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("RemoteBlockInputStream");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,7 +28,7 @@ TTLBlockInputStream::TTLBlockInputStream(
|
||||
, current_time(current_time_)
|
||||
, force(force_)
|
||||
, old_ttl_infos(data_part->ttl_infos)
|
||||
, log(&Logger::get(storage.getLogName() + " (TTLBlockInputStream)"))
|
||||
, log(&Poco::Logger::get(storage.getLogName() + " (TTLBlockInputStream)"))
|
||||
, date_lut(DateLUT::instance())
|
||||
{
|
||||
children.push_back(input_);
|
||||
|
@ -52,7 +52,7 @@ private:
|
||||
NameSet empty_columns;
|
||||
|
||||
size_t rows_removed = 0;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
const DateLUTImpl & date_lut;
|
||||
|
||||
/// TODO rewrite defaults logic to evaluteMissingDefaults
|
||||
|
@ -253,7 +253,7 @@ private:
|
||||
bool started = false;
|
||||
bool all_read = false;
|
||||
|
||||
Logger * log = &Logger::get("UnionBlockInputStream");
|
||||
Poco::Logger * log = &Poco::Logger::get("UnionBlockInputStream");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -20,8 +20,8 @@ try
|
||||
using namespace DB;
|
||||
|
||||
Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
|
||||
Logger::root().setChannel(channel);
|
||||
Logger::root().setLevel("trace");
|
||||
Poco::Logger::root().setChannel(channel);
|
||||
Poco::Logger::root().setLevel("trace");
|
||||
|
||||
Block block1;
|
||||
|
||||
|
@ -32,7 +32,7 @@ namespace
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & global_context_)
|
||||
: IDatabase(name_)
|
||||
, log(&Logger::get("DatabaseDictionary(" + database_name + ")"))
|
||||
, log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")"))
|
||||
, global_context(global_context_.getGlobalContext())
|
||||
{
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context)
|
||||
: IDatabase(name_), log(&Logger::get(logger)), global_context(context.getGlobalContext())
|
||||
: IDatabase(name_), log(&Poco::Logger::get(logger)), global_context(context.getGlobalContext())
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ CacheDictionary::CacheDictionary(
|
||||
, update_queue_push_timeout_milliseconds(update_queue_push_timeout_milliseconds_)
|
||||
, query_wait_timeout_milliseconds(query_wait_timeout_milliseconds_)
|
||||
, max_threads_for_updates(max_threads_for_updates_)
|
||||
, log(&Logger::get("ExternalDictionaries"))
|
||||
, log(&Poco::Logger::get("ExternalDictionaries"))
|
||||
, size{roundUpToPowerOfTwoOrZero(std::max(size_, size_t(max_collision_length)))}
|
||||
, size_overlap_mask{this->size - 1}
|
||||
, cells{this->size}
|
||||
|
@ -314,7 +314,7 @@ private:
|
||||
const size_t query_wait_timeout_milliseconds;
|
||||
const size_t max_threads_for_updates;
|
||||
|
||||
Logger * const log;
|
||||
Poco::Logger * log;
|
||||
|
||||
mutable std::shared_mutex rw_lock;
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider)
|
||||
{
|
||||
Logger * log = &Logger::get("RegionsHierarchies");
|
||||
Poco::Logger * log = &Poco::Logger::get("RegionsHierarchies");
|
||||
|
||||
LOG_DEBUG(log, "Adding default regions hierarchy");
|
||||
data.emplace("", data_provider->getDefaultHierarchySource());
|
||||
|
@ -23,7 +23,7 @@ RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_)
|
||||
|
||||
void RegionsHierarchy::reload()
|
||||
{
|
||||
Logger * log = &Logger::get("RegionsHierarchy");
|
||||
Poco::Logger * log = &Poco::Logger::get("RegionsHierarchy");
|
||||
|
||||
if (!data_source->isModified())
|
||||
return;
|
||||
|
@ -42,7 +42,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames()
|
||||
|
||||
void RegionsNames::reload()
|
||||
{
|
||||
Logger * log = &Logger::get("RegionsNames");
|
||||
Poco::Logger * log = &Poco::Logger::get("RegionsNames");
|
||||
LOG_DEBUG(log, "Reloading regions names");
|
||||
|
||||
RegionID max_region_id = 0;
|
||||
|
@ -53,7 +53,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block_,
|
||||
const Context & context_)
|
||||
: log(&Logger::get("ExecutableDictionarySource"))
|
||||
: log(&Poco::Logger::get("ExecutableDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, command{config.getString(config_prefix + ".command")}
|
||||
, update_field{config.getString(config_prefix + ".update_field", "")}
|
||||
@ -64,7 +64,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(
|
||||
}
|
||||
|
||||
ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other)
|
||||
: log(&Logger::get("ExecutableDictionarySource"))
|
||||
: log(&Poco::Logger::get("ExecutableDictionarySource"))
|
||||
, update_time{other.update_time}
|
||||
, dict_struct{other.dict_struct}
|
||||
, command{other.command}
|
||||
|
@ -31,7 +31,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
Block & sample_block_,
|
||||
const Context & context_,
|
||||
bool check_config)
|
||||
: log(&Logger::get("HTTPDictionarySource"))
|
||||
: log(&Poco::Logger::get("HTTPDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, url{config.getString(config_prefix + ".url", "")}
|
||||
@ -71,7 +71,7 @@ HTTPDictionarySource::HTTPDictionarySource(
|
||||
}
|
||||
|
||||
HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
|
||||
: log(&Logger::get("HTTPDictionarySource"))
|
||||
: log(&Poco::Logger::get("HTTPDictionarySource"))
|
||||
, update_time{other.update_time}
|
||||
, dict_struct{other.dict_struct}
|
||||
, url{other.url}
|
||||
|
@ -125,7 +125,7 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
Block & sample_block_,
|
||||
const Context & context,
|
||||
bool check_config)
|
||||
: log(&Logger::get("LibraryDictionarySource"))
|
||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{dict_struct_}
|
||||
, config_prefix{config_prefix_}
|
||||
, path{config.getString(config_prefix + ".path", "")}
|
||||
@ -157,7 +157,7 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
}
|
||||
|
||||
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
|
||||
: log(&Logger::get("LibraryDictionarySource"))
|
||||
: log(&Poco::Logger::get("LibraryDictionarySource"))
|
||||
, dict_struct{other.dict_struct}
|
||||
, config_prefix{other.config_prefix}
|
||||
, path{other.path}
|
||||
|
@ -10,7 +10,7 @@ void ClickHouseLibrary::log(ClickHouseLibrary::LogLevel level, ClickHouseLibrary
|
||||
{
|
||||
using ClickHouseLibrary::LogLevel;
|
||||
|
||||
auto & logger = Logger::get(DICT_LOGGER_NAME);
|
||||
auto & logger = Poco::Logger::get(DICT_LOGGER_NAME);
|
||||
switch (level)
|
||||
{
|
||||
case LogLevel::TRACE:
|
||||
|
@ -58,7 +58,7 @@ MySQLDictionarySource::MySQLDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Block & sample_block_)
|
||||
: log(&Logger::get("MySQLDictionarySource"))
|
||||
: log(&Poco::Logger::get("MySQLDictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, db{config.getString(config_prefix + ".db", "")}
|
||||
@ -77,7 +77,7 @@ MySQLDictionarySource::MySQLDictionarySource(
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other)
|
||||
: log(&Logger::get("MySQLDictionarySource"))
|
||||
: log(&Poco::Logger::get("MySQLDictionarySource"))
|
||||
, update_time{other.update_time}
|
||||
, dict_struct{other.dict_struct}
|
||||
, db{other.db}
|
||||
|
@ -253,7 +253,7 @@ private:
|
||||
|
||||
std::exception_ptr creation_exception;
|
||||
|
||||
Logger * logger;
|
||||
Poco::Logger * logger;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -72,7 +72,7 @@ XDBCDictionarySource::XDBCDictionarySource(
|
||||
const Block & sample_block_,
|
||||
const Context & context_,
|
||||
const BridgeHelperPtr bridge_)
|
||||
: log(&Logger::get(bridge_->getName() + "DictionarySource"))
|
||||
: log(&Poco::Logger::get(bridge_->getName() + "DictionarySource"))
|
||||
, update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, db{config_.getString(config_prefix_ + ".db", "")}
|
||||
@ -96,7 +96,7 @@ XDBCDictionarySource::XDBCDictionarySource(
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other)
|
||||
: log(&Logger::get(other.bridge_helper->getName() + "DictionarySource"))
|
||||
: log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource"))
|
||||
, update_time{other.update_time}
|
||||
, dict_struct{other.dict_struct}
|
||||
, db{other.db}
|
||||
|
@ -91,7 +91,7 @@ bool DiskLocal::tryReserve(UInt64 bytes)
|
||||
std::lock_guard lock(DiskLocal::reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name));
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
@ -100,7 +100,7 @@ bool DiskLocal::tryReserve(UInt64 bytes)
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
@ -310,7 +310,7 @@ DiskLocalReservation::~DiskLocalReservation()
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -318,7 +318,7 @@ DiskLocalReservation::~DiskLocalReservation()
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
|
@ -99,7 +99,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig(
|
||||
}
|
||||
|
||||
writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning);
|
||||
LOG_WARNING(&Logger::get("DiskSelector"), warning.str());
|
||||
LOG_WARNING(&Poco::Logger::get("DiskSelector"), warning.str());
|
||||
}
|
||||
|
||||
return result;
|
||||
|
@ -515,7 +515,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, si
|
||||
{
|
||||
Metadata metadata(metadata_path, path);
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}",
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}",
|
||||
backQuote(metadata_path + path), metadata.s3_objects.size());
|
||||
|
||||
return std::make_unique<ReadIndirectBufferFromS3>(client, bucket, metadata, buf_size);
|
||||
@ -536,7 +536,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
/// Save empty metadata to disk to have ability to get file size while buffer is not finalized.
|
||||
metadata.save();
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path);
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, min_upload_part_size, buf_size);
|
||||
}
|
||||
@ -544,7 +544,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
{
|
||||
Metadata metadata(metadata_path, path);
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.",
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.",
|
||||
backQuote(metadata_path + path), s3_path, metadata.s3_objects.size());
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, min_upload_part_size, buf_size);
|
||||
@ -553,7 +553,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
|
||||
void DiskS3::remove(const String & path)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path));
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path));
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
if (file.isFile())
|
||||
@ -607,7 +607,7 @@ bool DiskS3::tryReserve(UInt64 bytes)
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name));
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
@ -616,7 +616,7 @@ bool DiskS3::tryReserve(UInt64 bytes)
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
@ -672,7 +672,7 @@ DiskS3Reservation::~DiskS3Reservation()
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -680,7 +680,7 @@ DiskS3Reservation::~DiskS3Reservation()
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyListConfiguration::getConfigurat
|
||||
cfg.proxyHost = proxies[index].getHost();
|
||||
cfg.proxyPort = proxies[index].getPort();
|
||||
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString());
|
||||
LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString());
|
||||
|
||||
return cfg;
|
||||
}
|
||||
|
@ -21,7 +21,7 @@ ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoin
|
||||
|
||||
Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const Aws::Http::HttpRequest &)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString());
|
||||
LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString());
|
||||
|
||||
/// 1 second is enough for now.
|
||||
/// TODO: Make timeouts configurable.
|
||||
@ -49,7 +49,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig
|
||||
/// Read proxy host as string from response body.
|
||||
Poco::StreamCopier::copyToString(response_body_stream, proxy_host);
|
||||
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port);
|
||||
LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port);
|
||||
|
||||
cfg.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str());
|
||||
cfg.proxyHost = proxy_host;
|
||||
|
@ -46,7 +46,7 @@ namespace
|
||||
throw Exception("Only HTTP/HTTPS schemas allowed in proxy resolver config: " + proxy_scheme, ErrorCodes::BAD_ARGUMENTS);
|
||||
auto proxy_port = proxy_resolver_config.getUInt(prefix + ".proxy_port");
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port);
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port);
|
||||
|
||||
return std::make_shared<S3::ProxyResolverConfiguration>(endpoint, proxy_scheme, proxy_port);
|
||||
}
|
||||
@ -70,7 +70,7 @@ namespace
|
||||
|
||||
proxies.push_back(proxy_uri);
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString());
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString());
|
||||
}
|
||||
|
||||
if (!proxies.empty())
|
||||
|
@ -245,7 +245,7 @@ StoragePolicySelector::StoragePolicySelector(
|
||||
"StoragePolicy name can contain only alphanumeric and '_' (" + name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
policies.emplace(name, std::make_shared<StoragePolicy>(name, config, config_prefix + "." + name, disks));
|
||||
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name));
|
||||
LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name));
|
||||
}
|
||||
|
||||
constexpr auto default_storage_policy_name = "default";
|
||||
|
@ -20,7 +20,7 @@ VolumeJBOD::VolumeJBOD(
|
||||
DiskSelectorPtr disk_selector
|
||||
) : IVolume(name_, config, config_prefix, disk_selector)
|
||||
{
|
||||
Logger * logger = &Logger::get("StorageConfiguration");
|
||||
Poco::Logger * logger = &Poco::Logger::get("StorageConfiguration");
|
||||
|
||||
auto has_max_bytes = config.has(config_prefix + ".max_data_part_size_bytes");
|
||||
auto has_max_ratio = config.has(config_prefix + ".max_data_part_size_ratio");
|
||||
|
@ -173,7 +173,7 @@ namespace
|
||||
auto msg = Poco::AnyCast<std::string>(session_data);
|
||||
if (!msg.empty())
|
||||
{
|
||||
LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg);
|
||||
LOG_TRACE((&Poco::Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg);
|
||||
/// Host can change IP
|
||||
const auto ip = DNSResolver::instance().resolveHost(host).toString();
|
||||
if (ip != session->getHost())
|
||||
|
@ -33,7 +33,7 @@ private:
|
||||
Aws::S3::Model::GetObjectResult read_result;
|
||||
std::unique_ptr<ReadBuffer> impl;
|
||||
|
||||
Logger * log = &Logger::get("ReadBufferFromS3");
|
||||
Poco::Logger * log = &Poco::Logger::get("ReadBufferFromS3");
|
||||
|
||||
public:
|
||||
explicit ReadBufferFromS3(
|
||||
|
@ -127,7 +127,7 @@ namespace detail
|
||||
if (!credentials.getUsername().empty())
|
||||
credentials.authenticate(request);
|
||||
|
||||
LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString());
|
||||
LOG_TRACE((&Poco::Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString());
|
||||
|
||||
auto sess = session->getSession();
|
||||
|
||||
|
@ -16,16 +16,17 @@
|
||||
|
||||
namespace
|
||||
{
|
||||
const std::pair<LogsLevel, Message::Priority> & convertLogLevel(Aws::Utils::Logging::LogLevel log_level)
|
||||
const std::pair<DB::LogsLevel, Poco::Message::Priority> & convertLogLevel(Aws::Utils::Logging::LogLevel log_level)
|
||||
{
|
||||
static const std::unordered_map<Aws::Utils::Logging::LogLevel, std::pair<LogsLevel, Message::Priority>> mapping = {
|
||||
{Aws::Utils::Logging::LogLevel::Off, {LogsLevel::none, Message::PRIO_FATAL}},
|
||||
{Aws::Utils::Logging::LogLevel::Fatal, {LogsLevel::error, Message::PRIO_FATAL}},
|
||||
{Aws::Utils::Logging::LogLevel::Error, {LogsLevel::error, Message::PRIO_ERROR}},
|
||||
{Aws::Utils::Logging::LogLevel::Warn, {LogsLevel::warning, Message::PRIO_WARNING}},
|
||||
{Aws::Utils::Logging::LogLevel::Info, {LogsLevel::information, Message::PRIO_INFORMATION}},
|
||||
{Aws::Utils::Logging::LogLevel::Debug, {LogsLevel::debug, Message::PRIO_DEBUG}},
|
||||
{Aws::Utils::Logging::LogLevel::Trace, {LogsLevel::trace, Message::PRIO_TRACE}},
|
||||
static const std::unordered_map<Aws::Utils::Logging::LogLevel, std::pair<DB::LogsLevel, Poco::Message::Priority>> mapping =
|
||||
{
|
||||
{Aws::Utils::Logging::LogLevel::Off, {DB::LogsLevel::none, Poco::Message::PRIO_FATAL}},
|
||||
{Aws::Utils::Logging::LogLevel::Fatal, {DB::LogsLevel::error, Poco::Message::PRIO_FATAL}},
|
||||
{Aws::Utils::Logging::LogLevel::Error, {DB::LogsLevel::error, Poco::Message::PRIO_ERROR}},
|
||||
{Aws::Utils::Logging::LogLevel::Warn, {DB::LogsLevel::warning, Poco::Message::PRIO_WARNING}},
|
||||
{Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}},
|
||||
{Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG}},
|
||||
{Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TRACE}},
|
||||
};
|
||||
return mapping.at(log_level);
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP(
|
||||
request.setHost(uri.getHost());
|
||||
request.setChunkedTransferEncoding(true);
|
||||
|
||||
LOG_TRACE((&Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString());
|
||||
LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString());
|
||||
|
||||
ostr = &session->sendRequest(request);
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ private:
|
||||
String upload_id;
|
||||
std::vector<String> part_tags;
|
||||
|
||||
Logger * log = &Logger::get("WriteBufferFromS3");
|
||||
Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3");
|
||||
|
||||
public:
|
||||
explicit WriteBufferFromS3(
|
||||
|
@ -1565,7 +1565,7 @@ public:
|
||||
|
||||
~MergingAndConvertingBlockInputStream() override
|
||||
{
|
||||
LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
|
||||
LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish");
|
||||
|
||||
/// We need to wait for threads to finish before destructor of 'parallel_merge_data',
|
||||
/// because the threads access 'parallel_merge_data'.
|
||||
|
@ -1048,7 +1048,7 @@ protected:
|
||||
|
||||
std::mutex mutex;
|
||||
|
||||
Logger * log = &Logger::get("Aggregator");
|
||||
Poco::Logger * log = &Poco::Logger::get("Aggregator");
|
||||
|
||||
/// Returns true if you can abort the current task.
|
||||
CancellationHook isCancelled;
|
||||
|
@ -180,7 +180,7 @@ void SelectStreamFactory::createForShard(
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
if (shard_info.hasRemoteConnections())
|
||||
{
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num);
|
||||
LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num);
|
||||
emplace_remote_stream();
|
||||
}
|
||||
else
|
||||
@ -216,7 +216,7 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
/// If we reached this point, local replica is stale.
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay);
|
||||
LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay);
|
||||
|
||||
if (!settings.fallback_to_stale_replicas_for_distributed_queries)
|
||||
{
|
||||
@ -264,7 +264,7 @@ void SelectStreamFactory::createForShard(
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED)
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num);
|
||||
LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num);
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
@ -287,7 +287,7 @@ void NamedSession::release()
|
||||
*/
|
||||
struct ContextShared
|
||||
{
|
||||
Logger * log = &Logger::get("Context");
|
||||
Poco::Logger * log = &Poco::Logger::get("Context");
|
||||
|
||||
/// For access of most of shared objects. Recursive mutex.
|
||||
mutable std::recursive_mutex mutex;
|
||||
|
@ -220,7 +220,7 @@ static bool isSupportedAlterType(int type)
|
||||
|
||||
|
||||
DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix)
|
||||
: context(context_), log(&Logger::get("DDLWorker"))
|
||||
: context(context_), log(&Poco::Logger::get("DDLWorker"))
|
||||
{
|
||||
queue_dir = zk_root_dir;
|
||||
if (queue_dir.back() == '/')
|
||||
@ -1073,7 +1073,7 @@ class DDLQueryStatusInputStream : public IBlockInputStream
|
||||
public:
|
||||
|
||||
DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_)
|
||||
: node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Logger::get("DDLQueryStatusInputStream"))
|
||||
: node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Poco::Logger::get("DDLQueryStatusInputStream"))
|
||||
{
|
||||
sample = Block{
|
||||
{std::make_shared<DataTypeString>(), "host"},
|
||||
@ -1235,7 +1235,7 @@ private:
|
||||
String node_path;
|
||||
const Context & context;
|
||||
Stopwatch watch;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
Block sample;
|
||||
|
||||
|
@ -101,7 +101,7 @@ private:
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
std::unique_ptr<Context> current_context;
|
||||
|
||||
std::string host_fqdn; /// current host domain name
|
||||
|
@ -124,7 +124,7 @@ EmbeddedDictionaries::EmbeddedDictionaries(
|
||||
std::unique_ptr<GeoDictionariesLoader> geo_dictionaries_loader_,
|
||||
Context & context_,
|
||||
const bool throw_on_error)
|
||||
: log(&Logger::get("EmbeddedDictionaries"))
|
||||
: log(&Poco::Logger::get("EmbeddedDictionaries"))
|
||||
, context(context_)
|
||||
, geo_dictionaries_loader(std::move(geo_dictionaries_loader_))
|
||||
, reload_period(context_.getConfigRef().getInt("builtin_dictionaries_reload_interval", 3600))
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
|
||||
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
|
||||
ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
|
||||
: ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader"))
|
||||
: ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
setConfigSettings({"dictionary", "name", "database"});
|
||||
|
@ -100,7 +100,7 @@ namespace
|
||||
class ExternalLoader::LoadablesConfigReader : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
LoadablesConfigReader(const String & type_name_, Logger * log_)
|
||||
LoadablesConfigReader(const String & type_name_, Poco::Logger * log_)
|
||||
: type_name(type_name_), log(log_)
|
||||
{
|
||||
}
|
||||
@ -366,7 +366,7 @@ private:
|
||||
}
|
||||
|
||||
const String type_name;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
std::mutex mutex;
|
||||
ExternalLoaderConfigSettings settings;
|
||||
@ -389,7 +389,7 @@ public:
|
||||
LoadingDispatcher(
|
||||
const CreateObjectFunction & create_object_function_,
|
||||
const String & type_name_,
|
||||
Logger * log_)
|
||||
Poco::Logger * log_)
|
||||
: create_object(create_object_function_)
|
||||
, type_name(type_name_)
|
||||
, log(log_)
|
||||
@ -1140,7 +1140,7 @@ private:
|
||||
|
||||
const CreateObjectFunction create_object;
|
||||
const String type_name;
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
std::condition_variable event;
|
||||
@ -1220,7 +1220,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
ExternalLoader::ExternalLoader(const String & type_name_, Logger * log_)
|
||||
ExternalLoader::ExternalLoader(const String & type_name_, Poco::Logger * log_)
|
||||
: config_files_reader(std::make_unique<LoadablesConfigReader>(type_name_, log_))
|
||||
, loading_dispatcher(std::make_unique<LoadingDispatcher>(
|
||||
[this](auto && a, auto && b, auto && c) { return createObject(a, b, c); },
|
||||
|
@ -82,7 +82,7 @@ public:
|
||||
template <typename T>
|
||||
static constexpr bool is_vector_load_result_type = std::is_same_v<T, LoadResults> || std::is_same_v<T, Loadables>;
|
||||
|
||||
ExternalLoader(const String & type_name_, Logger * log);
|
||||
ExternalLoader(const String & type_name_, Poco::Logger * log);
|
||||
virtual ~ExternalLoader();
|
||||
|
||||
/// Adds a repository which will be used to read configurations from.
|
||||
|
@ -11,7 +11,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
ExternalModelsLoader::ExternalModelsLoader(Context & context_)
|
||||
: ExternalLoader("external model", &Logger::get("ExternalModelsLoader"))
|
||||
: ExternalLoader("external model", &Poco::Logger::get("ExternalModelsLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
setConfigSettings({"model", "name", {}});
|
||||
|
@ -203,7 +203,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
|
||||
, any_take_last_row(any_take_last_row_)
|
||||
, asof_inequality(table_join->getAsofInequality())
|
||||
, data(std::make_shared<RightTableData>())
|
||||
, log(&Logger::get("HashJoin"))
|
||||
, log(&Poco::Logger::get("HashJoin"))
|
||||
{
|
||||
setSampleBlock(right_sample_block);
|
||||
}
|
||||
|
@ -201,7 +201,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
, storage(storage_)
|
||||
, input(input_)
|
||||
, input_pipe(std::move(input_pipe_))
|
||||
, log(&Logger::get("InterpreterSelectQuery"))
|
||||
, log(&Poco::Logger::get("InterpreterSelectQuery"))
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
|
@ -139,7 +139,7 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__);
|
||||
tryLogCurrentException(part_log ? part_log->log : &Poco::Logger::get("PartLog"), __PRETTY_FUNCTION__);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -84,7 +84,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
|
||||
if (!is_unlimited_query && max_size && processes.size() >= max_size)
|
||||
{
|
||||
if (queue_max_wait_ms)
|
||||
LOG_WARNING(&Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms);
|
||||
LOG_WARNING(&Poco::Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms);
|
||||
if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; }))
|
||||
throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
|
||||
}
|
||||
@ -231,7 +231,7 @@ ProcessListEntry::~ProcessListEntry()
|
||||
auto user_process_list_it = parent.user_to_queries.find(user);
|
||||
if (user_process_list_it == parent.user_to_queries.end())
|
||||
{
|
||||
LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList");
|
||||
LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList");
|
||||
std::terminate();
|
||||
}
|
||||
|
||||
@ -250,7 +250,7 @@ ProcessListEntry::~ProcessListEntry()
|
||||
|
||||
if (!found)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
|
||||
LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
|
||||
std::terminate();
|
||||
}
|
||||
parent.have_space.notify_all();
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
/// store all set elements in explicit form.
|
||||
/// This is needed for subsequent use for index.
|
||||
Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_)
|
||||
: log(&Logger::get("Set")),
|
||||
: log(&Poco::Logger::get("Set")),
|
||||
limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_)
|
||||
{
|
||||
}
|
||||
@ -105,7 +105,7 @@ private:
|
||||
/// Types for set_elements.
|
||||
DataTypes set_elements_types;
|
||||
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Limitations on the maximum size of the set
|
||||
SizeLimits limits;
|
||||
|
@ -41,7 +41,7 @@ std::shared_ptr<TSystemLog> createSystemLog(
|
||||
if (database != default_database_name)
|
||||
{
|
||||
/// System tables must be loaded before other tables, but loading order is undefined for all databases except `system`
|
||||
LOG_ERROR(&Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database);
|
||||
LOG_ERROR(&Poco::Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database);
|
||||
database = default_database_name;
|
||||
}
|
||||
|
||||
|
@ -152,7 +152,7 @@ public:
|
||||
ASTPtr getCreateTableQuery() override;
|
||||
|
||||
protected:
|
||||
Logger * log;
|
||||
Poco::Logger * log;
|
||||
|
||||
private:
|
||||
/* Saving thread data */
|
||||
@ -207,7 +207,7 @@ SystemLog<LogElement>::SystemLog(Context & context_,
|
||||
, flush_interval_milliseconds(flush_interval_milliseconds_)
|
||||
{
|
||||
assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE);
|
||||
log = &Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")");
|
||||
log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")");
|
||||
}
|
||||
|
||||
|
||||
|
@ -105,7 +105,7 @@ static void logQuery(const String & query, const Context & context, bool interna
|
||||
{
|
||||
if (internal)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), "(internal) {}", joinLines(query));
|
||||
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(internal) {}", joinLines(query));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -113,7 +113,7 @@ static void logQuery(const String & query, const Context & context, bool interna
|
||||
const auto & initial_query_id = context.getClientInfo().initial_query_id;
|
||||
const auto & current_user = context.getClientInfo().current_user;
|
||||
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), "(from {}{}{}) {}",
|
||||
LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}) {}",
|
||||
context.getClientInfo().current_address.toString(),
|
||||
(current_user != "default" ? ", user: " + context.getClientInfo().current_user : ""),
|
||||
(!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()),
|
||||
@ -145,10 +145,10 @@ static void setExceptionStackTrace(QueryLogElement & elem)
|
||||
static void logException(Context & context, QueryLogElement & elem)
|
||||
{
|
||||
if (elem.stack_trace.empty())
|
||||
LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})",
|
||||
LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})",
|
||||
elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query));
|
||||
else
|
||||
LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})"
|
||||
LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})"
|
||||
", Stack trace (when copying this message, always include the lines below):\n\n{}",
|
||||
elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query), elem.stack_trace);
|
||||
}
|
||||
@ -505,7 +505,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
if (elem.read_rows != 0)
|
||||
{
|
||||
LOG_INFO(&Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds,
|
||||
static_cast<size_t>(elem.read_rows / elapsed_seconds),
|
||||
ReadableSize(elem.read_bytes / elapsed_seconds));
|
||||
@ -585,7 +585,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
std::stringstream log_str;
|
||||
log_str << "Query pipeline:\n";
|
||||
res.in->dumpTree(log_str);
|
||||
LOG_DEBUG(&Logger::get("executeQuery"), log_str.str());
|
||||
LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -24,8 +24,8 @@ int main(int, char **)
|
||||
try
|
||||
{
|
||||
Poco::AutoPtr<Poco::ConsoleChannel> channel = new Poco::ConsoleChannel(std::cerr);
|
||||
Logger::root().setChannel(channel);
|
||||
Logger::root().setLevel("trace");
|
||||
Poco::Logger::root().setChannel(channel);
|
||||
Poco::Logger::root().setLevel("trace");
|
||||
|
||||
/// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed.
|
||||
DateLUT::instance();
|
||||
|
@ -149,7 +149,7 @@ Chunk IRowInputFormat::generate()
|
||||
{
|
||||
if (num_errors && (params.allow_errors_num > 0 || params.allow_errors_ratio > 0))
|
||||
{
|
||||
Logger * log = &Logger::get("IRowInputFormat");
|
||||
Poco::Logger * log = &Poco::Logger::get("IRowInputFormat");
|
||||
LOG_TRACE(log, "Skipped {} rows with errors while reading the input stream", num_errors);
|
||||
}
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user