mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #11137 from ClickHouse/logger-fmt
fmt-style logging, part 1
This commit is contained in:
commit
bee5849c6a
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -157,3 +157,6 @@
|
||||
[submodule "contrib/openldap"]
|
||||
path = contrib/openldap
|
||||
url = https://github.com/openldap/openldap.git
|
||||
[submodule "contrib/fmtlib"]
|
||||
path = contrib/fmtlib
|
||||
url = https://github.com/fmtlib/fmt.git
|
||||
|
@ -79,6 +79,7 @@ target_link_libraries (common
|
||||
Poco::Util
|
||||
Poco::Foundation
|
||||
replxx
|
||||
fmt
|
||||
|
||||
PRIVATE
|
||||
cctz
|
||||
|
@ -2,16 +2,14 @@
|
||||
|
||||
/// Macros for convenient usage of Poco logger.
|
||||
|
||||
#include <sstream>
|
||||
#include <fmt/format.h>
|
||||
#include <fmt/ostream.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Message.h>
|
||||
#include <Poco/Version.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
|
||||
#ifndef QUERY_PREVIEW_LENGTH
|
||||
#define QUERY_PREVIEW_LENGTH 160
|
||||
#endif
|
||||
|
||||
/// TODO Remove this.
|
||||
using Poco::Logger;
|
||||
using Poco::Message;
|
||||
using DB::LogsLevel;
|
||||
@ -19,21 +17,20 @@ using DB::CurrentThread;
|
||||
|
||||
/// Logs a message to a specified logger with that level.
|
||||
|
||||
#define LOG_SIMPLE(logger, message, priority, PRIORITY) do \
|
||||
#define LOG_IMPL(logger, priority, PRIORITY, ...) do \
|
||||
{ \
|
||||
const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \
|
||||
(CurrentThread::getGroup()->client_logs_level >= (priority)); \
|
||||
if ((logger)->is((PRIORITY)) || is_clients_log) \
|
||||
{ \
|
||||
std::stringstream oss_internal_rare; \
|
||||
oss_internal_rare << message; \
|
||||
std::string formatted_message = fmt::format(__VA_ARGS__); \
|
||||
if (auto channel = (logger)->getChannel()) \
|
||||
{ \
|
||||
std::string file_function; \
|
||||
file_function += __FILE__; \
|
||||
file_function += "; "; \
|
||||
file_function += __PRETTY_FUNCTION__; \
|
||||
Message poco_message((logger)->name(), oss_internal_rare.str(), \
|
||||
Message poco_message((logger)->name(), formatted_message, \
|
||||
(PRIORITY), file_function.c_str(), __LINE__); \
|
||||
channel->log(poco_message); \
|
||||
} \
|
||||
@ -41,10 +38,9 @@ using DB::CurrentThread;
|
||||
} while (false)
|
||||
|
||||
|
||||
#define LOG_TRACE(logger, message) LOG_SIMPLE(logger, message, LogsLevel::trace, Message::PRIO_TRACE)
|
||||
#define LOG_DEBUG(logger, message) LOG_SIMPLE(logger, message, LogsLevel::debug, Message::PRIO_DEBUG)
|
||||
#define LOG_INFO(logger, message) LOG_SIMPLE(logger, message, LogsLevel::information, Message::PRIO_INFORMATION)
|
||||
#define LOG_WARNING(logger, message) LOG_SIMPLE(logger, message, LogsLevel::warning, Message::PRIO_WARNING)
|
||||
#define LOG_ERROR(logger, message) LOG_SIMPLE(logger, message, LogsLevel::error, Message::PRIO_ERROR)
|
||||
#define LOG_FATAL(logger, message) LOG_SIMPLE(logger, message, LogsLevel::error, Message::PRIO_FATAL)
|
||||
|
||||
#define LOG_TRACE(logger, ...) LOG_IMPL(logger, LogsLevel::trace, Message::PRIO_TRACE, __VA_ARGS__)
|
||||
#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, LogsLevel::debug, Message::PRIO_DEBUG, __VA_ARGS__)
|
||||
#define LOG_INFO(logger, ...) LOG_IMPL(logger, LogsLevel::information, Message::PRIO_INFORMATION, __VA_ARGS__)
|
||||
#define LOG_WARNING(logger, ...) LOG_IMPL(logger, LogsLevel::warning, Message::PRIO_WARNING, __VA_ARGS__)
|
||||
#define LOG_ERROR(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_ERROR, __VA_ARGS__)
|
||||
#define LOG_FATAL(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_FATAL, __VA_ARGS__)
|
||||
|
@ -24,6 +24,7 @@ PEERDIR(
|
||||
contrib/libs/cxxsupp/libcxx-filesystem
|
||||
contrib/libs/poco/Net
|
||||
contrib/libs/poco/Util
|
||||
contrib/libs/fmt
|
||||
contrib/restricted/boost
|
||||
contrib/restricted/cityhash-1.0.2
|
||||
)
|
||||
|
@ -180,7 +180,7 @@ public:
|
||||
// levels and more info, but for completeness we log all signals
|
||||
// here at trace level.
|
||||
// Don't use strsignal here, because it's not thread-safe.
|
||||
LOG_TRACE(log, "Received signal " << sig);
|
||||
LOG_TRACE(log, "Received signal {}", sig);
|
||||
|
||||
if (sig == Signals::StopThread)
|
||||
{
|
||||
@ -236,7 +236,7 @@ private:
|
||||
|
||||
void onTerminate(const std::string & message, UInt32 thread_num) const
|
||||
{
|
||||
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message);
|
||||
LOG_FATAL(log, "(version {}{}) (from thread {}) {}", VERSION_STRING, VERSION_OFFICIAL, thread_num, message);
|
||||
}
|
||||
|
||||
void onFault(
|
||||
@ -257,9 +257,9 @@ private:
|
||||
message << " (no query)";
|
||||
else
|
||||
message << " (query_id: " << query_id << ")";
|
||||
message << " Received signal " << strsignal(sig) << " (" << sig << ")" << ".";
|
||||
message << " Received signal " << strsignal(sig) << " (" << sig << ").";
|
||||
|
||||
LOG_FATAL(log, message.rdbuf());
|
||||
LOG_FATAL(log, message.str());
|
||||
}
|
||||
|
||||
LOG_FATAL(log, signalToErrorMessage(sig, info, context));
|
||||
@ -274,7 +274,7 @@ private:
|
||||
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
|
||||
bare_stacktrace << ' ' << stack_trace.getFrames()[i];
|
||||
|
||||
LOG_FATAL(log, bare_stacktrace.rdbuf());
|
||||
LOG_FATAL(log, bare_stacktrace.str());
|
||||
}
|
||||
|
||||
/// Write symbolized stack trace line by line for better grep-ability.
|
||||
@ -302,7 +302,7 @@ static void sanitizerDeathCallback()
|
||||
message << " (query_id: " << query_id << ")";
|
||||
message << " Sanitizer trap.";
|
||||
|
||||
LOG_FATAL(log, message.rdbuf());
|
||||
LOG_FATAL(log, message.str());
|
||||
}
|
||||
|
||||
/// Just in case print our own stack trace. In case when llvm-symbolizer does not work.
|
||||
@ -314,7 +314,7 @@ static void sanitizerDeathCallback()
|
||||
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
|
||||
bare_stacktrace << ' ' << stack_trace.getFrames()[i];
|
||||
|
||||
LOG_FATAL(log, bare_stacktrace.rdbuf());
|
||||
LOG_FATAL(log, bare_stacktrace.str());
|
||||
}
|
||||
|
||||
/// Write symbolized stack trace line by line for better grep-ability.
|
||||
@ -379,7 +379,7 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_WARNING(logger, __PRETTY_FUNCTION__ << ": when creating " << path << ", " << DB::getCurrentExceptionMessage(true));
|
||||
LOG_WARNING(logger, "{}: when creating {}, {}", __PRETTY_FUNCTION__, path, DB::getCurrentExceptionMessage(true));
|
||||
}
|
||||
return false;
|
||||
}
|
||||
@ -498,11 +498,10 @@ void debugIncreaseOOMScore()
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '" +
|
||||
e.displayText() + "'.");
|
||||
LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText());
|
||||
return;
|
||||
}
|
||||
LOG_INFO(&Logger::root(), "Set OOM score adjustment to " + new_score);
|
||||
LOG_INFO(&Logger::root(), "Set OOM score adjustment to {}", new_score);
|
||||
}
|
||||
#else
|
||||
void debugIncreaseOOMScore() {}
|
||||
@ -734,7 +733,7 @@ void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn)
|
||||
task_failed = true;
|
||||
Poco::AutoPtr<Poco::TaskFailedNotification> fn(_tfn);
|
||||
Logger *lg = &(logger());
|
||||
LOG_ERROR(lg, "Task '" << fn->task()->name() << "' failed. Daemon is shutting down. Reason - " << fn->reason().displayText());
|
||||
LOG_ERROR(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText());
|
||||
ServerApplication::terminate();
|
||||
}
|
||||
|
||||
@ -850,7 +849,7 @@ void BaseDaemon::handleSignal(int signal_id)
|
||||
void BaseDaemon::onInterruptSignals(int signal_id)
|
||||
{
|
||||
is_cancelled = true;
|
||||
LOG_INFO(&logger(), "Received termination signal (" << strsignal(signal_id) << ")");
|
||||
LOG_INFO(&logger(), "Received termination signal ({})", strsignal(signal_id));
|
||||
|
||||
if (sigint_signals_counter >= 2)
|
||||
{
|
||||
|
@ -52,8 +52,7 @@ private:
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
LOG_WARNING(&Poco::Util::Application::instance().logger(),
|
||||
"Fail to write to Graphite " << host << ":" << port << ". e.what() = " << e.what() << ", e.message() = " << e.message());
|
||||
LOG_WARNING(&Poco::Util::Application::instance().logger(), "Fail to write to Graphite {}:{}. e.what() = {}, e.message() = {}", host, port, e.what(), e.message());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <ctime>
|
||||
#include <string>
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
|
||||
|
||||
namespace ext
|
||||
|
2
contrib/CMakeLists.txt
vendored
2
contrib/CMakeLists.txt
vendored
@ -317,3 +317,5 @@ endif()
|
||||
if (USE_FASTOPS)
|
||||
add_subdirectory (fastops-cmake)
|
||||
endif()
|
||||
|
||||
add_subdirectory (fmtlib-cmake)
|
||||
|
1
contrib/fmtlib
vendored
Submodule
1
contrib/fmtlib
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit 297c3b2ed551a4989826fc8c4780bf533e964bd9
|
20
contrib/fmtlib-cmake/CMakeLists.txt
Normal file
20
contrib/fmtlib-cmake/CMakeLists.txt
Normal file
@ -0,0 +1,20 @@
|
||||
set (SRCS
|
||||
../fmtlib/src/format.cc
|
||||
../fmtlib/src/os.cc
|
||||
|
||||
../fmtlib/include/fmt/chrono.h
|
||||
../fmtlib/include/fmt/color.h
|
||||
../fmtlib/include/fmt/compile.h
|
||||
../fmtlib/include/fmt/core.h
|
||||
../fmtlib/include/fmt/format.h
|
||||
../fmtlib/include/fmt/format-inl.h
|
||||
../fmtlib/include/fmt/locale.h
|
||||
../fmtlib/include/fmt/os.h
|
||||
../fmtlib/include/fmt/ostream.h
|
||||
../fmtlib/include/fmt/posix.h
|
||||
../fmtlib/include/fmt/printf.h
|
||||
../fmtlib/include/fmt/ranges.h
|
||||
)
|
||||
|
||||
add_library(fmt ${SRCS})
|
||||
target_include_directories(fmt SYSTEM PUBLIC ../fmtlib/include)
|
@ -424,7 +424,7 @@ private:
|
||||
std::cerr << percent << "%\t\t";
|
||||
for (const auto & info : infos)
|
||||
{
|
||||
std::cerr << info->sampler.quantileNearest(percent / 100.0) << " sec." << "\t";
|
||||
std::cerr << info->sampler.quantileNearest(percent / 100.0) << " sec.\t";
|
||||
}
|
||||
std::cerr << "\n";
|
||||
};
|
||||
@ -459,7 +459,7 @@ private:
|
||||
|
||||
auto print_percentile = [&json_out](Stats & info, auto percent, bool with_comma = true)
|
||||
{
|
||||
json_out << "\"" << percent << "\"" << ": " << info.sampler.quantileNearest(percent / 100.0) << (with_comma ? ",\n" : "\n");
|
||||
json_out << "\"" << percent << "\": " << info.sampler.quantileNearest(percent / 100.0) << (with_comma ? ",\n" : "\n");
|
||||
};
|
||||
|
||||
json_out << "{\n";
|
||||
@ -469,7 +469,7 @@ private:
|
||||
const auto & info = infos[i];
|
||||
|
||||
json_out << double_quote << connections[i]->getDescription() << ": {\n";
|
||||
json_out << double_quote << "statistics" << ": {\n";
|
||||
json_out << double_quote << "statistics: {\n";
|
||||
|
||||
print_key_value("QPS", info->queries / info->work_time);
|
||||
print_key_value("RPS", info->read_rows / info->work_time);
|
||||
@ -479,7 +479,7 @@ private:
|
||||
print_key_value("num_queries", info->queries.load(), false);
|
||||
|
||||
json_out << "},\n";
|
||||
json_out << double_quote << "query_time_percentiles" << ": {\n";
|
||||
json_out << double_quote << "query_time_percentiles: {\n";
|
||||
|
||||
for (int percent = 0; percent <= 90; percent += 10)
|
||||
print_percentile(*info, percent);
|
||||
|
@ -26,7 +26,7 @@ void ClusterCopier::init()
|
||||
if (response.error != Coordination::ZOK)
|
||||
return;
|
||||
UInt64 version = ++task_description_version;
|
||||
LOG_DEBUG(log, "Task description should be updated, local version " << version);
|
||||
LOG_DEBUG(log, "Task description should be updated, local version {}", version);
|
||||
};
|
||||
|
||||
task_description_path = task_zookeeper_path + "/description";
|
||||
@ -47,7 +47,7 @@ void ClusterCopier::init()
|
||||
task_table.initShards(task_cluster->random_engine);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks");
|
||||
LOG_DEBUG(log, "Will process {} table tasks", task_cluster->table_tasks.size());
|
||||
|
||||
/// Do not initialize tables, will make deferred initialization in process()
|
||||
|
||||
@ -85,7 +85,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
|
||||
{
|
||||
TaskTable & task_table = task_shard->task_table;
|
||||
|
||||
LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription());
|
||||
LOG_INFO(log, "Discover partitions of shard {}", task_shard->getDescription());
|
||||
|
||||
auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); };
|
||||
auto existing_partitions_names = retry(get_partitions, 60);
|
||||
@ -132,8 +132,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
|
||||
{
|
||||
if (!task_table.enabled_partitions_set.count(partition_name))
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in "
|
||||
<< "enabled_partitions of " << task_table.table_id);
|
||||
LOG_DEBUG(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -165,11 +164,10 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
|
||||
for (const String & missing_partition : missing_partitions)
|
||||
ss << " " << missing_partition;
|
||||
|
||||
LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard "
|
||||
<< task_shard->getDescription() << " :" << ss.str());
|
||||
LOG_WARNING(log, "There are no {} partitions from enabled_partitions in shard {} :{}", missing_partitions.size(), task_shard->getDescription(), ss.str());
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription());
|
||||
LOG_DEBUG(log, "Will copy {} partitions from shard {}", task_shard->partition_tasks.size(), task_shard->getDescription());
|
||||
}
|
||||
|
||||
void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads)
|
||||
@ -181,7 +179,7 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts,
|
||||
for (const TaskShardPtr & task_shard : task_table.all_shards)
|
||||
thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); });
|
||||
|
||||
LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs");
|
||||
LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active());
|
||||
thread_pool.wait();
|
||||
}
|
||||
}
|
||||
@ -205,7 +203,8 @@ void ClusterCopier::uploadTaskDescription(const std::string & task_path, const s
|
||||
if (code && force)
|
||||
zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
|
||||
|
||||
LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")");
|
||||
LOG_DEBUG(log, "Task description {} uploaded to {} with result {} ({})",
|
||||
((code && !force) ? "not " : ""), local_task_description_path, code, zookeeper->error2string(code));
|
||||
}
|
||||
|
||||
void ClusterCopier::reloadTaskDescription()
|
||||
@ -221,7 +220,7 @@ void ClusterCopier::reloadTaskDescription()
|
||||
if (code)
|
||||
throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid);
|
||||
LOG_DEBUG(log, "Loading description, zxid={}", task_description_current_stat.czxid);
|
||||
auto config = getConfigurationFromXMLString(task_config_str);
|
||||
|
||||
/// Setup settings
|
||||
@ -251,9 +250,7 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts)
|
||||
{
|
||||
for (TaskTable & task_table : task_cluster->table_tasks)
|
||||
{
|
||||
LOG_INFO(log, "Process table task " << task_table.table_id << " with "
|
||||
<< task_table.all_shards.size() << " shards, "
|
||||
<< task_table.local_shards.size() << " of them are local ones");
|
||||
LOG_INFO(log, "Process table task {} with {} shards, {} of them are local ones", task_table.table_id, task_table.all_shards.size(), task_table.local_shards.size());
|
||||
|
||||
if (task_table.all_shards.empty())
|
||||
continue;
|
||||
@ -357,8 +354,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee
|
||||
|
||||
if (static_cast<UInt64>(stat.numChildren) >= task_cluster->max_workers)
|
||||
{
|
||||
LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")"
|
||||
<< ". Postpone processing " << description);
|
||||
LOG_DEBUG(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description);
|
||||
|
||||
if (unprioritized)
|
||||
current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time);
|
||||
@ -419,7 +415,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl
|
||||
{
|
||||
bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition);
|
||||
if (!piece_is_done)
|
||||
LOG_DEBUG(log, "Partition " << partition_name << " piece " + toString(piece_number) + " is not already done.");
|
||||
LOG_DEBUG(log, "Partition {} piece {} is not already done.", partition_name, piece_number);
|
||||
answer &= piece_is_done;
|
||||
}
|
||||
|
||||
@ -435,8 +431,7 @@ bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_tabl
|
||||
bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name,
|
||||
size_t piece_number, const TasksShard & shards_with_partition)
|
||||
{
|
||||
LOG_DEBUG(log, "Check that all shards processed partition " << partition_name
|
||||
<< " piece " + toString(piece_number) + " successfully");
|
||||
LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number);
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
|
||||
@ -465,8 +460,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
|
||||
TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data);
|
||||
if (status.state != TaskState::Finished)
|
||||
{
|
||||
LOG_INFO(log, "The task " << res.data << " is being rewritten by "
|
||||
<< status.owner << ". Partition piece will be rechecked");
|
||||
LOG_INFO(log, "The task {} is being rewritten by {}. Partition piece will be rechecked", res.data, status.owner);
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -484,7 +478,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
|
||||
|
||||
if (!is_clean)
|
||||
{
|
||||
LOG_INFO(log, "Partition " << partition_name << " become dirty");
|
||||
LOG_INFO(log, "Partition {} become dirty", partition_name);
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -501,8 +495,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
|
||||
}
|
||||
catch (const Coordination::Exception & e)
|
||||
{
|
||||
LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number "
|
||||
<< toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText());
|
||||
LOG_INFO(log, "A ZooKeeper error occurred while checking partition {} piece number {}. Will recheck the partition. Error: {}", partition_name, toString(piece_number), e.displayText());
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -511,12 +504,12 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons
|
||||
{
|
||||
if (zxid1[shard_num] != zxid2[shard_num])
|
||||
{
|
||||
LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked");
|
||||
LOG_INFO(log, "The task {} is being modified now. Partition piece will be rechecked", piece_status_paths[shard_num]);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " is copied successfully");
|
||||
LOG_INFO(log, "Partition {} piece number {} is copied successfully", partition_name, toString(piece_number));
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -530,7 +523,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
inject_fault = value < move_fault_probability;
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table");
|
||||
LOG_DEBUG(log, "Try to move {} to destionation table", partition_name);
|
||||
|
||||
auto zookeeper = context.getZooKeeper();
|
||||
|
||||
@ -548,7 +541,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
{
|
||||
if (e.code == Coordination::ZNODEEXISTS)
|
||||
{
|
||||
LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active);
|
||||
LOG_DEBUG(log, "Someone is already moving pieces {}", current_partition_attach_is_active);
|
||||
return TaskStatus::Active;
|
||||
}
|
||||
|
||||
@ -565,16 +558,13 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
|
||||
if (status.state == TaskState::Finished)
|
||||
{
|
||||
LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active
|
||||
<< " has been successfully moved to destination table by " << status.owner);
|
||||
LOG_DEBUG(log, "All pieces for partition from this task {} has been successfully moved to destination table by {}", current_partition_attach_is_active, status.owner);
|
||||
return TaskStatus::Finished;
|
||||
}
|
||||
|
||||
/// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
|
||||
/// Initialize DROP PARTITION
|
||||
LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active
|
||||
<< " has not been successfully finished by " << status.owner
|
||||
<< ". Will try to move by myself.");
|
||||
LOG_DEBUG(log, "Moving piece for partition {} has not been successfully finished by {}. Will try to move by myself.", current_partition_attach_is_active, status.owner);
|
||||
|
||||
/// Remove is_done marker.
|
||||
zookeeper->remove(current_partition_attach_is_done);
|
||||
@ -591,9 +581,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
/// Move partition to original destination table.
|
||||
for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
|
||||
{
|
||||
LOG_DEBUG(log, "Trying to move partition " << partition_name
|
||||
<< " piece " << toString(current_piece_number)
|
||||
<< " to original table");
|
||||
LOG_DEBUG(log, "Trying to move partition {} piece {} to original table", partition_name, toString(current_piece_number));
|
||||
|
||||
ASTPtr query_alter_ast;
|
||||
String query_alter_ast_string;
|
||||
@ -614,7 +602,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
" ATTACH PARTITION " + partition_name +
|
||||
" FROM " + getQuotedTable(helping_table);
|
||||
|
||||
LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string);
|
||||
LOG_DEBUG(log, "Executing ALTER query: {}", query_alter_ast_string);
|
||||
|
||||
try
|
||||
{
|
||||
@ -626,13 +614,11 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
PoolMode::GET_MANY,
|
||||
ClusterExecutionMode::ON_EACH_NODE);
|
||||
|
||||
LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes));
|
||||
LOG_INFO(log, "Number of nodes that executed ALTER query successfully : {}", toString(num_nodes));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_DEBUG(log, "Error while moving partition " << partition_name
|
||||
<< " piece " << toString(current_piece_number)
|
||||
<< "to original table");
|
||||
LOG_DEBUG(log, "Error while moving partition {} piece {} to original table", partition_name, toString(current_piece_number));
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -647,7 +633,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) +
|
||||
" PARTITION " + partition_name + " DEDUPLICATE;";
|
||||
|
||||
LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string);
|
||||
LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_alter_ast_string);
|
||||
|
||||
UInt64 num_nodes = executeQueryOnCluster(
|
||||
task_table.cluster_push,
|
||||
@ -656,14 +642,12 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t
|
||||
&task_cluster->settings_push,
|
||||
PoolMode::GET_MANY);
|
||||
|
||||
LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : "
|
||||
<< toString(num_nodes));
|
||||
LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes));
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name
|
||||
<< "in the original table");
|
||||
LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -759,8 +743,7 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
{
|
||||
if (e.code == Coordination::ZNODEEXISTS)
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " is cleaning now by somebody, sleep");
|
||||
LOG_DEBUG(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number));
|
||||
std::this_thread::sleep_for(default_sleep_time);
|
||||
return false;
|
||||
}
|
||||
@ -773,8 +756,7 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
{
|
||||
if (stat.numChildren != 0)
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren
|
||||
<< " active workers while trying to drop it. Going to sleep.");
|
||||
LOG_DEBUG(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren);
|
||||
std::this_thread::sleep_for(default_sleep_time);
|
||||
return false;
|
||||
}
|
||||
@ -794,7 +776,7 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
{
|
||||
if (e.code == Coordination::ZNODEEXISTS)
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep");
|
||||
LOG_DEBUG(log, "Partition {} is being filled now by somebody, sleep", task_partition.name);
|
||||
return false;
|
||||
}
|
||||
|
||||
@ -832,7 +814,7 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
/// It is important, DROP PARTITION must be done synchronously
|
||||
settings_push.replication_alter_partitions_sync = 2;
|
||||
|
||||
LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query);
|
||||
LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
|
||||
/// We have to drop partition_piece on each replica
|
||||
size_t num_shards = executeQueryOnCluster(
|
||||
cluster_push, query,
|
||||
@ -841,7 +823,7 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
PoolMode::GET_MANY,
|
||||
ClusterExecutionMode::ON_EACH_NODE);
|
||||
|
||||
LOG_INFO(log, "DROP PARTITION was successfully executed on " << num_shards << " nodes of a cluster.");
|
||||
LOG_INFO(log, "DROP PARTITION was successfully executed on {} nodes of a cluster.", num_shards);
|
||||
|
||||
/// Update the locking node
|
||||
if (!my_clock.is_stale())
|
||||
@ -859,13 +841,12 @@ bool ClusterCopier::tryDropPartitionPiece(
|
||||
return false;
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number)
|
||||
<< " was dropped on cluster " << task_table.cluster_push_name);
|
||||
LOG_INFO(log, "Partition {} piece {} was dropped on cluster {}", task_partition.name, toString(current_piece_number), task_table.cluster_push_name);
|
||||
if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS)
|
||||
zookeeper->set(current_shards_path, host_id);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is safe for work now.");
|
||||
LOG_INFO(log, "Partition {} piece {} is safe for work now.", task_partition.name, toString(current_piece_number));
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -889,7 +870,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
|
||||
|
||||
++cluster_partition.total_tries;
|
||||
|
||||
LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster");
|
||||
LOG_DEBUG(log, "Processing partition {} for the whole cluster", partition_name);
|
||||
|
||||
/// Process each source shard having current partition and copy current partition
|
||||
/// NOTE: shards are sorted by "distance" to current host
|
||||
@ -911,7 +892,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
|
||||
{
|
||||
const size_t number_of_splits = task_table.number_of_splits;
|
||||
shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits));
|
||||
LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription());
|
||||
LOG_DEBUG(log, "Discovered partition {} in shard {}", partition_name, shard->getDescription());
|
||||
/// To save references in the future.
|
||||
auto shard_partition_it = shard->partition_tasks.find(partition_name);
|
||||
PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces;
|
||||
@ -924,7 +905,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name);
|
||||
LOG_DEBUG(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
@ -1030,21 +1011,20 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
|
||||
task_table.rows_copied += cluster_partition.rows_copied;
|
||||
double elapsed = cluster_partition.elapsed_time_seconds;
|
||||
|
||||
LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name
|
||||
<< ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes"
|
||||
<< ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows"
|
||||
<< " and " << cluster_partition.blocks_copied << " source blocks are copied");
|
||||
LOG_INFO(log, "It took {} seconds to copy partition {}: {} uncompressed bytes, {} rows and {} source blocks are copied",
|
||||
elapsed, partition_name,
|
||||
formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied),
|
||||
formatReadableQuantity(cluster_partition.rows_copied),
|
||||
cluster_partition.blocks_copied);
|
||||
|
||||
if (cluster_partition.rows_copied)
|
||||
{
|
||||
LOG_INFO(log, "Average partition speed: "
|
||||
<< formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second.");
|
||||
LOG_INFO(log, "Average partition speed: {} per second.", formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed));
|
||||
}
|
||||
|
||||
if (task_table.rows_copied)
|
||||
{
|
||||
LOG_INFO(log, "Average table " << task_table.table_id << " speed: "
|
||||
<< formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second.");
|
||||
LOG_INFO(log, "Average table {} speed: {} per second.", task_table.table_id, formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -1055,8 +1035,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab
|
||||
|
||||
if (!table_is_done)
|
||||
{
|
||||
LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet."
|
||||
<< "Copied " << finished_partitions << " of " << required_partitions << ", will retry");
|
||||
LOG_INFO(log, "Table {} is not processed yet.Copied {} of {}, will retry", task_table.table_id, finished_partitions, required_partitions);
|
||||
}
|
||||
|
||||
return table_is_done;
|
||||
@ -1104,9 +1083,11 @@ TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTim
|
||||
{
|
||||
for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
|
||||
{
|
||||
LOG_INFO(log, "Attempt number " << try_num << " to process partition " << task_partition.name
|
||||
<< " piece number " << piece_number << " on shard number " << task_partition.task_shard.numberInCluster()
|
||||
<< " with index " << task_partition.task_shard.indexInCluster());
|
||||
LOG_INFO(log, "Attempt number {} to process partition {} piece number {} on shard number {} with index {}.",
|
||||
try_num, task_partition.name, piece_number,
|
||||
task_partition.task_shard.numberInCluster(),
|
||||
task_partition.task_shard.indexInCluster());
|
||||
|
||||
res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task);
|
||||
|
||||
/// Exit if success
|
||||
@ -1210,7 +1191,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
/// Load balancing
|
||||
auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task);
|
||||
|
||||
LOG_DEBUG(log, "Processing " << current_task_piece_status_path);
|
||||
LOG_DEBUG(log, "Processing {}", current_task_piece_status_path);
|
||||
|
||||
const String piece_status_path = partition_piece.getPartitionPieceShardsPath();
|
||||
|
||||
@ -1221,14 +1202,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
/// Do not start if partition piece is dirty, try to clean it
|
||||
if (is_clean)
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name
|
||||
<< " piece " + toString(current_piece_number) + " appears to be clean");
|
||||
LOG_DEBUG(log, "Partition {} piece {} appears to be clean", task_partition.name, current_piece_number);
|
||||
zookeeper->createAncestors(current_task_piece_status_path);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name
|
||||
<< " piece " + toString(current_piece_number) + " is dirty, try to drop it");
|
||||
LOG_DEBUG(log, "Partition {} piece {} is dirty, try to drop it", task_partition.name, current_piece_number);
|
||||
|
||||
try
|
||||
{
|
||||
@ -1253,7 +1232,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
{
|
||||
if (e.code == Coordination::ZNODEEXISTS)
|
||||
{
|
||||
LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path);
|
||||
LOG_DEBUG(log, "Someone is already processing {}", current_task_piece_is_active_path);
|
||||
return TaskStatus::Active;
|
||||
}
|
||||
|
||||
@ -1269,16 +1248,13 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
|
||||
if (status.state == TaskState::Finished)
|
||||
{
|
||||
LOG_DEBUG(log, "Task " << current_task_piece_status_path
|
||||
<< " has been successfully executed by " << status.owner);
|
||||
LOG_DEBUG(log, "Task {} has been successfully executed by {}", current_task_piece_status_path, status.owner);
|
||||
return TaskStatus::Finished;
|
||||
}
|
||||
|
||||
/// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
|
||||
/// Initialize DROP PARTITION
|
||||
LOG_DEBUG(log, "Task " << current_task_piece_status_path
|
||||
<< " has not been successfully finished by " << status.owner
|
||||
<< ". Partition will be dropped and refilled.");
|
||||
LOG_DEBUG(log, "Task {} has not been successfully finished by {}. Partition will be dropped and refilled.", current_task_piece_status_path, status.owner);
|
||||
|
||||
create_is_dirty_node(clean_state_clock);
|
||||
return TaskStatus::Error;
|
||||
@ -1293,11 +1269,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
|
||||
auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent);
|
||||
if (res == Coordination::ZNODEEXISTS)
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
|
||||
+ toString(current_piece_number) + " is absent on current replica of a shard. But other replicas have already marked it as done.");
|
||||
LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. But other replicas have already marked it as done.", task_partition.name, current_piece_number);
|
||||
if (res == Coordination::ZOK)
|
||||
LOG_DEBUG(log, "Partition " << task_partition.name << " piece "
|
||||
+ toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.");
|
||||
LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.", task_partition.name, current_piece_number);
|
||||
return TaskStatus::Finished;
|
||||
}
|
||||
|
||||
@ -1325,18 +1299,14 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
|
||||
if (count != 0)
|
||||
{
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< current_piece_number << "is not empty. In contains " << count << " rows.");
|
||||
LOG_INFO(log, "Partition {} piece {}is not empty. In contains {} rows.", task_partition.name, current_piece_number, count);
|
||||
Coordination::Stat stat_shards{};
|
||||
zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards);
|
||||
|
||||
/// NOTE: partition is still fresh if dirt discovery happens before cleaning
|
||||
if (stat_shards.numChildren == 0)
|
||||
{
|
||||
LOG_WARNING(log, "There are no workers for partition " << task_partition.name
|
||||
<< " piece " << toString(current_piece_number)
|
||||
<< ", but destination table contains " << count << " rows"
|
||||
<< ". Partition will be dropped and refilled.");
|
||||
LOG_WARNING(log, "There are no workers for partition {} piece {}, but destination table contains {} rows. Partition will be dropped and refilled.", task_partition.name, toString(current_piece_number), count);
|
||||
|
||||
create_is_dirty_node(clean_state_clock);
|
||||
return TaskStatus::Error;
|
||||
@ -1353,14 +1323,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
|
||||
if (clean_state_clock != new_clean_state_clock)
|
||||
{
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " clean state changed, cowardly bailing");
|
||||
LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
|
||||
return TaskStatus::Error;
|
||||
}
|
||||
else if (!new_clean_state_clock.is_clean())
|
||||
{
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " is dirty and will be dropped and refilled");
|
||||
LOG_INFO(log, "Partition {} piece {} is dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
|
||||
create_is_dirty_node(new_clean_state_clock);
|
||||
return TaskStatus::Error;
|
||||
}
|
||||
@ -1387,12 +1355,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
|
||||
String query = queryToString(create_query_push_ast);
|
||||
|
||||
LOG_DEBUG(log, "Create destination tables. Query: " << query);
|
||||
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
|
||||
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
|
||||
create_query_push_ast, &task_cluster->settings_push,
|
||||
PoolMode::GET_MANY);
|
||||
LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push)
|
||||
<< " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount());
|
||||
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
|
||||
}
|
||||
|
||||
/// Do the copying
|
||||
@ -1407,8 +1374,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
// Select all fields
|
||||
ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : "");
|
||||
|
||||
LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription()
|
||||
<< " : " << queryToString(query_select_ast));
|
||||
LOG_DEBUG(log, "Executing SELECT query and pull from {} : {}", task_shard.getDescription(), queryToString(query_select_ast));
|
||||
|
||||
ASTPtr query_insert_ast;
|
||||
{
|
||||
@ -1419,7 +1385,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
const auto & settings = context.getSettingsRef();
|
||||
query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
LOG_DEBUG(log, "Executing INSERT query: " << query);
|
||||
LOG_DEBUG(log, "Executing INSERT query: {}", query);
|
||||
}
|
||||
|
||||
try
|
||||
@ -1501,8 +1467,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " copied. But not moved to original destination table.");
|
||||
LOG_INFO(log, "Partition {} piece {} copied. But not moved to original destination table.", task_partition.name, toString(current_piece_number));
|
||||
|
||||
|
||||
/// Try create original table (if not exists) on each shard
|
||||
@ -1513,12 +1478,11 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
|
||||
String query = queryToString(create_query_push_ast);
|
||||
|
||||
LOG_DEBUG(log, "Create destination tables. Query: " << query);
|
||||
LOG_DEBUG(log, "Create destination tables. Query: {}", query);
|
||||
UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
|
||||
create_query_push_ast, &task_cluster->settings_push,
|
||||
PoolMode::GET_MANY);
|
||||
LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push)
|
||||
<< " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount());
|
||||
LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1531,14 +1495,12 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
||||
CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
|
||||
if (clean_state_clock != new_clean_state_clock)
|
||||
{
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " clean state changed, cowardly bailing");
|
||||
LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
|
||||
return TaskStatus::Error;
|
||||
}
|
||||
else if (!new_clean_state_clock.is_clean())
|
||||
{
|
||||
LOG_INFO(log, "Partition " << task_partition.name << " piece "
|
||||
<< toString(current_piece_number) << " became dirty and will be dropped and refilled");
|
||||
LOG_INFO(log, "Partition {} piece {} became dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
|
||||
create_is_dirty_node(new_clean_state_clock);
|
||||
return TaskStatus::Error;
|
||||
}
|
||||
@ -1582,7 +1544,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
|
||||
const ClusterPtr & cluster_push = task_table.cluster_push;
|
||||
Settings settings_push = task_cluster->settings_push;
|
||||
|
||||
LOG_DEBUG(log, "Execute distributed DROP TABLE: " << query);
|
||||
LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query);
|
||||
/// We have to drop partition_piece on each replica
|
||||
UInt64 num_nodes = executeQueryOnCluster(
|
||||
cluster_push, query,
|
||||
@ -1591,7 +1553,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
|
||||
PoolMode::GET_MANY,
|
||||
ClusterExecutionMode::ON_EACH_NODE);
|
||||
|
||||
LOG_DEBUG(log, "DROP TABLE query was successfully executed on " << toString(num_nodes) << " nodes.");
|
||||
LOG_DEBUG(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes));
|
||||
}
|
||||
}
|
||||
|
||||
@ -1609,7 +1571,7 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT
|
||||
const ClusterPtr & cluster_push = task_table.cluster_push;
|
||||
Settings settings_push = task_cluster->settings_push;
|
||||
|
||||
LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query);
|
||||
LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
|
||||
/// We have to drop partition_piece on each replica
|
||||
UInt64 num_nodes = executeQueryOnCluster(
|
||||
cluster_push, query,
|
||||
@ -1618,9 +1580,9 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT
|
||||
PoolMode::GET_MANY,
|
||||
ClusterExecutionMode::ON_EACH_NODE);
|
||||
|
||||
LOG_DEBUG(log, "DROP PARTITION query was successfully executed on " << toString(num_nodes) << " nodes.");
|
||||
LOG_DEBUG(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes));
|
||||
}
|
||||
LOG_DEBUG(log, "All helping tables dropped partition " << partition_name);
|
||||
LOG_DEBUG(log, "All helping tables dropped partition {}", partition_name);
|
||||
}
|
||||
|
||||
String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings)
|
||||
@ -1724,7 +1686,7 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
|
||||
const auto & settings = context.getSettingsRef();
|
||||
ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
|
||||
|
||||
LOG_DEBUG(log, "Computing destination partition set, executing query: " << query);
|
||||
LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query);
|
||||
|
||||
Context local_context = context;
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
@ -1744,7 +1706,7 @@ std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription());
|
||||
LOG_DEBUG(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription());
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -1765,8 +1727,7 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
|
||||
|
||||
query += " LIMIT 1";
|
||||
|
||||
LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition "
|
||||
<< partition_quoted_name << " existence, executing query: " << query);
|
||||
LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query);
|
||||
|
||||
ParserQuery parser_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
@ -1805,9 +1766,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
|
||||
|
||||
query += " LIMIT 1";
|
||||
|
||||
LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition "
|
||||
<< partition_quoted_name << " piece " << std::to_string(current_piece_number)
|
||||
<< "existence, executing query: " << query);
|
||||
LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query);
|
||||
|
||||
ParserQuery parser_query(query.data() + query.size());
|
||||
const auto & settings = context.getSettingsRef();
|
||||
@ -1817,11 +1776,9 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi
|
||||
local_context.setSettings(task_cluster->settings_pull);
|
||||
auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows();
|
||||
if (result != 0)
|
||||
LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number "
|
||||
<< std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription());
|
||||
LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
|
||||
else
|
||||
LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number "
|
||||
<< std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription());
|
||||
LOG_DEBUG(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
|
||||
return result != 0;
|
||||
}
|
||||
|
||||
@ -1938,8 +1895,7 @@ UInt64 ClusterCopier::executeQueryOnCluster(
|
||||
|
||||
if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number)
|
||||
{
|
||||
LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on "
|
||||
<< toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load()));
|
||||
LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load()));
|
||||
}
|
||||
|
||||
|
||||
|
@ -95,11 +95,7 @@ void ClusterCopierApp::mainImpl()
|
||||
ThreadStatus thread_status;
|
||||
|
||||
auto * log = &logger();
|
||||
LOG_INFO(log, "Starting clickhouse-copier ("
|
||||
<< "id " << process_id << ", "
|
||||
<< "host_id " << host_id << ", "
|
||||
<< "path " << process_path << ", "
|
||||
<< "revision " << ClickHouseRevision::get() << ")");
|
||||
LOG_INFO(log, "Starting clickhouse-copier (id {}, host_id {}, path {}, revision {})", process_id, host_id, process_path, ClickHouseRevision::get());
|
||||
|
||||
SharedContextHolder shared_context = Context::createShared();
|
||||
auto context = std::make_unique<Context>(Context::createGlobal(shared_context.get()));
|
||||
|
@ -183,11 +183,11 @@ public:
|
||||
switch (rsp.type)
|
||||
{
|
||||
case Coordination::CREATED:
|
||||
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path);
|
||||
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at {}", rsp.path);
|
||||
stale->store(true);
|
||||
break;
|
||||
case Coordination::CHANGED:
|
||||
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path);
|
||||
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at {}", rsp.path);
|
||||
stale->store(true);
|
||||
}
|
||||
}
|
||||
|
@ -211,7 +211,7 @@ try
|
||||
/// Lock path directory before read
|
||||
status.emplace(context->getPath() + "status");
|
||||
|
||||
LOG_DEBUG(log, "Loading metadata from " << context->getPath());
|
||||
LOG_DEBUG(log, "Loading metadata from {}", context->getPath());
|
||||
loadMetadataSystem(*context);
|
||||
attachSystemTables();
|
||||
loadMetadata(*context);
|
||||
|
@ -62,7 +62,7 @@ namespace
|
||||
void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
Poco::Net::HTMLForm params(request, request.stream());
|
||||
LOG_TRACE(log, "Request URI: " + request.getURI());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
{
|
||||
@ -89,11 +89,11 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
||||
if (params.has("schema"))
|
||||
{
|
||||
schema_name = params.get("schema");
|
||||
LOG_TRACE(log, "Will fetch info for table '" << schema_name + "." + table_name << "'");
|
||||
LOG_TRACE(log, "Will fetch info for table '{}'", schema_name + "." + table_name);
|
||||
}
|
||||
else
|
||||
LOG_TRACE(log, "Will fetch info for table '" << table_name << "'");
|
||||
LOG_TRACE(log, "Got connection str '" << connection_string << "'");
|
||||
LOG_TRACE(log, "Will fetch info for table '{}'", table_name);
|
||||
LOG_TRACE(log, "Got connection str '{}'", connection_string);
|
||||
|
||||
try
|
||||
{
|
||||
@ -124,7 +124,7 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
||||
select->format(settings);
|
||||
std::string query = ss.str();
|
||||
|
||||
LOG_TRACE(log, "Inferring structure with query '" << query << "'");
|
||||
LOG_TRACE(log, "Inferring structure with query '{}'", query);
|
||||
|
||||
if (POCO_SQL_ODBC_CLASS::Utility::isError(POCO_SQL_ODBC_CLASS::SQLPrepare(hstmt, reinterpret_cast<SQLCHAR *>(query.data()), query.size())))
|
||||
throw POCO_SQL_ODBC_CLASS::DescriptorException(session.dbc());
|
||||
|
@ -10,7 +10,7 @@ namespace DB
|
||||
Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
|
||||
{
|
||||
Poco::URI uri{request.getURI()};
|
||||
LOG_TRACE(log, "Request URI: " + uri.toString());
|
||||
LOG_TRACE(log, "Request URI: {}", uri.toString());
|
||||
|
||||
if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
|
||||
return new PingHandler(keep_alive_timeout);
|
||||
|
@ -25,7 +25,7 @@ namespace DB
|
||||
void IdentifierQuoteHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response)
|
||||
{
|
||||
Poco::Net::HTMLForm params(request, request.stream());
|
||||
LOG_TRACE(log, "Request URI: " + request.getURI());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
{
|
||||
|
@ -84,7 +84,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
Poco::Net::HTMLForm params(request);
|
||||
if (mode == "read")
|
||||
params.read(request.stream());
|
||||
LOG_TRACE(log, "Request URI: " + request.getURI());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
if (mode == "read" && !params.has("query"))
|
||||
{
|
||||
@ -132,7 +132,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
std::string format = params.get("format", "RowBinary");
|
||||
|
||||
std::string connection_string = params.get("connection_string");
|
||||
LOG_TRACE(log, "Connection string: '" << connection_string << "'");
|
||||
LOG_TRACE(log, "Connection string: '{}'", connection_string);
|
||||
|
||||
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
|
||||
|
||||
@ -152,7 +152,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
}
|
||||
std::string db_name = params.get("db_name");
|
||||
std::string table_name = params.get("table_name");
|
||||
LOG_TRACE(log, "DB name: '" << db_name << "', table name: '" << table_name << "'");
|
||||
LOG_TRACE(log, "DB name: '{}', table name: '{}'", db_name, table_name);
|
||||
|
||||
auto quoting_style = IdentifierQuotingStyle::None;
|
||||
#if USE_ODBC
|
||||
@ -171,7 +171,7 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
|
||||
else
|
||||
{
|
||||
std::string query = params.get("query");
|
||||
LOG_TRACE(log, "Query: " << query);
|
||||
LOG_TRACE(log, "Query: {}", query);
|
||||
|
||||
BlockOutputStreamPtr writer = FormatFactory::instance().getOutput(format, out, *sample_block, context);
|
||||
auto pool = getPool(connection_string);
|
||||
|
@ -48,12 +48,7 @@ namespace
|
||||
#endif
|
||||
)
|
||||
{
|
||||
LOG_ERROR(log,
|
||||
"Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message()
|
||||
<< ". "
|
||||
"If it is an IPv6 address and your host has disabled IPv6, then consider to "
|
||||
"specify IPv4 address to listen in <listen_host> element of configuration "
|
||||
"file. Example: <listen_host>0.0.0.0</listen_host>");
|
||||
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. If it is an IPv6 address and your host has disabled IPv6, then consider to specify IPv4 address to listen in <listen_host> element of configuration file. Example: <listen_host>0.0.0.0</listen_host>", host, e.code(), e.message());
|
||||
}
|
||||
|
||||
throw;
|
||||
@ -188,7 +183,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
|
||||
new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params);
|
||||
server.start();
|
||||
|
||||
LOG_INFO(log, "Listening http://" + address.toString());
|
||||
LOG_INFO(log, "Listening http://{}", address.toString());
|
||||
|
||||
SCOPE_EXIT({
|
||||
LOG_DEBUG(log, "Received termination signal.");
|
||||
@ -198,7 +193,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
if (server.currentConnections() == 0)
|
||||
break;
|
||||
LOG_DEBUG(log, "Waiting for " << server.currentConnections() << " connections, try " << count);
|
||||
LOG_DEBUG(log, "Waiting for {} connections, try {}", server.currentConnections(), count);
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
|
||||
}
|
||||
});
|
||||
|
@ -241,7 +241,7 @@ void HTTPHandler::processQuery(
|
||||
|
||||
CurrentThread::QueryScope query_scope(context);
|
||||
|
||||
LOG_TRACE(log, "Request URI: " << request.getURI());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
std::istream & istr = request.stream();
|
||||
|
||||
|
@ -30,13 +30,10 @@ HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string &
|
||||
|
||||
Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHandler(const Poco::Net::HTTPServerRequest & request)
|
||||
{
|
||||
LOG_TRACE(log, "HTTP Request for " << name << ". "
|
||||
<< "Method: " << request.getMethod()
|
||||
<< ", Address: " << request.clientAddress().toString()
|
||||
<< ", User-Agent: " << (request.has("User-Agent") ? request.get("User-Agent") : "none")
|
||||
<< (request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : (""))
|
||||
<< ", Content Type: " << request.getContentType()
|
||||
<< ", Transfer Encoding: " << request.getTransferEncoding());
|
||||
LOG_TRACE(log, "HTTP Request for {}. Method: {}, Address: {}, User-Agent: {}{}, Content Type: {}, Transfer Encoding: {}",
|
||||
name, request.getMethod(), request.clientAddress().toString(), request.has("User-Agent") ? request.get("User-Agent") : "none",
|
||||
(request.hasContentLength() ? (", Length: " + std::to_string(request.getContentLength())) : ("")),
|
||||
request.getContentType(), request.getTransferEncoding());
|
||||
|
||||
for (auto & handler_factory : child_factories)
|
||||
{
|
||||
|
@ -53,7 +53,7 @@ void InterserverIOHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & reque
|
||||
{
|
||||
HTMLForm params(request);
|
||||
|
||||
LOG_TRACE(log, "Request URI: " << request.getURI());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
String endpoint_name = params.get("endpoint");
|
||||
bool compress = params.get("compress") == "true";
|
||||
@ -103,7 +103,7 @@ void InterserverIOHTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & requ
|
||||
response.setStatusAndReason(Poco::Net::HTTPServerResponse::HTTP_UNAUTHORIZED);
|
||||
if (!response.sent())
|
||||
writeString(message, *used_output.out);
|
||||
LOG_WARNING(log, "Query processing failed request: '" << request.getURI() << "' authentication failed");
|
||||
LOG_WARNING(log, "Query processing failed request: '{}' authentication failed", request.getURI());
|
||||
}
|
||||
}
|
||||
catch (Exception & e)
|
||||
|
@ -83,19 +83,15 @@ void MySQLHandler::run()
|
||||
if (!connection_context.mysql.max_packet_size)
|
||||
connection_context.mysql.max_packet_size = MAX_PACKET_LENGTH;
|
||||
|
||||
LOG_TRACE(log, "Capabilities: " << handshake_response.capability_flags
|
||||
<< ", max_packet_size: "
|
||||
<< handshake_response.max_packet_size
|
||||
<< ", character_set: "
|
||||
<< static_cast<int>(handshake_response.character_set)
|
||||
<< ", user: "
|
||||
<< handshake_response.username
|
||||
<< ", auth_response length: "
|
||||
<< handshake_response.auth_response.length()
|
||||
<< ", database: "
|
||||
<< handshake_response.database
|
||||
<< ", auth_plugin_name: "
|
||||
<< handshake_response.auth_plugin_name);
|
||||
LOG_TRACE(log,
|
||||
"Capabilities: {}, max_packet_size: {}, character_set: {}, user: {}, auth_response length: {}, database: {}, auth_plugin_name: {}",
|
||||
handshake_response.capability_flags,
|
||||
handshake_response.max_packet_size,
|
||||
static_cast<int>(handshake_response.character_set),
|
||||
handshake_response.username,
|
||||
handshake_response.auth_response.length(),
|
||||
handshake_response.database,
|
||||
handshake_response.auth_plugin_name);
|
||||
|
||||
client_capability_flags = handshake_response.capability_flags;
|
||||
if (!(client_capability_flags & CLIENT_PROTOCOL_41))
|
||||
@ -129,7 +125,9 @@ void MySQLHandler::run()
|
||||
// For commands which are executed without MemoryTracker.
|
||||
LimitReadBuffer limited_payload(payload, 10000, true, "too long MySQL packet.");
|
||||
|
||||
LOG_DEBUG(log, "Received command: " << static_cast<int>(static_cast<unsigned char>(command)) << ". Connection id: " << connection_id << ".");
|
||||
LOG_DEBUG(log, "Received command: {}. Connection id: {}.",
|
||||
static_cast<int>(static_cast<unsigned char>(command)), connection_id);
|
||||
|
||||
try
|
||||
{
|
||||
switch (command)
|
||||
@ -197,7 +195,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet)
|
||||
read_bytes(3); /// We can find out whether it is SSLRequest of HandshakeResponse by first 3 bytes.
|
||||
|
||||
size_t payload_size = unalignedLoad<uint32_t>(buf) & 0xFFFFFFu;
|
||||
LOG_TRACE(log, "payload size: " << payload_size);
|
||||
LOG_TRACE(log, "payload size: {}", payload_size);
|
||||
|
||||
if (payload_size == SSL_REQUEST_PAYLOAD_SIZE)
|
||||
{
|
||||
@ -234,18 +232,18 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl
|
||||
}
|
||||
catch (const Exception & exc)
|
||||
{
|
||||
LOG_ERROR(log, "Authentication for user " << user_name << " failed.");
|
||||
LOG_ERROR(log, "Authentication for user {} failed.", user_name);
|
||||
packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true);
|
||||
throw;
|
||||
}
|
||||
LOG_INFO(log, "Authentication for user " << user_name << " succeeded.");
|
||||
LOG_INFO(log, "Authentication for user {} succeeded.", user_name);
|
||||
}
|
||||
|
||||
void MySQLHandler::comInitDB(ReadBuffer & payload)
|
||||
{
|
||||
String database;
|
||||
readStringUntilEOF(database, payload);
|
||||
LOG_DEBUG(log, "Setting current database to " << database);
|
||||
LOG_DEBUG(log, "Setting current database to {}", database);
|
||||
connection_context.setCurrentDatabase(database);
|
||||
packet_sender->sendPacket(OK_Packet(0, client_capability_flags, 0, 0, 1), true);
|
||||
}
|
||||
|
@ -32,7 +32,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_TRACE(log, "Failed to create SSL context. SSL will be disabled. Error: " << getCurrentExceptionMessage(false));
|
||||
LOG_TRACE(log, "Failed to create SSL context. SSL will be disabled. Error: {}", getCurrentExceptionMessage(false));
|
||||
ssl_enabled = false;
|
||||
}
|
||||
|
||||
@ -43,7 +43,7 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_)
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_TRACE(log, "Failed to read RSA key pair from server certificate. Error: " << getCurrentExceptionMessage(false));
|
||||
LOG_TRACE(log, "Failed to read RSA key pair from server certificate. Error: {}", getCurrentExceptionMessage(false));
|
||||
generateRSAKeys();
|
||||
}
|
||||
#endif
|
||||
@ -122,7 +122,7 @@ void MySQLHandlerFactory::generateRSAKeys()
|
||||
Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket)
|
||||
{
|
||||
size_t connection_id = last_connection_id++;
|
||||
LOG_TRACE(log, "MySQL connection. Id: " << connection_id << ". Address: " << socket.peerAddress().toString());
|
||||
LOG_TRACE(log, "MySQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString());
|
||||
#if USE_SSL
|
||||
return new MySQLHandlerSSL(server, socket, ssl_enabled, connection_id, *public_key, *private_key);
|
||||
#else
|
||||
|
@ -91,7 +91,7 @@ namespace
|
||||
|
||||
void setupTmpPath(Logger * log, const std::string & path)
|
||||
{
|
||||
LOG_DEBUG(log, "Setting up " << path << " to store temporary data in it");
|
||||
LOG_DEBUG(log, "Setting up {} to store temporary data in it", path);
|
||||
|
||||
Poco::File(path).createDirectories();
|
||||
|
||||
@ -101,11 +101,11 @@ void setupTmpPath(Logger * log, const std::string & path)
|
||||
{
|
||||
if (it->isFile() && startsWith(it.name(), "tmp"))
|
||||
{
|
||||
LOG_DEBUG(log, "Removing old temporary file " << it->path());
|
||||
LOG_DEBUG(log, "Removing old temporary file {}", it->path());
|
||||
it->remove();
|
||||
}
|
||||
else
|
||||
LOG_DEBUG(log, "Skipped file in temporary path " << it->path());
|
||||
LOG_DEBUG(log, "Skipped file in temporary path {}", it->path());
|
||||
}
|
||||
}
|
||||
|
||||
@ -276,7 +276,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds.");
|
||||
if (0 != mlockall(MCL_CURRENT))
|
||||
LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR));
|
||||
LOG_WARNING(log, "Failed mlockall: {}", errnoToString(ErrorCodes::SYSTEM_ERROR));
|
||||
else
|
||||
LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed");
|
||||
}
|
||||
@ -284,8 +284,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep " << executable_path << "'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.");
|
||||
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep {}'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.", executable_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -349,7 +349,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
if (rlim.rlim_cur == rlim.rlim_max)
|
||||
{
|
||||
LOG_DEBUG(log, "rlimit on number of file descriptors is " << rlim.rlim_cur);
|
||||
LOG_DEBUG(log, "rlimit on number of file descriptors is {}", rlim.rlim_cur);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -357,12 +357,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
rlim.rlim_cur = config().getUInt("max_open_files", rlim.rlim_max);
|
||||
int rc = setrlimit(RLIMIT_NOFILE, &rlim);
|
||||
if (rc != 0)
|
||||
LOG_WARNING(log,
|
||||
"Cannot set max number of file descriptors to " << rlim.rlim_cur
|
||||
<< ". Try to specify max_open_files according to your system limits. error: "
|
||||
<< strerror(errno));
|
||||
LOG_WARNING(log, "Cannot set max number of file descriptors to {}. Try to specify max_open_files according to your system limits. error: {}", rlim.rlim_cur, strerror(errno));
|
||||
else
|
||||
LOG_DEBUG(log, "Set max number of file descriptors to " << rlim.rlim_cur << " (was " << old << ").");
|
||||
LOG_DEBUG(log, "Set max number of file descriptors to {} (was {}).", rlim.rlim_cur, old);
|
||||
}
|
||||
}
|
||||
|
||||
@ -372,7 +369,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
/// Initialize DateLUT early, to not interfere with running time of first query.
|
||||
LOG_DEBUG(log, "Initializing DateLUT.");
|
||||
DateLUT::instance();
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '" << DateLUT::instance().getTimeZone() << "'.");
|
||||
LOG_TRACE(log, "Initialized DateLUT with time zone '{}'.", DateLUT::instance().getTimeZone());
|
||||
|
||||
|
||||
/// Storage with temporary data for processing of heavy queries.
|
||||
@ -431,9 +428,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
if (this_host.empty())
|
||||
{
|
||||
this_host = getFQDNOrHostName();
|
||||
LOG_DEBUG(log,
|
||||
"Configuration parameter '" + String(host_tag) + "' doesn't exist or exists and empty. Will use '" + this_host
|
||||
+ "' as replica host.");
|
||||
LOG_DEBUG(log, "Configuration parameter '{}' doesn't exist or exists and empty. Will use '{}' as replica host.",
|
||||
host_tag, this_host);
|
||||
}
|
||||
|
||||
String port_str = config().getString(port_tag);
|
||||
@ -538,8 +534,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
if (uncompressed_cache_size > max_cache_size)
|
||||
{
|
||||
uncompressed_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Uncompressed cache size was lowered to " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
|
||||
<< " because the system has low amount of memory");
|
||||
LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setUncompressedCache(uncompressed_cache_size);
|
||||
|
||||
@ -554,8 +549,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
if (mark_cache_size > max_cache_size)
|
||||
{
|
||||
mark_cache_size = max_cache_size;
|
||||
LOG_INFO(log, "Mark cache size was lowered to " << formatReadableSizeWithBinarySuffix(uncompressed_cache_size)
|
||||
<< " because the system has low amount of memory");
|
||||
LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size));
|
||||
}
|
||||
global_context->setMarkCache(mark_cache_size);
|
||||
|
||||
@ -579,20 +573,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
if (max_server_memory_usage == 0)
|
||||
{
|
||||
max_server_memory_usage = default_max_server_memory_usage;
|
||||
LOG_INFO(log, "Setting max_server_memory_usage was set to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage));
|
||||
LOG_INFO(log, "Setting max_server_memory_usage was set to {}", formatReadableSizeWithBinarySuffix(max_server_memory_usage));
|
||||
}
|
||||
else if (max_server_memory_usage > default_max_server_memory_usage)
|
||||
{
|
||||
max_server_memory_usage = default_max_server_memory_usage;
|
||||
LOG_INFO(log, "Setting max_server_memory_usage was lowered to " << formatReadableSizeWithBinarySuffix(max_server_memory_usage)
|
||||
<< " because the system has low amount of memory");
|
||||
LOG_INFO(log, "Setting max_server_memory_usage was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(max_server_memory_usage));
|
||||
}
|
||||
|
||||
total_memory_tracker.setOrRaiseHardLimit(max_server_memory_usage);
|
||||
total_memory_tracker.setDescription("(total)");
|
||||
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
|
||||
|
||||
LOG_INFO(log, "Loading metadata from " + path);
|
||||
LOG_INFO(log, "Loading metadata from {}", path);
|
||||
|
||||
try
|
||||
{
|
||||
@ -694,17 +687,19 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
" neither clickhouse-server process has CAP_NET_ADMIN capability."
|
||||
" 'taskstats' performance statistics will be disabled."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You can try to resolve the problem manually with 'sudo setcap cap_net_admin=+ep " << executable_path << "'."
|
||||
" You can try to resolve the problem manually with 'sudo setcap cap_net_admin=+ep {}'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems."
|
||||
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.");
|
||||
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.",
|
||||
executable_path);
|
||||
}
|
||||
|
||||
if (!hasLinuxCapability(CAP_SYS_NICE))
|
||||
{
|
||||
LOG_INFO(log, "It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_nice' will have no effect."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep " << executable_path << "'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.");
|
||||
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep {}'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.",
|
||||
executable_path);
|
||||
}
|
||||
#else
|
||||
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
|
||||
@ -746,11 +741,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
#endif
|
||||
)
|
||||
{
|
||||
LOG_ERROR(log,
|
||||
"Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message() << ". "
|
||||
LOG_ERROR(log, "Cannot resolve listen_host ({}), error {}: {}. "
|
||||
"If it is an IPv6 address and your host has disabled IPv6, then consider to "
|
||||
"specify IPv4 address to listen in <listen_host> element of configuration "
|
||||
"file. Example: <listen_host>0.0.0.0</listen_host>");
|
||||
"file. Example: <listen_host>0.0.0.0</listen_host>",
|
||||
host, e.code(), e.message());
|
||||
}
|
||||
|
||||
throw;
|
||||
@ -802,11 +797,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
if (listen_try)
|
||||
{
|
||||
LOG_ERROR(log, message
|
||||
<< ". If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
|
||||
LOG_ERROR(log, "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to "
|
||||
"specify not disabled IPv4 or IPv6 address to listen in <listen_host> element of configuration "
|
||||
"file. Example for disabled IPv6: <listen_host>0.0.0.0</listen_host> ."
|
||||
" Example for disabled IPv4: <listen_host>::</listen_host>");
|
||||
" Example for disabled IPv4: <listen_host>::</listen_host>",
|
||||
message);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -826,7 +821,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
|
||||
createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for http://" + address.toString());
|
||||
LOG_INFO(log, "Listening for http://{}", address.toString());
|
||||
});
|
||||
|
||||
/// HTTPS
|
||||
@ -840,7 +835,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
|
||||
createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for https://" + address.toString());
|
||||
LOG_INFO(log, "Listening for https://{}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
|
||||
@ -861,7 +856,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections with native protocol (tcp): " + address.toString());
|
||||
LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString());
|
||||
});
|
||||
|
||||
/// TCP with SSL
|
||||
@ -877,7 +872,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): " + address.toString());
|
||||
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
@ -895,7 +890,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for replica communication (interserver): http://" + address.toString());
|
||||
LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString());
|
||||
});
|
||||
|
||||
create_server("interserver_https_port", [&](UInt16 port)
|
||||
@ -908,7 +903,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for secure replica communication (interserver): https://" + address.toString());
|
||||
LOG_INFO(log, "Listening for secure replica communication (interserver): https://{}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
@ -928,7 +923,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for MySQL compatibility protocol: " + address.toString());
|
||||
LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString());
|
||||
});
|
||||
|
||||
/// Prometheus (if defined and not setup yet with http_port)
|
||||
@ -941,7 +936,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
servers.emplace_back(std::make_unique<Poco::Net::HTTPServer>(
|
||||
createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for Prometheus: http://" + address.toString());
|
||||
LOG_INFO(log, "Listening for Prometheus: http://{}", address.toString());
|
||||
});
|
||||
}
|
||||
|
||||
@ -966,12 +961,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
dns_cache_updater->start();
|
||||
|
||||
{
|
||||
std::stringstream message;
|
||||
message << "Available RAM: " << formatReadableSizeWithBinarySuffix(memory_amount) << ";"
|
||||
<< " physical cores: " << getNumberOfPhysicalCPUCores() << ";"
|
||||
// on ARM processors it can show only enabled at current moment cores
|
||||
<< " logical cores: " << std::thread::hardware_concurrency() << ".";
|
||||
LOG_INFO(log, message.str());
|
||||
LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.",
|
||||
formatReadableSizeWithBinarySuffix(memory_amount),
|
||||
getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
|
||||
std::thread::hardware_concurrency());
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Ready for connections.");
|
||||
@ -989,9 +982,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
current_connections += server->currentConnections();
|
||||
}
|
||||
|
||||
LOG_INFO(log,
|
||||
"Closed all listening sockets."
|
||||
<< (current_connections ? " Waiting for " + toString(current_connections) + " outstanding connections." : ""));
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
|
||||
else
|
||||
LOG_INFO(log, "Closed all listening sockets.");
|
||||
|
||||
/// Killing remaining queries.
|
||||
global_context->getProcessList().killAllQueries();
|
||||
@ -1013,9 +1007,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(
|
||||
log, "Closed connections." << (current_connections ? " But " + toString(current_connections) + " remains."
|
||||
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>" : ""));
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed connections. But {} remain."
|
||||
" Tip: To increase wait time add to config: <shutdown_wait_unfinished>60</shutdown_wait_unfinished>", current_connections);
|
||||
else
|
||||
LOG_INFO(log, "Closed connections.");
|
||||
|
||||
dns_cache_updater.reset();
|
||||
main_config_reloader.reset();
|
||||
|
@ -115,8 +115,7 @@ void TCPHandler::runImpl()
|
||||
if (!DatabaseCatalog::instance().isDatabaseExist(default_database))
|
||||
{
|
||||
Exception e("Database " + backQuote(default_database) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
|
||||
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText()
|
||||
<< ", Stack trace:\n\n" << e.getStackTraceString());
|
||||
LOG_ERROR(log, "Code: {}, e.displayText() = {}, Stack trace:\n\n{}", e.code(), e.displayText(), e.getStackTraceString());
|
||||
sendException(e, connection_context.getSettingsRef().calculate_text_stack_trace);
|
||||
return;
|
||||
}
|
||||
@ -379,8 +378,7 @@ void TCPHandler::runImpl()
|
||||
|
||||
watch.stop();
|
||||
|
||||
LOG_INFO(log, std::fixed << std::setprecision(3)
|
||||
<< "Processed in " << watch.elapsedSeconds() << " sec.");
|
||||
LOG_INFO(log, "Processed in {} sec.", watch.elapsedSeconds());
|
||||
|
||||
/// It is important to destroy query context here. We do not want it to live arbitrarily longer than the query.
|
||||
query_context.reset();
|
||||
@ -732,14 +730,12 @@ void TCPHandler::receiveHello()
|
||||
readStringBinary(user, *in);
|
||||
readStringBinary(password, *in);
|
||||
|
||||
LOG_DEBUG(log, "Connected " << client_name
|
||||
<< " version " << client_version_major
|
||||
<< "." << client_version_minor
|
||||
<< "." << client_version_patch
|
||||
<< ", revision: " << client_revision
|
||||
<< (!default_database.empty() ? ", database: " + default_database : "")
|
||||
<< (!user.empty() ? ", user: " + user : "")
|
||||
<< ".");
|
||||
LOG_DEBUG(log, "Connected {} version {}.{}.{}, revision: {}{}{}.",
|
||||
client_name,
|
||||
client_version_major, client_version_minor, client_version_patch,
|
||||
client_revision,
|
||||
(!default_database.empty() ? ", database: " + default_database : ""),
|
||||
(!user.empty() ? ", user: " + user : ""));
|
||||
|
||||
connection_context.setUser(user, password, socket().peerAddress());
|
||||
}
|
||||
@ -1205,8 +1201,7 @@ void TCPHandler::run()
|
||||
/// Timeout - not an error.
|
||||
if (!strcmp(e.what(), "Timeout"))
|
||||
{
|
||||
LOG_DEBUG(log, "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code()
|
||||
<< ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what());
|
||||
LOG_DEBUG(log, "Poco::Exception. Code: {}, e.code() = {}, e.displayText() = {}, e.what() = {}", ErrorCodes::POCO_EXCEPTION, e.code(), e.displayText(), e.what());
|
||||
}
|
||||
else
|
||||
throw;
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_TRACE(log, "TCP Request. Address: " << socket.peerAddress().toString());
|
||||
LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString());
|
||||
return new TCPHandler(server, socket);
|
||||
}
|
||||
catch (const Poco::Net::NetException &)
|
||||
|
@ -251,12 +251,11 @@ public:
|
||||
|
||||
void logTree(Poco::Logger * log) const
|
||||
{
|
||||
LOG_TRACE(log, "Tree(" << level << "): name=" << (node_name ? *node_name : "NULL")
|
||||
<< ", access=" << access.toString()
|
||||
<< ", final_access=" << final_access.toString()
|
||||
<< ", min_access=" << min_access.toString()
|
||||
<< ", max_access=" << max_access.toString()
|
||||
<< ", num_children=" << (children ? children->size() : 0));
|
||||
LOG_TRACE(log, "Tree({}): name={}, access={}, final_access={}, min_access={}, max_access={}, num_children={}",
|
||||
level, node_name ? *node_name : "NULL", access.toString(),
|
||||
final_access.toString(), min_access.toString(), max_access.toString(),
|
||||
(children ? children->size() : 0));
|
||||
|
||||
if (children)
|
||||
{
|
||||
for (auto & child : *children | boost::adaptors::map_values)
|
||||
|
@ -310,8 +310,8 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
|
||||
LOG_WARNING(
|
||||
&Logger::get("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address " << client_address.toString() << ". " << e.displayText()
|
||||
<< ", code = " << e.code());
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
}
|
||||
};
|
||||
@ -343,8 +343,8 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
|
||||
LOG_WARNING(
|
||||
&Logger::get("AddressPatterns"),
|
||||
"Failed to check if the allowed client hosts contain address " << client_address.toString() << ". " << e.displayText()
|
||||
<< ", code = " << e.code());
|
||||
"Failed to check if the allowed client hosts contain address {}. {}, code = {}",
|
||||
client_address.toString(), e.displayText(), e.code());
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
@ -200,7 +200,7 @@ bool ContextAccess::calculateResultAccessAndCheck(Poco::Logger * log_, const Acc
|
||||
bool is_granted = access->isGranted(flags, args...);
|
||||
|
||||
if (trace_log)
|
||||
LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{flags, args...}.toString()));
|
||||
LOG_TRACE(trace_log, "Access {}: {}", (is_granted ? "granted" : "denied"), (AccessRightsElement{flags, args...}.toString()));
|
||||
|
||||
if (is_granted)
|
||||
return true;
|
||||
@ -219,7 +219,7 @@ bool ContextAccess::calculateResultAccessAndCheck(Poco::Logger * log_, const Acc
|
||||
if constexpr (mode == THROW_IF_ACCESS_DENIED)
|
||||
throw Exception(user_name + ": " + msg, error_code);
|
||||
else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED)
|
||||
LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...));
|
||||
LOG_WARNING(log_, "{}: {}{}", user_name, msg, formatSkippedMessage(args...));
|
||||
};
|
||||
|
||||
if (!user)
|
||||
@ -451,15 +451,18 @@ boost::shared_ptr<const AccessRights> ContextAccess::calculateResultAccess(bool
|
||||
|
||||
if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_))
|
||||
{
|
||||
LOG_TRACE(trace_log, "List of all grants: " << merged_access->toString() << (grant_option ? " WITH GRANT OPTION" : ""));
|
||||
if (grant_option)
|
||||
LOG_TRACE(trace_log, "List of all grants: {} WITH GRANT OPTION", merged_access->toString());
|
||||
else
|
||||
LOG_TRACE(trace_log, "List of all grants: {}", merged_access->toString());
|
||||
|
||||
if (roles_info && !roles_info->getCurrentRolesNames().empty())
|
||||
{
|
||||
LOG_TRACE(
|
||||
trace_log,
|
||||
"Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ")
|
||||
<< ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
|
||||
LOG_TRACE(trace_log, "Current_roles: {}, enabled_roles: {}",
|
||||
boost::algorithm::join(roles_info->getCurrentRolesNames(), ", "),
|
||||
boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
|
||||
}
|
||||
LOG_TRACE(trace_log, "Settings: readonly=" << readonly_ << ", allow_ddl=" << allow_ddl_ << ", allow_introspection_functions=" << allow_introspection_);
|
||||
LOG_TRACE(trace_log, "Settings: readonly={}, allow_ddl={}, allow_introspection_functions={}", readonly_, allow_ddl_, allow_introspection_);
|
||||
}
|
||||
|
||||
res = std::move(merged_access);
|
||||
|
@ -367,7 +367,7 @@ bool DiskAccessStorage::readLists()
|
||||
auto file_path = getListFilePath(directory_path, type);
|
||||
if (!std::filesystem::exists(file_path))
|
||||
{
|
||||
LOG_WARNING(getLogger(), "File " + file_path.string() + " doesn't exist");
|
||||
LOG_WARNING(getLogger(), "File {} doesn't exist", file_path.string());
|
||||
ok = false;
|
||||
break;
|
||||
}
|
||||
@ -496,7 +496,7 @@ void DiskAccessStorage::listsWritingThreadFunc()
|
||||
/// and then saves the files "users.list", "roles.list", etc. to the same directory.
|
||||
bool DiskAccessStorage::rebuildLists()
|
||||
{
|
||||
LOG_WARNING(getLogger(), "Recovering lists in directory " + directory_path);
|
||||
LOG_WARNING(getLogger(), "Recovering lists in directory {}", directory_path);
|
||||
clear();
|
||||
|
||||
for (const auto & directory_entry : std::filesystem::directory_iterator(directory_path))
|
||||
|
@ -61,11 +61,11 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
|
||||
if (connected)
|
||||
disconnect();
|
||||
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: "
|
||||
<< (default_database.empty() ? "(not specified)" : default_database)
|
||||
<< ". User: " << user
|
||||
<< (static_cast<bool>(secure) ? ". Secure" : "")
|
||||
<< (static_cast<bool>(compression) ? "" : ". Uncompressed"));
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}",
|
||||
default_database.empty() ? "(not specified)" : default_database,
|
||||
user,
|
||||
static_cast<bool>(secure) ? ". Secure" : "",
|
||||
static_cast<bool>(compression) ? "" : ". Uncompressed");
|
||||
|
||||
if (static_cast<bool>(secure))
|
||||
{
|
||||
@ -107,11 +107,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
|
||||
sendHello();
|
||||
receiveHello();
|
||||
|
||||
LOG_TRACE(log_wrapper.get(), "Connected to " << server_name
|
||||
<< " server version " << server_version_major
|
||||
<< "." << server_version_minor
|
||||
<< "." << server_version_patch
|
||||
<< ".");
|
||||
LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.",
|
||||
server_name, server_version_major, server_version_minor, server_version_patch);
|
||||
}
|
||||
catch (Poco::Net::NetException & e)
|
||||
{
|
||||
@ -132,8 +129,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
|
||||
|
||||
void Connection::disconnect()
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Disconnecting");
|
||||
|
||||
in = nullptr;
|
||||
last_input_packet_type.reset();
|
||||
out = nullptr; // can write to socket
|
||||
@ -186,8 +181,6 @@ void Connection::sendHello()
|
||||
|
||||
void Connection::receiveHello()
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving hello");
|
||||
|
||||
/// Receive hello packet.
|
||||
UInt64 packet_type = 0;
|
||||
|
||||
@ -391,8 +384,6 @@ void Connection::sendQuery(
|
||||
|
||||
query_id = query_id_;
|
||||
|
||||
//LOG_TRACE(log_wrapper.get(), "Sending query");
|
||||
|
||||
writeVarUInt(Protocol::Client::Query, *out);
|
||||
writeStringBinary(query_id, *out);
|
||||
|
||||
@ -441,8 +432,6 @@ void Connection::sendCancel()
|
||||
if (!out)
|
||||
return;
|
||||
|
||||
//LOG_TRACE(log_wrapper.get(), "Sending cancel");
|
||||
|
||||
writeVarUInt(Protocol::Client::Cancel, *out);
|
||||
out->next();
|
||||
}
|
||||
@ -450,8 +439,6 @@ void Connection::sendCancel()
|
||||
|
||||
void Connection::sendData(const Block & block, const String & name, bool scalar)
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Sending data");
|
||||
|
||||
if (!block_out)
|
||||
{
|
||||
if (compression == Protocol::Compression::Enable)
|
||||
@ -516,19 +503,23 @@ void Connection::sendScalarsData(Scalars & data)
|
||||
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
|
||||
double elapsed = watch.elapsedSeconds();
|
||||
|
||||
std::stringstream msg;
|
||||
msg << std::fixed << std::setprecision(3);
|
||||
msg << "Sent data for " << data.size() << " scalars, total " << rows << " rows in " << elapsed << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
|
||||
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
|
||||
if (compression == Protocol::Compression::Enable)
|
||||
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
|
||||
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
LOG_DEBUG(log_wrapper.get(),
|
||||
"Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)",
|
||||
data.size(), rows, elapsed,
|
||||
static_cast<size_t>(rows / watch.elapsedSeconds()),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()),
|
||||
static_cast<double>(maybe_compressed_out_bytes) / out_bytes,
|
||||
formatReadableSizeWithBinarySuffix(out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds()));
|
||||
else
|
||||
msg << ", no compression.";
|
||||
|
||||
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
|
||||
LOG_DEBUG(log_wrapper.get(),
|
||||
"Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.",
|
||||
data.size(), rows, elapsed,
|
||||
static_cast<size_t>(rows / watch.elapsedSeconds()),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()));
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -616,19 +607,23 @@ void Connection::sendExternalTablesData(ExternalTablesData & data)
|
||||
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
|
||||
double elapsed = watch.elapsedSeconds();
|
||||
|
||||
std::stringstream msg;
|
||||
msg << std::fixed << std::setprecision(3);
|
||||
msg << "Sent data for " << data.size() << " external tables, total " << rows << " rows in " << elapsed << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
|
||||
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
|
||||
if (compression == Protocol::Compression::Enable)
|
||||
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
|
||||
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
LOG_DEBUG(log_wrapper.get(),
|
||||
"Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)",
|
||||
data.size(), rows, elapsed,
|
||||
static_cast<size_t>(rows / watch.elapsedSeconds()),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()),
|
||||
static_cast<double>(maybe_compressed_out_bytes) / out_bytes,
|
||||
formatReadableSizeWithBinarySuffix(out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds()));
|
||||
else
|
||||
msg << ", no compression.";
|
||||
|
||||
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
|
||||
LOG_DEBUG(log_wrapper.get(),
|
||||
"Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.",
|
||||
data.size(), rows, elapsed,
|
||||
static_cast<size_t>(rows / watch.elapsedSeconds()),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes),
|
||||
formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()));
|
||||
}
|
||||
|
||||
std::optional<Poco::Net::SocketAddress> Connection::getResolvedAddress() const
|
||||
@ -682,12 +677,9 @@ Packet Connection::receivePacket()
|
||||
}
|
||||
else
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving packet type");
|
||||
readVarUInt(res.type, *in);
|
||||
}
|
||||
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type));
|
||||
//std::cerr << "Client got packet: " << Protocol::Server::toString(res.type) << "\n";
|
||||
switch (res.type)
|
||||
{
|
||||
case Protocol::Server::Data: [[fallthrough]];
|
||||
@ -740,8 +732,6 @@ Packet Connection::receivePacket()
|
||||
|
||||
Block Connection::receiveData()
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving data");
|
||||
|
||||
initBlockInput();
|
||||
return receiveDataImpl(block_in);
|
||||
}
|
||||
@ -820,8 +810,6 @@ void Connection::setDescription()
|
||||
|
||||
std::unique_ptr<Exception> Connection::receiveException()
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving exception");
|
||||
|
||||
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription()));
|
||||
}
|
||||
|
||||
@ -838,8 +826,6 @@ std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
|
||||
|
||||
Progress Connection::receiveProgress()
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Receiving progress");
|
||||
|
||||
Progress progress;
|
||||
progress.read(*in, server_revision);
|
||||
return progress;
|
||||
|
@ -222,8 +222,8 @@ ConnectionPoolWithFailover::tryGetEntry(
|
||||
auto table_status_it = status_response.table_states_by_id.find(*table_to_check);
|
||||
if (table_status_it == status_response.table_states_by_id.end())
|
||||
{
|
||||
fail_message = "There is no table " + table_to_check->database + "." + table_to_check->table
|
||||
+ " on server: " + result.entry->getDescription();
|
||||
const char * message_pattern = "There is no table {}.{} on server: {}";
|
||||
fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription());
|
||||
LOG_WARNING(log, fail_message);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
|
||||
@ -248,10 +248,7 @@ ConnectionPoolWithFailover::tryGetEntry(
|
||||
result.is_up_to_date = false;
|
||||
result.staleness = delay;
|
||||
|
||||
LOG_TRACE(
|
||||
log, "Server " << result.entry->getDescription() << " has unacceptable replica delay "
|
||||
<< "for table " << table_to_check->database << "." << table_to_check->table
|
||||
<< ": " << delay);
|
||||
LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
}
|
||||
}
|
||||
|
@ -35,7 +35,7 @@ TimeoutSetter::~TimeoutSetter()
|
||||
catch (std::exception & e)
|
||||
{
|
||||
// Sometimes catched on macos
|
||||
LOG_ERROR(&Logger::get("Client"), std::string{"TimeoutSetter: Can't reset timeouts: "} + e.what());
|
||||
LOG_ERROR(&Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <cstdlib>
|
||||
#include <cstring>
|
||||
#include <algorithm>
|
||||
#include <sstream>
|
||||
#include <iostream>
|
||||
#include <functional>
|
||||
#include <Poco/DOM/Text.h>
|
||||
@ -303,7 +304,7 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
else if (throw_on_bad_incl)
|
||||
throw Poco::Exception(error_msg + name);
|
||||
else
|
||||
LOG_WARNING(log, error_msg << name);
|
||||
LOG_WARNING(log, "{}{}", error_msg, name);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -440,7 +441,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
zkutil::ZooKeeperNodeCache * zk_node_cache,
|
||||
const zkutil::EventPtr & zk_changed_event)
|
||||
{
|
||||
LOG_DEBUG(log, "Processing configuration file '" + path + "'.");
|
||||
LOG_DEBUG(log, "Processing configuration file '{}'.", path);
|
||||
|
||||
XMLDocumentPtr config = dom_parser.parse(path);
|
||||
|
||||
@ -451,7 +452,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Merging configuration file '" + merge_file + "'.");
|
||||
LOG_DEBUG(log, "Merging configuration file '{}'.", merge_file);
|
||||
|
||||
XMLDocumentPtr with = dom_parser.parse(merge_file);
|
||||
merge(config, with);
|
||||
@ -488,7 +489,7 @@ XMLDocumentPtr ConfigProcessor::processConfig(
|
||||
}
|
||||
if (!include_from_path.empty())
|
||||
{
|
||||
LOG_DEBUG(log, "Including configuration file '" + include_from_path + "'.");
|
||||
LOG_DEBUG(log, "Including configuration file '{}'.", include_from_path);
|
||||
|
||||
contributing_files.push_back(include_from_path);
|
||||
include_from = dom_parser.parse(include_from_path);
|
||||
@ -568,10 +569,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
|
||||
if (!zk_exception)
|
||||
throw;
|
||||
|
||||
LOG_WARNING(
|
||||
log,
|
||||
"Error while processing from_zk config includes: " + zk_exception->message() +
|
||||
". Config will be loaded from preprocessed file: " + preprocessed_path);
|
||||
LOG_WARNING(log, "Error while processing from_zk config includes: {}. Config will be loaded from preprocessed file: {}", zk_exception->message(), preprocessed_path);
|
||||
|
||||
config_xml = dom_parser.parse(preprocessed_path);
|
||||
}
|
||||
@ -619,11 +617,11 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
|
||||
Poco::File(preprocessed_path_parent).createDirectories();
|
||||
}
|
||||
DOMWriter().writeNode(preprocessed_path, loaded_config.preprocessed_xml);
|
||||
LOG_DEBUG(log, "Saved preprocessed configuration to '" << preprocessed_path << "'.");
|
||||
LOG_DEBUG(log, "Saved preprocessed configuration to '{}'.", preprocessed_path);
|
||||
}
|
||||
catch (Poco::Exception & e)
|
||||
{
|
||||
LOG_WARNING(log, "Couldn't save preprocessed config to " << preprocessed_path << ": " << e.displayText());
|
||||
LOG_WARNING(log, "Couldn't save preprocessed config to {}: {}", preprocessed_path, e.displayText());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -87,7 +87,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
|
||||
ConfigProcessor::LoadedConfig loaded_config;
|
||||
try
|
||||
{
|
||||
LOG_DEBUG(log, "Loading config '" << path << "'");
|
||||
LOG_DEBUG(log, "Loading config '{}'", path);
|
||||
|
||||
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
|
||||
if (loaded_config.has_zk_includes)
|
||||
|
@ -202,7 +202,7 @@ bool DNSResolver::updateCache()
|
||||
}
|
||||
|
||||
if (!lost_hosts.empty())
|
||||
LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: " << lost_hosts);
|
||||
LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts);
|
||||
|
||||
return updated;
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ Exception::Exception(const std::string & msg, int code)
|
||||
#ifndef NDEBUG
|
||||
if (code == ErrorCodes::LOGICAL_ERROR)
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::root(), "Logical error: '" + msg + "'.");
|
||||
LOG_ERROR(&Poco::Logger::root(), "Logical error: '{}'.", msg);
|
||||
assert(false);
|
||||
}
|
||||
#endif
|
||||
@ -125,7 +125,10 @@ void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_ERROR(logger, start_of_message << (start_of_message.empty() ? "" : ": ") << getCurrentExceptionMessage(true));
|
||||
if (start_of_message.empty())
|
||||
LOG_ERROR(logger, "{}", getCurrentExceptionMessage(true));
|
||||
else
|
||||
LOG_ERROR(logger, "{}: {}", start_of_message, getCurrentExceptionMessage(true));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -49,16 +49,12 @@ MemoryTracker::~MemoryTracker()
|
||||
|
||||
void MemoryTracker::logPeakMemoryUsage() const
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"),
|
||||
"Peak memory usage" << (description ? " " + std::string(description) : "")
|
||||
<< ": " << formatReadableSizeWithBinarySuffix(peak) << ".");
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(peak));
|
||||
}
|
||||
|
||||
void MemoryTracker::logMemoryUsage(Int64 current) const
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"),
|
||||
"Current memory usage" << (description ? " " + std::string(description) : "")
|
||||
<< ": " << formatReadableSizeWithBinarySuffix(current) << ".");
|
||||
LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(current));
|
||||
}
|
||||
|
||||
|
||||
|
@ -90,7 +90,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size)
|
||||
{
|
||||
if (errno == EINVAL)
|
||||
{
|
||||
LOG_INFO(log, "Cannot get pipe capacity, " << errnoToString(ErrorCodes::CANNOT_FCNTL) << ". Very old Linux kernels have no support for this fcntl.");
|
||||
LOG_INFO(log, "Cannot get pipe capacity, {}. Very old Linux kernels have no support for this fcntl.", errnoToString(ErrorCodes::CANNOT_FCNTL));
|
||||
/// It will work nevertheless.
|
||||
}
|
||||
else
|
||||
@ -102,7 +102,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size)
|
||||
if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
|
||||
throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
|
||||
|
||||
LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size)));
|
||||
LOG_TRACE(log, "Pipe capacity is {}", formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size)));
|
||||
}
|
||||
#else
|
||||
(void)desired_size;
|
||||
|
@ -239,8 +239,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "Connection failed at try №"
|
||||
<< (shuffled_pool.error_count + 1) << ", reason: " << fail_message);
|
||||
LOG_WARNING(log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message);
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
|
||||
|
||||
shuffled_pool.error_count = std::min(max_error_cap, shuffled_pool.error_count + 1);
|
||||
|
@ -165,10 +165,10 @@ void QueryProfilerBase<ProfilerImpl>::tryCleanup()
|
||||
{
|
||||
#if USE_UNWIND
|
||||
if (timer_id != nullptr && timer_delete(timer_id))
|
||||
LOG_ERROR(log, "Failed to delete query profiler timer " + errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
|
||||
LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
|
||||
|
||||
if (previous_handler != nullptr && sigaction(pause_signal, previous_handler, nullptr))
|
||||
LOG_ERROR(log, "Failed to restore signal handler after query profiler " + errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
|
||||
LOG_ERROR(log, "Failed to restore signal handler after query profiler {}", errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
|
||||
#endif
|
||||
}
|
||||
|
||||
|
@ -144,15 +144,13 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(logger, "Unused param " << config_prefix << '.' << rule);
|
||||
LOG_WARNING(logger, "Unused param {}.{}", config_prefix, rule);
|
||||
}
|
||||
}
|
||||
|
||||
auto rules_count = rulesCount();
|
||||
if (rules_count > 0)
|
||||
{
|
||||
LOG_INFO(logger, rules_count << " query masking rules loaded.");
|
||||
}
|
||||
LOG_INFO(logger, "{} query masking rules loaded.", rules_count);
|
||||
}
|
||||
|
||||
void SensitiveDataMasker::addMaskingRule(
|
||||
|
@ -50,10 +50,10 @@ ShellCommand::~ShellCommand()
|
||||
{
|
||||
if (terminate_in_destructor)
|
||||
{
|
||||
LOG_TRACE(getLogger(), "Will kill shell command pid " << pid << " with SIGTERM");
|
||||
LOG_TRACE(getLogger(), "Will kill shell command pid {} with SIGTERM", pid);
|
||||
int retcode = kill(pid, SIGTERM);
|
||||
if (retcode != 0)
|
||||
LOG_WARNING(getLogger(), "Cannot kill shell command pid " << pid << " errno '" << errnoToString(retcode) << "'");
|
||||
LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString(retcode));
|
||||
}
|
||||
else if (!wait_called)
|
||||
tryWait();
|
||||
@ -61,19 +61,16 @@ ShellCommand::~ShellCommand()
|
||||
|
||||
void ShellCommand::logCommand(const char * filename, char * const argv[])
|
||||
{
|
||||
std::stringstream log_message;
|
||||
log_message << "Will start shell command '" << filename << "' with arguments ";
|
||||
std::stringstream args;
|
||||
for (int i = 0; argv != nullptr && argv[i] != nullptr; ++i)
|
||||
{
|
||||
if (i > 0)
|
||||
{
|
||||
log_message << ", ";
|
||||
}
|
||||
args << ", ";
|
||||
|
||||
/// NOTE: No escaping is performed.
|
||||
log_message << "'" << argv[i] << "'";
|
||||
args << "'" << argv[i] << "'";
|
||||
}
|
||||
LOG_TRACE(ShellCommand::getLogger(), log_message.str());
|
||||
LOG_TRACE(ShellCommand::getLogger(), "Will start shell command '{}' with arguments {}", filename, args.str());
|
||||
}
|
||||
|
||||
std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor)
|
||||
@ -134,7 +131,7 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
|
||||
|
||||
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor));
|
||||
|
||||
LOG_TRACE(getLogger(), "Started shell command '" << filename << "' with pid " << pid);
|
||||
LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid);
|
||||
|
||||
/// Now the ownership of the file descriptors is passed to the result.
|
||||
pipe_stdin.fds_rw[1] = -1;
|
||||
@ -188,13 +185,13 @@ int ShellCommand::tryWait()
|
||||
{
|
||||
wait_called = true;
|
||||
|
||||
LOG_TRACE(getLogger(), "Will wait for shell command pid " << pid);
|
||||
LOG_TRACE(getLogger(), "Will wait for shell command pid {}", pid);
|
||||
|
||||
int status = 0;
|
||||
if (-1 == waitpid(pid, &status, 0))
|
||||
throwFromErrno("Cannot waitpid", ErrorCodes::CANNOT_WAITPID);
|
||||
|
||||
LOG_TRACE(getLogger(), "Wait for shell command pid " << pid << " completed with status " << status);
|
||||
LOG_TRACE(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);
|
||||
|
||||
if (WIFEXITED(status))
|
||||
return WEXITSTATUS(status);
|
||||
|
@ -43,9 +43,9 @@ StatusFile::StatusFile(const std::string & path_)
|
||||
}
|
||||
|
||||
if (!contents.empty())
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file " << path << " already exists - unclean restart. Contents:\n" << contents);
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents);
|
||||
else
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file " << path << " already exists and is empty - probably unclean hardware restart.");
|
||||
LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path);
|
||||
}
|
||||
|
||||
fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666);
|
||||
@ -90,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_)
|
||||
StatusFile::~StatusFile()
|
||||
{
|
||||
if (0 != close(fd))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
|
||||
if (0 != unlink(path.c_str()))
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -148,7 +148,7 @@ public:
|
||||
{
|
||||
if (!checkBridgeIsRunning())
|
||||
{
|
||||
LOG_TRACE(log, BridgeHelperMixin::serviceAlias() + " is not running, will try to start it");
|
||||
LOG_TRACE(log, "{} is not running, will try to start it", BridgeHelperMixin::serviceAlias());
|
||||
startBridge();
|
||||
bool started = false;
|
||||
|
||||
@ -157,7 +157,7 @@ public:
|
||||
while (milliseconds_to_wait < 10000)
|
||||
{
|
||||
++counter;
|
||||
LOG_TRACE(log, "Checking " + BridgeHelperMixin::serviceAlias() + " is running, try " << counter);
|
||||
LOG_TRACE(log, "Checking {} is running, try {}", BridgeHelperMixin::serviceAlias(), counter);
|
||||
if (checkBridgeIsRunning())
|
||||
{
|
||||
started = true;
|
||||
@ -309,7 +309,7 @@ struct ODBCBridgeMixin
|
||||
cmd_args.push_back(config.getString("logger." + configPrefix() + "_level"));
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Starting " + serviceAlias());
|
||||
LOG_TRACE(log, "Starting {}", serviceAlias());
|
||||
|
||||
return ShellCommand::executeDirect(path.toString(), cmd_args, true);
|
||||
}
|
||||
|
@ -64,7 +64,7 @@ Lock::Status Lock::tryCheck() const
|
||||
}
|
||||
|
||||
if (locked && lock_status != LOCKED_BY_ME)
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: " << lock_path << "/" << lock_message);
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message);
|
||||
|
||||
return lock_status;
|
||||
}
|
||||
|
@ -79,7 +79,7 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
|
||||
}
|
||||
catch (const Poco::Net::DNSException & e)
|
||||
{
|
||||
LOG_ERROR(log, "Cannot use ZooKeeper host " << host_string << ", reason: " << e.displayText());
|
||||
LOG_ERROR(log, "Cannot use ZooKeeper host {}, reason: {}", host_string, e.displayText());
|
||||
}
|
||||
}
|
||||
|
||||
@ -95,7 +95,10 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho
|
||||
Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000),
|
||||
Poco::Timespan(0, operation_timeout_ms_ * 1000));
|
||||
|
||||
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
|
||||
if (chroot.empty())
|
||||
LOG_TRACE(log, "Initialized, hosts: {}", hosts);
|
||||
else
|
||||
LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", hosts, chroot);
|
||||
}
|
||||
else if (implementation == "testkeeper")
|
||||
{
|
||||
|
@ -95,7 +95,7 @@ int main(int, char **)
|
||||
mut->set(3);
|
||||
|
||||
std::cerr << "refcounts: " << x->use_count() << ", " << mut->use_count() << "\n";
|
||||
std::cerr << "addresses: " << x.get() << ", " << ", " << mut.get() << "\n";
|
||||
std::cerr << "addresses: " << x.get() << ", " << mut.get() << "\n";
|
||||
y = std::move(mut);
|
||||
}
|
||||
|
||||
|
@ -62,6 +62,6 @@ TEST(ShellCommand, AutoWait)
|
||||
//command->wait(); // now automatic
|
||||
}
|
||||
|
||||
// std::cerr << "inspect me: ps auxwwf" << "\n";
|
||||
// std::cerr << "inspect me: ps auxwwf\n";
|
||||
// std::this_thread::sleep_for(std::chrono::seconds(100));
|
||||
}
|
||||
|
@ -220,7 +220,7 @@ template <typename T, typename ContainerLeft, typename ContainerRight>
|
||||
|
||||
if (l_size != r_size)
|
||||
{
|
||||
result = ::testing::AssertionFailure() << "size mismatch" << " expected: " << l_size << " got:" << r_size;
|
||||
result = ::testing::AssertionFailure() << "size mismatch expected: " << l_size << " got:" << r_size;
|
||||
}
|
||||
if (l_size == 0 || r_size == 0)
|
||||
{
|
||||
|
@ -111,7 +111,7 @@ void BackgroundSchedulePoolTaskInfo::execute()
|
||||
static const int32_t slow_execution_threshold_ms = 200;
|
||||
|
||||
if (milliseconds >= slow_execution_threshold_ms)
|
||||
LOG_TRACE(&Logger::get(log_name), "Execution took " << milliseconds << " ms.");
|
||||
LOG_TRACE(&Logger::get(log_name), "Execution took {} ms.", milliseconds);
|
||||
|
||||
{
|
||||
std::lock_guard lock_schedule(schedule_mutex);
|
||||
@ -156,7 +156,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met
|
||||
, memory_metric(memory_metric_)
|
||||
, thread_name(thread_name_)
|
||||
{
|
||||
LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with " << size << " threads");
|
||||
LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size);
|
||||
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
|
@ -1068,7 +1068,7 @@ public:
|
||||
# pragma GCC diagnostic pop
|
||||
String pem(pem_buf, pem_size);
|
||||
|
||||
LOG_TRACE(log, "Key: " << pem);
|
||||
LOG_TRACE(log, "Key: {}", pem);
|
||||
|
||||
AuthMoreData data(pem);
|
||||
packet_sender->sendPacket(data, true);
|
||||
|
@ -599,7 +599,7 @@ namespace details
|
||||
void SettingsCollectionUtils::warningNameNotFound(const StringRef & name)
|
||||
{
|
||||
static auto * log = &Logger::get("Settings");
|
||||
LOG_WARNING(log, "Unknown setting " << name << ", skipping");
|
||||
LOG_WARNING(log, "Unknown setting {}, skipping", name);
|
||||
}
|
||||
|
||||
void SettingsCollectionUtils::throwNameNotFound(const StringRef & name)
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
#include <DataStreams/BlocksListBlockInputStream.h>
|
||||
#include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
|
||||
@ -60,9 +60,7 @@ Block AggregatingBlockInputStream::readImpl()
|
||||
input_streams.emplace_back(temporary_inputs.back()->block_in);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Will merge " << files.files.size() << " temporary files of size "
|
||||
<< (files.sum_size_compressed / 1048576.0) << " MiB compressed, "
|
||||
<< (files.sum_size_uncompressed / 1048576.0) << " MiB uncompressed.");
|
||||
LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed));
|
||||
|
||||
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(input_streams, params, final, 1, 1);
|
||||
}
|
||||
|
@ -18,14 +18,12 @@ CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream()
|
||||
void CollapsingFinalBlockInputStream::reportBadCounts()
|
||||
{
|
||||
/// With inconsistent data, this is an unavoidable error that can not be easily fixed by admins. Therefore Warning.
|
||||
LOG_WARNING(log, "Incorrect data: number of rows with sign = 1 (" << count_positive
|
||||
<< ") differs with number of rows with sign = -1 (" << count_negative
|
||||
<< ") by more than one");
|
||||
LOG_WARNING(log, "Incorrect data: number of rows with sign = 1 ({}) differs with number of rows with sign = -1 ({}) by more than one", count_positive, count_negative);
|
||||
}
|
||||
|
||||
void CollapsingFinalBlockInputStream::reportBadSign(Int8 sign)
|
||||
{
|
||||
LOG_ERROR(log, "Invalid sign: " << static_cast<int>(sign));
|
||||
LOG_ERROR(log, "Invalid sign: {}", static_cast<int>(sign));
|
||||
}
|
||||
|
||||
void CollapsingFinalBlockInputStream::fetchNextBlock(size_t input_index)
|
||||
@ -135,7 +133,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
|
||||
if (output_blocks.empty())
|
||||
{
|
||||
if (blocks_fetched != blocks_output)
|
||||
LOG_ERROR(log, "Logical error: CollapsingFinalBlockInputStream has output " << blocks_output << " blocks instead of " << blocks_fetched);
|
||||
LOG_ERROR(log, "Logical error: CollapsingFinalBlockInputStream has output {} blocks instead of {}", blocks_output, blocks_fetched);
|
||||
|
||||
return Block();
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <iomanip>
|
||||
|
||||
@ -98,17 +99,13 @@ void ColumnGathererStream::readSuffixImpl()
|
||||
|
||||
double seconds = profile_info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
std::stringstream message;
|
||||
message << std::fixed << std::setprecision(2)
|
||||
<< "Gathered column " << column_name
|
||||
<< " (" << static_cast<double>(profile_info.bytes) / profile_info.rows << " bytes/elem.)"
|
||||
<< " in " << seconds << " sec.";
|
||||
|
||||
if (seconds)
|
||||
message << ", " << profile_info.rows / seconds << " rows/sec., "
|
||||
<< profile_info.bytes / 1048576.0 / seconds << " MiB/sec.";
|
||||
|
||||
LOG_TRACE(log, message.str());
|
||||
if (!seconds)
|
||||
LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in 0 sec.",
|
||||
column_name, static_cast<double>(profile_info.bytes) / profile_info.rows);
|
||||
else
|
||||
LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in {} sec., {} rows/sec., {}/sec.",
|
||||
column_name, static_cast<double>(profile_info.bytes) / profile_info.rows, seconds,
|
||||
profile_info.rows / seconds, formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -90,9 +90,13 @@ void CreatingSetsBlockInputStream::createAll()
|
||||
|
||||
void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
|
||||
{
|
||||
LOG_TRACE(log, (subquery.set ? "Creating set. " : "")
|
||||
<< (subquery.join ? "Creating join. " : "")
|
||||
<< (subquery.table ? "Filling temporary table. " : ""));
|
||||
if (subquery.set)
|
||||
LOG_TRACE(log, "Creating set.");
|
||||
if (subquery.join)
|
||||
LOG_TRACE(log, "Creating join.");
|
||||
if (subquery.table)
|
||||
LOG_TRACE(log, "Filling temporary table.");
|
||||
|
||||
Stopwatch watch;
|
||||
|
||||
BlockOutputStreamPtr table_out;
|
||||
@ -165,19 +169,14 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
|
||||
|
||||
if (head_rows != 0)
|
||||
{
|
||||
std::stringstream msg;
|
||||
msg << std::fixed << std::setprecision(3);
|
||||
msg << "Created. ";
|
||||
auto seconds = watch.elapsedSeconds();
|
||||
|
||||
if (subquery.set)
|
||||
msg << "Set with " << subquery.set->getTotalRowCount() << " entries from " << head_rows << " rows. ";
|
||||
LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), head_rows, seconds);
|
||||
if (subquery.join)
|
||||
msg << "Join with " << subquery.join->getTotalRowCount() << " entries from " << head_rows << " rows. ";
|
||||
LOG_DEBUG(log, "Created Join with {} entries from {} rows in {} sec.", subquery.join->getTotalRowCount(), head_rows, seconds);
|
||||
if (subquery.table)
|
||||
msg << "Table with " << head_rows << " rows. ";
|
||||
|
||||
msg << "In " << watch.elapsedSeconds() << " sec.";
|
||||
LOG_DEBUG(log, msg.rdbuf());
|
||||
LOG_DEBUG(log, "Created Table with {} rows in {} sec.", head_rows, seconds);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -95,10 +95,10 @@ Block MergeSortingBlockInputStream::readImpl()
|
||||
const std::string & path = temporary_files.back()->path();
|
||||
MergeSortingBlocksBlockInputStream block_in(blocks, description, max_merged_block_size, limit);
|
||||
|
||||
LOG_INFO(log, "Sorting and writing part of data into temporary file " + path);
|
||||
LOG_INFO(log, "Sorting and writing part of data into temporary file {}", path);
|
||||
ProfileEvents::increment(ProfileEvents::ExternalSortWritePart);
|
||||
TemporaryFileStream::write(path, header_without_constants, block_in, &is_cancelled, codec); /// NOTE. Possibly limit disk usage.
|
||||
LOG_INFO(log, "Done writing part of data into temporary file " + path);
|
||||
LOG_INFO(log, "Done writing part of data into temporary file {}", path);
|
||||
|
||||
blocks.clear();
|
||||
sum_bytes_in_blocks = 0;
|
||||
@ -118,7 +118,7 @@ Block MergeSortingBlockInputStream::readImpl()
|
||||
/// If there was temporary files.
|
||||
ProfileEvents::increment(ProfileEvents::ExternalSortMerge);
|
||||
|
||||
LOG_INFO(log, "There are " << temporary_files.size() << " temporary sorted parts to merge.");
|
||||
LOG_INFO(log, "There are {} temporary sorted parts to merge.", temporary_files.size());
|
||||
|
||||
/// Create sorted streams to merge.
|
||||
for (const auto & file : temporary_files)
|
||||
@ -246,7 +246,7 @@ Block MergeSortingBlocksBlockInputStream::mergeImpl(TSortingHeap & queue)
|
||||
|
||||
void MergeSortingBlockInputStream::remerge()
|
||||
{
|
||||
LOG_DEBUG(log, "Re-merging intermediate ORDER BY data (" << blocks.size() << " blocks with " << sum_rows_in_blocks << " rows) to save memory consumption");
|
||||
LOG_DEBUG(log, "Re-merging intermediate ORDER BY data ({} blocks with {} rows) to save memory consumption", blocks.size(), sum_rows_in_blocks);
|
||||
|
||||
/// NOTE Maybe concat all blocks and partial sort will be faster than merge?
|
||||
MergeSortingBlocksBlockInputStream merger(blocks, description, max_merged_block_size, limit);
|
||||
@ -264,9 +264,7 @@ void MergeSortingBlockInputStream::remerge()
|
||||
}
|
||||
merger.readSuffix();
|
||||
|
||||
LOG_DEBUG(log, "Memory usage is lowered from "
|
||||
<< formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks) << " to "
|
||||
<< formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks));
|
||||
LOG_DEBUG(log, "Memory usage is lowered from {} to {}", formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks), formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks));
|
||||
|
||||
/// If the memory consumption was not lowered enough - we will not perform remerge anymore. 2 is a guess.
|
||||
if (new_sum_bytes_in_blocks * 2 > sum_bytes_in_blocks)
|
||||
|
@ -149,7 +149,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::cancel(bool kill)
|
||||
* (example: connection reset during distributed query execution)
|
||||
* - then don't care.
|
||||
*/
|
||||
LOG_ERROR(log, "Exception while cancelling " << input.stream->getName());
|
||||
LOG_ERROR(log, "Exception while cancelling {}", input.stream->getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -263,17 +263,13 @@ void MergingSortedBlockInputStream::readSuffixImpl()
|
||||
const BlockStreamProfileInfo & profile_info = getProfileInfo();
|
||||
double seconds = profile_info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
std::stringstream message;
|
||||
message << std::fixed << std::setprecision(2)
|
||||
<< "Merge sorted " << profile_info.blocks << " blocks, " << profile_info.rows << " rows"
|
||||
<< " in " << seconds << " sec.";
|
||||
|
||||
if (seconds)
|
||||
message << ", "
|
||||
<< profile_info.rows / seconds << " rows/sec., "
|
||||
<< profile_info.bytes / 1000000.0 / seconds << " MB/sec.";
|
||||
|
||||
LOG_DEBUG(log, message.str());
|
||||
if (!seconds)
|
||||
LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in 0 sec.", profile_info.blocks, profile_info.rows);
|
||||
else
|
||||
LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec",
|
||||
profile_info.blocks, profile_info.rows, seconds,
|
||||
profile_info.rows / seconds,
|
||||
formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <DataStreams/NativeBlockInputStream.h>
|
||||
#include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
|
||||
#include <DataStreams/ParallelAggregatingBlockInputStream.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/formatReadable.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -82,9 +82,7 @@ Block ParallelAggregatingBlockInputStream::readImpl()
|
||||
input_streams.emplace_back(temporary_inputs.back()->block_in);
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Will merge " << files.files.size() << " temporary files of size "
|
||||
<< (files.sum_size_compressed / 1048576.0) << " MiB compressed, "
|
||||
<< (files.sum_size_uncompressed / 1048576.0) << " MiB uncompressed.");
|
||||
LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed));
|
||||
|
||||
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(
|
||||
input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads);
|
||||
@ -179,20 +177,17 @@ void ParallelAggregatingBlockInputStream::execute()
|
||||
for (size_t i = 0; i < max_threads; ++i)
|
||||
{
|
||||
size_t rows = many_data[i]->size();
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Aggregated. " << threads_data[i].src_rows << " to " << rows << " rows"
|
||||
<< " (from " << threads_data[i].src_bytes / 1048576.0 << " MiB)"
|
||||
<< " in " << elapsed_seconds << " sec."
|
||||
<< " (" << threads_data[i].src_rows / elapsed_seconds << " rows/sec., "
|
||||
<< threads_data[i].src_bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
||||
LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)",
|
||||
threads_data[i].src_rows, rows, formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes),
|
||||
elapsed_seconds, threads_data[i].src_rows / elapsed_seconds,
|
||||
formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes / elapsed_seconds));
|
||||
|
||||
total_src_rows += threads_data[i].src_rows;
|
||||
total_src_bytes += threads_data[i].src_bytes;
|
||||
}
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Total aggregated. " << total_src_rows << " rows (from " << total_src_bytes / 1048576.0 << " MiB)"
|
||||
<< " in " << elapsed_seconds << " sec."
|
||||
<< " (" << total_src_rows / elapsed_seconds << " rows/sec., " << total_src_bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
||||
LOG_TRACE(log, "Total aggregated. {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)",
|
||||
total_src_rows, formatReadableSizeWithBinarySuffix(total_src_bytes), elapsed_seconds,
|
||||
total_src_rows / elapsed_seconds, formatReadableSizeWithBinarySuffix(total_src_bytes / elapsed_seconds));
|
||||
|
||||
/// If there was no data, and we aggregate without keys, we must return single row with the result of empty aggregation.
|
||||
/// To do this, we pass a block with zero rows to aggregate.
|
||||
|
@ -134,7 +134,7 @@ public:
|
||||
* (for example, the connection is broken for distributed query processing)
|
||||
* - then do not care.
|
||||
*/
|
||||
LOG_ERROR(log, "Exception while cancelling " << input->getName());
|
||||
LOG_ERROR(log, "Exception while cancelling {}", input->getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -372,7 +372,7 @@ void RemoteBlockInputStream::tryCancel(const char * reason)
|
||||
multiplexed_connections->sendCancel();
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "(" << multiplexed_connections->dumpAddresses() << ") " << reason);
|
||||
LOG_TRACE(log, "({}) {}", multiplexed_connections->dumpAddresses(), reason);
|
||||
}
|
||||
|
||||
bool RemoteBlockInputStream::isQueryPending() const
|
||||
|
@ -108,7 +108,7 @@ void TTLBlockInputStream::readSuffixImpl()
|
||||
data_part->expired_columns = std::move(empty_columns);
|
||||
|
||||
if (rows_removed)
|
||||
LOG_INFO(log, "Removed " << rows_removed << " rows with expired TTL from part " << data_part->name);
|
||||
LOG_INFO(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name);
|
||||
}
|
||||
|
||||
void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)
|
||||
|
@ -152,7 +152,7 @@ bool DatabaseLazy::empty() const
|
||||
|
||||
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &)
|
||||
{
|
||||
LOG_DEBUG(log, "Attach table " << backQuote(table_name) << ".");
|
||||
LOG_DEBUG(log, "Attach table {}.", backQuote(table_name));
|
||||
std::lock_guard lock(mutex);
|
||||
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
|
||||
|
||||
@ -169,7 +169,7 @@ StoragePtr DatabaseLazy::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
LOG_DEBUG(log, "Detach table " << backQuote(table_name) << ".");
|
||||
LOG_DEBUG(log, "Detach table {}.", backQuote(table_name));
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables_cache.find(table_name);
|
||||
if (it == tables_cache.end())
|
||||
@ -216,7 +216,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
|
||||
LOG_DEBUG(log, "Load table " << backQuote(table_name) << " to cache.");
|
||||
LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name));
|
||||
|
||||
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
|
||||
|
||||
@ -277,14 +277,14 @@ void DatabaseLazy::clearExpiredTables() const
|
||||
|
||||
if (!it->second.table || it->second.table.unique())
|
||||
{
|
||||
LOG_DEBUG(log, "Drop table " << backQuote(it->first) << " from cache.");
|
||||
LOG_DEBUG(log, "Drop table {} from cache.", backQuote(it->first));
|
||||
it->second.table.reset();
|
||||
expired_tables.erase(it->second.expiration_iterator);
|
||||
it->second.expiration_iterator = cache_expiration_queue.end();
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Table " << backQuote(it->first) << " is busy.");
|
||||
LOG_DEBUG(log, "Table {} is busy.", backQuote(it->first));
|
||||
busy_tables.splice(busy_tables.end(), expired_tables, it->second.expiration_iterator);
|
||||
}
|
||||
}
|
||||
|
@ -376,12 +376,12 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql");
|
||||
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored");
|
||||
LOG_WARNING(log, "Object {} was not dropped previously and will be restored", backQuote(object_name));
|
||||
process_metadata_file(object_name + ".sql");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Removing file " << getMetadataPath() + file_name);
|
||||
LOG_INFO(log, "Removing file {}", getMetadataPath() + file_name);
|
||||
Poco::File(getMetadataPath() + file_name).remove();
|
||||
}
|
||||
};
|
||||
@ -406,7 +406,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
else if (endsWith(dir_it.name(), ".sql.tmp"))
|
||||
{
|
||||
/// There are files .sql.tmp - delete
|
||||
LOG_INFO(log, "Removing file " << dir_it->path());
|
||||
LOG_INFO(log, "Removing file {}", dir_it->path());
|
||||
Poco::File(dir_it->path()).remove();
|
||||
}
|
||||
else if (endsWith(dir_it.name(), ".sql"))
|
||||
@ -442,7 +442,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * loger, const Contex
|
||||
*/
|
||||
if (remove_empty && query.empty())
|
||||
{
|
||||
LOG_ERROR(loger, "File " << metadata_file_path << " is empty. Removing.");
|
||||
LOG_ERROR(loger, "File {} is empty. Removing.", metadata_file_path);
|
||||
Poco::File(metadata_file_path).remove();
|
||||
return nullptr;
|
||||
}
|
||||
@ -466,8 +466,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * loger, const Contex
|
||||
table_name = unescapeForFileName(table_name);
|
||||
|
||||
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)
|
||||
LOG_WARNING(loger, "File " << metadata_file_path << " contains both UUID and table name. "
|
||||
"Will use name `" << table_name << "` instead of `" << create.table << "`");
|
||||
LOG_WARNING(loger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table);
|
||||
create.table = table_name;
|
||||
}
|
||||
|
||||
|
@ -95,7 +95,7 @@ namespace
|
||||
{
|
||||
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << processed * 100.0 / total << "%");
|
||||
LOG_INFO(log, "{}%", processed * 100.0 / total);
|
||||
watch.restart();
|
||||
}
|
||||
}
|
||||
@ -150,7 +150,7 @@ void DatabaseOrdinary::loadStoredObjects(
|
||||
|
||||
size_t total_tables = file_names.size() - total_dictionaries;
|
||||
|
||||
LOG_INFO(log, "Total " << total_tables << " tables and " << total_dictionaries << " dictionaries.");
|
||||
LOG_INFO(log, "Total {} tables and {} dictionaries.", total_tables, total_dictionaries);
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed{0};
|
||||
|
@ -768,8 +768,7 @@ void CacheDictionary::updateThreadFunction()
|
||||
const size_t current_queue_size = update_queue.size();
|
||||
|
||||
if (current_queue_size > 0)
|
||||
LOG_TRACE(log, "Performing bunch of keys update in cache dictionary with "
|
||||
<< current_queue_size + 1 << " keys");
|
||||
LOG_TRACE(log, "Performing bunch of keys update in cache dictionary with {} keys", current_queue_size + 1);
|
||||
|
||||
std::vector<UpdateUnitPtr> update_request;
|
||||
update_request.reserve(current_queue_size + 1);
|
||||
|
@ -168,7 +168,7 @@ bool ClickHouseDictionarySource::isModified() const
|
||||
if (!invalidate_query.empty())
|
||||
{
|
||||
auto response = doInvalidateQuery(invalidate_query);
|
||||
LOG_TRACE(log, "Invalidate query has returned: " << response << ", previous value: " << invalidate_query_response);
|
||||
LOG_TRACE(log, "Invalidate query has returned: {}, previous value: {}", response, invalidate_query_response);
|
||||
if (invalidate_query_response == response)
|
||||
return false;
|
||||
invalidate_query_response = response;
|
||||
|
@ -43,9 +43,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
|
||||
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context, check_source_config);
|
||||
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"),
|
||||
"Created dictionary source '" << source_ptr->toString()
|
||||
<< "' for dictionary '" << name << "'");
|
||||
LOG_TRACE(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
|
||||
|
||||
const auto & layout_type = keys.front();
|
||||
|
||||
|
@ -13,7 +13,7 @@ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_p
|
||||
|
||||
for (const auto & name : data_provider->listCustomHierarchies())
|
||||
{
|
||||
LOG_DEBUG(log, "Adding regions hierarchy for " << name);
|
||||
LOG_DEBUG(log, "Adding regions hierarchy for {}", name);
|
||||
data.emplace(name, data_provider->getHierarchySource(name));
|
||||
}
|
||||
|
||||
|
@ -55,7 +55,7 @@ void RegionsNames::reload()
|
||||
if (!names_source || !names_source->isModified())
|
||||
continue;
|
||||
|
||||
LOG_DEBUG(log, "Reloading regions names for language: " << language);
|
||||
LOG_DEBUG(log, "Reloading regions names for language: {}", language);
|
||||
|
||||
auto names_reader = names_source->createReader();
|
||||
|
||||
|
@ -77,7 +77,7 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar
|
||||
|
||||
BlockInputStreamPtr ExecutableDictionarySource::loadAll()
|
||||
{
|
||||
LOG_TRACE(log, "loadAll " + toString());
|
||||
LOG_TRACE(log, "loadAll {}", toString());
|
||||
auto process = ShellCommand::execute(command);
|
||||
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
||||
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));
|
||||
@ -92,7 +92,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll()
|
||||
if (update_time)
|
||||
command_with_update_field += " " + update_field + " " + DB::toString(LocalDateTime(update_time - 1));
|
||||
|
||||
LOG_TRACE(log, "loadUpdatedAll " + command_with_update_field);
|
||||
LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field);
|
||||
auto process = ShellCommand::execute(command_with_update_field);
|
||||
auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size);
|
||||
return std::make_shared<ShellCommandOwningBlockInputStream>(input_stream, std::move(process));
|
||||
@ -163,7 +163,7 @@ namespace
|
||||
|
||||
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
{
|
||||
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
|
||||
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
||||
auto process = ShellCommand::execute(command);
|
||||
|
||||
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
|
||||
@ -175,7 +175,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
|
||||
|
||||
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
||||
{
|
||||
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
|
||||
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
||||
auto process = ShellCommand::execute(command);
|
||||
|
||||
auto output_stream = context.getOutputFormat(format, process->in, sample_block);
|
||||
|
@ -49,7 +49,7 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other)
|
||||
|
||||
BlockInputStreamPtr FileDictionarySource::loadAll()
|
||||
{
|
||||
LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll " + toString());
|
||||
LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString());
|
||||
auto in_ptr = std::make_unique<ReadBufferFromFile>(filepath);
|
||||
auto stream = context.getInputFormat(format, *in_ptr, sample_block, max_block_size);
|
||||
last_modification = getLastModification();
|
||||
|
@ -105,7 +105,7 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri)
|
||||
|
||||
BlockInputStreamPtr HTTPDictionarySource::loadAll()
|
||||
{
|
||||
LOG_TRACE(log, "loadAll " + toString());
|
||||
LOG_TRACE(log, "loadAll {}", toString());
|
||||
Poco::URI uri(url);
|
||||
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
|
||||
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
|
||||
@ -118,7 +118,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
|
||||
{
|
||||
Poco::URI uri(url);
|
||||
getUpdateFieldAndDate(uri);
|
||||
LOG_TRACE(log, "loadUpdatedAll " + uri.toString());
|
||||
LOG_TRACE(log, "loadUpdatedAll {}", uri.toString());
|
||||
auto in_ptr = std::make_unique<ReadWriteBufferFromHTTP>(
|
||||
uri, Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), timeouts,
|
||||
0, credentials, DBMS_DEFAULT_BUFFER_SIZE, header_entries);
|
||||
@ -128,7 +128,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll()
|
||||
|
||||
BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
{
|
||||
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
|
||||
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
||||
|
||||
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
|
||||
{
|
||||
@ -147,7 +147,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & id
|
||||
|
||||
BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
|
||||
{
|
||||
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
|
||||
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
||||
|
||||
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
|
||||
{
|
||||
|
@ -182,7 +182,7 @@ LibraryDictionarySource::~LibraryDictionarySource()
|
||||
|
||||
BlockInputStreamPtr LibraryDictionarySource::loadAll()
|
||||
{
|
||||
LOG_TRACE(log, "loadAll " + toString());
|
||||
LOG_TRACE(log, "loadAll {}", toString());
|
||||
|
||||
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
|
||||
ClickHouseLibrary::CStrings columns{static_cast<decltype(ClickHouseLibrary::CStrings::data)>(columns_holder.get()),
|
||||
@ -207,7 +207,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll()
|
||||
|
||||
BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
|
||||
{
|
||||
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
|
||||
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
|
||||
|
||||
const ClickHouseLibrary::VectorUInt64 ids_data{ext::bit_cast<decltype(ClickHouseLibrary::VectorUInt64::data)>(ids.data()), ids.size()};
|
||||
auto columns_holder = std::make_unique<ClickHouseLibrary::CString[]>(dict_struct.attributes.size());
|
||||
@ -234,7 +234,7 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> &
|
||||
|
||||
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||
{
|
||||
LOG_TRACE(log, "loadKeys " << toString() << " size = " << requested_rows.size());
|
||||
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
|
||||
|
||||
auto holder = std::make_unique<ClickHouseLibrary::Row[]>(key_columns.size());
|
||||
std::vector<std::unique_ptr<ClickHouseLibrary::Field[]>> column_data_holders;
|
||||
|
@ -227,7 +227,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
|
||||
if (!update_time_value.isNull())
|
||||
{
|
||||
modification_time = update_time_value.getDateTime();
|
||||
LOG_TRACE(log, "Got modification time: " << modification_time);
|
||||
LOG_TRACE(log, "Got modification time: {}", modification_time);
|
||||
}
|
||||
|
||||
/// fetch remaining rows to avoid "commands out of sync" error
|
||||
|
@ -91,7 +91,7 @@ bool DiskLocal::tryReserve(UInt64 bytes)
|
||||
std::lock_guard lock(DiskLocal::reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk " << backQuote(name));
|
||||
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
@ -100,10 +100,8 @@ bool DiskLocal::tryReserve(UInt64 bytes)
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskLocal"),
|
||||
"Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name) << ", having unreserved "
|
||||
<< formatReadableSizeWithBinarySuffix(unreserved_space) << ".");
|
||||
LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return true;
|
||||
@ -312,7 +310,7 @@ DiskLocalReservation::~DiskLocalReservation()
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'.");
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -320,7 +318,7 @@ DiskLocalReservation::~DiskLocalReservation()
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'.");
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
|
@ -15,9 +15,7 @@ bool IDisk::isDirectoryEmpty(const String & path)
|
||||
|
||||
void copyFile(IDisk & from_disk, const String & from_path, IDisk & to_disk, const String & to_path)
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Poco::Logger::get("IDisk"),
|
||||
"Copying from " << from_disk.getName() << " " << from_path << " to " << to_disk.getName() << " " << to_path);
|
||||
LOG_DEBUG(&Poco::Logger::get("IDisk"), "Copying from {} {} to {} {}.", from_disk.getName(), from_path, to_disk.getName(), to_path);
|
||||
|
||||
auto in = from_disk.readFile(from_path);
|
||||
auto out = to_disk.writeFile(to_path);
|
||||
|
@ -515,9 +515,8 @@ std::unique_ptr<ReadBufferFromFileBase> DiskS3::readFile(const String & path, si
|
||||
{
|
||||
Metadata metadata(metadata_path, path);
|
||||
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskS3"),
|
||||
"Read from file by path: " << backQuote(metadata_path + path) << " Existing S3 objects: " << metadata.s3_objects.size());
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}",
|
||||
backQuote(metadata_path + path), metadata.s3_objects.size());
|
||||
|
||||
return std::make_unique<ReadIndirectBufferFromS3>(client, bucket, metadata, buf_size);
|
||||
}
|
||||
@ -537,7 +536,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
/// Save empty metadata to disk to have ability to get file size while buffer is not finalized.
|
||||
metadata.save();
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Write to file by path: " << backQuote(metadata_path + path) << " New S3 path: " << s3_path);
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, min_upload_part_size, buf_size);
|
||||
}
|
||||
@ -545,10 +544,8 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
{
|
||||
Metadata metadata(metadata_path, path);
|
||||
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskS3"),
|
||||
"Append to file by path: " << backQuote(metadata_path + path) << " New S3 path: " << s3_path
|
||||
<< " Existing S3 objects: " << metadata.s3_objects.size());
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.",
|
||||
backQuote(metadata_path + path), s3_path, metadata.s3_objects.size());
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, min_upload_part_size, buf_size);
|
||||
}
|
||||
@ -556,7 +553,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
|
||||
void DiskS3::remove(const String & path)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Remove file by path: " << backQuote(metadata_path + path));
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path));
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
if (file.isFile())
|
||||
@ -610,7 +607,7 @@ bool DiskS3::tryReserve(UInt64 bytes)
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk " << backQuote(name));
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
@ -619,10 +616,8 @@ bool DiskS3::tryReserve(UInt64 bytes)
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskS3"),
|
||||
"Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name) << ", having unreserved "
|
||||
<< formatReadableSizeWithBinarySuffix(unreserved_space) << ".");
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.",
|
||||
formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return true;
|
||||
@ -677,7 +672,7 @@ DiskS3Reservation::~DiskS3Reservation()
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'.");
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -685,7 +680,7 @@ DiskS3Reservation::~DiskS3Reservation()
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'.");
|
||||
LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
|
@ -20,7 +20,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyListConfiguration::getConfigurat
|
||||
cfg.proxyHost = proxies[index].getHost();
|
||||
cfg.proxyPort = proxies[index].getPort();
|
||||
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: " << proxies[index].toString());
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString());
|
||||
|
||||
return cfg;
|
||||
}
|
||||
|
@ -21,7 +21,7 @@ ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoin
|
||||
|
||||
Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const Aws::Http::HttpRequest &)
|
||||
{
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Obtain proxy using resolver: " << endpoint.toString());
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString());
|
||||
|
||||
/// 1 second is enough for now.
|
||||
/// TODO: Make timeouts configurable.
|
||||
@ -49,7 +49,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig
|
||||
/// Read proxy host as string from response body.
|
||||
Poco::StreamCopier::copyToString(response_body_stream, proxy_host);
|
||||
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: " << proxy_scheme << "://" << proxy_host << ":" << proxy_port);
|
||||
LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port);
|
||||
|
||||
cfg.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str());
|
||||
cfg.proxyHost = proxy_host;
|
||||
|
@ -46,8 +46,7 @@ namespace
|
||||
throw Exception("Only HTTP/HTTPS schemas allowed in proxy resolver config: " + proxy_scheme, ErrorCodes::BAD_ARGUMENTS);
|
||||
auto proxy_port = proxy_resolver_config.getUInt(prefix + ".proxy_port");
|
||||
|
||||
LOG_DEBUG(
|
||||
&Logger::get("DiskS3"), "Configured proxy resolver: " << endpoint.toString() << ", Scheme: " << proxy_scheme << ", Port: " << proxy_port);
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port);
|
||||
|
||||
return std::make_shared<S3::ProxyResolverConfiguration>(endpoint, proxy_scheme, proxy_port);
|
||||
}
|
||||
@ -71,7 +70,7 @@ namespace
|
||||
|
||||
proxies.push_back(proxy_uri);
|
||||
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy: " << proxy_uri.toString());
|
||||
LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString());
|
||||
}
|
||||
|
||||
if (!proxies.empty())
|
||||
|
@ -245,7 +245,7 @@ StoragePolicySelector::StoragePolicySelector(
|
||||
"StoragePolicy name can contain only alphanumeric and '_' (" + name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
policies.emplace(name, std::make_shared<StoragePolicy>(name, config, config_prefix + "." + name, disks));
|
||||
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy " << backQuote(name) << " loaded");
|
||||
LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name));
|
||||
}
|
||||
|
||||
constexpr auto default_storage_policy_name = "default";
|
||||
|
@ -48,18 +48,11 @@ VolumeJBOD::VolumeJBOD(
|
||||
max_data_part_size = static_cast<decltype(max_data_part_size)>(sum_size * ratio / disks.size());
|
||||
for (size_t i = 0; i < disks.size(); ++i)
|
||||
if (sizes[i] < max_data_part_size)
|
||||
LOG_WARNING(
|
||||
logger,
|
||||
"Disk " << backQuote(disks[i]->getName()) << " on volume " << backQuote(config_prefix) << " have not enough space ("
|
||||
<< formatReadableSizeWithBinarySuffix(sizes[i]) << ") for containing part the size of max_data_part_size ("
|
||||
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << ")");
|
||||
LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})", backQuote(disks[i]->getName()), backQuote(config_prefix), formatReadableSizeWithBinarySuffix(sizes[i]), formatReadableSizeWithBinarySuffix(max_data_part_size));
|
||||
}
|
||||
static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
|
||||
if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
|
||||
LOG_WARNING(
|
||||
logger,
|
||||
"Volume " << backQuote(name) << " max_data_part_size is too low (" << formatReadableSizeWithBinarySuffix(max_data_part_size)
|
||||
<< " < " << formatReadableSizeWithBinarySuffix(MIN_PART_SIZE) << ")");
|
||||
LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), formatReadableSizeWithBinarySuffix(max_data_part_size), formatReadableSizeWithBinarySuffix(MIN_PART_SIZE));
|
||||
}
|
||||
|
||||
DiskPtr VolumeJBOD::getNextDisk()
|
||||
|
@ -98,7 +98,7 @@ void AIOContextPool::fulfillPromises(const io_event events[], const int num_even
|
||||
const auto it = promises.find(completed_id);
|
||||
if (it == std::end(promises))
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("AIOcontextPool"), "Found io_event with unknown id " << completed_id);
|
||||
LOG_ERROR(&Poco::Logger::get("AIOcontextPool"), "Found io_event with unknown id {}", completed_id);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -173,7 +173,7 @@ namespace
|
||||
auto msg = Poco::AnyCast<std::string>(session_data);
|
||||
if (!msg.empty())
|
||||
{
|
||||
LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with " << host << " with error '" << msg << "' will try to reconnect session");
|
||||
LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg);
|
||||
/// Host can change IP
|
||||
const auto ip = DNSResolver::instance().resolveHost(host).toString();
|
||||
if (ip != session->getHost())
|
||||
|
@ -66,7 +66,7 @@ off_t ReadBufferFromS3::getPosition()
|
||||
|
||||
std::unique_ptr<ReadBuffer> ReadBufferFromS3::initialize()
|
||||
{
|
||||
LOG_TRACE(log, "Read S3 object. Bucket: " + bucket + ", Key: " + key + ", Offset: " + std::to_string(offset));
|
||||
LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, std::to_string(offset));
|
||||
|
||||
Aws::S3::Model::GetObjectRequest req;
|
||||
req.SetBucket(bucket);
|
||||
|
@ -127,7 +127,7 @@ namespace detail
|
||||
if (!credentials.getUsername().empty())
|
||||
credentials.authenticate(request);
|
||||
|
||||
LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to " << uri.toString());
|
||||
LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString());
|
||||
|
||||
auto sess = session->getSession();
|
||||
|
||||
|
@ -40,13 +40,13 @@ public:
|
||||
void Log(Aws::Utils::Logging::LogLevel log_level, const char * tag, const char * format_str, ...) final // NOLINT
|
||||
{
|
||||
const auto & [level, prio] = convertLogLevel(log_level);
|
||||
LOG_SIMPLE(log, std::string(tag) + ": " + format_str, level, prio);
|
||||
LOG_IMPL(log, level, prio, "{}: {}", tag, format_str);
|
||||
}
|
||||
|
||||
void LogStream(Aws::Utils::Logging::LogLevel log_level, const char * tag, const Aws::OStringStream & message_stream) final
|
||||
{
|
||||
const auto & [level, prio] = convertLogLevel(log_level);
|
||||
LOG_SIMPLE(log, std::string(tag) + ": " + message_stream.str(), level, prio);
|
||||
LOG_IMPL(log, level, prio, "{}: {}", tag, message_stream.str());
|
||||
}
|
||||
|
||||
void Flush() final {}
|
||||
|
@ -15,7 +15,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP(
|
||||
request.setHost(uri.getHost());
|
||||
request.setChunkedTransferEncoding(true);
|
||||
|
||||
LOG_TRACE((&Logger::get("WriteBufferToHTTP")), "Sending request to " << uri.toString());
|
||||
LOG_TRACE((&Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString());
|
||||
|
||||
ostr = &session->sendRequest(request);
|
||||
}
|
||||
|
@ -100,7 +100,7 @@ void WriteBufferFromS3::initiate()
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
upload_id = outcome.GetResult().GetUploadId();
|
||||
LOG_DEBUG(log, "Multipart upload initiated. Upload id: " << upload_id);
|
||||
LOG_DEBUG(log, "Multipart upload initiated. Upload id: {}", upload_id);
|
||||
}
|
||||
else
|
||||
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
@ -129,17 +129,13 @@ void WriteBufferFromS3::writePart(const String & data)
|
||||
|
||||
auto outcome = client_ptr->UploadPart(req);
|
||||
|
||||
LOG_TRACE(
|
||||
log, "Writing part. Bucket: " << bucket << ", Key: " << key << ", Upload_id: " << upload_id << ", Data size: " << data.size());
|
||||
LOG_TRACE(log, "Writing part. Bucket: {}, Key: {}, Upload_id: {}, Data size: {}", bucket, key, upload_id, data.size());
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
auto etag = outcome.GetResult().GetETag();
|
||||
part_tags.push_back(etag);
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Writing part finished. "
|
||||
<< "Total parts: " << part_tags.size() << ", Upload_id: " << upload_id << ", Etag: " << etag);
|
||||
LOG_DEBUG(log, "Writing part finished. Total parts: {}, Upload_id: {}, Etag: {}", part_tags.size(), upload_id, etag);
|
||||
}
|
||||
else
|
||||
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
@ -148,7 +144,7 @@ void WriteBufferFromS3::writePart(const String & data)
|
||||
|
||||
void WriteBufferFromS3::complete()
|
||||
{
|
||||
LOG_DEBUG(log, "Completing multipart upload. Bucket: " + bucket + ", Key: " + key + ", Upload_id: " + upload_id);
|
||||
LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, upload_id);
|
||||
|
||||
Aws::S3::Model::CompleteMultipartUploadRequest req;
|
||||
req.SetBucket(bucket);
|
||||
@ -170,7 +166,7 @@ void WriteBufferFromS3::complete()
|
||||
auto outcome = client_ptr->CompleteMultipartUpload(req);
|
||||
|
||||
if (outcome.IsSuccess())
|
||||
LOG_DEBUG(log, "Multipart upload completed. Upload_id: " << upload_id);
|
||||
LOG_DEBUG(log, "Multipart upload completed. Upload_id: {}", upload_id);
|
||||
else
|
||||
throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
@ -551,7 +552,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData
|
||||
result.init(method_chosen);
|
||||
result.keys_size = params.keys_size;
|
||||
result.key_sizes = key_sizes;
|
||||
LOG_TRACE(log, "Aggregation method: " << result.getMethodName());
|
||||
LOG_TRACE(log, "Aggregation method: {}", result.getMethodName());
|
||||
}
|
||||
|
||||
if (isCancelled())
|
||||
@ -722,7 +723,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co
|
||||
CompressedWriteBuffer compressed_buf(file_buf);
|
||||
NativeBlockOutputStream block_out(compressed_buf, ClickHouseRevision::get(), getHeader(false));
|
||||
|
||||
LOG_DEBUG(log, "Writing part of aggregation data into temporary file " << path << ".");
|
||||
LOG_DEBUG(log, "Writing part of aggregation data into temporary file {}.", path);
|
||||
ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart);
|
||||
|
||||
/// Flush only two-level data and possibly overflow data.
|
||||
@ -761,17 +762,20 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co
|
||||
ProfileEvents::increment(ProfileEvents::ExternalAggregationCompressedBytes, compressed_bytes);
|
||||
ProfileEvents::increment(ProfileEvents::ExternalAggregationUncompressedBytes, uncompressed_bytes);
|
||||
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Written part in " << elapsed_seconds << " sec., "
|
||||
<< rows << " rows, "
|
||||
<< (uncompressed_bytes / 1048576.0) << " MiB uncompressed, "
|
||||
<< (compressed_bytes / 1048576.0) << " MiB compressed, "
|
||||
<< (uncompressed_bytes / rows) << " uncompressed bytes per row, "
|
||||
<< (compressed_bytes / rows) << " compressed bytes per row, "
|
||||
<< "compression rate: " << (uncompressed_bytes / compressed_bytes)
|
||||
<< " (" << (rows / elapsed_seconds) << " rows/sec., "
|
||||
<< (uncompressed_bytes / elapsed_seconds / 1048576.0) << " MiB/sec. uncompressed, "
|
||||
<< (compressed_bytes / elapsed_seconds / 1048576.0) << " MiB/sec. compressed)");
|
||||
LOG_TRACE(log,
|
||||
"Written part in {} sec., {} rows, {} uncompressed, {} compressed,"
|
||||
" {} uncompressed bytes per row, {} compressed bytes per row, compression rate: {}"
|
||||
" ({} rows/sec., {}/sec. uncompressed, {}/sec. compressed)",
|
||||
elapsed_seconds,
|
||||
rows,
|
||||
formatReadableSizeWithBinarySuffix(uncompressed_bytes),
|
||||
formatReadableSizeWithBinarySuffix(compressed_bytes),
|
||||
uncompressed_bytes / rows,
|
||||
compressed_bytes / rows,
|
||||
uncompressed_bytes / compressed_bytes,
|
||||
rows / elapsed_seconds,
|
||||
formatReadableSizeWithBinarySuffix(uncompressed_bytes / elapsed_seconds),
|
||||
formatReadableSizeWithBinarySuffix(compressed_bytes / elapsed_seconds));
|
||||
}
|
||||
void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants)
|
||||
{
|
||||
@ -867,9 +871,7 @@ void Aggregator::writeToTemporaryFileImpl(
|
||||
/// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects.
|
||||
data_variants.aggregator = nullptr;
|
||||
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Max size of temporary block: " << max_temporary_block_size_rows << " rows, "
|
||||
<< (max_temporary_block_size_bytes / 1048576.0) << " MiB.");
|
||||
LOG_TRACE(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, formatReadableSizeWithBinarySuffix(max_temporary_block_size_bytes));
|
||||
}
|
||||
|
||||
|
||||
@ -939,10 +941,11 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria
|
||||
|
||||
double elapsed_seconds = watch.elapsedSeconds();
|
||||
size_t rows = result.sizeWithoutOverflowRow();
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Aggregated. " << src_rows << " to " << rows << " rows (from " << src_bytes / 1048576.0 << " MiB)"
|
||||
<< " in " << elapsed_seconds << " sec."
|
||||
<< " (" << src_rows / elapsed_seconds << " rows/sec., " << src_bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
||||
|
||||
LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)",
|
||||
src_rows, rows, formatReadableSizeWithBinarySuffix(src_bytes),
|
||||
elapsed_seconds, src_rows / elapsed_seconds,
|
||||
formatReadableSizeWithBinarySuffix(src_bytes / elapsed_seconds));
|
||||
}
|
||||
|
||||
|
||||
@ -1308,11 +1311,11 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
|
||||
}
|
||||
|
||||
double elapsed_seconds = watch.elapsedSeconds();
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Converted aggregated data to blocks. "
|
||||
<< rows << " rows, " << bytes / 1048576.0 << " MiB"
|
||||
<< " in " << elapsed_seconds << " sec."
|
||||
<< " (" << rows / elapsed_seconds << " rows/sec., " << bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
||||
LOG_TRACE(log,
|
||||
"Converted aggregated data to blocks. {} rows, {} in {} sec. ({} rows/sec., {}/sec.)",
|
||||
rows, formatReadableSizeWithBinarySuffix(bytes),
|
||||
elapsed_seconds, rows / elapsed_seconds,
|
||||
formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds));
|
||||
|
||||
return blocks;
|
||||
}
|
||||
@ -1943,8 +1946,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
|
||||
bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Read " << total_input_blocks << " blocks of partially aggregated data, total " << total_input_rows
|
||||
<< " rows.");
|
||||
LOG_TRACE(log, "Read {} blocks of partially aggregated data, total {} rows.", total_input_blocks, total_input_rows);
|
||||
|
||||
mergeBlocks(bucket_to_blocks, result, max_threads);
|
||||
}
|
||||
@ -2099,7 +2101,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
auto bucket_num = blocks.front().info.bucket_num;
|
||||
bool is_overflows = blocks.front().info.is_overflows;
|
||||
|
||||
LOG_TRACE(log, "Merging partially aggregated blocks (bucket = " << bucket_num << ").");
|
||||
LOG_TRACE(log, "Merging partially aggregated blocks (bucket = {}).", bucket_num);
|
||||
Stopwatch watch;
|
||||
|
||||
/** If possible, change 'method' to some_hash64. Otherwise, leave as is.
|
||||
@ -2175,11 +2177,10 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
size_t rows = block.rows();
|
||||
size_t bytes = block.bytes();
|
||||
double elapsed_seconds = watch.elapsedSeconds();
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
<< "Merged partially aggregated blocks. "
|
||||
<< rows << " rows, " << bytes / 1048576.0 << " MiB."
|
||||
<< " in " << elapsed_seconds << " sec."
|
||||
<< " (" << rows / elapsed_seconds << " rows/sec., " << bytes / elapsed_seconds / 1048576.0 << " MiB/sec.)");
|
||||
LOG_TRACE(log, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({} rows/sec., {}/sec.)",
|
||||
rows, formatReadableSizeWithBinarySuffix(bytes),
|
||||
elapsed_seconds, rows / elapsed_seconds,
|
||||
formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds));
|
||||
|
||||
if (isCancelled())
|
||||
return {};
|
||||
|
@ -200,10 +200,7 @@ void SelectStreamFactory::createForShard(
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable);
|
||||
if (shard_info.hasRemoteConnections())
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"There is no table " << main_table.getNameForLogs()
|
||||
<< " on local replica of shard " << shard_info.shard_num << ", will try remote replicas.");
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num);
|
||||
emplace_remote_stream();
|
||||
}
|
||||
else
|
||||
@ -239,9 +236,7 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
/// If we reached this point, local replica is stale.
|
||||
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"Local replica of shard " << shard_info.shard_num << " is stale (delay: " << local_delay << "s.)");
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay);
|
||||
|
||||
if (!settings.fallback_to_stale_replicas_for_distributed_queries)
|
||||
{
|
||||
@ -289,9 +284,7 @@ void SelectStreamFactory::createForShard(
|
||||
catch (const Exception & ex)
|
||||
{
|
||||
if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED)
|
||||
LOG_WARNING(
|
||||
&Logger::get("ClusterProxy::SelectStreamFactory"),
|
||||
"Connections to remote replicas of local shard " << shard_num << " failed, will use stale local replica");
|
||||
LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num);
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
@ -1761,7 +1761,7 @@ void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
LOG_ERROR(shared->log, "An error has occured while reloading storage policies, storage policies were not applied: " << e.message());
|
||||
LOG_ERROR(shared->log, "An error has occured while reloading storage policies, storage policies were not applied: {}", e.message());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -238,8 +238,7 @@ DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const
|
||||
|
||||
if (context.getSettingsRef().readonly)
|
||||
{
|
||||
LOG_WARNING(log, "Distributed DDL worker is run with readonly settings, it will not be able to execute DDL queries"
|
||||
<< " Set appropriate system_profile or distributed_ddl.profile to fix this.");
|
||||
LOG_WARNING(log, "Distributed DDL worker is run with readonly settings, it will not be able to execute DDL queries Set appropriate system_profile or distributed_ddl.profile to fix this.");
|
||||
}
|
||||
|
||||
host_fqdn = getFQDNOrHostName();
|
||||
@ -335,8 +334,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason,
|
||||
if (host_in_hostlist)
|
||||
{
|
||||
/// This check could be slow a little bit
|
||||
LOG_WARNING(log, "There are two the same ClickHouse instances in task " << entry_name
|
||||
<< ": " << task->host_id.readableString() << " and " << host.readableString() << ". Will use the first one only.");
|
||||
LOG_WARNING(log, "There are two the same ClickHouse instances in task {}: {} and {}. Will use the first one only.", entry_name, task->host_id.readableString(), host.readableString());
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -386,11 +384,11 @@ void DDLWorker::processTasks()
|
||||
{
|
||||
if (current_task->entry_name == entry_name)
|
||||
{
|
||||
LOG_INFO(log, "Trying to process task " << entry_name << " again");
|
||||
LOG_INFO(log, "Trying to process task {} again", entry_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Task " << current_task->entry_name << " was deleted from ZooKeeper before current host committed it");
|
||||
LOG_INFO(log, "Task {} was deleted from ZooKeeper before current host committed it", current_task->entry_name);
|
||||
current_task = nullptr;
|
||||
}
|
||||
}
|
||||
@ -400,7 +398,7 @@ void DDLWorker::processTasks()
|
||||
String reason;
|
||||
if (!initAndCheckTask(entry_name, reason, zookeeper))
|
||||
{
|
||||
LOG_DEBUG(log, "Will not execute task " << entry_name << ": " << reason);
|
||||
LOG_DEBUG(log, "Will not execute task {}: {}", entry_name, reason);
|
||||
last_processed_task_name = entry_name;
|
||||
continue;
|
||||
}
|
||||
@ -426,8 +424,7 @@ void DDLWorker::processTasks()
|
||||
{
|
||||
if (server_startup && e.code == Coordination::ZNONODE)
|
||||
{
|
||||
LOG_WARNING(log, "ZooKeeper NONODE error during startup. Ignoring entry " <<
|
||||
task.entry_name << " (" << task.entry.query << ") : " << getCurrentExceptionMessage(true));
|
||||
LOG_WARNING(log, "ZooKeeper NONODE error during startup. Ignoring entry {} ({}) : {}", task.entry_name, task.entry.query, getCurrentExceptionMessage(true));
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -436,14 +433,13 @@ void DDLWorker::processTasks()
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_WARNING(log, "An error occurred while processing task " << task.entry_name << " (" << task.entry.query << ") : "
|
||||
<< getCurrentExceptionMessage(true));
|
||||
LOG_WARNING(log, "An error occurred while processing task {} ({}) : {}", task.entry_name, task.entry.query, getCurrentExceptionMessage(true));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_DEBUG(log, "Task " << task.entry_name << " (" << task.entry.query << ") has been already processed");
|
||||
LOG_DEBUG(log, "Task {} ({}) has been already processed", task.entry_name, task.entry.query);
|
||||
}
|
||||
|
||||
last_processed_task_name = task.entry_name;
|
||||
@ -510,8 +506,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
|
||||
if (found_exact_match)
|
||||
return;
|
||||
|
||||
LOG_WARNING(log, "Not found the exact match of host " << task.host_id.readableString() << " from task " << task.entry_name
|
||||
<< " in cluster " << task.cluster_name << " definition. Will try to find it using host name resolving.");
|
||||
LOG_WARNING(log, "Not found the exact match of host {} from task {} in cluster {} definition. Will try to find it using host name resolving.", task.host_id.readableString(), task.entry_name, task.cluster_name);
|
||||
|
||||
bool found_via_resolving = false;
|
||||
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num)
|
||||
@ -547,8 +542,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Resolved host " << task.host_id.readableString() << " from task " << task.entry_name
|
||||
<< " as host " << task.address_in_cluster.readableString() << " in definition of cluster " << task.cluster_name);
|
||||
LOG_INFO(log, "Resolved host {} from task {} as host {} in definition of cluster {}", task.host_id.readableString(), task.entry_name, task.address_in_cluster.readableString(), task.cluster_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -579,7 +573,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec
|
||||
}
|
||||
|
||||
status = ExecutionStatus(0);
|
||||
LOG_DEBUG(log, "Executed query: " << query);
|
||||
LOG_DEBUG(log, "Executed query: {}", query);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -601,7 +595,7 @@ void DDLWorker::attachToThreadGroup()
|
||||
|
||||
void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
|
||||
{
|
||||
LOG_DEBUG(log, "Processing task " << task.entry_name << " (" << task.entry.query << ")");
|
||||
LOG_DEBUG(log, "Processing task {} ({})", task.entry_name, task.entry.query);
|
||||
|
||||
String dummy;
|
||||
String active_node_path = task.entry_path + "/active/" + task.host_id_str;
|
||||
@ -631,7 +625,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
|
||||
|
||||
ASTPtr rewritten_ast = task.query_on_cluster->getRewrittenASTWithoutOnCluster(task.address_in_cluster.default_database);
|
||||
String rewritten_query = queryToString(rewritten_ast);
|
||||
LOG_DEBUG(log, "Executing query: " << rewritten_query);
|
||||
LOG_DEBUG(log, "Executing query: {}", rewritten_query);
|
||||
|
||||
if (auto * query_with_table = dynamic_cast<ASTQueryWithTableAndOutput *>(rewritten_ast.get()); query_with_table)
|
||||
{
|
||||
@ -701,7 +695,7 @@ void DDLWorker::checkShardConfig(const String & table, const DDLTask & task, Sto
|
||||
|
||||
if (dynamic_cast<const StorageDistributed *>(storage.get()))
|
||||
{
|
||||
LOG_TRACE(log, "Table " + backQuote(table) + " is distributed, skip checking config.");
|
||||
LOG_TRACE(log, "Table {} is distributed, skip checking config.", backQuote(table));
|
||||
return;
|
||||
}
|
||||
|
||||
@ -761,8 +755,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
|
||||
String executed_by;
|
||||
if (zookeeper->tryGet(is_executed_path, executed_by))
|
||||
{
|
||||
LOG_DEBUG(log, "Task " << task.entry_name << " has already been executed by leader replica ("
|
||||
<< executed_by << ") of the same shard.");
|
||||
LOG_DEBUG(log, "Task {} has already been executed by leader replica ({}) of the same shard.", task.entry_name, executed_by);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -863,7 +856,7 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
|
||||
/// Skip if there are active nodes (it is weak guard)
|
||||
if (zookeeper->exists(node_path + "/active", &stat) && stat.numChildren > 0)
|
||||
{
|
||||
LOG_INFO(log, "Task " << node_name << " should be deleted, but there are active workers. Skipping it.");
|
||||
LOG_INFO(log, "Task {} should be deleted, but there are active workers. Skipping it.", node_name);
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -872,14 +865,14 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
|
||||
auto lock = createSimpleZooKeeperLock(zookeeper, node_path, "lock", host_fqdn_id);
|
||||
if (!lock->tryLock())
|
||||
{
|
||||
LOG_INFO(log, "Task " << node_name << " should be deleted, but it is locked. Skipping it.");
|
||||
LOG_INFO(log, "Task {} should be deleted, but it is locked. Skipping it.", node_name);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (node_lifetime_is_expired)
|
||||
LOG_INFO(log, "Lifetime of task " << node_name << " is expired, deleting it");
|
||||
LOG_INFO(log, "Lifetime of task {} is expired, deleting it", node_name);
|
||||
else if (node_is_outside_max_window)
|
||||
LOG_INFO(log, "Task " << node_name << " is outdated, deleting it");
|
||||
LOG_INFO(log, "Task {} is outdated, deleting it", node_name);
|
||||
|
||||
/// Deleting
|
||||
{
|
||||
@ -901,7 +894,7 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_INFO(log, "An error occured while checking and cleaning task " + node_name + " from queue: " + getCurrentExceptionMessage(false));
|
||||
LOG_INFO(log, "An error occured while checking and cleaning task {} from queue: {}", node_name, getCurrentExceptionMessage(false));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -947,8 +940,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_INFO(log, "An error occurred while creating auxiliary ZooKeeper directories in " << node_path << " . They will be created later"
|
||||
<< ". Error : " << getCurrentExceptionMessage(true));
|
||||
LOG_INFO(log, "An error occurred while creating auxiliary ZooKeeper directories in {} . They will be created later. Error : {}", node_path, getCurrentExceptionMessage(true));
|
||||
}
|
||||
|
||||
return node_path;
|
||||
@ -1003,7 +995,7 @@ void DDLWorker::runMainThread()
|
||||
{
|
||||
if (Coordination::isHardwareError(e.code))
|
||||
{
|
||||
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));
|
||||
LOG_DEBUG(log, "Recovering ZooKeeper session after: {}", getCurrentExceptionMessage(false));
|
||||
|
||||
while (!stop_flag)
|
||||
{
|
||||
@ -1023,11 +1015,11 @@ void DDLWorker::runMainThread()
|
||||
}
|
||||
else if (e.code == Coordination::ZNONODE)
|
||||
{
|
||||
LOG_ERROR(log, "ZooKeeper error: " << getCurrentExceptionMessage(true));
|
||||
LOG_ERROR(log, "ZooKeeper error: {}", getCurrentExceptionMessage(true));
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_ERROR(log, "Unexpected ZooKeeper error: " << getCurrentExceptionMessage(true) << ". Terminating.");
|
||||
LOG_ERROR(log, "Unexpected ZooKeeper error: {}. Terminating.", getCurrentExceptionMessage(true));
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1225,7 +1217,7 @@ private:
|
||||
if (!ignoring_hosts.count(host))
|
||||
{
|
||||
ignoring_hosts.emplace(host);
|
||||
LOG_INFO(log, "Unexpected host " << host << " appeared " << " in task " << node_path);
|
||||
LOG_INFO(log, "Unexpected host {} appeared in task {}", host, node_path);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
@ -21,8 +21,7 @@ void DNSCacheUpdater::run()
|
||||
/// Reload cluster config if IP of any host has been changed since last update.
|
||||
if (resolver.updateCache())
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"),
|
||||
"IPs of some hosts have been changed. Will reload cluster config.");
|
||||
LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "IPs of some hosts have been changed. Will reload cluster config.");
|
||||
try
|
||||
{
|
||||
context.reloadClusterConfig();
|
||||
|
@ -572,7 +572,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
|
||||
else
|
||||
{
|
||||
/// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes)
|
||||
LOG_INFO(log, "Trying load partially dropped table " << table_id.getNameForLogs() << " from " << dropped_metadata_path);
|
||||
LOG_INFO(log, "Trying load partially dropped table {} from {}", table_id.getNameForLogs(), dropped_metadata_path);
|
||||
ASTPtr ast = DatabaseOnDisk::parseQueryFromMetadata(log, *global_context, dropped_metadata_path, /*throw_on_error*/ false, /*remove_empty*/false);
|
||||
auto * create = typeid_cast<ASTCreateQuery *>(ast.get());
|
||||
assert(!create || create->uuid == table_id.uuid);
|
||||
@ -597,9 +597,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "Cannot parse metadata of partially dropped table "
|
||||
<< table_id.getNameForLogs() << " from " << dropped_metadata_path
|
||||
<< ". Will remove metadata file and data directory. Garbage may be left in /store directory and ZooKeeper.");
|
||||
LOG_WARNING(log, "Cannot parse metadata of partially dropped table {} from {}. Will remove metadata file and data directory. Garbage may be left in /store directory and ZooKeeper.", table_id.getNameForLogs(), dropped_metadata_path);
|
||||
}
|
||||
|
||||
drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime();
|
||||
@ -635,7 +633,7 @@ void DatabaseCatalog::dropTableDataTask()
|
||||
if (it != tables_marked_dropped.end())
|
||||
{
|
||||
table = std::move(*it);
|
||||
LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs());
|
||||
LOG_INFO(log, "Will try drop {}", table.table_id.getNameForLogs());
|
||||
tables_marked_dropped.erase(it);
|
||||
}
|
||||
need_reschedule = !tables_marked_dropped.empty();
|
||||
@ -685,11 +683,11 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const
|
||||
Poco::File table_data_dir{data_path};
|
||||
if (table_data_dir.exists())
|
||||
{
|
||||
LOG_INFO(log, "Removing data directory " << data_path << " of dropped table " << table.table_id.getNameForLogs());
|
||||
LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path, table.table_id.getNameForLogs());
|
||||
table_data_dir.remove(true);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Removing metadata " << table.metadata_path << " of dropped table " << table.table_id.getNameForLogs());
|
||||
LOG_INFO(log, "Removing metadata {} of dropped table {}", table.metadata_path, table.table_id.getNameForLogs());
|
||||
Poco::File(table.metadata_path).remove();
|
||||
}
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user