ClickHouse/src/Interpreters/PartLog.cpp

180 lines
6.9 KiB
C++
Raw Normal View History

#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeEnum.h>
2019-10-10 16:30:30 +00:00
#include <Storages/MergeTree/IMergeTreeDataPart.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
namespace DB
{
Block PartLogElement::createBlock()
{
auto event_type_datatype = std::make_shared<DataTypeEnum8>(
2018-09-09 02:23:24 +00:00
DataTypeEnum8::Values
{
2019-08-01 16:37:50 +00:00
{"NewPart", static_cast<Int8>(NEW_PART)},
{"MergeParts", static_cast<Int8>(MERGE_PARTS)},
{"DownloadPart", static_cast<Int8>(DOWNLOAD_PART)},
{"RemovePart", static_cast<Int8>(REMOVE_PART)},
{"MutatePart", static_cast<Int8>(MUTATE_PART)},
2019-09-03 11:32:25 +00:00
{"MovePart", static_cast<Int8>(MOVE_PART)},
}
);
2021-02-03 08:29:34 +00:00
ColumnsWithTypeAndName columns_with_type_and_name;
2021-02-03 08:29:34 +00:00
return {
2021-02-03 08:29:34 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "query_id"),
columns_with_type_and_name.emplace_back(std::move(event_type_datatype), "event_type"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDate>(), "event_date"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime>(), "event_time"),
2021-02-03 08:29:34 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"),
2021-02-03 08:29:34 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "duration_ms"),
2021-02-03 08:29:34 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "database"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "table"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "part_name"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "partition_id"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "path_on_disk"),
2021-02-03 14:55:52 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "rows"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "size_in_bytes"), // On disk
/// Merge-specific info
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "merged_from"),
2021-02-03 14:55:52 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "bytes_uncompressed"), // Result bytes
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "read_rows"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "read_bytes"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "peak_memory_usage"),
/// Is there an error during the execution or commit
2021-02-03 14:55:52 +00:00
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt16>(), "error"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "exception"),
2018-05-07 02:01:11 +00:00
};
}
void PartLogElement::appendToBlock(MutableColumns & columns) const
{
size_t i = 0;
columns[i++]->insert(query_id);
columns[i++]->insert(event_type);
Extended range of DateTime64 to years 1925 - 2238 The Year 1925 is a starting point because most of the timezones switched to saner (mostly 15-minutes based) offsets somewhere during 1924 or before. And that significantly simplifies implementation. 2238 is to simplify arithmetics for sanitizing LUT index access; there are less than 0x1ffff days from 1925. * Extended DateLUTImpl internal LUT to 0x1ffff items, some of which represent negative (pre-1970) time values. As a collateral benefit, Date now correctly supports dates up to 2149 (instead of 2106). * Added a new strong typedef ExtendedDayNum, which represents dates pre-1970 and post 2149. * Functions that used to return DayNum now return ExtendedDayNum. * Refactored DateLUTImpl to untie DayNum from the dual role of being a value and an index (due to negative time). Index is now a different type LUTIndex with explicit conversion functions from DatNum, time_t, and ExtendedDayNum. * Updated DateLUTImpl to properly support values close to epoch start (1970-01-01 00:00), including negative ones. * Reduced resolution of DateLUTImpl::Values::time_at_offset_change to multiple of 15-minutes to allow storing 64-bits of time_t in DateLUTImpl::Value while keeping same size. * Minor performance updates to DateLUTImpl when building month LUT by skipping non-start-of-month days. * Fixed extractTimeZoneFromFunctionArguments to work correctly with DateTime64. * New unit-tests and stateless integration tests for both DateTime and DateTime64.
2020-04-17 13:26:44 +00:00
columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType());
columns[i++]->insert(event_time);
columns[i++]->insert(event_time_microseconds);
columns[i++]->insert(duration_ms);
columns[i++]->insert(database_name);
columns[i++]->insert(table_name);
columns[i++]->insert(part_name);
2018-09-13 12:48:46 +00:00
columns[i++]->insert(partition_id);
2019-09-03 11:32:25 +00:00
columns[i++]->insert(path_on_disk);
columns[i++]->insert(rows);
columns[i++]->insert(bytes_compressed_on_disk);
Array source_part_names_array;
source_part_names_array.reserve(source_part_names.size());
for (const auto & name : source_part_names)
source_part_names_array.push_back(name);
columns[i++]->insert(source_part_names_array);
columns[i++]->insert(bytes_uncompressed);
columns[i++]->insert(rows_read);
columns[i++]->insert(bytes_read_uncompressed);
2020-03-19 11:31:21 +00:00
columns[i++]->insert(peak_memory_usage);
columns[i++]->insert(error);
columns[i++]->insert(exception);
}
bool PartLog::addNewPart(
Context & current_context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
{
return addNewParts(current_context, {part}, elapsed_ns, execution_status);
}
inline UInt64 time_in_microseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::microseconds>(timepoint.time_since_epoch()).count();
}
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
{
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
}
bool PartLog::addNewParts(
Context & current_context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
{
if (parts.empty())
return true;
2019-03-21 19:22:38 +00:00
std::shared_ptr<PartLog> part_log;
try
{
auto table_id = parts.front()->storage.getStorageID();
part_log = current_context.getPartLog(table_id.database_name); // assume parts belong to the same table
if (!part_log)
return false;
auto query_id = CurrentThread::getQueryId();
for (const auto & part : parts)
{
PartLogElement elem;
if (query_id.data && query_id.size)
elem.query_id.insert(0, query_id.data, query_id.size);
elem.event_type = PartLogElement::NEW_PART;
// construct event_time and event_time_microseconds using the same time point
// so that the two times will always be equal up to a precision of a second.
const auto time_now = std::chrono::system_clock::now();
elem.event_time = time_in_seconds(time_now);
elem.event_time_microseconds = time_in_microseconds(time_now);
elem.duration_ms = elapsed_ns / 1000000;
elem.database_name = table_id.database_name;
elem.table_name = table_id.table_name;
2018-09-13 12:48:46 +00:00
elem.partition_id = part->info.partition_id;
2019-01-31 17:30:56 +00:00
elem.part_name = part->name;
2019-09-03 11:32:25 +00:00
elem.path_on_disk = part->getFullPath();
2020-03-23 13:32:02 +00:00
elem.bytes_compressed_on_disk = part->getBytesOnDisk();
elem.rows = part->rows_count;
elem.error = static_cast<UInt16>(execution_status.code);
elem.exception = execution_status.message;
part_log->add(elem);
}
}
catch (...)
{
2020-05-30 21:57:37 +00:00
tryLogCurrentException(part_log ? part_log->log : &Poco::Logger::get("PartLog"), __PRETTY_FUNCTION__);
return false;
}
return true;
}
}