Revert "Add total_bytes_with_inactive to system.tables"

This commit is contained in:
Robert Schulze 2024-12-09 22:55:34 +01:00 committed by GitHub
parent d9fad57bde
commit c56ece0780
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 9 additions and 59 deletions

View File

@ -727,15 +727,6 @@ public:
/// Does not take underlying Storage (if any) into account /// Does not take underlying Storage (if any) into account
virtual std::optional<UInt64> totalBytesUncompressed(const Settings &) const { return {}; } 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. /// Number of rows INSERTed since server start.
/// ///
/// Does not take the underlying Storage (if any) into account. /// 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; 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 SinkToStoragePtr
StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) 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> totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr) const override;
std::optional<UInt64> totalBytes(const Settings &) const override; std::optional<UInt64> totalBytes(const Settings &) const override;
std::optional<UInt64> totalBytesUncompressed(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; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override;

View File

@ -5757,15 +5757,6 @@ std::optional<UInt64> StorageReplicatedMergeTree::totalBytesUncompressed(const S
return res; 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 void StorageReplicatedMergeTree::assertNotReadonly() const
{ {
if (is_readonly) if (is_readonly)

View File

@ -162,7 +162,6 @@ public:
std::optional<UInt64> totalRowsByPartitionPredicate(const ActionsDAG & filter_actions_dag, ContextPtr context) const override; 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> totalBytes(const Settings & settings) const override;
std::optional<UInt64> totalBytesUncompressed(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; 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 " "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)." "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."}, {"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."}, {"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."}, {"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(); 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); auto table_merge_tree = std::dynamic_pointer_cast<MergeTreeData>(table);
if (columns_mask[src_index++]) if (columns_mask[src_index++])
{ {

View File

@ -43,7 +43,7 @@ def test_two_new_versions(start_cluster):
query_id = str(uuid.uuid4()) query_id = str(uuid.uuid4())
current.query( current.query(
"SELECT name FROM clusterAllReplicas('test_cluster_mixed', system.tables);", "SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables);",
query_id=query_id, query_id=query_id,
) )
@ -73,7 +73,7 @@ WHERE initial_query_id = '{query_id}';"""
query_id = str(uuid.uuid4()) query_id = str(uuid.uuid4())
backward.query( backward.query(
"SELECT name FROM clusterAllReplicas('test_cluster_mixed', system.tables)", "SELECT * FROM clusterAllReplicas('test_cluster_mixed', system.tables)",
query_id=query_id, query_id=query_id,
) )
@ -108,7 +108,7 @@ WHERE initial_query_id = '{query_id}';"""
# to the remote server. # to the remote server.
query_id = str(uuid.uuid4()) query_id = str(uuid.uuid4())
current.query( 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, 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─ ┌─name────────────────┬─partition_key─┬─sorting_key─┬─primary_key─┬─sampling_key─┬─storage_policy─┬─total_rows─┐
│ check_system_tables │ name2 │ name1 │ name1 │ name1 │ default │ 0 │ 0 │ │ 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─┐ ┌─name──┬─is_in_partition_key─┬─is_in_sorting_key─┬─is_in_primary_key─┬─is_in_sampling_key─┐
│ name1 │ 0 │ 1 │ 1 │ 1 │ │ name1 │ 0 │ 1 │ 1 │ 1 │
│ name2 │ 1 │ 0 │ 0 │ 0 │ │ name2 │ 1 │ 0 │ 0 │ 0 │
│ name3 │ 0 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │
└───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘
1 1 3 231 1 0 3 231 1
3 1 6 234 2 462
┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐ ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┐
│ check_system_tables │ date │ date, version │ date │ │ │ check_system_tables │ date │ date, version │ date │ │
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘

View File

@ -15,7 +15,7 @@ CREATE TABLE check_system_tables
SAMPLE BY name1 SAMPLE BY name1
SETTINGS min_bytes_for_wide_part = 0, compress_marks = false, compress_primary_key = false, ratio_of_defaults_for_sparse_serialization = 1; 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() FROM system.tables WHERE name = 'check_system_tables' AND database = currentDatabase()
FORMAT PrettyCompactNoEscapes; FORMAT PrettyCompactNoEscapes;
@ -24,12 +24,7 @@ FROM system.columns WHERE table = 'check_system_tables' AND database = currentDa
FORMAT PrettyCompactNoEscapes; FORMAT PrettyCompactNoEscapes;
INSERT INTO check_system_tables VALUES (1, 1, 1); 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(); SELECT total_bytes_uncompressed, total_bytes, total_rows 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();
DROP TABLE IF EXISTS check_system_tables; DROP TABLE IF EXISTS check_system_tables;

View File

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