Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions ydb/core/base/statestorage.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -356,13 +356,13 @@ TIntrusivePtr<TStateStorageInfo> BuildStateStorageInfoImpl(const char* namePrefi
memset(name + offset, 0, TActorId::MaxServiceIDLength - offset);
for (size_t i = 0; i < config.RingGroupsSize(); i++) {
auto& ringGroup = config.GetRingGroups(i);
info->RingGroups.push_back({GetRingGroupState(ringGroup), ringGroup.GetWriteOnly(), ringGroup.GetNToSelect(), {}});
info->RingGroups.push_back({GetRingGroupState(ringGroup), ringGroup.GetWriteOnly(), ringGroup.GetNToSelect(), TBridgePileId::FromProto(&ringGroup, &NKikimrConfig::TDomainsConfig::TStateStorage::TRing::GetBridgePileId), {}});
CopyStateStorageRingInfo(ringGroup, info->RingGroups.back(), name, offset, ringGroup.GetRingGroupActorIdOffset());
memset(name + offset, 0, TActorId::MaxServiceIDLength - offset);
}
if (config.HasRing()) {
auto& ring = config.GetRing();
info->RingGroups.push_back({ERingGroupState::PRIMARY, false, ring.GetNToSelect(), {}});
info->RingGroups.push_back({ERingGroupState::PRIMARY, false, ring.GetNToSelect(), {}, {}});
CopyStateStorageRingInfo(ring, info->RingGroups.back(), name, offset, ring.GetRingGroupActorIdOffset());
}
return info;
Expand Down
6 changes: 6 additions & 0 deletions ydb/core/base/statestorage.h
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
#pragma once
#include "bridge.h"
#include "defs.h"
#include "events.h"
#include <ydb/core/protos/statestorage.pb.h>
Expand Down Expand Up @@ -30,6 +31,7 @@ struct TEvStateStorage {
EvPublishActorGone,
EvRingGroupPassAway,
EvConfigVersionInfo,
EvListBoard,

// replies (local, from proxy)
EvInfo = EvLookup + 512,
Expand All @@ -39,6 +41,7 @@ struct TEvStateStorage {
EvDeleteResult,
EvListSchemeBoardResult,
EvListStateStorageResult,
EvListBoardResult,

// replicas interface
EvReplicaLookup = EvLock + 2 * 512,
Expand Down Expand Up @@ -383,6 +386,8 @@ struct TEvStateStorage {
struct TEvPublishActorGone;
struct TEvUpdateGroupConfig;
struct TEvRingGroupPassAway;
struct TEvListBoard;
struct TEvListBoardResult;

struct TEvReplicaShutdown : public TEventPB<TEvStateStorage::TEvReplicaShutdown, NKikimrStateStorage::TEvReplicaShutdown, TEvStateStorage::EvReplicaShutdown> {
};
Expand Down Expand Up @@ -514,6 +519,7 @@ struct TStateStorageInfo : public TThrRefBase {
ERingGroupState State;
bool WriteOnly = false;
ui32 NToSelect = 0;
TBridgePileId BridgePileId;
TVector<TRing> Rings;

TString ToString() const;
Expand Down
11 changes: 11 additions & 0 deletions ydb/core/base/statestorage_impl.h
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,17 @@ struct TEvStateStorage::TEvListStateStorageResult : public TEventLocal<TEvListSt
{}
};

struct TEvStateStorage::TEvListBoard : public TEventLocal<TEvListBoard, EvListBoard> {
};

struct TEvStateStorage::TEvListBoardResult : public TEventLocal<TEvListBoardResult, EvListBoardResult> {
TIntrusiveConstPtr<TStateStorageInfo> Info;

TEvListBoardResult(const TIntrusiveConstPtr<TStateStorageInfo> &info)
: Info(info)
{}
};

struct TEvStateStorage::TEvPublishActorGone : public TEventLocal<TEvPublishActorGone, EvPublishActorGone> {
TActorId Replica;

Expand Down
5 changes: 5 additions & 0 deletions ydb/core/base/statestorage_proxy.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1070,6 +1070,10 @@ class TStateStorageProxy : public TActor<TStateStorageProxy> {
Send(ev->Sender, new TEvStateStorage::TEvListStateStorageResult(Info), 0, ev->Cookie);
}

void Handle(TEvStateStorage::TEvListBoard::TPtr &ev) {
Send(ev->Sender, new TEvStateStorage::TEvListBoardResult(BoardInfo), 0, ev->Cookie);
}

void Handle(TEvStateStorage::TEvUpdateGroupConfig::TPtr &ev) {
auto *msg = ev->Get();
Info = msg->GroupConfig;
Expand Down Expand Up @@ -1142,6 +1146,7 @@ class TStateStorageProxy : public TActor<TStateStorageProxy> {
hFunc(TEvStateStorage::TEvListStateStorage, Handle);
hFunc(TEvStateStorage::TEvUpdateGroupConfig, Handle);
hFunc(TEvStateStorage::TEvRingGroupPassAway, Handle);
hFunc(TEvStateStorage::TEvListBoard, Handle);
fFunc(TEvents::TSystem::Unsubscribe, HandleUnsubscribe);
default:
if (Info->RingGroups.size() > 1)
Expand Down
141 changes: 139 additions & 2 deletions ydb/core/health_check/health_check.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
#include <ydb/core/base/hive.h>
#include <ydb/core/base/path.h>
#include <ydb/core/base/statestorage.h>
#include <ydb/core/base/statestorage_impl.h>
#include <ydb/core/base/tablet_pipe.h>
#include <ydb/core/cms/console/configs_dispatcher.h>
#include <ydb/core/mon/mon.h>
Expand Down Expand Up @@ -155,6 +156,9 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
QuotaUsage,
BridgeGroupState,
PileComputeState,
StateStorage,
StateStorageRing,
StateStorageNode,
};

enum ETimeoutTag {
Expand Down Expand Up @@ -316,7 +320,7 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
if (issueLog.status() != Ydb::Monitoring::StatusFlag::UNSPECIFIED) {
id << Ydb::Monitoring::StatusFlag_Status_Name(issueLog.status()) << '-';
}
id << crc16(issueLog.message());
id << crc16(TStringBuilder() << issueLog.message() << issueLog.type());
if (location.database().name()) {
id << '-' << crc32(location.database().name());
}
Expand Down Expand Up @@ -363,6 +367,15 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
if (location.compute().schema().path()) {
id << '-' << crc32(location.compute().schema().path());
}
if (location.compute().state_storage().pile().name()) {
id << '-' << location.compute().state_storage().pile().name();
}
if (location.compute().state_storage().ring()) {
id << '-' << location.compute().state_storage().ring();
}
if (location.compute().state_storage().node().id()) {
id << '-' << location.compute().state_storage().node().id();
}
return id.Str();
}

Expand Down Expand Up @@ -392,10 +405,10 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
if (Location.ByteSizeLong() > 0) {
issueLog.mutable_location()->CopyFrom(Location);
}
issueLog.set_id(GetIssueId(issueLog));
if (Type) {
issueLog.set_type(Type);
}
issueLog.set_id(GetIssueId(issueLog));
issueLog.set_level(Level);
if (!reason.empty()) {
for (const TString& r : reason) {
Expand Down Expand Up @@ -675,6 +688,9 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
std::optional<TRequestResponse<TEvSysView::TEvGetPDisksResponse>> PDisks;
std::optional<TRequestResponse<TEvNodeWardenStorageConfig>> NodeWardenStorageConfig;
std::optional<TRequestResponse<TEvStateStorage::TEvBoardInfo>> DatabaseBoardInfo;
std::optional<TRequestResponse<TEvStateStorage::TEvListStateStorageResult>> StateStorageInfo;
std::optional<TRequestResponse<TEvStateStorage::TEvListSchemeBoardResult>> SchemeBoardInfo;
std::optional<TRequestResponse<TEvStateStorage::TEvListBoardResult>> BoardInfo;
THashSet<TNodeId> UnknownStaticGroups;

const NKikimrConfig::THealthCheckConfig& HealthCheckConfig;
Expand Down Expand Up @@ -837,6 +853,16 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
NodeWardenStorageConfig = RequestStorageConfig();
}

if (!IsSpecificDatabaseFilter()) {
StateStorageInfo = TRequestResponse<TEvStateStorage::TEvListStateStorageResult>(Span.CreateChild(TComponentTracingLevels::TTablet::Detailed, "TEvStateStorage::TEvListStateStorageResult"));
Send(MakeStateStorageProxyID(), new TEvStateStorage::TEvListStateStorage(), 0/*flags*/, 0/*cookie*/, Span.GetTraceId());
SchemeBoardInfo = TRequestResponse<TEvStateStorage::TEvListSchemeBoardResult>(Span.CreateChild(TComponentTracingLevels::TTablet::Detailed, "TEvStateStorage::TEvListSchemeBoardResult"));
Send(MakeStateStorageProxyID(), new TEvStateStorage::TEvListSchemeBoard(false), 0/*flags*/, 0/*cookie*/, Span.GetTraceId());
BoardInfo = TRequestResponse<TEvStateStorage::TEvListBoardResult>(Span.CreateChild(TComponentTracingLevels::TTablet::Detailed, "TEvStateStorage::TEvListBoardResult"));
Send(MakeStateStorageProxyID(), new TEvStateStorage::TEvListBoard(), 0/*flags*/, 0/*cookie*/, Span.GetTraceId());
Requests += 3;
}


NodesInfo = TRequestResponse<TEvInterconnect::TEvNodesInfo>(Span.CreateChild(TComponentTracingLevels::TTablet::Detailed, "TEvInterconnect::TEvListNodes"));
Send(GetNameserviceActorId(), new TEvInterconnect::TEvListNodes(), 0/*flags*/, 0/*cookie*/, Span.GetTraceId());
Expand Down Expand Up @@ -921,6 +947,37 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
}
}

void RequestNodes(TIntrusiveConstPtr<TStateStorageInfo> info) {
for (const auto& group : info->RingGroups) {
for (const auto& ring : group.Rings) {
for (const auto& replica : ring.Replicas) {
RequestGenericNode(replica.NodeId());
}
}
}
}

void Handle(TEvStateStorage::TEvListStateStorageResult::TPtr& ev) {
if (StateStorageInfo->Set(std::move(ev))) {
RequestNodes(StateStorageInfo->Get()->Info);
RequestDone("TEvListStateStorageResult");
}
}

void Handle(TEvStateStorage::TEvListSchemeBoardResult::TPtr& ev) {
if (SchemeBoardInfo->Set(std::move(ev))) {
RequestNodes(SchemeBoardInfo->Get()->Info);
RequestDone("TEvListSсhemeBoardResult");
}
}

void Handle(TEvStateStorage::TEvListBoardResult::TPtr& ev) {
if (BoardInfo->Set(std::move(ev))) {
RequestNodes(BoardInfo->Get()->Info);
RequestDone("TEvListBoardResult");
}
}

STATEFN(StateWait) {
switch (ev->GetTypeRewrite()) {
hFunc(TEvents::TEvUndelivered, Handle);
Expand All @@ -946,6 +1003,9 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
hFunc(TEvStateStorage::TEvBoardInfo, Handle);
hFunc(TEvents::TEvWakeup, HandleTimeout);
hFunc(TEvNodeWardenStorageConfig, Handle);
hFunc(TEvStateStorage::TEvListStateStorageResult, Handle);
hFunc(TEvStateStorage::TEvListSchemeBoardResult, Handle);
hFunc(TEvStateStorage::TEvListBoardResult, Handle);
}
}

Expand Down Expand Up @@ -2999,6 +3059,18 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
message = std::regex_replace(message.c_str(), std::regex("^PDisk "), "PDisks ");
break;
}
case ETags::StateStorageRing: {
message = std::regex_replace(message.c_str(), std::regex("^Ring has "), "Rings have ");
message = std::regex_replace(message.c_str(), std::regex("^Ring is "), "Rings are ");
message = std::regex_replace(message.c_str(), std::regex("^Ring "), "Rings ");
break;
}
case ETags::StateStorageNode: {
message = std::regex_replace(message.c_str(), std::regex("^Ring has "), "Rings have ");
message = std::regex_replace(message.c_str(), std::regex("^Ring is "), "Rings are ");
message = std::regex_replace(message.c_str(), std::regex("^Ring "), "Rings ");
break;
}
default:
break;
}
Expand Down Expand Up @@ -3054,6 +3126,10 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
isSimilar = it->IssueLog.location().storage().pool().group().pile().name()
== similar.begin()->IssueLog.location().storage().pool().group().pile().name();
}
if (isSimilar && similar.begin()->IssueLog.location().compute().state_storage().has_pile()) {
isSimilar = it->IssueLog.location().compute().state_storage().pile().name()
== similar.begin()->IssueLog.location().compute().state_storage().pile().name();
}
if (isSimilar) {
auto move = it++;
similar.splice(similar.end(), records, move);
Expand Down Expand Up @@ -3295,6 +3371,8 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
MergeLevelRecords(mergeContext, ETags::VDiskState, ETags::BridgeGroupState);
MergeLevelRecords(mergeContext, ETags::VDiskState, ETags::GroupState);
MergeLevelRecords(mergeContext, ETags::PDiskState, ETags::VDiskState);
MergeLevelRecords(mergeContext, ETags::StateStorageRing);
MergeLevelRecords(mergeContext, ETags::StateStorageNode, ETags::StateStorageRing);
}
mergeContext.FillRecords(records);
}
Expand Down Expand Up @@ -3463,13 +3541,72 @@ class TSelfCheckRequest : public TActorBootstrapped<TSelfCheckRequest> {
}
}

void FillStateStorage(TOverallStateContext& context, TString type, TIntrusiveConstPtr<TStateStorageInfo> info) {
TSelfCheckResult ssContext;
ssContext.Type = type;
for (const auto& ringGroup : info->RingGroups) {
if (ringGroup.State != ERingGroupState::PRIMARY && ringGroup.State != ERingGroupState::SYNCHRONIZED) {
continue;
}
TSelfCheckResult* currentContext = &ssContext;
TSelfCheckContext pileContext(&ssContext, TStringBuilder() << "PILE_" << type);
if ((bool)ringGroup.BridgePileId && NodeWardenStorageConfig && NodeWardenStorageConfig->IsOk()) {
const auto& pileName = NodeWardenStorageConfig->Get()->BridgeInfo->GetPile(ringGroup.BridgePileId)->Name;
pileContext.Location.mutable_compute()->mutable_state_storage()->mutable_pile()->set_name(pileName);
currentContext = &pileContext;
}
ui32 disabledRings = 0;
ui32 badRings = 0;
for (size_t ringIdx = 0; ringIdx < ringGroup.Rings.size(); ++ringIdx) {
const auto& ring = ringGroup.Rings[ringIdx];
TSelfCheckContext ringContext(currentContext, TStringBuilder() << type << "_RING");
ringContext.Location.mutable_compute()->mutable_state_storage()->set_ring(ringIdx + 1);
if (ring.IsDisabled) {
++disabledRings;
continue;
}
for (const auto& replica : ring.Replicas) {
const auto node = replica.NodeId();
if (!NodeSystemState[node].IsOk()) {
TSelfCheckContext nodeContext(&ringContext, TStringBuilder() << type << "_NODE");
nodeContext.Location.mutable_compute()->mutable_state_storage()->mutable_node()->set_id(node);
nodeContext.ReportStatus(Ydb::Monitoring::StatusFlag::RED, "Node is not available", ETags::StateStorageNode);
}
}
ringContext.ReportWithMaxChildStatus("Ring has unavailable nodes", ETags::StateStorageRing, {ETags::StateStorageNode});
if (ringContext.GetOverallStatus() == Ydb::Monitoring::StatusFlag::RED) {
++badRings;
}
}
if (disabledRings + badRings > (ringGroup.NToSelect - 1) / 2) {
currentContext->ReportStatus(Ydb::Monitoring::StatusFlag::RED, "There is not enough functional rings", ETags::StateStorage);
} else if (badRings > 1) {
currentContext->ReportStatus(Ydb::Monitoring::StatusFlag::YELLOW, "Multiple rings have unavailable replicas", ETags::StateStorage);
} else if (badRings > 0) {
currentContext->ReportStatus(Ydb::Monitoring::StatusFlag::BLUE, "One ring has unavailable replicas", ETags::StateStorage);
}
}
MergeRecords(ssContext.IssueRecords);
context.UpdateMaxStatus(ssContext.GetOverallStatus());
context.AddIssues(ssContext.IssueRecords);
}

void FillResult(TOverallStateContext context) {
if (IsSpecificDatabaseFilter()) {
FillDatabaseResult(context, FilterDatabase, DatabaseState[FilterDatabase]);
} else {
for (auto& [path, state] : DatabaseState) {
FillDatabaseResult(context, path, state);
}
if (StateStorageInfo && StateStorageInfo->IsOk()) {
FillStateStorage(context, "STATE_STORAGE", StateStorageInfo->Get()->Info);
}
if (SchemeBoardInfo && SchemeBoardInfo->IsOk()) {
FillStateStorage(context, "SCHEME_BOARD", SchemeBoardInfo->Get()->Info);
}
if (BoardInfo && BoardInfo->IsOk()) {
FillStateStorage(context, "BOARD", BoardInfo->Get()->Info);
}
}
if (DatabaseState.empty()) {
Ydb::Monitoring::DatabaseStatus& databaseStatus(*context.Result->add_database_status());
Expand Down
Loading
Loading