mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
commit
78ca7bb770
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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.'
|
||||
|
Loading…
Reference in New Issue
Block a user