find {base,src,programs} -name '*.h' -or -name '*.cpp' | xargs grep -l -P 'LOG_\w+\([^,]+, "[^"]+" << [^<]+ << "[^"]+" << [^<]+\);' | xargs sed -i -r -e 's/(LOG_\w+)\(([^,]+), "([^"]+)" << ([^<]+) << "([^"]+)" << ([^<]+)\);/\1_FORMATTED(\2, "\3{}\5{}", \4, \6);/'

This commit is contained in:
Alexey Milovidov 2020-05-23 19:58:15 +03:00
parent e391b77d81
commit 8042e5febe
31 changed files with 58 additions and 58 deletions

View File

@ -734,7 +734,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_FORMATTED(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText());
ServerApplication::terminate();
}

View File

@ -169,7 +169,7 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts,
<< task_shard->getDescription() << " :" << ss.str());
}
LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription());
LOG_DEBUG_FORMATTED(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)
@ -911,7 +911,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_FORMATTED(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 +924,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_FORMATTED(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name);
continue;
}
}
@ -1744,7 +1744,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_FORMATTED(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription());
return res;
}

View File

@ -198,7 +198,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_FORMATTED(log, "Waiting for {} connections, try {}", server.currentConnections(), count);
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
}
});

View File

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

View File

@ -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_FORMATTED(trace_log, "Access {}: {}", (is_granted ? "granted" : "denied"), (AccessRightsElement{flags, args...}.toString()));
if (is_granted)
return true;

View File

@ -623,7 +623,7 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
}
catch (Poco::Exception & e)
{
LOG_WARNING(log, "Couldn't save preprocessed config to " << preprocessed_path << ": " << e.displayText());
LOG_WARNING_FORMATTED(log, "Couldn't save preprocessed config to {}: {}", preprocessed_path, e.displayText());
}
}

View File

@ -134,7 +134,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_FORMATTED(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;
@ -194,7 +194,7 @@ int ShellCommand::tryWait()
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_FORMATTED(getLogger(), "Wait for shell command pid {} completed with status {}", pid, status);
if (WIFEXITED(status))
return WEXITSTATUS(status);

View File

@ -43,7 +43,7 @@ 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_FORMATTED(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents);
else
LOG_INFO_FORMATTED(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path);
}
@ -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_FORMATTED(&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_FORMATTED(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE));
}
}

View File

@ -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_FORMATTED(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message);
return lock_status;
}

View File

@ -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_FORMATTED(log, "Cannot use ZooKeeper host {}, reason: {}", host_string, e.displayText());
}
}

View File

@ -135,7 +135,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_FORMATTED(log, "Logical error: CollapsingFinalBlockInputStream has output {} blocks instead of {}", blocks_output, blocks_fetched);
return Block();
}

View File

@ -372,7 +372,7 @@ void RemoteBlockInputStream::tryCancel(const char * reason)
multiplexed_connections->sendCancel();
}
LOG_TRACE(log, "(" << multiplexed_connections->dumpAddresses() << ") " << reason);
LOG_TRACE_FORMATTED(log, "({}) {}", multiplexed_connections->dumpAddresses(), reason);
}
bool RemoteBlockInputStream::isQueryPending() const

View File

@ -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_FORMATTED(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name);
}
void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block)

View File

@ -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_FORMATTED(log, "Invalidate query has returned: {}, previous value: {}", response, invalidate_query_response);
if (invalidate_query_response == response)
return false;
invalidate_query_response = response;

View File

@ -163,7 +163,7 @@ namespace
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds " << toString() << " size = " << ids.size());
LOG_TRACE_FORMATTED(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_FORMATTED(log, "loadKeys {} size = {}", toString(), requested_rows.size());
auto process = ShellCommand::execute(command);
auto output_stream = context.getOutputFormat(format, process->in, sample_block);

View File

@ -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_FORMATTED(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_FORMATTED(log, "loadKeys {} size = {}", toString(), requested_rows.size());
ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr)
{

View File

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

View File

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

View File

@ -400,7 +400,7 @@ void DDLWorker::processTasks()
String reason;
if (!initAndCheckTask(entry_name, reason, zookeeper))
{
LOG_DEBUG(log, "Will not execute task " << entry_name << ": " << reason);
LOG_DEBUG_FORMATTED(log, "Will not execute task {}: {}", entry_name, reason);
last_processed_task_name = entry_name;
continue;
}

View File

@ -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_FORMATTED(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);
@ -685,11 +685,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_FORMATTED(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_FORMATTED(log, "Removing metadata {} of dropped table {}", table.metadata_path, table.table_id.getNameForLogs());
Poco::File(table.metadata_path).remove();
}

View File

@ -229,7 +229,7 @@ void DistributedBlockOutputStream::waitForJobs()
size_t num_finished_jobs = finished_jobs_count;
if (num_finished_jobs < jobs_count)
LOG_WARNING(log, "Expected " << jobs_count << " writing jobs, but finished only " << num_finished_jobs);
LOG_WARNING_FORMATTED(log, "Expected {} writing jobs, but finished only {}", jobs_count, num_finished_jobs);
}

View File

@ -329,7 +329,7 @@ bool ReadBufferFromKafkaConsumer::nextImpl()
{
messages = std::move(new_messages);
current = messages.begin();
LOG_TRACE(log, "Polled batch of " << messages.size() << " messages. Offset position: " << consumer->get_offsets_position(consumer->get_assignment()));
LOG_TRACE_FORMATTED(log, "Polled batch of {} messages. Offset position: {}", messages.size(), consumer->get_offsets_position(consumer->get_assignment()));
break;
}
}

View File

@ -721,7 +721,7 @@ void IMergeTreeDataPart::remove() const
}
catch (...)
{
LOG_ERROR(storage.log, "Cannot recursively remove directory " << fullPath(volume->getDisk(), to) << ". Exception: " << getCurrentExceptionMessage(false));
LOG_ERROR_FORMATTED(storage.log, "Cannot recursively remove directory {}. Exception: {}", fullPath(volume->getDisk(), to), getCurrentExceptionMessage(false));
throw;
}
}

View File

@ -1838,7 +1838,7 @@ void MergeTreeData::renameTempPartAndReplace(
if (covering_part)
{
LOG_WARNING(log, "Tried to add obsolete part " << part_name << " covered by " << covering_part->getNameWithState());
LOG_WARNING_FORMATTED(log, "Tried to add obsolete part {} covered by {}", part_name, covering_part->getNameWithState());
return;
}
@ -2802,7 +2802,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
else
{
String partition_id = getPartitionIDFromQuery(partition, context);
LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
LOG_DEBUG_FORMATTED(log, "Looking for parts for partition {} in {}", partition_id, source_dir);
ActiveDataPartSet active_parts(format_version);
const auto disks = getStoragePolicy()->getDisks();
@ -3205,7 +3205,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
if (disk->exists(dst_part_path))
throw Exception("Part in " + fullPath(disk, dst_part_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
LOG_DEBUG(log, "Cloning part " << fullPath(disk, src_part_path) << " to " << fullPath(disk, dst_part_path));
LOG_DEBUG_FORMATTED(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path));
localBackup(disk, src_part_path, dst_part_path);
disk->removeIfExists(dst_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH);

View File

@ -268,7 +268,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge(
if (prev_part && part->info.partition_id == (*prev_part)->info.partition_id
&& part->info.min_block <= (*prev_part)->info.max_block)
{
LOG_ERROR(log, "Part " << part->name << " intersects previous part " << (*prev_part)->name);
LOG_ERROR_FORMATTED(log, "Part {} intersects previous part {}", part->name, (*prev_part)->name);
}
prev_part = &part;
@ -1000,12 +1000,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading))
{
LOG_TRACE(log, "Part " << source_part->name << " doesn't change up to mutation version " << future_part.part_info.mutation);
LOG_TRACE_FORMATTED(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation);
return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info);
}
else
{
LOG_TRACE(log, "Mutating part " << source_part->name << " to mutation version " << future_part.part_info.mutation);
LOG_TRACE_FORMATTED(log, "Mutating part {} to mutation version {}", source_part->name, future_part.part_info.mutation);
}
BlockInputStreamPtr in = nullptr;

View File

@ -199,7 +199,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt
auto single_disk_volume = std::make_shared<SingleDiskVolume>("volume_" + moving_part.part->name, moving_part.reserved_space->getDisk());
MergeTreeData::MutableDataPartPtr cloned_part =
data->createPart(moving_part.part->name, single_disk_volume, "detached/" + moving_part.part->name);
LOG_TRACE(log, "Part " << moving_part.part->name << " was cloned to " << cloned_part->getFullPath());
LOG_TRACE_FORMATTED(log, "Part {} was cloned to {}", moving_part.part->name, cloned_part->getFullPath());
cloned_part->loadColumnsChecksumsIndexes(true, true);
return cloned_part;
@ -228,7 +228,7 @@ void MergeTreePartsMover::swapClonedPart(const MergeTreeData::DataPartPtr & clon
/// TODO what happen if server goes down here?
data->swapActivePart(cloned_part);
LOG_TRACE(log, "Part " << cloned_part->name << " was moved to " << cloned_part->getFullPath());
LOG_TRACE_FORMATTED(log, "Part {} was moved to {}", cloned_part->name, cloned_part->getFullPath());
}
}

View File

@ -298,7 +298,7 @@ CheckResult ReplicatedMergeTreePartCheckThread::checkPart(const String & part_na
{
/// If we have a covering part, ignore all the problems with this part.
/// In the worst case, errors will still appear `old_parts_lifetime` seconds in error log until the part is removed as the old one.
LOG_WARNING(log, "We have part " << part->name << " covering part " << part_name);
LOG_WARNING_FORMATTED(log, "We have part {} covering part {}", part->name, part_name);
}
return {part_name, true, ""};

View File

@ -1434,7 +1434,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
it->second.is_done = true;
if (entry->isAlterMutation())
{
LOG_TRACE(log, "Finishing data alter with version " << entry->alter_version << " for entry " << entry->znode_name);
LOG_TRACE_FORMATTED(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name);
alter_sequence.finishDataAlter(entry->alter_version, lock);
}
}

View File

@ -215,7 +215,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
}
catch (const Coordination::Exception & e)
{
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true));
LOG_ERROR_FORMATTED(log, "Couldn't start replication: {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
return false;
}
catch (const Exception & e)
@ -223,7 +223,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup()
if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
throw;
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true));
LOG_ERROR_FORMATTED(log, "Couldn't start replication: {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
return false;
}
}

View File

@ -534,7 +534,7 @@ void StorageMergeTree::loadMutations()
{
MergeTreeMutationEntry entry(disk, path, it->name());
Int64 block_number = entry.block_number;
LOG_DEBUG(log, "Loading mutation: " << it->name() << " entry, commands size: " << entry.commands.size());
LOG_DEBUG_FORMATTED(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size());
auto insertion = current_mutations_by_id.emplace(it->name(), std::move(entry));
current_mutations_by_version.emplace(block_number, insertion.first->second);
}
@ -1071,7 +1071,7 @@ void StorageMergeTree::attachPartition(const ASTPtr & partition, bool attach_par
for (size_t i = 0; i < loaded_parts.size(); ++i)
{
LOG_INFO(log, "Attaching part " << loaded_parts[i]->name << " from " << renamed_parts.old_and_new_names[i].second);
LOG_INFO_FORMATTED(log, "Attaching part {} from {}", loaded_parts[i]->name, renamed_parts.old_and_new_names[i].second);
renameTempPartAndAdd(loaded_parts[i], &increment);
renamed_parts.old_and_new_names[i].first.clear();
LOG_INFO_FORMATTED(log, "Finished attaching part");

View File

@ -1036,7 +1036,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
String replica = findReplicaHavingPart(entry.new_part_name, true); /// NOTE excessive ZK requests for same data later, may remove.
if (!replica.empty())
{
LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
LOG_DEBUG_FORMATTED(log, "Prefer to fetch {} from replica {}", entry.new_part_name, replica);
return false;
}
}
@ -1151,7 +1151,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
{
const String & source_part_name = entry.source_parts.at(0);
const auto storage_settings_ptr = getSettings();
LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name);
LOG_TRACE_FORMATTED(log, "Executing log entry to mutate part {} to {}", source_part_name, entry.new_part_name);
DataPartPtr source_part = getActiveContainingPart(source_part_name);
if (!source_part)
@ -1178,7 +1178,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
String replica = findReplicaHavingPart(entry.new_part_name, true); /// NOTE excessive ZK requests for same data later, may remove.
if (!replica.empty())
{
LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
LOG_DEBUG_FORMATTED(log, "Prefer to fetch {} from replica {}", entry.new_part_name, replica);
return false;
}
}
@ -1623,7 +1623,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
auto src_part = src_data->getPartIfExists(part_desc->src_part_info, valid_states);
if (!src_part)
{
LOG_DEBUG(log, "There is no part " << part_desc->src_part_name << " in " << source_table_id.getNameForLogs());
LOG_DEBUG_FORMATTED(log, "There is no part {} in {}", part_desc->src_part_name, source_table_id.getNameForLogs());
continue;
}
@ -2793,7 +2793,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
currently_fetching_parts.erase(part_name);
});
LOG_DEBUG(log, "Fetching part " << part_name << " from " << source_replica_path);
LOG_DEBUG_FORMATTED(log, "Fetching part {} from {}", part_name, source_replica_path);
TableStructureReadLockHolder table_lock_holder;
if (!to_detached)
@ -2841,7 +2841,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
if (source_part_checksums == desired_checksums)
{
LOG_TRACE(log, "Found local part " << source_part->name << " with the same checksums as " << part_name);
LOG_TRACE_FORMATTED(log, "Found local part {} with the same checksums as {}", source_part->name, part_name);
part_to_clone = source_part;
}
}
@ -2903,7 +2903,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin
for (const auto & replaced_part : replaced_parts)
{
LOG_DEBUG(log, "Part " << replaced_part->name << " is rendered obsolete by fetching part " << part_name);
LOG_DEBUG_FORMATTED(log, "Part {} is rendered obsolete by fetching part {}", replaced_part->name, part_name);
ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
}
@ -3682,7 +3682,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool
String old_name = loaded_parts[i]->name;
output.writeExistingPart(loaded_parts[i]);
renamed_parts.old_and_new_names[i].first.clear();
LOG_DEBUG(log, "Attached part " << old_name << " as " << loaded_parts[i]->name);
LOG_DEBUG_FORMATTED(log, "Attached part {} as {}", old_name, loaded_parts[i]->name);
}
}
@ -4271,7 +4271,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const
if (from.back() == '/')
from.resize(from.size() - 1);
LOG_INFO(log, "Will fetch partition " << partition_id << " from shard " << from_);
LOG_INFO_FORMATTED(log, "Will fetch partition {} from shard {}", partition_id, from_);
/** Let's check that there is no such partition in the `detached` directory (where we will write the downloaded parts).
* Unreliable (there is a race condition) - such a partition may appear a little later.
@ -4869,7 +4869,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
"Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring.");
}
LOG_TRACE(log, "Deleted " << to_delete_futures.size() << " deduplication block IDs in partition ID " << partition_id);
LOG_TRACE_FORMATTED(log, "Deleted {} deduplication block IDs in partition ID {}", to_delete_futures.size(), partition_id);
}
void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace,