Merge pull request #57186 from chenlx0/bytes_uncompressed

Show uncompressed data size in system.tables
This commit is contained in:
Yakov Olkhovskiy 2023-12-19 19:04:47 -05:00 committed by GitHub
commit 43fcea64b6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 90 additions and 2 deletions

View File

@ -57,6 +57,8 @@ Columns:
- If the table stores data on disk, returns used space on disk (i.e. compressed).
- If the table stores data in memory, returns approximated number of used bytes in memory.
- `total_bytes_uncompressed` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of uncompressed bytes, if it's possible to quickly determine the exact number of bytes from the part checksums for the table on storage, otherwise `NULL` (does not take underlying storage (if any) into account).
- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows INSERTed since server start (only for `Buffer` tables).
- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes INSERTed since server start (only for `Buffer` tables).

View File

@ -692,6 +692,15 @@ public:
/// when considering in-memory blocks.
virtual std::optional<UInt64> totalBytes(const Settings &) const { return {}; }
/// If it is possible to quickly determine exact number of uncompressed bytes for the table on storage:
/// - disk (uncompressed)
///
/// Used for:
/// - For total_bytes_uncompressed column in system.tables
///
/// Does not take underlying Storage (if any) into account
virtual std::optional<UInt64> totalBytesUncompressed(const Settings &) const { return {}; }
/// Number of rows INSERTed since server start.
///
/// Does not take the underlying Storage (if any) into account.

View File

@ -1142,6 +1142,7 @@ void IMergeTreeDataPart::loadChecksums(bool require)
{
assertEOF(*buf);
bytes_on_disk = checksums.getTotalSizeOnDisk();
bytes_uncompressed_on_disk = checksums.getTotalSizeUncompressedOnDisk();
}
else
bytes_on_disk = getDataPartStorage().calculateTotalSizeOnDisk();
@ -1159,6 +1160,7 @@ void IMergeTreeDataPart::loadChecksums(bool require)
writeChecksums(checksums, {});
bytes_on_disk = checksums.getTotalSizeOnDisk();
bytes_uncompressed_on_disk = checksums.getTotalSizeUncompressedOnDisk();
}
}

View File

@ -370,7 +370,9 @@ public:
UInt64 getIndexSizeFromFile() const;
UInt64 getBytesOnDisk() const { return bytes_on_disk; }
UInt64 getBytesUncompressedOnDisk() const { return bytes_uncompressed_on_disk; }
void setBytesOnDisk(UInt64 bytes_on_disk_) { bytes_on_disk = bytes_on_disk_; }
void setBytesUncompressedOnDisk(UInt64 bytes_uncompressed_on_disk_) { bytes_uncompressed_on_disk = bytes_uncompressed_on_disk_; }
size_t getFileSizeOrZero(const String & file_name) const;
auto getFilesChecksums() const { return checksums.files; }
@ -566,6 +568,7 @@ protected:
/// Total size on disk, not only columns. May not contain size of
/// checksums.txt and columns.txt. 0 - if not counted;
UInt64 bytes_on_disk{0};
UInt64 bytes_uncompressed_on_disk{0};
/// Columns description. Cannot be changed, after part initialization.
NamesAndTypesList columns;

View File

@ -112,6 +112,14 @@ UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const
return res;
}
UInt64 MergeTreeDataPartChecksums::getTotalSizeUncompressedOnDisk() const
{
UInt64 res = 0;
for (const auto & [_, checksum] : files)
res += checksum.uncompressed_size;
return res;
}
bool MergeTreeDataPartChecksums::read(ReadBuffer & in, size_t format_version)
{
switch (format_version)

View File

@ -88,6 +88,7 @@ struct MergeTreeDataPartChecksums
static MergeTreeDataPartChecksums deserializeFrom(const String & s);
UInt64 getTotalSizeOnDisk() const;
UInt64 getTotalSizeUncompressedOnDisk() const;
};
/// A kind of MergeTreeDataPartChecksums intended to be stored in ZooKeeper (to save its RAM)

View File

@ -184,6 +184,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync(
new_part->index = writer->releaseIndexColumns();
new_part->checksums = checksums;
new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk());
new_part->setBytesUncompressedOnDisk(checksums.getTotalSizeUncompressedOnDisk());
new_part->index_granularity = writer->getIndexGranularity();
new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk();

View File

@ -460,6 +460,16 @@ std::optional<UInt64> StorageMaterializedView::totalBytes(const Settings & setti
return {};
}
std::optional<UInt64> StorageMaterializedView::totalBytesUncompressed(const Settings & settings) const
{
if (hasInnerTable())
{
if (auto table = tryGetTargetTable())
return table->totalBytesUncompressed(settings);
}
return {};
}
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
{
if (has_inner_table)

View File

@ -102,6 +102,7 @@ public:
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings & settings) const override;
private:
/// Will be initialized in constructor

View File

@ -272,6 +272,15 @@ std::optional<UInt64> StorageMergeTree::totalBytes(const Settings &) const
return getTotalActiveSizeInBytes();
}
std::optional<UInt64> StorageMergeTree::totalBytesUncompressed(const Settings &) const
{
UInt64 res = 0;
auto parts = getDataPartsForInternalUsage();
for (const auto & part : parts)
res += part->getBytesUncompressedOnDisk();
return res;
}
SinkToStoragePtr
StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
{

View File

@ -70,6 +70,7 @@ public:
std::optional<UInt64> totalRows(const Settings &) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override;
std::optional<UInt64> totalBytes(const Settings &) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings &) const override;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override;

View File

@ -5459,6 +5459,12 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalBytes(const Settings & se
return res;
}
std::optional<UInt64> StorageReplicatedMergeTree::totalBytesUncompressed(const Settings & settings) const
{
UInt64 res = 0;
foreachActiveParts([&res](auto & part) { res += part->getBytesUncompressedOnDisk(); }, settings.select_sequential_consistency);
return res;
}
void StorageReplicatedMergeTree::assertNotReadonly() const
{

View File

@ -165,6 +165,7 @@ public:
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override;
std::optional<UInt64> totalBytes(const Settings & settings) const override;
std::optional<UInt64> totalBytesUncompressed(const Settings & settings) const override;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override;

View File

@ -54,6 +54,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
{"storage_policy", std::make_shared<DataTypeString>()},
{"total_rows", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"total_bytes", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"total_bytes_uncompressed", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"parts", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"active_parts", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
{"total_marks", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
@ -518,6 +519,15 @@ protected:
res_columns[res_index++]->insertDefault();
}
if (columns_mask[src_index++])
{
auto total_bytes_uncompressed = table->totalBytesUncompressed(settings);
if (total_bytes_uncompressed)
res_columns[res_index++]->insert(*total_bytes_uncompressed);
else
res_columns[res_index++]->insertDefault();
}
auto table_merge_tree = std::dynamic_pointer_cast<MergeTreeData>(table);
if (columns_mask[src_index++])
{

View File

@ -6,7 +6,7 @@
│ name2 │ 1 │ 0 │ 0 │ 0 │
│ name3 │ 0 │ 0 │ 0 │ 0 │
└───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
231 1
3 231 1
┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐
│ check_system_tables │ date │ date, version │ date │ │
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘
@ -51,3 +51,6 @@ Check total_bytes/total_rows for Set
Check total_bytes/total_rows for Join
1 50
1 100
Check total_uncompressed_bytes/total_bytes/total_rows for Materialized views
0 0 0
1 1 1

View File

@ -23,7 +23,7 @@ FROM system.columns WHERE table = 'check_system_tables' AND database = currentDa
FORMAT PrettyCompactNoEscapes;
INSERT INTO check_system_tables VALUES (1, 1, 1);
SELECT total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
SELECT total_bytes_uncompressed, total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
DROP TABLE IF EXISTS check_system_tables;
@ -138,3 +138,23 @@ SELECT total_bytes BETWEEN 5000 AND 15000, total_rows FROM system.tables WHERE n
INSERT INTO check_system_tables SELECT number+50 FROM numbers(50);
SELECT total_bytes BETWEEN 5000 AND 15000, total_rows FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
DROP TABLE check_system_tables;
-- Build MergeTree table for Materialized view
CREATE TABLE check_system_tables
(
name1 UInt8,
name2 UInt8,
name3 UInt8
) ENGINE = MergeTree()
ORDER BY name1
PARTITION BY name2
SAMPLE BY name1
SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1;
SELECT 'Check total_uncompressed_bytes/total_bytes/total_rows for Materialized views';
CREATE MATERIALIZED VIEW check_system_tables_mv ENGINE = MergeTree() ORDER BY name2 AS SELECT name1, name2, name3 FROM check_system_tables;
SELECT total_bytes_uncompressed, total_bytes, total_rows FROM system.tables WHERE name = 'check_system_tables_mv' AND database = currentDatabase();
INSERT INTO check_system_tables VALUES (1, 1, 1);
SELECT total_bytes_uncompressed > 0, total_bytes > 0, total_rows FROM system.tables WHERE name = 'check_system_tables_mv' AND database = currentDatabase();
DROP TABLE check_system_tables_mv;
DROP TABLE check_system_tables;

View File

@ -1087,6 +1087,7 @@ CREATE TABLE system.tables
`storage_policy` String,
`total_rows` Nullable(UInt64),
`total_bytes` Nullable(UInt64),
`total_bytes_uncompressed` Nullable(UInt64),
`parts` Nullable(UInt64),
`active_parts` Nullable(UInt64),
`total_marks` Nullable(UInt64),