Merge pull request #23160 from azat/logging-v2

[RFC] Change logging from trace to debug for messages with rows/bytes
This commit is contained in:
alexey-milovidov 2021-04-16 21:18:32 +03:00 committed by GitHub
commit 74a0ac6917
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 23 additions and 23 deletions

View File

@ -834,7 +834,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co
ProfileEvents::increment(ProfileEvents::ExternalAggregationCompressedBytes, compressed_bytes);
ProfileEvents::increment(ProfileEvents::ExternalAggregationUncompressedBytes, uncompressed_bytes);
LOG_TRACE(log,
LOG_DEBUG(log,
"Written part in {} sec., {} rows, {} uncompressed, {} compressed,"
" {} uncompressed bytes per row, {} compressed bytes per row, compression rate: {}"
" ({} rows/sec., {}/sec. uncompressed, {}/sec. compressed)",
@ -947,7 +947,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, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes));
LOG_DEBUG(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes));
}
@ -1481,7 +1481,7 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
}
double elapsed_seconds = watch.elapsedSeconds();
LOG_TRACE(log,
LOG_DEBUG(log,
"Converted aggregated data to blocks. {} rows, {} in {} sec. ({} rows/sec., {}/sec.)",
rows, ReadableSize(bytes),
elapsed_seconds, rows / elapsed_seconds,
@ -2109,7 +2109,7 @@ 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, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({} rows/sec., {}/sec.)",
LOG_DEBUG(log, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({} rows/sec., {}/sec.)",
rows, ReadableSize(bytes),
elapsed_seconds, rows / elapsed_seconds,
ReadableSize(bytes / elapsed_seconds));

View File

@ -190,7 +190,7 @@ Chunk IRowInputFormat::generate()
if (num_errors && (params.allow_errors_num > 0 || params.allow_errors_ratio > 0))
{
Poco::Logger * log = &Poco::Logger::get("IRowInputFormat");
LOG_TRACE(log, "Skipped {} rows with errors while reading the input stream", num_errors);
LOG_DEBUG(log, "Skipped {} rows with errors while reading the input stream", num_errors);
}
readSuffix();

View File

@ -214,8 +214,8 @@ IProcessor::Status AggregatingInOrderTransform::prepare()
{
output.push(std::move(to_push_chunk));
output.finish();
LOG_TRACE(log, "Aggregated. {} to {} rows (from {})", src_rows, res_rows,
formatReadableSizeWithBinarySuffix(src_bytes));
LOG_DEBUG(log, "Aggregated. {} to {} rows (from {})",
src_rows, res_rows, formatReadableSizeWithBinarySuffix(src_bytes));
return Status::Finished;
}
if (input.isFinished())

View File

@ -541,7 +541,7 @@ void AggregatingTransform::initGenerate()
double elapsed_seconds = watch.elapsedSeconds();
size_t rows = variants.sizeWithoutOverflowRow();
LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)",
LOG_DEBUG(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)",
src_rows, rows, ReadableSize(src_bytes),
elapsed_seconds, src_rows / elapsed_seconds,
ReadableSize(src_bytes / elapsed_seconds));
@ -599,7 +599,7 @@ void AggregatingTransform::initGenerate()
pipe = Pipe::unitePipes(std::move(pipes));
}
LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed));
LOG_DEBUG(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed));
addMergingAggregatedMemoryEfficientTransform(pipe, params, temporary_data_merge_threads);

View File

@ -52,7 +52,7 @@ Chunk MergingAggregatedTransform::generate()
if (!generate_started)
{
generate_started = true;
LOG_TRACE(log, "Read {} blocks of partially aggregated data, total {} rows.", total_input_blocks, total_input_rows);
LOG_DEBUG(log, "Read {} blocks of partially aggregated data, total {} rows.", total_input_blocks, total_input_rows);
/// Exception safety. Make iterator valid in case any method below throws.
next_block = blocks.begin();

View File

@ -535,7 +535,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa
ReadBufferFromFile in(file_path);
const auto & distributed_header = readDistributedHeader(in, log);
LOG_TRACE(log, "Started processing `{}` ({} rows, {} bytes)", file_path,
LOG_DEBUG(log, "Started processing `{}` ({} rows, {} bytes)", file_path,
formatReadableQuantity(distributed_header.rows),
formatReadableSizeWithBinarySuffix(distributed_header.bytes));
@ -631,7 +631,7 @@ struct StorageDistributedDirectoryMonitor::Batch
Stopwatch watch;
LOG_TRACE(parent.log, "Sending a batch of {} files ({} rows, {} bytes).", file_indices.size(),
LOG_DEBUG(parent.log, "Sending a batch of {} files ({} rows, {} bytes).", file_indices.size(),
formatReadableQuantity(total_rows),
formatReadableSizeWithBinarySuffix(total_bytes));
@ -876,7 +876,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map
if (!total_rows || !header)
{
LOG_TRACE(log, "Processing batch {} with old format (no header/rows)", in.getFileName());
LOG_DEBUG(log, "Processing batch {} with old format (no header/rows)", in.getFileName());
CompressedReadBuffer decompressing_in(in);
NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION);

View File

@ -1054,7 +1054,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams(
false);
/// Let's estimate total number of rows for progress bar.
LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows, num_streams);
LOG_DEBUG(log, "Reading approx. {} rows with {} streams", total_rows, num_streams);
for (size_t i = 0; i < num_streams; ++i)
{
@ -1576,7 +1576,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal(
settings.preferred_block_size_bytes,
false);
LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows_in_lonely_parts, num_streams_for_lonely_parts);
LOG_DEBUG(log, "Reading approx. {} rows with {} streams", total_rows_in_lonely_parts, num_streams_for_lonely_parts);
for (size_t i = 0; i < num_streams_for_lonely_parts; ++i)
{

View File

@ -182,7 +182,7 @@ bool MergeTreePartsMover::selectPartsForMove(
if (!parts_to_move.empty())
{
LOG_TRACE(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, ReadableSize(parts_to_move_total_size_bytes));
LOG_DEBUG(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, ReadableSize(parts_to_move_total_size_bytes));
return true;
}
else

View File

@ -47,7 +47,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor(
size_t total_rows = data_part->index_granularity.getRowsCountInRanges(all_mark_ranges);
if (!quiet)
LOG_TRACE(log, "Reading {} ranges in reverse order from part {}, approx. {} rows starting from {}",
LOG_DEBUG(log, "Reading {} ranges in reverse order from part {}, approx. {} rows starting from {}",
all_mark_ranges.size(), data_part->name, total_rows,
data_part->index_granularity.getMarkStartingRow(all_mark_ranges.front().begin));

View File

@ -47,7 +47,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
size_t total_rows = data_part->index_granularity.getRowsCountInRanges(all_mark_ranges);
if (!quiet)
LOG_TRACE(log, "Reading {} ranges from part {}, approx. {} rows starting from {}",
LOG_DEBUG(log, "Reading {} ranges from part {}, approx. {} rows starting from {}",
all_mark_ranges.size(), data_part->name, total_rows,
data_part->index_granularity.getMarkStartingRow(all_mark_ranges.front().begin));

View File

@ -29,10 +29,10 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
{
/// Print column name but don't pollute logs in case of many columns.
if (columns_to_read.size() == 1)
LOG_TRACE(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part, column {}",
LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part, column {}",
data_part->getMarksCount(), data_part->name, data_part->rows_count, columns_to_read.front());
else
LOG_TRACE(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part",
LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part",
data_part->getMarksCount(), data_part->name, data_part->rows_count);
}

View File

@ -547,7 +547,7 @@ public:
{
if (storage.destination_id)
{
LOG_TRACE(storage.log, "Writing block with {} rows, {} bytes directly.", rows, bytes);
LOG_DEBUG(storage.log, "Writing block with {} rows, {} bytes directly.", rows, bytes);
storage.writeBlockToDestination(block, destination);
}
return;
@ -830,7 +830,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc
if (!destination_id)
{
LOG_TRACE(log, "Flushing buffer with {} rows (discarded), {} bytes, age {} seconds {}.", rows, bytes, time_passed, (check_thresholds ? "(bg)" : "(direct)"));
LOG_DEBUG(log, "Flushing buffer with {} rows (discarded), {} bytes, age {} seconds {}.", rows, bytes, time_passed, (check_thresholds ? "(bg)" : "(direct)"));
return;
}
@ -867,7 +867,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc
}
UInt64 milliseconds = watch.elapsedMilliseconds();
LOG_TRACE(log, "Flushing buffer with {} rows, {} bytes, age {} seconds, took {} ms {}.", rows, bytes, time_passed, milliseconds, (check_thresholds ? "(bg)" : "(direct)"));
LOG_DEBUG(log, "Flushing buffer with {} rows, {} bytes, age {} seconds, took {} ms {}.", rows, bytes, time_passed, milliseconds, (check_thresholds ? "(bg)" : "(direct)"));
}