mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-19 21:03:51 +00:00
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:
commit
ccbbce2b72
@ -731,15 +731,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.
|
||||||
|
@ -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*/)
|
||||||
{
|
{
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -5883,15 +5883,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)
|
||||||
|
@ -164,7 +164,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;
|
||||||
|
|
||||||
|
@ -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++])
|
||||||
{
|
{
|
||||||
|
@ -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,
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -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 │ │
|
||||||
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘
|
└─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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),
|
||||||
|
Loading…
Reference in New Issue
Block a user