2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypeDate.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
2018-01-23 22:56:46 +00:00
|
|
|
#include <DataTypes/DataTypeEnum.h>
|
2017-06-25 00:01:10 +00:00
|
|
|
#include <Storages/MergeTree/MergeTreeDataPart.h>
|
|
|
|
#include <Storages/MergeTree/MergeTreeData.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/PartLog.h>
|
2017-03-07 17:13:54 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2018-11-20 20:09:20 +00:00
|
|
|
template <> struct NearestFieldTypeImpl<PartLogElement::Type> { using Type = UInt64; };
|
2018-10-22 08:54:54 +00:00
|
|
|
|
2017-03-07 17:13:54 +00:00
|
|
|
Block PartLogElement::createBlock()
|
|
|
|
{
|
2018-01-23 22:56:46 +00:00
|
|
|
auto event_type_datatype = std::make_shared<DataTypeEnum8>(
|
2018-09-09 02:23:24 +00:00
|
|
|
DataTypeEnum8::Values
|
|
|
|
{
|
2018-01-23 22:56:46 +00:00
|
|
|
{"NEW_PART", static_cast<Int8>(NEW_PART)},
|
|
|
|
{"MERGE_PARTS", static_cast<Int8>(MERGE_PARTS)},
|
|
|
|
{"DOWNLOAD_PART", static_cast<Int8>(DOWNLOAD_PART)},
|
2018-05-25 19:44:14 +00:00
|
|
|
{"REMOVE_PART", static_cast<Int8>(REMOVE_PART)},
|
|
|
|
{"MUTATE_PART", static_cast<Int8>(MUTATE_PART)},
|
2018-09-09 02:23:24 +00:00
|
|
|
});
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return
|
|
|
|
{
|
2018-01-23 22:56:46 +00:00
|
|
|
{ColumnInt8::create(), std::move(event_type_datatype), "event_type"},
|
|
|
|
{ColumnUInt16::create(), std::make_shared<DataTypeDate>(), "event_date"},
|
|
|
|
{ColumnUInt32::create(), std::make_shared<DataTypeDateTime>(), "event_time"},
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "duration_ms"},
|
|
|
|
|
|
|
|
{ColumnString::create(), std::make_shared<DataTypeString>(), "database"},
|
|
|
|
{ColumnString::create(), std::make_shared<DataTypeString>(), "table"},
|
|
|
|
{ColumnString::create(), std::make_shared<DataTypeString>(), "part_name"},
|
2018-09-13 12:48:46 +00:00
|
|
|
{ColumnString::create(), std::make_shared<DataTypeString>(), "partition_id"},
|
2018-01-23 22:56:46 +00:00
|
|
|
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "rows"},
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "size_in_bytes"}, // On disk
|
|
|
|
|
|
|
|
/// Merge-specific info
|
|
|
|
{ColumnArray::create(ColumnString::create()), std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "merged_from"},
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "bytes_uncompressed"}, // Result bytes
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_rows"},
|
|
|
|
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_bytes"},
|
|
|
|
|
|
|
|
/// Is there an error during the execution or commit
|
|
|
|
{ColumnUInt16::create(), std::make_shared<DataTypeUInt16>(), "error"},
|
|
|
|
{ColumnString::create(), std::make_shared<DataTypeString>(), "exception"},
|
2018-05-07 02:01:11 +00:00
|
|
|
};
|
2017-03-07 17:13:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void PartLogElement::appendToBlock(Block & block) const
|
|
|
|
{
|
2017-12-15 18:23:05 +00:00
|
|
|
MutableColumns columns = block.mutateColumns();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t i = 0;
|
2017-03-07 17:13:54 +00:00
|
|
|
|
2018-10-22 08:54:54 +00:00
|
|
|
columns[i++]->insert(event_type);
|
|
|
|
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
|
|
|
|
columns[i++]->insert(event_time);
|
|
|
|
columns[i++]->insert(duration_ms);
|
2017-03-07 17:13:54 +00:00
|
|
|
|
2017-12-15 18:23:05 +00:00
|
|
|
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);
|
2017-03-07 17:13:54 +00:00
|
|
|
|
2018-10-22 08:54:54 +00:00
|
|
|
columns[i++]->insert(rows);
|
|
|
|
columns[i++]->insert(bytes_compressed_on_disk);
|
2018-01-23 22:56:46 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
|
2018-10-22 08:54:54 +00:00
|
|
|
columns[i++]->insert(bytes_uncompressed);
|
|
|
|
columns[i++]->insert(rows_read);
|
|
|
|
columns[i++]->insert(bytes_read_uncompressed);
|
2017-12-15 18:23:05 +00:00
|
|
|
|
2018-10-22 08:54:54 +00:00
|
|
|
columns[i++]->insert(error);
|
2018-01-23 22:56:46 +00:00
|
|
|
columns[i++]->insert(exception);
|
2017-12-15 18:23:05 +00:00
|
|
|
|
|
|
|
block.setColumns(std::move(columns));
|
2017-03-07 17:13:54 +00:00
|
|
|
}
|
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
bool PartLog::addNewPart(Context & context, const MutableDataPartPtr & part, UInt64 elapsed_ns, const ExecutionStatus & execution_status)
|
2017-06-25 00:01:10 +00:00
|
|
|
{
|
2018-05-21 13:49:54 +00:00
|
|
|
return addNewParts(context, {part}, elapsed_ns, execution_status);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool PartLog::addNewParts(Context & context, const PartLog::MutableDataPartsVector & parts, UInt64 elapsed_ns,
|
|
|
|
const ExecutionStatus & execution_status)
|
|
|
|
{
|
|
|
|
if (parts.empty())
|
|
|
|
return true;
|
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
PartLog * part_log = nullptr;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2018-05-21 13:49:54 +00:00
|
|
|
part_log = context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table
|
2018-01-23 22:56:46 +00:00
|
|
|
if (!part_log)
|
|
|
|
return false;
|
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
for (const auto & part : parts)
|
|
|
|
{
|
|
|
|
PartLogElement elem;
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
elem.event_type = PartLogElement::NEW_PART;
|
|
|
|
elem.event_time = time(nullptr);
|
|
|
|
elem.duration_ms = elapsed_ns / 1000000;
|
2017-06-25 00:01:10 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
elem.database_name = part->storage.getDatabaseName();
|
|
|
|
elem.table_name = part->storage.getTableName();
|
|
|
|
elem.part_name = part->name;
|
2018-09-13 12:48:46 +00:00
|
|
|
elem.partition_id = part->info.partition_id;
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
elem.bytes_compressed_on_disk = part->bytes_on_disk;
|
|
|
|
elem.rows = part->rows_count;
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
elem.error = static_cast<UInt16>(execution_status.code);
|
|
|
|
elem.exception = execution_status.message;
|
2018-01-23 22:56:46 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
part_log->add(elem);
|
|
|
|
}
|
2018-01-23 22:56:46 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__);
|
|
|
|
return false;
|
|
|
|
}
|
2017-03-07 17:13:54 +00:00
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
return true;
|
2017-06-25 00:01:10 +00:00
|
|
|
}
|
2017-03-07 17:13:54 +00:00
|
|
|
|
|
|
|
}
|