mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
find {base,src,programs} -name '*.h' -or -name '*.cpp' | xargs grep -l -P 'LOG_[^\_(]+\([^,]+, "[^"]+" << [^<]+ << "[^"]+" << [^<]+ << "[^"]+" << [^<]+ << "[^"]+"\);' | xargs sed -i -r -e 's/(LOG_[^\_(]+)\(([^,]+), "([^"]+)" << ([^<]+) << "([^"]+)" << ([^<]+) << "([^"]+)" << ([^<]+) << "([^"]+)"\);/\1_FORMATTED(\2, "\3{}\5{}\7{}\9", \4, \6, \8);/'
This commit is contained in:
parent
7e2fb9ad65
commit
d5b7883451
@ -249,7 +249,7 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts)
|
|||||||
{
|
{
|
||||||
for (TaskTable & task_table : task_cluster->table_tasks)
|
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_FORMATTED(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())
|
if (task_table.all_shards.empty())
|
||||||
continue;
|
continue;
|
||||||
@ -1290,7 +1290,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
|
|||||||
|
|
||||||
if (count != 0)
|
if (count != 0)
|
||||||
{
|
{
|
||||||
LOG_INFO(log, "Partition " << task_partition.name << " piece " << current_piece_number << "is not empty. In contains " << count << " rows.");
|
LOG_INFO_FORMATTED(log, "Partition {} piece {}is not empty. In contains {} rows.", task_partition.name, current_piece_number, count);
|
||||||
Coordination::Stat stat_shards{};
|
Coordination::Stat stat_shards{};
|
||||||
zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards);
|
zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards);
|
||||||
|
|
||||||
|
@ -60,7 +60,7 @@ Block AggregatingBlockInputStream::readImpl()
|
|||||||
input_streams.emplace_back(temporary_inputs.back()->block_in);
|
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_FORMATTED(log, "Will merge {} temporary files of size {} MiB compressed, {} MiB uncompressed.", files.files.size(), (files.sum_size_compressed / 1048576.0), (files.sum_size_uncompressed / 1048576.0));
|
||||||
|
|
||||||
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(input_streams, params, final, 1, 1);
|
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(input_streams, params, final, 1, 1);
|
||||||
}
|
}
|
||||||
|
@ -82,7 +82,7 @@ Block ParallelAggregatingBlockInputStream::readImpl()
|
|||||||
input_streams.emplace_back(temporary_inputs.back()->block_in);
|
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_FORMATTED(log, "Will merge {} temporary files of size {} MiB compressed, {} MiB uncompressed.", files.files.size(), (files.sum_size_compressed / 1048576.0), (files.sum_size_uncompressed / 1048576.0));
|
||||||
|
|
||||||
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(
|
impl = std::make_unique<MergingAggregatedMemoryEfficientBlockInputStream>(
|
||||||
input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads);
|
input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads);
|
||||||
|
@ -466,7 +466,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * loger, const Contex
|
|||||||
table_name = unescapeForFileName(table_name);
|
table_name = unescapeForFileName(table_name);
|
||||||
|
|
||||||
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)
|
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_FORMATTED(loger, "File {} contains both UUID and table name. Will use name `{}` instead of `{}`", metadata_file_path, table_name, create.table);
|
||||||
create.table = table_name;
|
create.table = table_name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -52,7 +52,7 @@ VolumeJBOD::VolumeJBOD(
|
|||||||
}
|
}
|
||||||
static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
|
static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
|
||||||
if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
|
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_FORMATTED(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), formatReadableSizeWithBinarySuffix(max_data_part_size), formatReadableSizeWithBinarySuffix(MIN_PART_SIZE));
|
||||||
}
|
}
|
||||||
|
|
||||||
DiskPtr VolumeJBOD::getNextDisk()
|
DiskPtr VolumeJBOD::getNextDisk()
|
||||||
|
@ -334,7 +334,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason,
|
|||||||
if (host_in_hostlist)
|
if (host_in_hostlist)
|
||||||
{
|
{
|
||||||
/// This check could be slow a little bit
|
/// 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_FORMATTED(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
|
else
|
||||||
{
|
{
|
||||||
@ -506,7 +506,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
|
|||||||
if (found_exact_match)
|
if (found_exact_match)
|
||||||
return;
|
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_FORMATTED(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;
|
bool found_via_resolving = false;
|
||||||
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num)
|
for (size_t shard_num = 0; shard_num < shards.size(); ++shard_num)
|
||||||
|
@ -1037,7 +1037,7 @@ private:
|
|||||||
}
|
}
|
||||||
if (info->loading_id != loading_id)
|
if (info->loading_id != loading_id)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Next update time for '" << name << "' will not be set because this object's current loading_id " << info->loading_id << " is different from the specified " << loading_id << ".");
|
LOG_TRACE_FORMATTED(log, "Next update time for '{}' will not be set because this object's current loading_id {} is different from the specified {}.", name, info->loading_id, loading_id);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -589,7 +589,7 @@ void AggregatingTransform::initGenerate()
|
|||||||
for (const auto & file : files.files)
|
for (const auto & file : files.files)
|
||||||
processors.emplace_back(std::make_unique<SourceFromNativeStream>(header, file->path()));
|
processors.emplace_back(std::make_unique<SourceFromNativeStream>(header, file->path()));
|
||||||
|
|
||||||
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_FORMATTED(log, "Will merge {} temporary files of size {} MiB compressed, {} MiB uncompressed.", files.files.size(), (files.sum_size_compressed / 1048576.0), (files.sum_size_uncompressed / 1048576.0));
|
||||||
|
|
||||||
auto pipe = createMergingAggregatedMemoryEfficientPipe(
|
auto pipe = createMergingAggregatedMemoryEfficientPipe(
|
||||||
header, params, files.files.size(), temporary_data_merge_threads);
|
header, params, files.files.size(), temporary_data_merge_threads);
|
||||||
|
@ -179,7 +179,7 @@ bool MergeTreePartsMover::selectPartsForMove(
|
|||||||
|
|
||||||
if (!parts_to_move.empty())
|
if (!parts_to_move.empty())
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Selected " << parts_to_move_by_policy_rules << " parts to move according to storage policy rules and " << parts_to_move_by_ttl_rules << " parts according to TTL rules, " << formatReadableSizeWithBinarySuffix(parts_to_move_total_size_bytes) << " total");
|
LOG_TRACE_FORMATTED(log, "Selected {} parts to move according to storage policy rules and {} parts according to TTL rules, {} total", parts_to_move_by_policy_rules, parts_to_move_by_ttl_rules, formatReadableSizeWithBinarySuffix(parts_to_move_total_size_bytes));
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -365,7 +365,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep
|
|||||||
|
|
||||||
auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
|
auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
|
||||||
if (code)
|
if (code)
|
||||||
LOG_ERROR(log, "Couldn't remove " << replica_path << "/queue/" << entry->znode_name << ": " << zkutil::ZooKeeper::error2string(code) << ". This shouldn't happen often.");
|
LOG_ERROR_FORMATTED(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, zkutil::ZooKeeper::error2string(code));
|
||||||
|
|
||||||
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed);
|
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed);
|
||||||
}
|
}
|
||||||
@ -1337,7 +1337,7 @@ MutationCommands ReplicatedMergeTreeQueue::getMutationCommands(
|
|||||||
auto in_partition = mutations_by_partition.find(part->info.partition_id);
|
auto in_partition = mutations_by_partition.find(part->info.partition_id);
|
||||||
if (in_partition == mutations_by_partition.end())
|
if (in_partition == mutations_by_partition.end())
|
||||||
{
|
{
|
||||||
LOG_WARNING(log, "There are no mutations for partition ID " << part->info.partition_id << " (trying to mutate part " << part->name << " to " << toString(desired_mutation_version) << ")");
|
LOG_WARNING_FORMATTED(log, "There are no mutations for partition ID {} (trying to mutate part {} to {})", part->info.partition_id, part->name, toString(desired_mutation_version));
|
||||||
return MutationCommands{};
|
return MutationCommands{};
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1885,7 +1885,7 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const ReplicatedMerge
|
|||||||
partition_it->second.begin(), partition_it->second.lower_bound(block_num));
|
partition_it->second.begin(), partition_it->second.lower_bound(block_num));
|
||||||
if (blocks_count)
|
if (blocks_count)
|
||||||
{
|
{
|
||||||
LOG_TRACE(queue.log, "Mutation " << mutation.znode_name << " is not done yet because in partition ID " << partition_id << " there are still " << blocks_count << " uncommitted blocks.");
|
LOG_TRACE_FORMATTED(queue.log, "Mutation {} is not done yet because in partition ID {} there are still {} uncommitted blocks.", mutation.znode_name, partition_id, blocks_count);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1003,7 +1003,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
|
|||||||
}
|
}
|
||||||
if (part->name != name)
|
if (part->name != name)
|
||||||
{
|
{
|
||||||
LOG_WARNING(log, "Part " << name << " is covered by " << part->name << " but should be merged into " << entry.new_part_name << ". This shouldn't happen often.");
|
LOG_WARNING_FORMATTED(log, "Part {} is covered by {} but should be merged into {}. This shouldn't happen often.", name, part->name, entry.new_part_name);
|
||||||
have_all_parts = false;
|
have_all_parts = false;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
@ -1370,7 +1370,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
|
|||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
LOG_WARNING(log, "No active replica has part " << entry.new_part_name << ", but that part needs quorum and /quorum/status contains entry about another part " << quorum_entry.part_name << ". It means that part was successfully written to " << entry.quorum << " replicas, but then all of them goes offline. Or it is a bug.");
|
LOG_WARNING_FORMATTED(log, "No active replica has part {}, but that part needs quorum and /quorum/status contains entry about another part {}. It means that part was successfully written to {} replicas, but then all of them goes offline. Or it is a bug.", entry.new_part_name, quorum_entry.part_name, entry.quorum);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -2187,7 +2187,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask()
|
|||||||
size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second;
|
size_t merges_and_mutations_sum = merges_and_mutations_queued.first + merges_and_mutations_queued.second;
|
||||||
if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue)
|
if (merges_and_mutations_sum >= storage_settings_ptr->max_replicated_merges_in_queue)
|
||||||
{
|
{
|
||||||
LOG_TRACE(log, "Number of queued merges (" << merges_and_mutations_queued.first << ") and part mutations (" << merges_and_mutations_queued.second << ") is greater than max_replicated_merges_in_queue (" << storage_settings_ptr->max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
|
LOG_TRACE_FORMATTED(log, "Number of queued merges ({}) and part mutations ({}) is greater than max_replicated_merges_in_queue ({}), so won't select new parts to merge or mutate.", merges_and_mutations_queued.first, merges_and_mutations_queued.second, storage_settings_ptr->max_replicated_merges_in_queue);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -4305,7 +4305,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
|
|||||||
if (best_replica.empty())
|
if (best_replica.empty())
|
||||||
throw Exception("Logical error: cannot choose best replica.", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Logical error: cannot choose best replica.", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
LOG_INFO(log, "Found " << replicas.size() << " replicas, " << active_replicas.size() << " of them are active. Selected " << best_replica << " to fetch from.");
|
LOG_INFO_FORMATTED(log, "Found {} replicas, {} of them are active. Selected {} to fetch from.", replicas.size(), active_replicas.size(), best_replica);
|
||||||
|
|
||||||
String best_replica_path = from + "/replicas/" + best_replica;
|
String best_replica_path = from + "/replicas/" + best_replica;
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user