Progress on task

This commit is contained in:
Alexey Milovidov 2020-05-23 20:52:11 +03:00
parent 3a09f9ca1c
commit 9d24908e53
18 changed files with 70 additions and 83 deletions

View File

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

View File

@ -308,10 +308,10 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
if (e.code() != ErrorCodes::DNS_ERROR)
throw;
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
LOG_WARNING_FORMATTED(
&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;
}
};
@ -341,10 +341,10 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
if (e.code() != ErrorCodes::DNS_ERROR)
throw;
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
LOG_WARNING_FORMATTED(
&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;
}
};

View File

@ -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_FORMATTED(log_, "{}: {}{}", user_name, msg, formatSkippedMessage(args...));
};
if (!user)
@ -451,13 +451,16 @@ 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_FORMATTED(trace_log, "List of all grants: {} WITH GRANT OPTION", merged_access->toString());
else
LOG_TRACE_FORMATTED(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_FORMATTED(trace_log, "Current_roles: {}, enabled_roles: {}",
boost::algorithm::join(roles_info->getCurrentRolesNames(), ", "),
boost::algorithm::join(roles_info->getEnabledRolesNames(), ", "));
}
LOG_TRACE_FORMATTED(trace_log, "Settings: readonly={}, allow_ddl={}, allow_introspection_functions={}", readonly_, allow_ddl_, allow_introspection_);
}

View File

@ -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_FORMATTED(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_FORMATTED(log_wrapper.get(), "Connected to {} server version {}.{}.{}.",
server_name, server_version_major, server_version_minor, server_version_patch);
}
catch (Poco::Net::NetException & e)
{
@ -324,7 +321,7 @@ bool Connection::ping()
}
catch (const Poco::Exception & e)
{
LOG_TRACE(log_wrapper.get(), e.displayText());
LOG_TRACE_FORMATTED(log_wrapper.get(), e.displayText());
return false;
}

View File

@ -179,8 +179,8 @@ 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"
LOG_TRACE(log,
"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., "
@ -189,8 +189,8 @@ void ParallelAggregatingBlockInputStream::execute()
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)"
LOG_TRACE(log,
"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.)");

View File

@ -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_FORMATTED(log, "{}%", processed * 100.0 / total);
watch.restart();
}
}

View File

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

View File

@ -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_FORMATTED(&Poco::Logger::get("DictionaryFactory"), "Created dictionary source '{}' for dictionary '{}'", source_ptr->toString(), name);
const auto & layout_type = keys.front();

View File

@ -213,7 +213,7 @@ LocalDateTime MySQLDictionarySource::getLastModification(mysqlxx::Pool::Entry &
{
auto query = connection->query("SHOW TABLE STATUS LIKE " + quoteForLike(table));
LOG_TRACE(log, query.str());
LOG_TRACE_FORMATTED(log, query.str());
auto result = query.use();

View File

@ -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_FORMATTED(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.",
formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space));
++reservation_count;
reserved_bytes += bytes;
return true;

View File

@ -99,7 +99,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig(
}
writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning);
LOG_WARNING(&Logger::get("DiskSelector"), warning.str());
LOG_WARNING_FORMATTED(&Logger::get("DiskSelector"), warning.str());
}
return result;

View File

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

View File

@ -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_FORMATTED(&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);
}
@ -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_FORMATTED(&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);
}
@ -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_FORMATTED(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.",
formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space));
++reservation_count;
reserved_bytes += bytes;
return true;

View File

@ -761,8 +761,8 @@ 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., "
LOG_TRACE(log,
"Written part in " << elapsed_seconds << " sec., "
<< rows << " rows, "
<< (uncompressed_bytes / 1048576.0) << " MiB uncompressed, "
<< (compressed_bytes / 1048576.0) << " MiB compressed, "
@ -867,8 +867,8 @@ 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, "
LOG_TRACE(log,
"Max size of temporary block: " << max_temporary_block_size_rows << " rows, "
<< (max_temporary_block_size_bytes / 1048576.0) << " MiB.");
}
@ -939,8 +939,8 @@ 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)"
LOG_TRACE(log,
"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.)");
}
@ -1308,8 +1308,8 @@ 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. "
LOG_TRACE(log,
"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.)");
@ -2175,8 +2175,8 @@ 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. "
LOG_TRACE(log,
"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.)");

View File

@ -547,12 +547,12 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
#ifndef NDEBUG
auto & context = executor_contexts[thread_num];
LOG_TRACE(log, std::fixed << std::setprecision(3)
<< "Thread finished."
<< " Total time: " << (context->total_time_ns / 1e9) << " sec."
<< " Execution time: " << (context->execution_time_ns / 1e9) << " sec."
<< " Processing time: " << (context->processing_time_ns / 1e9) << " sec."
<< " Wait time: " << (context->wait_time_ns / 1e9) << " sec.");
LOG_TRACE(log,
"Thread finished."
<< " Total time: " << (context->total_time_ns / 1e9) << " sec."
<< " Execution time: " << (context->execution_time_ns / 1e9) << " sec."
<< " Processing time: " << (context->processing_time_ns / 1e9) << " sec."
<< " Wait time: " << (context->wait_time_ns / 1e9) << " sec.");
#endif
}

View File

@ -540,10 +540,10 @@ void AggregatingTransform::initGenerate()
double elapsed_seconds = watch.elapsedSeconds();
size_t rows = variants.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. " << 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.)");
if (params->aggregator.hasTemporaryFiles())
{

View File

@ -949,8 +949,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
/// Print overall profiling info. NOTE: it may duplicates previous messages
{
double elapsed_seconds = merge_entry->watch.elapsedSeconds();
LOG_DEBUG(log, std::fixed << std::setprecision(2)
<< "Merge sorted " << merge_entry->rows_read << " rows"
LOG_DEBUG(log,
"Merge sorted " << merge_entry->rows_read << " rows"
<< ", containing " << all_column_names.size() << " columns"
<< " (" << merging_column_names.size() << " merged, " << gathering_column_names.size() << " gathered)"
<< " in " << elapsed_seconds << " sec., "

View File

@ -166,8 +166,8 @@ void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInf
++backoff_state.num_events;
ProfileEvents::increment(ProfileEvents::SlowRead);
LOG_DEBUG(log, std::fixed << std::setprecision(3)
<< "Slow read, event №" << backoff_state.num_events
LOG_DEBUG(log,
"Slow read, event №" << backoff_state.num_events
<< ": read " << info.bytes_read << " bytes in " << info.nanoseconds / 1000000000.0 << " sec., "
<< info.bytes_read * 1000.0 / info.nanoseconds << " MB/s.");