mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 21:12:28 +00:00
Merge branch 'ClickHouse:master' into #31363_format_template_configure_in_settings
This commit is contained in:
commit
e6844a5412
@ -33,7 +33,8 @@ namespace Poco
|
||||
|
||||
|
||||
class Exception;
|
||||
|
||||
class Logger;
|
||||
using LoggerPtr = std::shared_ptr<Logger>;
|
||||
|
||||
class Foundation_API Logger : public Channel
|
||||
/// Logger is a special Channel that acts as the main
|
||||
@ -870,6 +871,11 @@ public:
|
||||
/// If the Logger does not yet exist, it is created, based
|
||||
/// on its parent logger.
|
||||
|
||||
static LoggerPtr getShared(const std::string & name);
|
||||
/// Returns a shared pointer to the Logger with the given name.
|
||||
/// If the Logger does not yet exist, it is created, based
|
||||
/// on its parent logger.
|
||||
|
||||
static Logger & unsafeGet(const std::string & name);
|
||||
/// Returns a reference to the Logger with the given name.
|
||||
/// If the Logger does not yet exist, it is created, based
|
||||
@ -885,6 +891,11 @@ public:
|
||||
/// given name. The Logger's Channel and log level as set as
|
||||
/// specified.
|
||||
|
||||
static LoggerPtr createShared(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
|
||||
/// Creates and returns a shared pointer to a Logger with the
|
||||
/// given name. The Logger's Channel and log level as set as
|
||||
/// specified.
|
||||
|
||||
static Logger & root();
|
||||
/// Returns a reference to the root logger, which is the ultimate
|
||||
/// ancestor of all Loggers.
|
||||
@ -893,7 +904,7 @@ public:
|
||||
/// Returns a pointer to the Logger with the given name if it
|
||||
/// exists, or a null pointer otherwise.
|
||||
|
||||
static void destroy(const std::string & name);
|
||||
static bool destroy(const std::string & name);
|
||||
/// Destroys the logger with the specified name. Does nothing
|
||||
/// if the logger is not found.
|
||||
///
|
||||
@ -938,6 +949,7 @@ protected:
|
||||
void log(const std::string & text, Message::Priority prio, const char * file, int line);
|
||||
|
||||
static std::string format(const std::string & fmt, int argc, std::string argv[]);
|
||||
static Logger & unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION);
|
||||
static Logger & parent(const std::string & name);
|
||||
static void add(Logger * pLogger);
|
||||
static Logger * find(const std::string & name);
|
||||
@ -952,7 +964,6 @@ private:
|
||||
std::atomic_int _level;
|
||||
|
||||
static LoggerMap * _pLoggerMap;
|
||||
static Mutex _mapMtx;
|
||||
};
|
||||
|
||||
|
||||
|
@ -38,15 +38,15 @@ public:
|
||||
/// Creates the RefCountedObject.
|
||||
/// The initial reference count is one.
|
||||
|
||||
void duplicate() const;
|
||||
/// Increments the object's reference count.
|
||||
size_t duplicate() const;
|
||||
/// Increments the object's reference count, returns reference count before call.
|
||||
|
||||
void release() const throw();
|
||||
size_t release() const throw();
|
||||
/// Decrements the object's reference count
|
||||
/// and deletes the object if the count
|
||||
/// reaches zero.
|
||||
/// reaches zero, returns reference count before call.
|
||||
|
||||
int referenceCount() const;
|
||||
size_t referenceCount() const;
|
||||
/// Returns the reference count.
|
||||
|
||||
protected:
|
||||
@ -57,36 +57,40 @@ private:
|
||||
RefCountedObject(const RefCountedObject &);
|
||||
RefCountedObject & operator=(const RefCountedObject &);
|
||||
|
||||
mutable AtomicCounter _counter;
|
||||
mutable std::atomic<size_t> _counter;
|
||||
};
|
||||
|
||||
|
||||
//
|
||||
// inlines
|
||||
//
|
||||
inline int RefCountedObject::referenceCount() const
|
||||
inline size_t RefCountedObject::referenceCount() const
|
||||
{
|
||||
return _counter.value();
|
||||
return _counter.load(std::memory_order_acquire);
|
||||
}
|
||||
|
||||
|
||||
inline void RefCountedObject::duplicate() const
|
||||
inline size_t RefCountedObject::duplicate() const
|
||||
{
|
||||
++_counter;
|
||||
return _counter.fetch_add(1, std::memory_order_acq_rel);
|
||||
}
|
||||
|
||||
|
||||
inline void RefCountedObject::release() const throw()
|
||||
inline size_t RefCountedObject::release() const throw()
|
||||
{
|
||||
size_t reference_count_before = _counter.fetch_sub(1, std::memory_order_acq_rel);
|
||||
|
||||
try
|
||||
{
|
||||
if (--_counter == 0)
|
||||
if (reference_count_before == 1)
|
||||
delete this;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
poco_unexpected();
|
||||
}
|
||||
|
||||
return reference_count_before;
|
||||
}
|
||||
|
||||
|
||||
|
@ -20,12 +20,29 @@
|
||||
#include "Poco/NumberParser.h"
|
||||
#include "Poco/String.h"
|
||||
|
||||
#include <mutex>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
std::mutex & getLoggerMutex()
|
||||
{
|
||||
auto get_logger_mutex_placeholder_memory = []()
|
||||
{
|
||||
static char buffer[sizeof(std::mutex)]{};
|
||||
return buffer;
|
||||
};
|
||||
|
||||
static std::mutex * logger_mutex = new (get_logger_mutex_placeholder_memory()) std::mutex();
|
||||
return *logger_mutex;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace Poco {
|
||||
|
||||
|
||||
Logger::LoggerMap* Logger::_pLoggerMap = 0;
|
||||
Mutex Logger::_mapMtx;
|
||||
const std::string Logger::ROOT;
|
||||
|
||||
|
||||
@ -112,7 +129,7 @@ void Logger::dump(const std::string& msg, const void* buffer, std::size_t length
|
||||
|
||||
void Logger::setLevel(const std::string& name, int level)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
@ -131,7 +148,7 @@ void Logger::setLevel(const std::string& name, int level)
|
||||
|
||||
void Logger::setChannel(const std::string& name, Channel* pChannel)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
@ -150,7 +167,7 @@ void Logger::setChannel(const std::string& name, Channel* pChannel)
|
||||
|
||||
void Logger::setProperty(const std::string& loggerName, const std::string& propertyName, const std::string& value)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
@ -280,13 +297,41 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
struct LoggerDeleter
|
||||
{
|
||||
void operator()(Poco::Logger * logger)
|
||||
{
|
||||
if (Logger::destroy(logger->name()))
|
||||
return;
|
||||
|
||||
logger->release();
|
||||
}
|
||||
};
|
||||
|
||||
inline LoggerPtr makeLoggerPtr(Logger & logger)
|
||||
{
|
||||
logger.duplicate();
|
||||
return std::shared_ptr<Logger>(&logger, LoggerDeleter());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Logger& Logger::get(const std::string& name)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return unsafeGet(name);
|
||||
}
|
||||
|
||||
LoggerPtr Logger::getShared(const std::string & name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return makeLoggerPtr(unsafeGet(name));
|
||||
}
|
||||
|
||||
Logger& Logger::unsafeGet(const std::string& name)
|
||||
{
|
||||
@ -310,18 +355,21 @@ Logger& Logger::unsafeGet(const std::string& name)
|
||||
|
||||
Logger& Logger::create(const std::string& name, Channel* pChannel, int level)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (find(name)) throw ExistsException();
|
||||
Logger* pLogger = new Logger(name, pChannel, level);
|
||||
add(pLogger);
|
||||
return *pLogger;
|
||||
return unsafeCreate(name, pChannel, level);
|
||||
}
|
||||
|
||||
LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return makeLoggerPtr(unsafeCreate(name, pChannel, level));
|
||||
}
|
||||
|
||||
Logger& Logger::root()
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return unsafeGet(ROOT);
|
||||
}
|
||||
@ -329,7 +377,7 @@ Logger& Logger::root()
|
||||
|
||||
Logger* Logger::has(const std::string& name)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
return find(name);
|
||||
}
|
||||
@ -337,7 +385,7 @@ Logger* Logger::has(const std::string& name)
|
||||
|
||||
void Logger::shutdown()
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
@ -363,25 +411,29 @@ Logger* Logger::find(const std::string& name)
|
||||
}
|
||||
|
||||
|
||||
void Logger::destroy(const std::string& name)
|
||||
bool Logger::destroy(const std::string& name)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
if (_pLoggerMap)
|
||||
{
|
||||
LoggerMap::iterator it = _pLoggerMap->find(name);
|
||||
if (it != _pLoggerMap->end())
|
||||
{
|
||||
it->second->release();
|
||||
_pLoggerMap->erase(it);
|
||||
if (it->second->release() == 1)
|
||||
_pLoggerMap->erase(it);
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void Logger::names(std::vector<std::string>& names)
|
||||
{
|
||||
Mutex::ScopedLock lock(_mapMtx);
|
||||
std::lock_guard<std::mutex> lock(getLoggerMutex());
|
||||
|
||||
names.clear();
|
||||
if (_pLoggerMap)
|
||||
@ -393,6 +445,14 @@ void Logger::names(std::vector<std::string>& names)
|
||||
}
|
||||
}
|
||||
|
||||
Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level)
|
||||
{
|
||||
if (find(name)) throw ExistsException();
|
||||
Logger* pLogger = new Logger(name, pChannel, level);
|
||||
add(pLogger);
|
||||
|
||||
return *pLogger;
|
||||
}
|
||||
|
||||
Logger& Logger::parent(const std::string& name)
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ public:
|
||||
const String & host_id_,
|
||||
const String & proxy_database_name_,
|
||||
ContextMutablePtr context_,
|
||||
Poco::Logger * log_)
|
||||
LoggerRawPtr log_)
|
||||
: WithMutableContext(context_),
|
||||
task_zookeeper_path(task_path_),
|
||||
host_id(host_id_),
|
||||
@ -230,7 +230,7 @@ private:
|
||||
|
||||
bool experimental_use_sample_offset{false};
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerRawPtr log;
|
||||
|
||||
UInt64 max_table_tries = 3;
|
||||
UInt64 max_shard_partition_tries = 3;
|
||||
|
@ -177,7 +177,7 @@ public:
|
||||
auto watch_callback =
|
||||
[my_stale = stale] (const Coordination::WatchResponse & rsp)
|
||||
{
|
||||
auto logger = &Poco::Logger::get("ClusterCopier");
|
||||
auto logger = getLogger("ClusterCopier");
|
||||
if (rsp.error == Coordination::Error::ZOK)
|
||||
{
|
||||
switch (rsp.type)
|
||||
|
@ -375,7 +375,7 @@ int KeeperClient::main(const std::vector<String> & /* args */)
|
||||
|
||||
if (!config().has("host") && !config().has("port") && !keys.empty())
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("KeeperClient"), "Found keeper node in the config.xml, will use it for connection");
|
||||
LOG_INFO(getLogger("KeeperClient"), "Found keeper node in the config.xml, will use it for connection");
|
||||
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
|
@ -28,7 +28,7 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
|
||||
po::store(po::command_line_parser(argc, argv).options(desc).run(), options);
|
||||
Poco::AutoPtr<Poco::ConsoleChannel> console_channel(new Poco::ConsoleChannel);
|
||||
|
||||
Poco::Logger * logger = &Poco::Logger::get("KeeperConverter");
|
||||
LoggerPtr logger = getLogger("KeeperConverter");
|
||||
logger->setChannel(console_channel);
|
||||
|
||||
if (options.count("help"))
|
||||
|
@ -624,7 +624,7 @@ catch (...)
|
||||
|
||||
void Keeper::logRevision() const
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("Application"),
|
||||
LOG_INFO(getLogger("Application"),
|
||||
"Starting ClickHouse Keeper {} (revision: {}, git hash: {}, build id: {}), PID {}",
|
||||
VERSION_STRING,
|
||||
ClickHouseRevision::getVersionRevision(),
|
||||
|
@ -13,7 +13,7 @@ CatBoostLibraryHandlerFactory & CatBoostLibraryHandlerFactory::instance()
|
||||
}
|
||||
|
||||
CatBoostLibraryHandlerFactory::CatBoostLibraryHandlerFactory()
|
||||
: log(&Poco::Logger::get("CatBoostLibraryHandlerFactory"))
|
||||
: log(getLogger("CatBoostLibraryHandlerFactory"))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -31,7 +31,7 @@ private:
|
||||
/// map: model path --> catboost library handler
|
||||
std::unordered_map<String, CatBoostLibraryHandlerPtr> library_handlers TSA_GUARDED_BY(mutex);
|
||||
std::mutex mutex;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,40 +9,40 @@ const char DICT_LOGGER_NAME[] = "LibraryDictionarySourceExternal";
|
||||
|
||||
void ExternalDictionaryLibraryAPI::log(LogLevel level, CString msg)
|
||||
{
|
||||
auto & logger = Poco::Logger::get(DICT_LOGGER_NAME);
|
||||
auto logger = getLogger(DICT_LOGGER_NAME);
|
||||
switch (level)
|
||||
{
|
||||
case LogLevel::TRACE:
|
||||
if (logger.trace())
|
||||
logger.trace(msg);
|
||||
if (logger->trace())
|
||||
logger->trace(msg);
|
||||
break;
|
||||
case LogLevel::DEBUG:
|
||||
if (logger.debug())
|
||||
logger.debug(msg);
|
||||
if (logger->debug())
|
||||
logger->debug(msg);
|
||||
break;
|
||||
case LogLevel::INFORMATION:
|
||||
if (logger.information())
|
||||
logger.information(msg);
|
||||
if (logger->information())
|
||||
logger->information(msg);
|
||||
break;
|
||||
case LogLevel::NOTICE:
|
||||
if (logger.notice())
|
||||
logger.notice(msg);
|
||||
if (logger->notice())
|
||||
logger->notice(msg);
|
||||
break;
|
||||
case LogLevel::WARNING:
|
||||
if (logger.warning())
|
||||
logger.warning(msg);
|
||||
if (logger->warning())
|
||||
logger->warning(msg);
|
||||
break;
|
||||
case LogLevel::ERROR:
|
||||
if (logger.error())
|
||||
logger.error(msg);
|
||||
if (logger->error())
|
||||
logger->error(msg);
|
||||
break;
|
||||
case LogLevel::CRITICAL:
|
||||
if (logger.critical())
|
||||
logger.critical(msg);
|
||||
if (logger->critical())
|
||||
logger->critical(msg);
|
||||
break;
|
||||
case LogLevel::FATAL:
|
||||
if (logger.fatal())
|
||||
logger.fatal(msg);
|
||||
if (logger->fatal())
|
||||
logger->fatal(msg);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -26,7 +26,7 @@ void ExternalDictionaryLibraryHandlerFactory::create(
|
||||
|
||||
if (library_handlers.contains(dictionary_id))
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id);
|
||||
LOG_WARNING(getLogger("ExternalDictionaryLibraryHandlerFactory"), "Library handler with dictionary id {} already exists", dictionary_id);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,7 @@ LibraryBridgeHandlerFactory::LibraryBridgeHandlerFactory(
|
||||
size_t keep_alive_timeout_,
|
||||
ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get(name_))
|
||||
, log(getLogger(name_))
|
||||
, name(name_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ public:
|
||||
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
const std::string name;
|
||||
const size_t keep_alive_timeout;
|
||||
};
|
||||
|
@ -47,7 +47,7 @@ namespace
|
||||
if (!response.sent())
|
||||
*response.send() << message << '\n';
|
||||
|
||||
LOG_WARNING(&Poco::Logger::get("LibraryBridge"), fmt::runtime(message));
|
||||
LOG_WARNING(getLogger("LibraryBridge"), fmt::runtime(message));
|
||||
}
|
||||
|
||||
std::shared_ptr<Block> parseColumns(String && column_string)
|
||||
@ -92,7 +92,7 @@ static void writeData(Block data, OutputFormatPtr format)
|
||||
ExternalDictionaryLibraryBridgeRequestHandler::ExternalDictionaryLibraryBridgeRequestHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeRequestHandler"))
|
||||
, log(getLogger("ExternalDictionaryLibraryBridgeRequestHandler"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -380,7 +380,7 @@ void ExternalDictionaryLibraryBridgeRequestHandler::handleRequest(HTTPServerRequ
|
||||
ExternalDictionaryLibraryBridgeExistsHandler::ExternalDictionaryLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, log(&Poco::Logger::get("ExternalDictionaryLibraryBridgeExistsHandler"))
|
||||
, log(getLogger("ExternalDictionaryLibraryBridgeExistsHandler"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -419,7 +419,7 @@ CatBoostLibraryBridgeRequestHandler::CatBoostLibraryBridgeRequestHandler(
|
||||
size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, log(&Poco::Logger::get("CatBoostLibraryBridgeRequestHandler"))
|
||||
, log(getLogger("CatBoostLibraryBridgeRequestHandler"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -623,7 +623,7 @@ void CatBoostLibraryBridgeRequestHandler::handleRequest(HTTPServerRequest & requ
|
||||
CatBoostLibraryBridgeExistsHandler::CatBoostLibraryBridgeExistsHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, log(&Poco::Logger::get("CatBoostLibraryBridgeExistsHandler"))
|
||||
, log(getLogger("CatBoostLibraryBridgeExistsHandler"))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ private:
|
||||
static constexpr inline auto FORMAT = "RowBinary";
|
||||
|
||||
const size_t keep_alive_timeout;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
|
||||
@ -40,7 +40,7 @@ public:
|
||||
|
||||
private:
|
||||
const size_t keep_alive_timeout;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
|
||||
@ -69,7 +69,7 @@ public:
|
||||
|
||||
private:
|
||||
const size_t keep_alive_timeout;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
|
||||
@ -83,7 +83,7 @@ public:
|
||||
|
||||
private:
|
||||
const size_t keep_alive_timeout;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -221,7 +221,7 @@ void LocalServer::tryInitPath()
|
||||
{
|
||||
// The path is not provided explicitly - use a unique path in the system temporary directory
|
||||
// (or in the current dir if temporary don't exist)
|
||||
Poco::Logger * log = &logger();
|
||||
LoggerRawPtr log = &logger();
|
||||
std::filesystem::path parent_folder;
|
||||
std::filesystem::path default_path;
|
||||
|
||||
@ -631,7 +631,7 @@ void LocalServer::processConfig()
|
||||
|
||||
tryInitPath();
|
||||
|
||||
Poco::Logger * log = &logger();
|
||||
LoggerRawPtr log = &logger();
|
||||
|
||||
/// Maybe useless
|
||||
if (config().has("macros"))
|
||||
|
@ -18,7 +18,7 @@ class ODBCColumnsInfoHandler : public HTTPRequestHandler, WithContext
|
||||
public:
|
||||
ODBCColumnsInfoHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("ODBCColumnsInfoHandler"))
|
||||
, log(getLogger("ODBCColumnsInfoHandler"))
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
@ -26,7 +26,7 @@ public:
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
size_t keep_alive_timeout;
|
||||
};
|
||||
|
||||
|
@ -16,7 +16,7 @@ class IdentifierQuoteHandler : public HTTPRequestHandler, WithContext
|
||||
public:
|
||||
IdentifierQuoteHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("IdentifierQuoteHandler"))
|
||||
, log(getLogger("IdentifierQuoteHandler"))
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
@ -24,7 +24,7 @@ public:
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
size_t keep_alive_timeout;
|
||||
};
|
||||
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
ContextPtr context_,
|
||||
const String & mode_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("ODBCHandler"))
|
||||
, log(getLogger("ODBCHandler"))
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
, mode(mode_)
|
||||
{
|
||||
@ -33,7 +33,7 @@ public:
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
size_t keep_alive_timeout;
|
||||
String mode;
|
||||
|
@ -23,7 +23,7 @@ namespace ErrorCodes
|
||||
ODBCSource::ODBCSource(
|
||||
nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_)
|
||||
: ISource(sample_block)
|
||||
, log(&Poco::Logger::get("ODBCSource"))
|
||||
, log(getLogger("ODBCSource"))
|
||||
, max_block_size{max_block_size_}
|
||||
, query(query_str)
|
||||
{
|
||||
|
@ -30,7 +30,7 @@ private:
|
||||
column.insertFrom(sample_column, 0);
|
||||
}
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
const UInt64 max_block_size;
|
||||
ExternalResultDescription description;
|
||||
|
||||
|
@ -19,7 +19,7 @@ ODBCSink::ODBCSink(
|
||||
ContextPtr local_context_,
|
||||
IdentifierQuotingStyle quoting_)
|
||||
: ISink(sample_block_)
|
||||
, log(&Poco::Logger::get("ODBCSink"))
|
||||
, log(getLogger("ODBCSink"))
|
||||
, connection_holder(std::move(connection_holder_))
|
||||
, db_name(remote_database_name_)
|
||||
, table_name(remote_table_name_)
|
||||
|
@ -30,7 +30,7 @@ protected:
|
||||
void consume(Chunk chunk) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
nanodbc::ConnectionHolderPtr connection_holder;
|
||||
std::string db_name;
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
|
||||
ODBCBridgeHandlerFactory::ODBCBridgeHandlerFactory(const std::string & name_, size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get(name_))
|
||||
, log(getLogger(name_))
|
||||
, name(name_)
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
std::unique_ptr<HTTPRequestHandler> createRequestHandler(const HTTPServerRequest & request) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
std::string name;
|
||||
size_t keep_alive_timeout;
|
||||
};
|
||||
|
@ -97,7 +97,7 @@ T execute(nanodbc::ConnectionHolderPtr connection_holder, std::function<T(nanodb
|
||||
/// https://docs.microsoft.com/ru-ru/sql/odbc/reference/appendixes/appendix-a-odbc-error-codes?view=sql-server-ver15
|
||||
bool is_retriable = e.state().starts_with("08") || e.state().starts_with("24") || e.state().starts_with("25");
|
||||
LOG_ERROR(
|
||||
&Poco::Logger::get("ODBCConnection"),
|
||||
getLogger("ODBCConnection"),
|
||||
"ODBC query failed with error: {}, state: {}, native code: {}{}",
|
||||
e.what(), e.state(), e.native(), is_retriable ? ", will retry" : "");
|
||||
|
||||
|
@ -19,7 +19,7 @@ class SchemaAllowedHandler : public HTTPRequestHandler, WithContext
|
||||
public:
|
||||
SchemaAllowedHandler(size_t keep_alive_timeout_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, log(&Poco::Logger::get("SchemaAllowedHandler"))
|
||||
, log(getLogger("SchemaAllowedHandler"))
|
||||
, keep_alive_timeout(keep_alive_timeout_)
|
||||
{
|
||||
}
|
||||
@ -27,7 +27,7 @@ public:
|
||||
void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
size_t keep_alive_timeout;
|
||||
};
|
||||
|
||||
|
@ -26,7 +26,7 @@ std::string getIdentifierQuote(nanodbc::ConnectionHolderPtr connection_holder)
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("ODBCGetIdentifierQuote"), "Cannot fetch identifier quote. Default double quote is used. Reason: {}", getCurrentExceptionMessage(false));
|
||||
LOG_WARNING(getLogger("ODBCGetIdentifierQuote"), "Cannot fetch identifier quote. Default double quote is used. Reason: {}", getCurrentExceptionMessage(false));
|
||||
return "\"";
|
||||
}
|
||||
|
||||
|
@ -365,7 +365,7 @@ void Server::createServer(
|
||||
namespace
|
||||
{
|
||||
|
||||
void setOOMScore(int value, Poco::Logger * log)
|
||||
void setOOMScore(int value, LoggerRawPtr log)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -450,7 +450,7 @@ void checkForUsersNotInMainConfig(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_path,
|
||||
const std::string & users_config_path,
|
||||
Poco::Logger * log)
|
||||
LoggerPtr log)
|
||||
{
|
||||
if (config.getBool("skip_check_for_incorrect_settings", false))
|
||||
return;
|
||||
@ -2491,7 +2491,7 @@ void Server::stopServers(
|
||||
const ServerType & server_type
|
||||
) const
|
||||
{
|
||||
Poco::Logger * log = &logger();
|
||||
LoggerRawPtr log = &logger();
|
||||
|
||||
/// Remove servers once all their connections are closed
|
||||
auto check_server = [&log](const char prefix[], auto & server)
|
||||
@ -2530,7 +2530,7 @@ void Server::updateServers(
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
std::vector<ProtocolServerAdapter> & servers_to_start_before_tables)
|
||||
{
|
||||
Poco::Logger * log = &logger();
|
||||
LoggerRawPtr log = &logger();
|
||||
|
||||
const auto listen_hosts = getListenHosts(config);
|
||||
const auto interserver_listen_hosts = getInterserverListenHosts(config);
|
||||
|
@ -47,7 +47,7 @@ namespace
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_path,
|
||||
const std::string & users_config_path,
|
||||
Poco::Logger * log)
|
||||
LoggerPtr log)
|
||||
{
|
||||
if (config.getBool("skip_check_for_incorrect_settings", false))
|
||||
return;
|
||||
|
@ -443,7 +443,7 @@ public:
|
||||
optimizeTree();
|
||||
}
|
||||
|
||||
void logTree(Poco::Logger * log, const String & title) const
|
||||
void logTree(LoggerPtr log, const String & title) const
|
||||
{
|
||||
LOG_TRACE(log, "Tree({}): level={}, name={}, flags={}, min_flags={}, max_flags={}, num_children={}",
|
||||
title, level, node_name ? *node_name : "NULL", flags.toString(),
|
||||
@ -1158,7 +1158,7 @@ AccessRights AccessRights::getFullAccess()
|
||||
|
||||
void AccessRights::logTree() const
|
||||
{
|
||||
auto * log = &Poco::Logger::get("AccessRights");
|
||||
auto log = getLogger("AccessRights");
|
||||
if (root)
|
||||
{
|
||||
root->logTree(log, "");
|
||||
|
@ -514,7 +514,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(
|
||||
&Poco::Logger::get("AddressPatterns"),
|
||||
getLogger("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
@ -556,7 +556,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(
|
||||
&Poco::Logger::get("AddressPatterns"),
|
||||
getLogger("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
|
@ -298,7 +298,7 @@ void ContextAccess::setUser(const UserPtr & user_) const
|
||||
}
|
||||
|
||||
user_name = user->getName();
|
||||
trace_log = &Poco::Logger::get("ContextAccess (" + user_name + ")");
|
||||
trace_log = getLogger("ContextAccess (" + user_name + ")");
|
||||
|
||||
std::vector<UUID> current_roles, current_roles_with_admin_option;
|
||||
if (params.use_default_roles)
|
||||
|
@ -185,9 +185,10 @@ private:
|
||||
|
||||
mutable std::atomic<bool> initialized = false; // can be removed after Bug 5504 is resolved
|
||||
mutable std::atomic<bool> user_was_dropped = false;
|
||||
mutable std::atomic<Poco::Logger *> trace_log = nullptr;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
/// TODO: Fix race
|
||||
mutable LoggerPtr trace_log;
|
||||
mutable UserPtr user TSA_GUARDED_BY(mutex);
|
||||
mutable String user_name TSA_GUARDED_BY(mutex);
|
||||
mutable scope_guard subscription_for_user_change TSA_GUARDED_BY(mutex);
|
||||
|
@ -47,7 +47,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
AccessEntityPtr tryReadEntityFile(const String & file_path, Poco::Logger & log)
|
||||
AccessEntityPtr tryReadEntityFile(const String & file_path, LoggerPtr log)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -55,7 +55,7 @@ namespace
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(&log);
|
||||
tryLogCurrentException(log);
|
||||
return nullptr;
|
||||
}
|
||||
}
|
||||
@ -378,7 +378,7 @@ void DiskAccessStorage::reloadAllAndRebuildLists()
|
||||
continue;
|
||||
|
||||
const auto access_entity_file_path = getEntityFilePath(directory_path, id);
|
||||
auto entity = tryReadEntityFile(access_entity_file_path, *getLogger());
|
||||
auto entity = tryReadEntityFile(access_entity_file_path, getLogger());
|
||||
if (!entity)
|
||||
continue;
|
||||
|
||||
|
@ -279,7 +279,7 @@ void ExternalAuthenticators::reset()
|
||||
resetImpl();
|
||||
}
|
||||
|
||||
void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log)
|
||||
void ExternalAuthenticators::setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
resetImpl();
|
||||
|
@ -36,7 +36,7 @@ class ExternalAuthenticators
|
||||
{
|
||||
public:
|
||||
void reset();
|
||||
void setConfiguration(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log);
|
||||
void setConfiguration(const Poco::Util::AbstractConfiguration & config, LoggerPtr log);
|
||||
|
||||
// The name and readiness of the credentials must be verified before calling these.
|
||||
bool checkLDAPCredentials(const String & server, const BasicCredentials & credentials,
|
||||
|
@ -328,7 +328,7 @@ void GSSAcceptorContext::initHandles()
|
||||
}
|
||||
}
|
||||
|
||||
String GSSAcceptorContext::processToken(const String & input_token, Poco::Logger * log)
|
||||
String GSSAcceptorContext::processToken(const String & input_token, LoggerPtr log)
|
||||
{
|
||||
std::lock_guard lock(gss_global_mutex);
|
||||
|
||||
@ -455,7 +455,7 @@ void GSSAcceptorContext::initHandles()
|
||||
{
|
||||
}
|
||||
|
||||
String GSSAcceptorContext::processToken(const String &, Poco::Logger *)
|
||||
String GSSAcceptorContext::processToken(const String &, LoggerPtr)
|
||||
{
|
||||
throw Exception(ErrorCodes::FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME, "ClickHouse was built without GSS-API/Kerberos support");
|
||||
}
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include "config.h"
|
||||
|
||||
#include <Access/Credentials.h>
|
||||
#include <Common/Logger.h>
|
||||
#include <base/types.h>
|
||||
#include <memory>
|
||||
|
||||
@ -42,7 +43,7 @@ public:
|
||||
|
||||
const String & getRealm() const;
|
||||
bool isFailed() const;
|
||||
MAYBE_NORETURN String processToken(const String & input_token, Poco::Logger * log);
|
||||
MAYBE_NORETURN String processToken(const String & input_token, LoggerPtr log);
|
||||
|
||||
private:
|
||||
void reset();
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/callOnce.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/UUIDGenerator.h>
|
||||
@ -615,7 +616,7 @@ UUID IAccessStorage::generateRandomID()
|
||||
}
|
||||
|
||||
|
||||
void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const Poco::Logger * log_)
|
||||
void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const LoggerPtr log_)
|
||||
{
|
||||
std::unordered_map<UUID, size_t> positions_by_id;
|
||||
std::unordered_map<std::string_view, size_t> positions_by_type_and_name[static_cast<size_t>(AccessEntityType::MAX)];
|
||||
@ -671,12 +672,13 @@ void IAccessStorage::clearConflictsInEntitiesList(std::vector<std::pair<UUID, Ac
|
||||
}
|
||||
|
||||
|
||||
Poco::Logger * IAccessStorage::getLogger() const
|
||||
LoggerPtr IAccessStorage::getLogger() const
|
||||
{
|
||||
Poco::Logger * ptr = log.load();
|
||||
if (!ptr)
|
||||
log.store(ptr = &Poco::Logger::get("Access(" + storage_name + ")"), std::memory_order_relaxed);
|
||||
return ptr;
|
||||
callOnce(log_initialized, [&] {
|
||||
log = ::getLogger("Access(" + storage_name + ")");
|
||||
});
|
||||
|
||||
return log;
|
||||
}
|
||||
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Parsers/IParser.h>
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Common/callOnce.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <functional>
|
||||
@ -225,9 +226,9 @@ protected:
|
||||
SettingsChanges & settings) const;
|
||||
virtual bool isAddressAllowed(const User & user, const Poco::Net::IPAddress & address) const;
|
||||
static UUID generateRandomID();
|
||||
Poco::Logger * getLogger() const;
|
||||
LoggerPtr getLogger() const;
|
||||
static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); }
|
||||
static void clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const Poco::Logger * log_);
|
||||
static void clearConflictsInEntitiesList(std::vector<std::pair<UUID, AccessEntityPtr>> & entities, const LoggerPtr log_);
|
||||
[[noreturn]] void throwNotFound(const UUID & id) const;
|
||||
[[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const;
|
||||
[[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type);
|
||||
@ -246,7 +247,9 @@ protected:
|
||||
|
||||
private:
|
||||
const String storage_name;
|
||||
mutable std::atomic<Poco::Logger *> log = nullptr;
|
||||
|
||||
mutable OnceFlag log_initialized;
|
||||
mutable LoggerPtr log = nullptr;
|
||||
};
|
||||
|
||||
|
||||
|
@ -63,7 +63,7 @@ String KerberosInit::fmtError(krb5_error_code code) const
|
||||
|
||||
void KerberosInit::init(const String & keytab_file, const String & principal, const String & cache_name)
|
||||
{
|
||||
auto * log = &Poco::Logger::get("KerberosInit");
|
||||
auto log = getLogger("KerberosInit");
|
||||
LOG_TRACE(log,"Trying to authenticate with Kerberos v5");
|
||||
|
||||
krb5_error_code ret;
|
||||
|
@ -532,7 +532,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params)
|
||||
|
||||
for (size_t i = 0; referrals[i]; ++i)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]);
|
||||
LOG_WARNING(getLogger("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -91,7 +91,7 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_)
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(
|
||||
&Poco::Logger::get("RowPolicy"),
|
||||
getLogger("RowPolicy"),
|
||||
String("Could not parse the condition ") + toString(filter_type) + " of row policy "
|
||||
+ backQuote(policy->getName()));
|
||||
}
|
||||
|
@ -37,7 +37,7 @@ SettingsAuthResponseParser::parse(const Poco::Net::HTTPResponse & response, std:
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it.");
|
||||
LOG_INFO(getLogger("HTTPAuthentication"), "Failed to parse settings from authentication response. Skip it.");
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
@ -440,7 +440,7 @@ struct GroupArrayNodeGeneral : public GroupArrayNodeBase<GroupArrayNodeGeneral>
|
||||
return node;
|
||||
}
|
||||
|
||||
void insertInto(IColumn & column) { column.deserializeAndInsertFromArena(data()); }
|
||||
void insertInto(IColumn & column) { std::ignore = column.deserializeAndInsertFromArena(data()); }
|
||||
};
|
||||
|
||||
template <typename Node, bool has_sampler>
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/KeyHolderHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -153,8 +154,6 @@ private:
|
||||
UInt64 threshold;
|
||||
UInt64 reserved;
|
||||
|
||||
static void deserializeAndInsert(StringRef str, IColumn & data_to);
|
||||
|
||||
public:
|
||||
AggregateFunctionTopKGeneric(
|
||||
UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params)
|
||||
@ -251,12 +250,7 @@ public:
|
||||
offsets_to.push_back(offsets_to.back() + result_vec.size());
|
||||
|
||||
for (auto & elem : result_vec)
|
||||
{
|
||||
if constexpr (is_plain_column)
|
||||
data_to.insertData(elem.key.data, elem.key.size);
|
||||
else
|
||||
data_to.deserializeAndInsertFromArena(elem.key.data);
|
||||
}
|
||||
deserializeAndInsert<is_plain_column>(elem.key, data_to);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -29,7 +29,7 @@ static void deserializeAndInsert(StringRef str, IColumn & data_to)
|
||||
if constexpr (is_plain_column)
|
||||
data_to.insertData(str.data, str.size);
|
||||
else
|
||||
data_to.deserializeAndInsertFromArena(str.data);
|
||||
std::ignore = data_to.deserializeAndInsertFromArena(str.data);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -326,6 +326,68 @@ void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_q
|
||||
}
|
||||
}
|
||||
|
||||
QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree)
|
||||
{
|
||||
QueryTreeNodes result;
|
||||
|
||||
QueryTreeNodes nodes_to_process;
|
||||
nodes_to_process.push_back(tree);
|
||||
|
||||
while (!nodes_to_process.empty())
|
||||
{
|
||||
auto node_to_process = std::move(nodes_to_process.back());
|
||||
nodes_to_process.pop_back();
|
||||
|
||||
auto node_type = node_to_process->getNodeType();
|
||||
|
||||
switch (node_type)
|
||||
{
|
||||
case QueryTreeNodeType::TABLE:
|
||||
{
|
||||
result.push_back(std::move(node_to_process));
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::QUERY:
|
||||
{
|
||||
nodes_to_process.push_back(node_to_process->as<QueryNode>()->getJoinTree());
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::UNION:
|
||||
{
|
||||
for (const auto & union_node : node_to_process->as<UnionNode>()->getQueries().getNodes())
|
||||
nodes_to_process.push_back(union_node);
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::TABLE_FUNCTION:
|
||||
{
|
||||
// Arguments of table function can't contain TableNodes.
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::ARRAY_JOIN:
|
||||
{
|
||||
nodes_to_process.push_back(node_to_process->as<ArrayJoinNode>()->getTableExpression());
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::JOIN:
|
||||
{
|
||||
auto & join_node = node_to_process->as<JoinNode &>();
|
||||
nodes_to_process.push_back(join_node.getRightTableExpression());
|
||||
nodes_to_process.push_back(join_node.getLeftTableExpression());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Unexpected node type for table expression. "
|
||||
"Expected table, table function, query, union, join or array join. Actual {}",
|
||||
node_to_process->getNodeTypeName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join)
|
||||
{
|
||||
QueryTreeNodes result;
|
||||
|
@ -50,10 +50,13 @@ std::optional<bool> tryExtractConstantFromConditionNode(const QueryTreeNodePtr &
|
||||
*/
|
||||
void addTableExpressionOrJoinIntoTablesInSelectQuery(ASTPtr & tables_in_select_query_ast, const QueryTreeNodePtr & table_expression, const IQueryTreeNode::ConvertToASTOptions & convert_to_ast_options);
|
||||
|
||||
/// Extract table, table function, query, union from join tree
|
||||
/// Extract all TableNodes from the query tree.
|
||||
QueryTreeNodes extractAllTableReferences(const QueryTreeNodePtr & tree);
|
||||
|
||||
/// Extract table, table function, query, union from join tree.
|
||||
QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, bool add_array_join = false);
|
||||
|
||||
/// Extract left table expression from join tree
|
||||
/// Extract left table expression from join tree.
|
||||
QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node);
|
||||
|
||||
/** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree.
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
{
|
||||
|
||||
BackupCoordinationLocal::BackupCoordinationLocal(bool plain_backup_)
|
||||
: log(&Poco::Logger::get("BackupCoordinationLocal")), file_infos(plain_backup_)
|
||||
: log(getLogger("BackupCoordinationLocal")), file_infos(plain_backup_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -57,7 +57,7 @@ public:
|
||||
bool hasConcurrentBackups(const std::atomic<size_t> & num_active_backups) const override;
|
||||
|
||||
private:
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
|
||||
BackupCoordinationReplicatedTables TSA_GUARDED_BY(replicated_tables_mutex) replicated_tables;
|
||||
BackupCoordinationReplicatedAccess TSA_GUARDED_BY(replicated_access_mutex) replicated_access;
|
||||
|
@ -173,7 +173,7 @@ BackupCoordinationRemote::BackupCoordinationRemote(
|
||||
, current_host_index(findCurrentHostIndex(all_hosts, current_host))
|
||||
, plain_backup(plain_backup_)
|
||||
, is_internal(is_internal_)
|
||||
, log(&Poco::Logger::get("BackupCoordinationRemote"))
|
||||
, log(getLogger("BackupCoordinationRemote"))
|
||||
, with_retries(
|
||||
log,
|
||||
get_zookeeper_,
|
||||
|
@ -102,7 +102,7 @@ private:
|
||||
const size_t current_host_index;
|
||||
const bool plain_backup;
|
||||
const bool is_internal;
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
|
||||
/// The order of these two fields matters, because stage_sync holds a reference to with_retries object
|
||||
mutable WithRetries with_retries;
|
||||
|
@ -24,7 +24,7 @@ namespace ErrorCodes
|
||||
BackupCoordinationStageSync::BackupCoordinationStageSync(
|
||||
const String & root_zookeeper_path_,
|
||||
WithRetries & with_retries_,
|
||||
Poco::Logger * log_)
|
||||
LoggerPtr log_)
|
||||
: zookeeper_path(root_zookeeper_path_ + "/stage")
|
||||
, with_retries(with_retries_)
|
||||
, log(log_)
|
||||
|
@ -12,7 +12,7 @@ public:
|
||||
BackupCoordinationStageSync(
|
||||
const String & root_zookeeper_path_,
|
||||
WithRetries & with_retries_,
|
||||
Poco::Logger * log_);
|
||||
LoggerPtr log_);
|
||||
|
||||
/// Sets the stage of the current host and signal other hosts if there were other hosts waiting for that.
|
||||
void set(const String & current_host, const String & new_stage, const String & message, const bool & all_hosts = false);
|
||||
@ -36,7 +36,7 @@ private:
|
||||
String zookeeper_path;
|
||||
/// A reference to the field of parent object - BackupCoordinationRemote or RestoreCoordinationRemote
|
||||
WithRetries & with_retries;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -97,7 +97,7 @@ BackupEntriesCollector::BackupEntriesCollector(
|
||||
, max_sleep_before_next_attempt_to_collect_metadata(
|
||||
context->getConfigRef().getUInt64("backups.max_sleep_before_next_attempt_to_collect_metadata", 5000))
|
||||
, compare_collected_metadata(context->getConfigRef().getBool("backups.compare_collected_metadata", true))
|
||||
, log(&Poco::Logger::get("BackupEntriesCollector"))
|
||||
, log(getLogger("BackupEntriesCollector"))
|
||||
, global_zookeeper_retries_info(
|
||||
context->getSettingsRef().backup_restore_keeper_max_retries,
|
||||
context->getSettingsRef().backup_restore_keeper_retry_initial_backoff_ms,
|
||||
|
@ -129,7 +129,7 @@ private:
|
||||
/// Whether we should collect the metadata after a successful attempt one more time and check that nothing has changed.
|
||||
const bool compare_collected_metadata;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
/// Unfortunately we can use ZooKeeper for collecting information for backup
|
||||
/// and we need to retry...
|
||||
ZooKeeperRetriesInfo global_zookeeper_retries_info;
|
||||
|
@ -102,7 +102,7 @@ BackupFileInfo buildFileInfoForBackupEntry(
|
||||
const BackupEntryPtr & backup_entry,
|
||||
const BackupPtr & base_backup,
|
||||
const ReadSettings & read_settings,
|
||||
Poco::Logger * log)
|
||||
LoggerPtr log)
|
||||
{
|
||||
auto adjusted_path = removeLeadingSlash(file_name);
|
||||
|
||||
@ -129,7 +129,7 @@ BackupFileInfo buildFileInfoForBackupEntry(
|
||||
}
|
||||
|
||||
if (!log)
|
||||
log = &Poco::Logger::get("FileInfoFromBackupEntry");
|
||||
log = getLogger("FileInfoFromBackupEntry");
|
||||
|
||||
std::optional<SizeAndChecksum> base_backup_file_info = getInfoAboutFileFromBaseBackupIfExists(base_backup, adjusted_path);
|
||||
|
||||
@ -216,7 +216,7 @@ BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entr
|
||||
std::exception_ptr exception;
|
||||
|
||||
auto thread_group = CurrentThread::getGroup();
|
||||
Poco::Logger * log = &Poco::Logger::get("FileInfosFromBackupEntries");
|
||||
LoggerPtr log = getLogger("FileInfosFromBackupEntries");
|
||||
|
||||
for (size_t i = 0; i != backup_entries.size(); ++i)
|
||||
{
|
||||
|
@ -2,8 +2,7 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
#include <Common/Logger.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -77,7 +76,7 @@ struct BackupFileInfo
|
||||
using BackupFileInfos = std::vector<BackupFileInfo>;
|
||||
|
||||
/// Builds a BackupFileInfo for a specified backup entry.
|
||||
BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, Poco::Logger * log);
|
||||
BackupFileInfo buildFileInfoForBackupEntry(const String & file_name, const BackupEntryPtr & backup_entry, const BackupPtr & base_backup, const ReadSettings & read_settings, LoggerPtr log);
|
||||
|
||||
/// Builds a vector of BackupFileInfos for specified backup entries.
|
||||
BackupFileInfos buildFileInfosForBackupEntries(const BackupEntries & backup_entries, const BackupPtr & base_backup, const ReadSettings & read_settings, ThreadPool & thread_pool, QueryStatusPtr process_list_element);
|
||||
|
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_)
|
||||
BackupReaderDefault::BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_)
|
||||
: log(log_)
|
||||
, read_settings(read_settings_)
|
||||
, write_settings(write_settings_)
|
||||
@ -36,7 +36,7 @@ void BackupReaderDefault::copyFileToDisk(const String & path_in_backup, size_t f
|
||||
write_buffer->finalize();
|
||||
}
|
||||
|
||||
BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_)
|
||||
BackupWriterDefault::BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_)
|
||||
: log(log_)
|
||||
, read_settings(read_settings_)
|
||||
, write_settings(write_settings_)
|
||||
|
@ -18,7 +18,7 @@ enum class WriteMode;
|
||||
class BackupReaderDefault : public IBackupReader
|
||||
{
|
||||
public:
|
||||
BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_);
|
||||
BackupReaderDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_);
|
||||
~BackupReaderDefault() override = default;
|
||||
|
||||
/// The function copyFileToDisk() can be much faster than reading the file with readFile() and then writing it to some disk.
|
||||
@ -33,7 +33,7 @@ public:
|
||||
size_t getWriteBufferSize() const override { return write_buffer_size; }
|
||||
|
||||
protected:
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
const ReadSettings read_settings;
|
||||
|
||||
/// The write settings are used to write to the source disk in copyFileToDisk().
|
||||
@ -45,7 +45,7 @@ protected:
|
||||
class BackupWriterDefault : public IBackupWriter
|
||||
{
|
||||
public:
|
||||
BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, Poco::Logger * log_);
|
||||
BackupWriterDefault(const ReadSettings & read_settings_, const WriteSettings & write_settings_, LoggerPtr log_);
|
||||
~BackupWriterDefault() override = default;
|
||||
|
||||
bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override;
|
||||
@ -60,7 +60,7 @@ protected:
|
||||
/// Here readFile() is used only to implement fileContentsEqual().
|
||||
virtual std::unique_ptr<ReadBuffer> readFile(const String & file_name, size_t expected_file_size) = 0;
|
||||
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
|
||||
/// The read settings are used to read from the source disk in copyFileFromDisk().
|
||||
const ReadSettings read_settings;
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
{
|
||||
|
||||
BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
|
||||
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderDisk"))
|
||||
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderDisk"))
|
||||
, disk(disk_)
|
||||
, root_path(root_path_)
|
||||
, data_source_description(disk->getDataSourceDescription())
|
||||
@ -57,7 +57,7 @@ void BackupReaderDisk::copyFileToDisk(const String & path_in_backup, size_t file
|
||||
|
||||
|
||||
BackupWriterDisk::BackupWriterDisk(const DiskPtr & disk_, const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
|
||||
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterDisk"))
|
||||
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterDisk"))
|
||||
, disk(disk_)
|
||||
, root_path(root_path_)
|
||||
, data_source_description(disk->getDataSourceDescription())
|
||||
|
@ -17,7 +17,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
BackupReaderFile::BackupReaderFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
|
||||
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderFile"))
|
||||
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderFile"))
|
||||
, root_path(root_path_)
|
||||
, data_source_description(DiskLocal::getLocalDataSourceDescription(root_path))
|
||||
{
|
||||
@ -75,7 +75,7 @@ void BackupReaderFile::copyFileToDisk(const String & path_in_backup, size_t file
|
||||
|
||||
|
||||
BackupWriterFile::BackupWriterFile(const String & root_path_, const ReadSettings & read_settings_, const WriteSettings & write_settings_)
|
||||
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterFile"))
|
||||
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterFile"))
|
||||
, root_path(root_path_)
|
||||
, data_source_description(DiskLocal::getLocalDataSourceDescription(root_path))
|
||||
{
|
||||
|
@ -124,7 +124,7 @@ BackupReaderS3::BackupReaderS3(
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_)
|
||||
: BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderS3"))
|
||||
: BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3"))
|
||||
, s3_uri(s3_uri_)
|
||||
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()))
|
||||
@ -214,7 +214,7 @@ BackupWriterS3::BackupWriterS3(
|
||||
const ReadSettings & read_settings_,
|
||||
const WriteSettings & write_settings_,
|
||||
const ContextPtr & context_)
|
||||
: BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterS3"))
|
||||
: BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3"))
|
||||
, s3_uri(s3_uri_)
|
||||
, data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false}
|
||||
, s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()))
|
||||
|
@ -105,7 +105,7 @@ BackupImpl::BackupImpl(
|
||||
, version(INITIAL_BACKUP_VERSION)
|
||||
, base_backup_info(base_backup_info_)
|
||||
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
|
||||
, log(&Poco::Logger::get("BackupImpl"))
|
||||
, log(getLogger("BackupImpl"))
|
||||
{
|
||||
open();
|
||||
}
|
||||
@ -136,7 +136,7 @@ BackupImpl::BackupImpl(
|
||||
, base_backup_info(base_backup_info_)
|
||||
, deduplicate_files(deduplicate_files_)
|
||||
, use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_)
|
||||
, log(&Poco::Logger::get("BackupImpl"))
|
||||
, log(getLogger("BackupImpl"))
|
||||
{
|
||||
open();
|
||||
}
|
||||
|
@ -153,7 +153,7 @@ private:
|
||||
bool writing_finalized = false;
|
||||
bool deduplicate_files = true;
|
||||
bool use_same_s3_credentials_for_base_backup = false;
|
||||
const Poco::Logger * log;
|
||||
const LoggerPtr log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -380,7 +380,7 @@ BackupsWorker::BackupsWorker(ContextMutablePtr global_context, size_t num_backup
|
||||
, allow_concurrent_backups(allow_concurrent_backups_)
|
||||
, allow_concurrent_restores(allow_concurrent_restores_)
|
||||
, test_inject_sleep(test_inject_sleep_)
|
||||
, log(&Poco::Logger::get("BackupsWorker"))
|
||||
, log(getLogger("BackupsWorker"))
|
||||
, backup_log(global_context->getBackupLog())
|
||||
, process_list(global_context->getProcessList())
|
||||
{
|
||||
|
@ -127,7 +127,7 @@ private:
|
||||
const bool allow_concurrent_restores;
|
||||
const bool test_inject_sleep;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
struct ExtendedOperationInfo
|
||||
{
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal"))
|
||||
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(getLogger("RestoreCoordinationLocal"))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -51,7 +51,7 @@ public:
|
||||
bool hasConcurrentRestores(const std::atomic<size_t> & num_active_restores) const override;
|
||||
|
||||
private:
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
|
||||
std::set<std::pair<String /* database_zk_path */, String /* table_name */>> acquired_tables_in_replicated_databases;
|
||||
std::unordered_set<String /* table_zk_path */> acquired_data_in_replicated_tables;
|
||||
|
@ -32,7 +32,7 @@ RestoreCoordinationRemote::RestoreCoordinationRemote(
|
||||
, current_host(current_host_)
|
||||
, current_host_index(BackupCoordinationRemote::findCurrentHostIndex(all_hosts, current_host))
|
||||
, is_internal(is_internal_)
|
||||
, log(&Poco::Logger::get("RestoreCoordinationRemote"))
|
||||
, log(getLogger("RestoreCoordinationRemote"))
|
||||
, with_retries(
|
||||
log,
|
||||
get_zookeeper_,
|
||||
|
@ -73,7 +73,7 @@ private:
|
||||
const String current_host;
|
||||
const size_t current_host_index;
|
||||
const bool is_internal;
|
||||
Poco::Logger * const log;
|
||||
LoggerPtr const log;
|
||||
|
||||
mutable WithRetries with_retries;
|
||||
std::optional<BackupCoordinationStageSync> stage_sync;
|
||||
|
@ -89,7 +89,7 @@ RestorerFromBackup::RestorerFromBackup(
|
||||
, process_list_element(context->getProcessListElement())
|
||||
, on_cluster_first_sync_timeout(context->getConfigRef().getUInt64("backups.on_cluster_first_sync_timeout", 180000))
|
||||
, create_table_timeout(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000))
|
||||
, log(&Poco::Logger::get("RestorerFromBackup"))
|
||||
, log(getLogger("RestorerFromBackup"))
|
||||
, tables_dependencies("RestorerFromBackup")
|
||||
{
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ private:
|
||||
QueryStatusPtr process_list_element;
|
||||
std::chrono::milliseconds on_cluster_first_sync_timeout;
|
||||
std::chrono::milliseconds create_table_timeout;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
Strings all_hosts;
|
||||
DDLRenamingMap renaming_map;
|
||||
|
@ -21,7 +21,7 @@ WithRetries::KeeperSettings WithRetries::KeeperSettings::fromContext(ContextPtr
|
||||
}
|
||||
|
||||
WithRetries::WithRetries(
|
||||
Poco::Logger * log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_)
|
||||
LoggerPtr log_, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings_, QueryStatusPtr process_list_element_, RenewerCallback callback_)
|
||||
: log(log_)
|
||||
, get_zookeeper(get_zookeeper_)
|
||||
, settings(settings_)
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
};
|
||||
|
||||
RetriesControlHolder createRetriesControlHolder(const String & name);
|
||||
WithRetries(Poco::Logger * log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback);
|
||||
WithRetries(LoggerPtr log, zkutil::GetZooKeeper get_zookeeper_, const KeeperSettings & settings, QueryStatusPtr process_list_element_, RenewerCallback callback);
|
||||
|
||||
/// Used to re-establish new connection inside a retry loop.
|
||||
void renewZooKeeper(FaultyKeeper my_faulty_zookeeper) const;
|
||||
@ -62,7 +62,7 @@ private:
|
||||
/// This will provide a special wrapper which is useful for testing
|
||||
FaultyKeeper getFaultyZooKeeper() const;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
zkutil::GetZooKeeper get_zookeeper;
|
||||
KeeperSettings settings;
|
||||
QueryStatusPtr process_list_element;
|
||||
|
@ -51,7 +51,7 @@ protected:
|
||||
|
||||
virtual const Poco::Util::AbstractConfiguration & getConfig() const = 0;
|
||||
|
||||
virtual Poco::Logger * getLog() const = 0;
|
||||
virtual LoggerPtr getLog() const = 0;
|
||||
|
||||
virtual Poco::Timespan getHTTPTimeout() const = 0;
|
||||
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_)
|
||||
: IBridgeHelper(context_)
|
||||
, config(context_->getConfigRef())
|
||||
, log(&Poco::Logger::get("LibraryBridgeHelper"))
|
||||
, log(getLogger("LibraryBridgeHelper"))
|
||||
, http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value)
|
||||
, bridge_host(config.getString("library_bridge.host", DEFAULT_HOST))
|
||||
, bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT))
|
||||
|
@ -31,7 +31,7 @@ protected:
|
||||
|
||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||
|
||||
Poco::Logger * getLog() const override { return log; }
|
||||
LoggerPtr getLog() const override { return log; }
|
||||
|
||||
Poco::Timespan getHTTPTimeout() const override { return http_timeout; }
|
||||
|
||||
@ -40,7 +40,7 @@ protected:
|
||||
static constexpr inline size_t DEFAULT_PORT = 9012;
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
const Poco::Timespan http_timeout;
|
||||
std::string bridge_host;
|
||||
size_t bridge_port;
|
||||
|
@ -65,7 +65,7 @@ public:
|
||||
const std::string & connection_string_,
|
||||
bool use_connection_pooling_)
|
||||
: IXDBCBridgeHelper(context_->getGlobalContext())
|
||||
, log(&Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||
, log(getLogger(BridgeHelperMixin::getName() + "BridgeHelper"))
|
||||
, connection_string(connection_string_)
|
||||
, use_connection_pooling(use_connection_pooling_)
|
||||
, http_timeout(http_timeout_)
|
||||
@ -123,7 +123,7 @@ protected:
|
||||
|
||||
const Poco::Util::AbstractConfiguration & getConfig() const override { return config; }
|
||||
|
||||
Poco::Logger * getLog() const override { return log; }
|
||||
LoggerPtr getLog() const override { return log; }
|
||||
|
||||
bool startBridgeManually() const override { return BridgeHelperMixin::startBridgeManually(); }
|
||||
|
||||
@ -146,7 +146,7 @@ protected:
|
||||
private:
|
||||
using Configuration = Poco::Util::AbstractConfiguration;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
std::string connection_string;
|
||||
bool use_connection_pooling;
|
||||
Poco::Timespan http_timeout;
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
|
||||
#include <Common/SSH/Wrappers.h>
|
||||
#include <Common/callOnce.h>
|
||||
#include <Client/IServerConnection.h>
|
||||
#include <Core/Defines.h>
|
||||
|
||||
@ -244,16 +245,18 @@ private:
|
||||
{
|
||||
}
|
||||
|
||||
Poco::Logger * get()
|
||||
LoggerPtr get()
|
||||
{
|
||||
if (!log)
|
||||
log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")");
|
||||
callOnce(log_initialized, [&] {
|
||||
log = getLogger("Connection (" + parent.getDescription() + ")");
|
||||
});
|
||||
|
||||
return log;
|
||||
}
|
||||
|
||||
private:
|
||||
std::atomic<Poco::Logger *> log;
|
||||
OnceFlag log_initialized;
|
||||
LoggerPtr log;
|
||||
Connection & parent;
|
||||
};
|
||||
|
||||
|
@ -25,7 +25,7 @@ ConnectionEstablisher::ConnectionEstablisher(
|
||||
IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings & settings_,
|
||||
Poco::Logger * log_,
|
||||
LoggerPtr log_,
|
||||
const QualifiedTableName * table_to_check_)
|
||||
: pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_), is_finished(false)
|
||||
{
|
||||
@ -114,7 +114,7 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync(
|
||||
IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings & settings_,
|
||||
Poco::Logger * log_,
|
||||
LoggerPtr log_,
|
||||
const QualifiedTableName * table_to_check_)
|
||||
: AsyncTaskExecutor(std::make_unique<Task>(*this)), connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_)
|
||||
{
|
||||
|
@ -23,7 +23,7 @@ public:
|
||||
ConnectionEstablisher(IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings & settings_,
|
||||
Poco::Logger * log,
|
||||
LoggerPtr log,
|
||||
const QualifiedTableName * table_to_check = nullptr);
|
||||
|
||||
/// Establish connection and save it in result, write possible exception message in fail_message.
|
||||
@ -38,7 +38,7 @@ private:
|
||||
IConnectionPool * pool;
|
||||
const ConnectionTimeouts * timeouts;
|
||||
const Settings & settings;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
const QualifiedTableName * table_to_check;
|
||||
|
||||
bool is_finished;
|
||||
@ -61,7 +61,7 @@ public:
|
||||
ConnectionEstablisherAsync(IConnectionPool * pool_,
|
||||
const ConnectionTimeouts * timeouts_,
|
||||
const Settings & settings_,
|
||||
Poco::Logger * log_,
|
||||
LoggerPtr log_,
|
||||
const QualifiedTableName * table_to_check_ = nullptr);
|
||||
|
||||
/// Get file descriptor that can be added in epoll and be polled,
|
||||
|
@ -64,7 +64,7 @@ public:
|
||||
Protocol::Secure secure_,
|
||||
Priority priority_ = Priority{1})
|
||||
: Base(max_connections_,
|
||||
&Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
getLogger("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_),
|
||||
port(port_),
|
||||
default_database(default_database_),
|
||||
|
@ -29,7 +29,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_, &Poco::Logger::get("ConnectionPoolWithFailover"))
|
||||
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, getLogger("ConnectionPoolWithFailover"))
|
||||
, get_priority_load_balancing(load_balancing)
|
||||
{
|
||||
const std::string & local_hostname = getFQDNOrHostName();
|
||||
|
@ -34,7 +34,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory(
|
||||
: pool(pool_)
|
||||
, timeouts(timeouts_)
|
||||
, table_to_check(table_to_check_)
|
||||
, log(&Poco::Logger::get("HedgedConnectionsFactory"))
|
||||
, log(getLogger("HedgedConnectionsFactory"))
|
||||
, max_tries(max_tries_)
|
||||
, fallback_to_stale_replicas(fallback_to_stale_replicas_)
|
||||
, max_parallel_replicas(max_parallel_replicas_)
|
||||
|
@ -133,7 +133,7 @@ private:
|
||||
std::shared_ptr<QualifiedTableName> table_to_check;
|
||||
int last_used_index = -1;
|
||||
Epoll epoll;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
std::string fail_messages;
|
||||
|
||||
/// The maximum number of attempts to connect to replicas.
|
||||
|
@ -222,11 +222,11 @@ public:
|
||||
|
||||
/// Deserializes a value that was serialized using IColumn::serializeValueIntoArena method.
|
||||
/// Returns pointer to the position after the read data.
|
||||
virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
|
||||
[[nodiscard]] virtual const char * deserializeAndInsertFromArena(const char * pos) = 0;
|
||||
|
||||
/// Skip previously serialized value that was serialized using IColumn::serializeValueIntoArena method.
|
||||
/// Returns a pointer to the position after the deserialized data.
|
||||
virtual const char * skipSerializedInArena(const char *) const = 0;
|
||||
[[nodiscard]] virtual const char * skipSerializedInArena(const char *) const = 0;
|
||||
|
||||
/// Update state of hash function with value of n-th element.
|
||||
/// On subsequent calls of this method for sequence of column values of arbitrary types,
|
||||
|
@ -50,7 +50,7 @@ void prefaultPages([[maybe_unused]] void * buf_, [[maybe_unused]] size_t len_)
|
||||
auto [buf, len] = adjustToPageSize(buf_, len_, page_size);
|
||||
if (::madvise(buf, len, MADV_POPULATE_WRITE) < 0)
|
||||
LOG_TRACE(
|
||||
LogFrequencyLimiter(&Poco::Logger::get("Allocator"), 1),
|
||||
LogFrequencyLimiter(getLogger("Allocator"), 1),
|
||||
"Attempt to populate pages failed: {} (EINVAL is expected for kernels < 5.14)",
|
||||
errnoToString(errno));
|
||||
#endif
|
||||
|
@ -34,7 +34,7 @@ namespace ErrorCodes
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
|
||||
|
||||
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
|
||||
void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch)
|
||||
{
|
||||
if (total && (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS)))
|
||||
{
|
||||
@ -205,7 +205,7 @@ void LoadTask::detach()
|
||||
AsyncLoader::AsyncLoader(std::vector<PoolInitializer> pool_initializers, bool log_failures_, bool log_progress_)
|
||||
: log_failures(log_failures_)
|
||||
, log_progress(log_progress_)
|
||||
, log(&Poco::Logger::get("AsyncLoader"))
|
||||
, log(getLogger("AsyncLoader"))
|
||||
{
|
||||
pools.reserve(pool_initializers.size());
|
||||
for (auto && init : pool_initializers)
|
||||
|
@ -15,6 +15,7 @@
|
||||
#include <Common/Priority.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Common/Logger.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -40,7 +41,7 @@ using LoadTaskPtr = std::shared_ptr<LoadTask>;
|
||||
using LoadTaskPtrs = std::vector<LoadTaskPtr>;
|
||||
class AsyncLoader;
|
||||
|
||||
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch);
|
||||
void logAboutProgress(LoggerPtr log, size_t processed, size_t total, AtomicStopwatch & watch);
|
||||
|
||||
// Execution status of a load job.
|
||||
enum class LoadStatus
|
||||
@ -419,7 +420,7 @@ private:
|
||||
// Logging
|
||||
const bool log_failures; // Worker should log all exceptions caught from job functions.
|
||||
const bool log_progress; // Periodically log total progress
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
mutable std::mutex mutex; // Guards all the fields below.
|
||||
bool is_running = true;
|
||||
|
@ -58,7 +58,7 @@ AsynchronousMetrics::AsynchronousMetrics(
|
||||
int update_period_seconds,
|
||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
|
||||
: update_period(update_period_seconds)
|
||||
, log(&Poco::Logger::get("AsynchronousMetrics"))
|
||||
, log(getLogger("AsynchronousMetrics"))
|
||||
, protocol_server_metrics_func(protocol_server_metrics_func_)
|
||||
{
|
||||
#if defined(OS_LINUX)
|
||||
@ -125,7 +125,7 @@ void AsynchronousMetrics::openSensors() TSA_REQUIRES(data_mutex)
|
||||
catch (const ErrnoException & e)
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Poco::Logger::get("AsynchronousMetrics"),
|
||||
getLogger("AsynchronousMetrics"),
|
||||
"Thermal monitor '{}' exists but could not be read: {}.",
|
||||
thermal_device_index,
|
||||
errnoToString(e.getErrno()));
|
||||
@ -254,7 +254,7 @@ void AsynchronousMetrics::openSensorsChips() TSA_REQUIRES(data_mutex)
|
||||
catch (const ErrnoException & e)
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Poco::Logger::get("AsynchronousMetrics"),
|
||||
getLogger("AsynchronousMetrics"),
|
||||
"Hardware monitor '{}', sensor '{}' exists but could not be read: {}.",
|
||||
hwmon_name,
|
||||
sensor_index,
|
||||
|
@ -82,7 +82,7 @@ public:
|
||||
protected:
|
||||
const Duration update_period;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
private:
|
||||
virtual void updateImpl(TimePoint update_time, TimePoint current_time, bool force_update, bool first_run, AsynchronousMetricValues & new_values) = 0;
|
||||
virtual void logImpl(AsynchronousMetricValues &) {}
|
||||
|
@ -77,23 +77,17 @@ ConfigProcessor::ConfigProcessor(
|
||||
, name_pool(new Poco::XML::NamePool(65521))
|
||||
, dom_parser(name_pool)
|
||||
{
|
||||
if (log_to_console && !Poco::Logger::has("ConfigProcessor"))
|
||||
if (log_to_console && !hasLogger("ConfigProcessor"))
|
||||
{
|
||||
channel_ptr = new Poco::ConsoleChannel;
|
||||
log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
|
||||
log = createLogger("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE);
|
||||
}
|
||||
else
|
||||
{
|
||||
log = &Poco::Logger::get("ConfigProcessor");
|
||||
log = getLogger("ConfigProcessor");
|
||||
}
|
||||
}
|
||||
|
||||
ConfigProcessor::~ConfigProcessor()
|
||||
{
|
||||
if (channel_ptr) /// This means we have created a new console logger in the constructor.
|
||||
Poco::Logger::destroy("ConfigProcessor");
|
||||
}
|
||||
|
||||
static std::unordered_map<std::string, std::string_view> embedded_configs;
|
||||
|
||||
void ConfigProcessor::registerEmbeddedConfig(std::string name, std::string_view content)
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
|
||||
#include <Common/Logger.h>
|
||||
|
||||
#include <Poco/DOM/Document.h>
|
||||
#include <Poco/DOM/DOMParser.h>
|
||||
#include <Poco/DOM/DOMWriter.h>
|
||||
@ -44,8 +46,6 @@ public:
|
||||
bool log_to_console = false,
|
||||
const Substitutions & substitutions = Substitutions());
|
||||
|
||||
~ConfigProcessor();
|
||||
|
||||
/// Perform config includes and substitutions and return the resulting XML-document.
|
||||
///
|
||||
/// Suppose path is "/path/file.xml"
|
||||
@ -125,7 +125,7 @@ private:
|
||||
|
||||
bool throw_on_bad_incl;
|
||||
|
||||
Poco::Logger * log;
|
||||
LoggerPtr 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 = &Poco::Logger::get("ConfigReloader");
|
||||
LoggerPtr log = getLogger("ConfigReloader");
|
||||
|
||||
std::string config_path;
|
||||
std::vector<std::string> extra_paths;
|
||||
|
@ -104,7 +104,7 @@ DNSResolver::IPAddresses hostByName(const std::string & host)
|
||||
}
|
||||
catch (const Poco::Net::DNSException & e)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name());
|
||||
LOG_WARNING(getLogger("DNSResolver"), "Cannot resolve host ({}), error {}: {}.", host, e.code(), e.name());
|
||||
addresses.clear();
|
||||
}
|
||||
|
||||
@ -201,7 +201,7 @@ struct DNSResolver::Impl
|
||||
};
|
||||
|
||||
|
||||
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(&Poco::Logger::get("DNSResolver")) {}
|
||||
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(getLogger("DNSResolver")) {}
|
||||
|
||||
Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host)
|
||||
{
|
||||
|
@ -73,7 +73,7 @@ private:
|
||||
|
||||
struct Impl;
|
||||
std::unique_ptr<Impl> impl;
|
||||
Poco::Logger * log;
|
||||
LoggerPtr log;
|
||||
|
||||
/// Updates cached value and returns true it has been changed.
|
||||
bool updateHost(const String & host);
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user