Merge pull request #70046 from ClickHouse/backport/24.8/69871

Backport #69871 to 24.8: add `readonly_duration` to the `system.replicas` table
This commit is contained in:
robot-ch-test-poll 2024-09-27 01:05:52 +02:00 committed by GitHub
commit 78ca7bb770
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 34 additions and 4 deletions

View File

@ -1,3 +1,4 @@
#include <atomic>
#include <IO/Operators.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
@ -362,6 +363,13 @@ void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown)
bool old_val = false;
bool became_readonly = storage.is_readonly.compare_exchange_strong(old_val, true);
if (became_readonly)
{
const UInt32 now = static_cast<UInt32>(
std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()));
storage.readonly_start_time.store(now, std::memory_order_relaxed);
}
/// Do not increment the metric if replica became readonly due to shutdown.
if (became_readonly && on_shutdown)
return;
@ -394,6 +402,8 @@ void ReplicatedMergeTreeRestartingThread::setNotReadonly()
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
chassert(CurrentMetrics::get(CurrentMetrics::ReadonlyReplica) >= 0);
}
storage.readonly_start_time.store(0, std::memory_order_relaxed);
}
}

View File

@ -27,6 +27,7 @@ struct ReplicatedTableStatus
UInt32 total_replicas;
UInt32 active_replicas;
UInt64 lost_part_count;
UInt32 readonly_start_time;
String last_queue_update_exception;
/// If the error has happened fetching the info from ZooKeeper, this field will be set.
String zookeeper_exception;

View File

@ -1,5 +1,6 @@
#include <Core/Defines.h>
#include <atomic>
#include <ranges>
#include <chrono>
@ -7048,6 +7049,7 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit
res.active_replicas = 0;
res.lost_part_count = 0;
res.last_queue_update_exception = getLastQueueUpdateException();
res.readonly_start_time = readonly_start_time.load(std::memory_order_relaxed);
if (with_zk_fields && !res.is_session_expired)
{

View File

@ -411,6 +411,8 @@ private:
/// If true, the table is offline and can not be written to it.
/// This flag is managed by RestartingThread.
std::atomic_bool is_readonly {true};
std::atomic_uint32_t readonly_start_time{0};
/// If nullopt - ZooKeeper is not available, so we don't know if there is table metadata.
/// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case.
std::optional<bool> has_metadata_in_zookeeper;

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeMap.h>
#include <Storages/System/StorageSystemReplicas.h>
@ -200,8 +201,8 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_)
: IStorage(table_id_)
, impl(std::make_unique<StorageSystemReplicasImpl>(128))
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription({
ColumnsDescription description = {
{ "database", std::make_shared<DataTypeString>(), "Database name."},
{ "table", std::make_shared<DataTypeString>(), "Table name."},
{ "engine", std::make_shared<DataTypeString>(), "Table engine name."},
@ -212,6 +213,7 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_)
{ "can_become_leader", std::make_shared<DataTypeUInt8>(), "Whether the replica can be a leader."},
{ "is_readonly", std::make_shared<DataTypeUInt8>(), "Whether the replica is in read-only mode. This mode is turned on if the config does not have sections with ClickHouse Keeper, "
"if an unknown error occurred when reinitializing sessions in ClickHouse Keeper, and during session reinitialization in ClickHouse Keeper."},
{ "readonly_start_time", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>()), "The timestamp when the replica transitioned into readonly mode. Null if the replica is not in readonly mode." },
{ "is_session_expired", std::make_shared<DataTypeUInt8>(), "Whether the session with ClickHouse Keeper has expired. Basically the same as `is_readonly`."},
{ "future_parts", std::make_shared<DataTypeUInt32>(), "The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet."},
{ "parts_to_check", std::make_shared<DataTypeUInt32>(), "The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged."},
@ -243,7 +245,14 @@ StorageSystemReplicas::StorageSystemReplicas(const StorageID & table_id_)
{ "last_queue_update_exception", std::make_shared<DataTypeString>(), "When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions."},
{ "zookeeper_exception", std::make_shared<DataTypeString>(), "The last exception message, got if the error happened when fetching the info from ClickHouse Keeper."},
{ "replica_is_active", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt8>()), "Map between replica name and is replica active."}
}));
};
description.setAliases({
{"readonly_duration", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>()), "if(isNull(readonly_start_time), NULL, now() - readonly_start_time)"},
});
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(description);
setInMemoryMetadata(storage_metadata);
}
@ -527,6 +536,10 @@ Chunk SystemReplicasSource::generate()
res_columns[col_num++]->insert(status.is_leader);
res_columns[col_num++]->insert(status.can_become_leader);
res_columns[col_num++]->insert(status.is_readonly);
if (status.readonly_start_time != 0)
res_columns[col_num++]->insert(status.readonly_start_time);
else
res_columns[col_num++]->insertDefault();
res_columns[col_num++]->insert(status.is_session_expired);
res_columns[col_num++]->insert(status.queue.future_parts);
res_columns[col_num++]->insert(status.parts_to_check);

View File

@ -861,6 +861,7 @@ CREATE TABLE system.replicas
`is_leader` UInt8,
`can_become_leader` UInt8,
`is_readonly` UInt8,
`readonly_start_time` Nullable(DateTime),
`is_session_expired` UInt8,
`future_parts` UInt32,
`parts_to_check` UInt32,
@ -889,7 +890,8 @@ CREATE TABLE system.replicas
`lost_part_count` UInt64,
`last_queue_update_exception` String,
`zookeeper_exception` String,
`replica_is_active` Map(String, UInt8)
`replica_is_active` Map(String, UInt8),
`readonly_duration` Nullable(DateTime) ALIAS if(readonly_start_time IS NULL, NULL, now() - readonly_start_time)
)
ENGINE = SystemReplicas
COMMENT 'Contains information and status of all table replicas on current server. Each replica is represented by a single row.'