Progress on task

This commit is contained in:
Alexey Milovidov 2020-05-23 22:55:06 +03:00
parent f13ccfc8e8
commit cd6d3c7509
8 changed files with 11 additions and 11 deletions

View File

@ -52,7 +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_FORMATTED(&Poco::Util::Application::instance().logger(), "Fail to write to Graphite {}:{}. e.what() = {}, e.message() = {}", host, port, e.what(), e.message());
}
}

View File

@ -132,7 +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_FORMATTED(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id);
}
}
}
@ -353,7 +353,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_FORMATTED(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);
@ -1297,7 +1297,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
/// 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_FORMATTED(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;
@ -1757,7 +1757,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_FORMATTED(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();

View File

@ -1196,7 +1196,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_FORMATTED(log, "Poco::Exception. Code: {}, e.code() = {}, e.displayText() = {}, e.what() = {}", ErrorCodes::POCO_EXCEPTION, e.code(), e.displayText(), e.what());
}
else
throw;

View File

@ -248,7 +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_FORMATTED(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay);
ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica);
}
}

View File

@ -129,7 +129,7 @@ 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_FORMATTED(log, "Writing part. Bucket: {}, Key: {}, Upload_id: {}, Data size: {}", bucket, key, upload_id, data.size());
if (outcome.IsSuccess())
{

View File

@ -542,7 +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_FORMATTED(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);
}
}

View File

@ -885,7 +885,7 @@ private:
info.loading_start_time = std::chrono::system_clock::now();
info.loading_end_time = TimePoint{};
LOG_TRACE(log, "Will load the object '" << info.name << "' " << (enable_async_loading ? std::string("in background") : "immediately") << ", force = " << forced_to_reload << ", loading_id = " << info.loading_id);
LOG_TRACE_FORMATTED(log, "Will load the object '{}' {}, force = {}, loading_id = {}", info.name, (enable_async_loading ? std::string("in background") : "immediately"), forced_to_reload, info.loading_id);
if (enable_async_loading)
{

View File

@ -583,7 +583,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor
const MergeTreeData::DataPartsVector & parts = future_part.parts;
LOG_DEBUG(log, "Merging " << parts.size() << " parts: from " << parts.front()->name << " to " << parts.back()->name << " into " << TMP_PREFIX + future_part.name + " with type " + future_part.type.toString());
LOG_DEBUG_FORMATTED(log, "Merging {} parts: from {} to {} into {}", parts.size(), parts.front()->name, parts.back()->name, future_part.type.toString());
auto disk = space_reservation->getDisk();
String part_path = data.relative_data_path;