Merge pull request #72992 from ClickHouse/revert-72690-add_totalbytes_of_inactive_parts

Revert "Add total_bytes_with_inactive to system.tables"
This commit is contained in:
Robert Schulze 2024-12-11 15:58:50 +00:00 committed by GitHub
commit ccbbce2b72
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 9 additions and 59 deletions

View File

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

View File

@ -319,16 +319,6 @@ std::optional<UInt64> StorageMergeTree::totalBytesUncompressed(const Settings &)
return res;
}
std::optional<UInt64> StorageMergeTree::totalBytesWithInactive(const Settings &) const
{
UInt64 res = 0;
auto outdated_parts = getDataPartsVectorForInternalUsage({DataPartState::Outdated});
for (const auto & part : outdated_parts)
res += part->getBytesOnDisk();
return res;
}
SinkToStoragePtr
StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/)
{

View File

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

View File

@ -5883,15 +5883,6 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalBytesUncompressed(const S
return res;
}
std::optional<UInt64> StorageReplicatedMergeTree::totalBytesWithInactive(const Settings &) const
{
UInt64 res = 0;
auto outdated_parts = getDataPartsStateRange(DataPartState::Outdated);
for (const auto & part : outdated_parts)
res += part->getBytesOnDisk();
return res;
}
void StorageReplicatedMergeTree::assertNotReadonly() const
{
if (is_readonly)

View File

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

View File

@ -179,10 +179,6 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
"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)."
},
{"total_bytes_with_inactive", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()),
"Total number of bytes with inactive parts, if it is possible to quickly determine exact number "
"of bytes for the table on storage, otherwise NULL (does not includes any underlying storage). "
},
{"parts", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()), "The total number of parts in this table."},
{"active_parts", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()), "The number of active parts in this table."},
{"total_marks", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()), "The total number of marks in all parts in this table."},
@ -601,15 +597,6 @@ protected:
res_columns[res_index++]->insertDefault();
}
if (columns_mask[src_index++])
{
auto total_bytes_with_inactive = table->totalBytesWithInactive(settings);
if (total_bytes_with_inactive)
res_columns[res_index++]->insert(*total_bytes_with_inactive);
else
res_columns[res_index++]->insertDefault();
}
auto table_merge_tree = std::dynamic_pointer_cast<MergeTreeData>(table);
if (columns_mask[src_index++])
{

View File

@ -43,7 +43,7 @@ def test_two_new_versions(start_cluster):
query_id = str(uuid.uuid4())
current.query(
"SELECT name FROM clusterAllReplicas('test_cluster_mixed', system.tables);",
"SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables);",
query_id=query_id,
)
@ -73,7 +73,7 @@ WHERE initial_query_id = '{query_id}';"""
query_id = str(uuid.uuid4())
backward.query(
"SELECT name FROM clusterAllReplicas('test_cluster_mixed', system.tables)",
"SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables)",
query_id=query_id,
)
@ -108,7 +108,7 @@ WHERE initial_query_id = '{query_id}';"""
# to the remote server.
query_id = str(uuid.uuid4())
current.query(
"SELECT name FROM clusterAllReplicas('test_cluster_mixed', system.tables) SETTINGS enable_analyzer = 1;",
"SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables) SETTINGS enable_analyzer = 1;",
query_id=query_id,
)

View File

@ -1,13 +1,12 @@
┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┬─storage_policy─┬─total_rows─┬─total_bytes_with_inactive─
│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ default │ 0 │ 0 │
└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┴────────────────┴────────────┴───────────────────────────
┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┬─storage_policy─┬─total_rows─┐
│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ default │ 0 │
└─────────────────────┴───────────────┴─────────────┴─────────────┴──────────────┴────────────────┴────────────┘
┌─name──┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐
│ name1 │ 0 │ 1 │ 1 │ 1 │
│ name2 │ 1 │ 0 │ 0 │ 0 │
│ name3 │ 0 │ 0 │ 0 │ 0 │
└───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
1 1 3 231 1 0
3 1 6 234 2 462
3 231 1
┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐
│ check_system_tables │ date │ date, version │ date │ │
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘

View File

@ -15,7 +15,7 @@ CREATE TABLE check_system_tables
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 name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows, total_bytes_with_inactive
SELECT name, partition_key, sorting_key, primary_key, sampling_key, storage_policy, total_rows
FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
FORMAT PrettyCompactNoEscapes;
@ -24,12 +24,7 @@ FROM system.columns WHERE table = 'check_system_tables' AND database = currentDa
FORMAT PrettyCompactNoEscapes;
INSERT INTO check_system_tables VALUES (1, 1, 1);
SELECT parts, active_parts, total_bytes_uncompressed, total_bytes, total_rows, total_bytes_with_inactive FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase();
INSERT INTO check_system_tables VALUES (2, 1, 3);
OPTIMIZE TABLE check_system_tables;
SELECT parts, active_parts, total_bytes_uncompressed, total_bytes, total_rows, total_bytes_with_inactive 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;

View File

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