mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #57186 from chenlx0/bytes_uncompressed
Show uncompressed data size in system.tables
This commit is contained in:
commit
43fcea64b6
@ -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).
|
||||
|
@ -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.
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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*/)
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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++])
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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),
|
||||
|
Loading…
Reference in New Issue
Block a user