Merge pull request #60981 from nickitat/debug_info

Add debug info to exceptions in `IMergeTreeDataPart::checkConsistency()`
This commit is contained in:
Nikita Taranov 2024-03-20 15:23:51 +01:00 committed by GitHub
commit f80c3e5941
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 99 additions and 43 deletions

View File

@ -2,38 +2,41 @@
#include <Storages/MergeTree/IDataPartStorage.h>
#include <base/types.h>
#include <exception>
#include <optional>
#include <boost/algorithm/string/join.hpp>
#include <string_view>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/getCompressionCodecForFile.h>
#include <Core/Defines.h>
#include <IO/HashingWriteBuffer.h>
#include <Core/NamesAndTypes.h>
#include <Core/SettingsEnums.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/NestedUtils.h>
#include <IO/HashingReadBuffer.h>
#include <IO/HashingWriteBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/localBackup.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
#include <Core/NamesAndTypes.h>
#include <Storages/ColumnsDescription.h>
#include <Compression/CompressedReadBuffer.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/CurrentMetrics.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <base/JSON.h>
#include <Common/logger_useful.h>
#include <Compression/getCompressionCodecForFile.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Parsers/ExpressionElementParsers.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Interpreters/MergeTreeTransaction.h>
#include <Interpreters/TransactionLog.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/PartMetadataManagerOrdinary.h>
#include <Storages/MergeTree/checkDataPart.h>
#include <Storages/MergeTree/localBackup.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <base/JSON.h>
#include <boost/algorithm/string/join.hpp>
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/MemoryTrackerBlockerInThread.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/logger_useful.h>
#include <Disks/IO/CachedOnDiskReadBufferFromFile.h>
@ -2076,7 +2079,39 @@ void IMergeTreeDataPart::checkConsistencyBase() const
}
}
void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const
void IMergeTreeDataPart::checkConsistency(bool require_part_metadata) const
{
try
{
checkConsistencyBase();
doCheckConsistency(require_part_metadata);
}
catch (Exception & e)
{
const auto part_state = fmt::format(
"state: {}, is_unexpected_local_part: {}, is_frozen: {}, is_duplicate: {}",
stateString(),
is_unexpected_local_part,
is_frozen,
is_duplicate,
is_temp);
const auto debug_info = fmt::format(
"columns: {}, getMarkSizeInBytes: {}, getMarksCount: {}, index_granularity_info: [{}], index_granularity: [{}], "
"part_state: [{}]",
columns.toString(),
index_granularity_info.getMarkSizeInBytes(columns.size()),
index_granularity.getMarksCount(),
index_granularity_info.describe(),
index_granularity.describe(),
part_state);
e.addMessage(debug_info);
e.rethrow();
}
}
void IMergeTreeDataPart::doCheckConsistency(bool /* require_part_metadata */) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'checkConsistency' is not implemented for part with type {}", getType().toString());
}

View File

@ -511,7 +511,7 @@ public:
void writeChecksums(const MergeTreeDataPartChecksums & checksums_, const WriteSettings & settings);
/// Checks the consistency of this data part.
virtual void checkConsistency(bool require_part_metadata) const;
void checkConsistency(bool require_part_metadata) const;
/// Checks the consistency of this data part, and check the consistency of its projections (if any) as well.
void checkConsistencyWithProjections(bool require_part_metadata) const;
@ -599,8 +599,6 @@ protected:
void removeIfNeeded();
void checkConsistencyBase() const;
/// Fill each_columns_size and total_size with sizes from columns files on
/// disk using columns and checksums.
virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0;
@ -623,6 +621,8 @@ protected:
void initializeIndexGranularityInfo();
virtual void doCheckConsistency(bool require_part_metadata) const;
private:
String mutable_name;
mutable MergeTreeDataPartState state{MergeTreeDataPartState::Temporary};
@ -713,6 +713,8 @@ private:
void incrementStateMetric(MergeTreeDataPartState state) const;
void decrementStateMetric(MergeTreeDataPartState state) const;
void checkConsistencyBase() const;
/// This ugly flag is needed for debug assertions only
mutable bool part_is_probably_removed_from_disk = false;
};

View File

@ -160,9 +160,8 @@ std::optional<time_t> MergeTreeDataPartCompact::getColumnModificationTime(const
return getDataPartStorage().getFileLastModified(DATA_FILE_NAME_WITH_EXTENSION).epochTime();
}
void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) const
void MergeTreeDataPartCompact::doCheckConsistency(bool require_part_metadata) const
{
checkConsistencyBase();
String mrk_file_name = DATA_FILE_NAME + getMarksFileExtension();
if (!checksums.empty())

View File

@ -68,14 +68,14 @@ protected:
MergeTreeIndexGranularity & index_granularity_, const MergeTreeIndexGranularityInfo & index_granularity_info_,
size_t columns_count, const IDataPartStorage & data_part_storage_);
void doCheckConsistency(bool require_part_metadata) const override;
private:
void checkConsistency(bool require_part_metadata) const override;
/// Loads marks index granularity into memory
void loadIndexGranularity() override;
/// Loads marks index granularity into memory
void loadIndexGranularity() override;
/// Compact parts doesn't support per column size, only total size
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
/// Compact parts doesn't support per column size, only total size
void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override;
};
}

View File

@ -189,9 +189,8 @@ MergeTreeDataPartWide::~MergeTreeDataPartWide()
removeIfNeeded();
}
void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const
{
checkConsistencyBase();
std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension();
if (!checksums.empty())

View File

@ -63,9 +63,9 @@ protected:
MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_,
const IDataPartStorage & data_part_storage_, const std::string & any_column_file_name);
private:
void checkConsistency(bool require_part_metadata) const override;
void doCheckConsistency(bool require_part_metadata) const override;
private:
/// Loads marks index granularity into memory
void loadIndexGranularity() override;

View File

@ -121,5 +121,8 @@ void MergeTreeIndexGranularity::resizeWithFixedGranularity(size_t size, size_t f
}
}
std::string MergeTreeIndexGranularity::describe() const
{
return fmt::format("initialized: {}, marks_rows_partial_sums: [{}]", initialized, fmt::join(marks_rows_partial_sums, ", "));
}
}

View File

@ -95,6 +95,8 @@ public:
/// Add `size` of marks with `fixed_granularity` rows
void resizeWithFixedGranularity(size_t size, size_t fixed_granularity);
std::string describe() const;
};
}

View File

@ -86,6 +86,10 @@ std::string MarkType::getFileExtension() const
}
}
std::string MarkType::describe() const
{
return fmt::format("adaptive: {}, compressed: {}, part_type: {}", adaptive, compressed, part_type);
}
std::optional<MarkType> MergeTreeIndexGranularityInfo::getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage)
{
@ -128,10 +132,18 @@ size_t MergeTreeIndexGranularityInfo::getMarkSizeInBytes(size_t columns_num) con
throw Exception(ErrorCodes::UNKNOWN_PART_TYPE, "Unknown part type");
}
std::string MergeTreeIndexGranularityInfo::describe() const
{
return fmt::format(
"mark_type: [{}], index_granularity_bytes: {}, fixed_index_granularity: {}",
mark_type.describe(),
index_granularity_bytes,
fixed_index_granularity);
}
size_t getAdaptiveMrkSizeCompact(size_t columns_num)
{
/// Each mark contains number of rows in granule and two offsets for every column.
return sizeof(UInt64) * (columns_num * 2 + 1);
}
}

View File

@ -24,6 +24,8 @@ struct MarkType
static bool isMarkFileExtension(std::string_view extension);
std::string getFileExtension() const;
std::string describe() const;
bool adaptive = false;
bool compressed = false;
MergeTreeDataPartType::Value part_type = MergeTreeDataPartType::Unknown;
@ -58,6 +60,8 @@ public:
size_t getMarkSizeInBytes(size_t columns_num = 1) const;
static std::optional<MarkType> getMarksTypeFromFilesystem(const IDataPartStorage & data_part_storage);
std::string describe() const;
};
constexpr inline auto getNonAdaptiveMrkSizeWide() { return sizeof(UInt64) * 2; }

View File

@ -1,4 +1,4 @@
data.bin doesn't exist: while restoring part all_2_2_0
OK
RESTORED
1
3

View File

@ -26,7 +26,7 @@ ln -s "$SRC_BACKUP_DIR/$SRC_BACKUP_FILENAME" "$BACKUPS_DIR/$BACKUP_FILENAME"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl"
# First try to restore with the setting `restore_broken_parts_as_detached` set to false.
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | grep -o -m 1 "data.bin doesn't exist: while restoring part all_2_2_0"
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl"