Merge remote-tracking branch 'origin/master' into remove-kql

This commit is contained in:
Alexey Milovidov 2024-01-28 14:24:26 +01:00
commit 267b077038
771 changed files with 3344 additions and 1828 deletions

View File

@ -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;
};

View File

@ -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;
}

View File

@ -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;
@ -73,7 +90,7 @@ void Logger::setProperty(const std::string& name, const std::string& value)
setChannel(LoggingRegistry::defaultRegistry().channelForName(value));
else if (name == "level")
setLevel(value);
else
else
Channel::setProperty(name, value);
}
@ -112,14 +129,14 @@ 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)
{
std::string::size_type len = name.length();
for (LoggerMap::iterator it = _pLoggerMap->begin(); it != _pLoggerMap->end(); ++it)
{
if (len == 0 ||
if (len == 0 ||
(it->first.compare(0, len, name) == 0 && (it->first.length() == len || it->first[len] == '.')))
{
it->second->setLevel(level);
@ -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)
{

View File

@ -1,8 +1,5 @@
if (NOT ENABLE_LIBRARIES)
set(DEFAULT_ENABLE_RUST FALSE)
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "aarch64"))
message(STATUS "Rust is not available on aarch64-apple-darwin")
set(DEFAULT_ENABLE_RUST FALSE)
else()
list (APPEND CMAKE_MODULE_PATH "${ClickHouse_SOURCE_DIR}/contrib/corrosion/cmake")
find_package(Rust)
@ -19,7 +16,9 @@ message(STATUS "Checking Rust toolchain for current target")
# See https://doc.rust-lang.org/nightly/rustc/platform-support.html
if((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl"))
if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le")
set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64") AND (CMAKE_TOOLCHAIN_FILE MATCHES "musl"))
set(Rust_CARGO_TARGET "x86_64-unknown-linux-musl")
elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-x86_64")
set(Rust_CARGO_TARGET "x86_64-unknown-linux-gnu")
@ -29,14 +28,14 @@ elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-aarch64")
set(Rust_CARGO_TARGET "aarch64-unknown-linux-gnu")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "darwin") AND (CMAKE_TOOLCHAIN_FILE MATCHES "darwin"))
set(Rust_CARGO_TARGET "aarch64-apple-darwin")
elseif((CMAKE_TOOLCHAIN_FILE MATCHES "freebsd") AND (CMAKE_TOOLCHAIN_FILE MATCHES "x86_64"))
set(Rust_CARGO_TARGET "x86_64-unknown-freebsd")
elseif(CMAKE_TOOLCHAIN_FILE MATCHES "linux/toolchain-riscv64")
set(Rust_CARGO_TARGET "riscv64gc-unknown-linux-gnu")
endif()
if(CMAKE_TOOLCHAIN_FILE MATCHES "ppc64le")
set(Rust_CARGO_TARGET "powerpc64le-unknown-linux-gnu")
else()
message(FATAL_ERROR "Unsupported rust target")
endif()
message(STATUS "Switched Rust target to ${Rust_CARGO_TARGET}")

View File

@ -22,7 +22,7 @@ RUN apt-get update \
zstd \
--yes --no-install-recommends
RUN pip3 install numpy scipy pandas Jinja2
RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3
ARG odbc_driver_url="https://github.com/ClickHouse/clickhouse-odbc/releases/download/v1.1.4.20200302/clickhouse-odbc-1.1.4-Linux.tar.gz"

View File

@ -247,7 +247,7 @@ stop_logs_replication
# Try to get logs while server is running
successfuly_saved=0
for table in query_log zookeeper_log trace_log transactions_info_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log
do
clickhouse-client -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst
successfuly_saved=$?
@ -288,7 +288,7 @@ if [ $successfuly_saved -ne 0 ]; then
# directly
# - even though ci auto-compress some files (but not *.tsv) it does this only
# for files >64MB, we want this files to be compressed explicitly
for table in query_log zookeeper_log trace_log transactions_info_log
for table in query_log zookeeper_log trace_log transactions_info_log metric_log
do
clickhouse-local "$data_path_config" --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.tsv.zst ||:
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then

View File

@ -5197,3 +5197,13 @@ The value 0 means that you can delete all tables without any restrictions.
:::note
This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop)
:::
## iceberg_engine_ignore_schema_evolution {#iceberg_engine_ignore_schema_evolution}
Allow to ignore schema evolution in Iceberg table engine and read all data using schema specified by the user on table creation or latest schema parsed from metadata on table creation.
:::note
Enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema.
:::
Default value: 'false'.

View File

@ -1805,6 +1805,7 @@ Example of settings:
``` xml
<source>
<postgresql>
<host>postgresql-hostname</hoat>
<port>5432</port>
<user>clickhouse</user>
<password>qwerty</password>

View File

@ -8,8 +8,6 @@ sidebar_label: VIEW
You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process.
The `allow_experimental_alter_materialized_view_structure` setting must be enabled.
This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underling storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause.
**Example with TO table**

View File

@ -97,7 +97,7 @@ This feature is deprecated and will be removed in the future.
For your convenience, the old documentation is located [here](https://pastila.nl/?00f32652/fdf07272a7b54bda7e13b919264e449f.md)
## Refreshable Materialized View {#refreshable-materialized-view}
## Refreshable Materialized View [Experimental] {#refreshable-materialized-view}
```sql
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name
@ -120,7 +120,8 @@ Differences from regular non-refreshable materialized views:
:::note
Refreshable materialized views are a work in progress. Setting `allow_experimental_refreshable_materialized_view = 1` is required for creating one. Current limitations:
* not compatible with Replicated database or table engines,
* not compatible with Replicated database or table engines
* It is not supported in ClickHouse Cloud
* require [Atomic database engine](../../../engines/database-engines/atomic.md),
* no retries for failed refresh - we just skip to the next scheduled refresh time,
* no limit on number of concurrent refreshes.

View File

@ -9,10 +9,6 @@ sidebar_label: RENAME
Renames databases, tables, or dictionaries. Several entities can be renamed in a single query.
Note that the `RENAME` query with several entities is non-atomic operation. To swap entities names atomically, use the [EXCHANGE](./exchange.md) statement.
:::note
The `RENAME` query is supported by the [Atomic](../../engines/database-engines/atomic.md) database engine only.
:::
**Syntax**
```sql

View File

@ -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;

View File

@ -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)

View File

@ -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)
{

View File

@ -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"))

View File

@ -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(),

View File

@ -13,7 +13,7 @@ CatBoostLibraryHandlerFactory & CatBoostLibraryHandlerFactory::instance()
}
CatBoostLibraryHandlerFactory::CatBoostLibraryHandlerFactory()
: log(&Poco::Logger::get("CatBoostLibraryHandlerFactory"))
: log(getLogger("CatBoostLibraryHandlerFactory"))
{
}

View File

@ -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;
};
}

View File

@ -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;
}
}

View File

@ -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;
}

View File

@ -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_)
{

View File

@ -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;
};

View File

@ -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"))
{
}

View File

@ -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;
};
}

View File

@ -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"))

View File

@ -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;
};

View File

@ -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;
};

View File

@ -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;

View File

@ -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)
{

View File

@ -30,7 +30,7 @@ private:
column.insertFrom(sample_column, 0);
}
Poco::Logger * log;
LoggerPtr log;
const UInt64 max_block_size;
ExternalResultDescription description;

View File

@ -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_)

View File

@ -30,7 +30,7 @@ protected:
void consume(Chunk chunk) override;
private:
Poco::Logger * log;
LoggerPtr log;
nanodbc::ConnectionHolderPtr connection_holder;
std::string db_name;

View File

@ -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_)
{

View File

@ -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;
};

View File

@ -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" : "");

View File

@ -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;
};

View File

@ -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 "\"";
}

View File

@ -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;
@ -1748,7 +1748,6 @@ try
LOG_INFO(log, "Stopping AsyncLoader.");
// Waits for all currently running jobs to finish and do not run any other pending jobs.
// Pending jobs will be canceled and destructed later by `load_metadata_tasks` dtor.
global_context->getAsyncLoader().stop();
);
@ -2491,7 +2490,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 +2529,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);

View File

@ -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;

View File

@ -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, "");

View File

@ -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;

View File

@ -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)

View File

@ -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);

View File

@ -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;

View File

@ -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();

View File

@ -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,

View File

@ -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");
}

View File

@ -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();

View File

@ -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;
}

View File

@ -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;
};

View File

@ -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;

View File

@ -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]);
}
}

View File

@ -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()));
}

View File

@ -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;
}

View File

@ -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>

View File

@ -10,12 +10,28 @@ struct Settings;
void registerAggregateFunctionNothing(AggregateFunctionFactory & factory)
{
factory.registerFunction("nothing", [](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
factory.registerFunction(NameAggregateFunctionNothing::name,
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
return std::make_shared<AggregateFunctionNothing>(argument_types, parameters);
});
auto result_type = argument_types.empty() ? std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>()) : argument_types.front();
return std::make_shared<AggregateFunctionNothing>(argument_types, parameters, result_type);
factory.registerFunction(NameAggregateFunctionNothingNull::name,
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
return std::make_shared<AggregateFunctionNothingNull>(argument_types, parameters);
});
factory.registerFunction(NameAggregateFunctionNothingUInt64::name, {
[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
return std::make_shared<AggregateFunctionNothingUInt64>(argument_types, parameters);
},
AggregateFunctionProperties{ .returns_default_when_only_null = true }
});
}

View File

@ -6,7 +6,8 @@
#include <AggregateFunctions/IAggregateFunction.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include "DataTypes/IDataType.h"
#include <DataTypes/IDataType.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
@ -18,20 +19,42 @@ namespace ErrorCodes
extern const int INCORRECT_DATA;
}
/// Returns the same type as the first argument
struct NameAggregateFunctionNothing { static constexpr auto name = "nothing"; };
/// Always returns Nullable(Nothing)
struct NameAggregateFunctionNothingNull { static constexpr auto name = "nothingNull"; };
/// Always returns UInt64
struct NameAggregateFunctionNothingUInt64 { static constexpr auto name = "nothingUInt64"; };
template <typename Name> class AggregateFunctionNothingImpl;
using AggregateFunctionNothing = AggregateFunctionNothingImpl<NameAggregateFunctionNothing>;
using AggregateFunctionNothingNull = AggregateFunctionNothingImpl<NameAggregateFunctionNothingNull>;
using AggregateFunctionNothingUInt64 = AggregateFunctionNothingImpl<NameAggregateFunctionNothingUInt64>;
/** Aggregate function that takes arbitrary number of arbitrary arguments and does nothing.
*/
class AggregateFunctionNothing final : public IAggregateFunctionHelper<AggregateFunctionNothing>
template <typename Name>
class AggregateFunctionNothingImpl final : public IAggregateFunctionHelper<AggregateFunctionNothingImpl<Name>>
{
public:
AggregateFunctionNothing(const DataTypes & arguments, const Array & params, const DataTypePtr & result_type_)
: IAggregateFunctionHelper<AggregateFunctionNothing>(arguments, params, result_type_) {}
String getName() const override
static DataTypePtr getReturnType(const DataTypes & arguments [[maybe_unused]])
{
return "nothing";
if constexpr (std::is_same_v<Name, NameAggregateFunctionNothingUInt64>)
return std::make_shared<DataTypeUInt64>();
else if constexpr (std::is_same_v<Name, NameAggregateFunctionNothingNull>)
return std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>());
return arguments.empty() ? std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>()) : arguments.front();
}
public:
AggregateFunctionNothingImpl(const DataTypes & arguments, const Array & params)
: IAggregateFunctionHelper<AggregateFunctionNothingImpl<Name>>(arguments, params, getReturnType(arguments))
{
}
String getName() const override { return Name::name; }
bool allocatesMemoryInArena() const override { return false; }
void create(AggregateDataPtr __restrict) const override
@ -75,7 +98,8 @@ public:
[[maybe_unused]] char symbol;
readChar(symbol, buf);
if (symbol != '\0')
throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect state of aggregate function 'nothing', it should contain exactly one zero byte, while it is {}.", static_cast<UInt32>(symbol));
throw Exception(ErrorCodes::INCORRECT_DATA, "Incorrect state of aggregate function '{}', it should contain exactly one zero byte, while it is {}",
getName(), static_cast<UInt32>(symbol));
}
void insertResultInto(AggregateDataPtr __restrict, IColumn & to, Arena *) const override

View File

@ -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);
}
};

View File

@ -111,9 +111,9 @@ public:
* To address this, we handle `nothing` in a special way in `FunctionNode::toASTImpl`.
*/
if (properties.returns_default_when_only_null)
return std::make_shared<AggregateFunctionNothing>(arguments, params, std::make_shared<DataTypeUInt64>());
return std::make_shared<AggregateFunctionNothingUInt64>(arguments, params);
else
return std::make_shared<AggregateFunctionNothing>(arguments, params, std::make_shared<DataTypeNullable>(std::make_shared<DataTypeNothing>()));
return std::make_shared<AggregateFunctionNothingNull>(arguments, params);
}
assert(nested_function);

View File

@ -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);
}
}

View File

@ -210,18 +210,6 @@ ASTPtr FunctionNode::toASTImpl(const ConvertToASTOptions & options) const
function_ast->name = function_name;
function_ast->nulls_action = nulls_action;
if (function_name == "nothing")
{
/** Inside AggregateFunctionCombinatorNull we may replace functions with `NULL` in arguments with `nothing`.
* Result type of `nothing` depends on `returns_default_when_only_null` property of nested function.
* If we convert `nothing` to AST, we will lose this information, so we use original function name instead.
*/
const auto & original_ast = getOriginalAST();
const auto & original_function_ast = original_ast ? original_ast->as<ASTFunction>() : nullptr;
if (original_function_ast)
function_ast->name = original_function_ast->name;
}
if (isWindowFunction())
{
function_ast->is_window_function = true;

View File

@ -0,0 +1,124 @@
#include <Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/IQueryTreeNode.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/ExternalDictionariesLoader.h>
namespace DB
{
namespace
{
const std::unordered_set<String> possibly_injective_function_names
{
"dictGet",
"dictGetString",
"dictGetUInt8",
"dictGetUInt16",
"dictGetUInt32",
"dictGetUInt64",
"dictGetInt8",
"dictGetInt16",
"dictGetInt32",
"dictGetInt64",
"dictGetFloat32",
"dictGetFloat64",
"dictGetDate",
"dictGetDateTime"
};
class OptimizeGroupByInjectiveFunctionsVisitor : public InDepthQueryTreeVisitorWithContext<OptimizeGroupByInjectiveFunctionsVisitor>
{
using Base = InDepthQueryTreeVisitorWithContext<OptimizeGroupByInjectiveFunctionsVisitor>;
public:
explicit OptimizeGroupByInjectiveFunctionsVisitor(ContextPtr context)
: Base(std::move(context))
{}
void enterImpl(QueryTreeNodePtr & node)
{
if (!getSettings().optimize_injective_functions_in_group_by)
return;
auto * query = node->as<QueryNode>();
if (!query)
return;
if (!query->hasGroupBy())
return;
if (query->isGroupByWithCube() || query->isGroupByWithRollup())
return;
auto & group_by = query->getGroupBy().getNodes();
if (query->isGroupByWithGroupingSets())
{
for (auto & set : group_by)
{
auto & grouping_set = set->as<ListNode>()->getNodes();
optimizeGroupingSet(grouping_set);
}
}
else
optimizeGroupingSet(group_by);
}
private:
void optimizeGroupingSet(QueryTreeNodes & grouping_set)
{
auto context = getContext();
QueryTreeNodes new_group_by_keys;
new_group_by_keys.reserve(grouping_set.size());
for (auto & group_by_elem : grouping_set)
{
std::queue<QueryTreeNodePtr> nodes_to_process;
nodes_to_process.push(group_by_elem);
while (!nodes_to_process.empty())
{
auto node_to_process = nodes_to_process.front();
nodes_to_process.pop();
auto const * function_node = node_to_process->as<FunctionNode>();
if (!function_node)
{
// Constant aggregation keys are removed in PlannerExpressionAnalysis.cpp
new_group_by_keys.push_back(node_to_process);
continue;
}
// Aggregate functions are not allowed in GROUP BY clause
auto function = function_node->getFunctionOrThrow();
bool can_be_eliminated = function->isInjective(function_node->getArgumentColumns());
if (can_be_eliminated)
{
for (auto const & argument : function_node->getArguments())
{
// We can skip constants here because aggregation key is already not a constant.
if (argument->getNodeType() != QueryTreeNodeType::CONSTANT)
nodes_to_process.push(argument);
}
}
else
new_group_by_keys.push_back(node_to_process);
}
}
grouping_set = std::move(new_group_by_keys);
}
};
}
void OptimizeGroupByInjectiveFunctionsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context)
{
OptimizeGroupByInjectiveFunctionsVisitor visitor(std::move(context));
visitor.visit(query_tree_node);
}
}

View File

@ -0,0 +1,20 @@
#pragma once
#include <Analyzer/IQueryTreePass.h>
namespace DB
{
/* Eliminates injective functions in GROUP BY section.
*/
class OptimizeGroupByInjectiveFunctionsPass final : public IQueryTreePass
{
public:
String getName() override { return "OptimizeGroupByInjectiveFunctionsPass"; }
String getDescription() override { return "Replaces injective functions by it's arguments in GROUP BY section."; }
void run(QueryTreeNodePtr query_tree_node, ContextPtr context) override;
};
}

View File

@ -2321,11 +2321,15 @@ std::pair<bool, UInt64> QueryAnalyzer::recursivelyCollectMaxOrdinaryExpressions(
*/
void QueryAnalyzer::expandGroupByAll(QueryNode & query_tree_node_typed)
{
if (!query_tree_node_typed.isGroupByAll())
return;
auto & group_by_nodes = query_tree_node_typed.getGroupBy().getNodes();
auto & projection_list = query_tree_node_typed.getProjection();
for (auto & node : projection_list.getNodes())
recursivelyCollectMaxOrdinaryExpressions(node, group_by_nodes);
query_tree_node_typed.setIsGroupByAll(false);
}
void QueryAnalyzer::expandOrderByAll(QueryNode & query_tree_node_typed)
@ -7422,8 +7426,7 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier
node->removeAlias();
}
if (query_node_typed.isGroupByAll())
expandGroupByAll(query_node_typed);
expandGroupByAll(query_node_typed);
validateFilters(query_node);
validateAggregates(query_node, { .group_by_use_nulls = scope.group_by_use_nulls });

View File

@ -3,6 +3,7 @@
#include <memory>
#include <Common/Exception.h>
#include "Analyzer/Passes/OptimizeGroupByInjectiveFunctionsPass.h"
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
@ -163,8 +164,6 @@ private:
/** ClickHouse query tree pass manager.
*
* TODO: Support setting optimize_substitute_columns.
* TODO: Support GROUP BY injective function elimination.
* TODO: Support setting optimize_aggregators_of_group_by_keys.
* TODO: Support setting optimize_monotonous_functions_in_order_by.
* TODO: Add optimizations based on function semantics. Example: SELECT * FROM test_table WHERE id != id. (id is not nullable column).
@ -268,6 +267,7 @@ void addQueryTreePasses(QueryTreePassManager & manager)
manager.addPass(std::make_unique<AggregateFunctionsArithmericOperationsPass>());
manager.addPass(std::make_unique<UniqInjectiveFunctionsEliminationPass>());
manager.addPass(std::make_unique<OptimizeGroupByFunctionKeysPass>());
manager.addPass(std::make_unique<OptimizeGroupByInjectiveFunctionsPass>());
manager.addPass(std::make_unique<MultiIfToIfPass>());
manager.addPass(std::make_unique<IfConstantConditionPass>());

View File

@ -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;
@ -626,7 +688,7 @@ void rerunFunctionResolve(FunctionNode * function_node, ContextPtr context)
}
else if (function_node->isAggregateFunction())
{
if (name == "nothing")
if (name == "nothing" || name == "nothingUInt64" || name == "nothingNull")
return;
function_node->resolveAsAggregateFunction(resolveAggregateFunction(function_node));
}

View File

@ -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.

View File

@ -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_)
{
}

View File

@ -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;

View File

@ -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_,

View File

@ -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;

View File

@ -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_)

View File

@ -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;
};
}

View File

@ -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,

View File

@ -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;

View File

@ -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)
{

View File

@ -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);

View File

@ -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_)

View File

@ -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;

View File

@ -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())

View File

@ -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))
{

View File

@ -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()))

View File

@ -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();
}

View File

@ -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;
};
}

View File

@ -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())
{

View File

@ -127,7 +127,7 @@ private:
const bool allow_concurrent_restores;
const bool test_inject_sleep;
Poco::Logger * log;
LoggerPtr log;
struct ExtendedOperationInfo
{

View File

@ -6,7 +6,7 @@
namespace DB
{
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(&Poco::Logger::get("RestoreCoordinationLocal"))
RestoreCoordinationLocal::RestoreCoordinationLocal() : log(getLogger("RestoreCoordinationLocal"))
{
}

View File

@ -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;

View File

@ -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_,

View File

@ -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;

View File

@ -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")
{
}

View File

@ -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;

View File

@ -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_)

View File

@ -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;

View File

@ -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;

View File

@ -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))

View File

@ -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;

View File

@ -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;

View File

@ -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;
};

View File

@ -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_)
{

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