aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorinnokentii <innokentii@yandex-team.com>2022-11-17 15:52:18 +0300
committerinnokentii <innokentii@yandex-team.com>2022-11-17 15:52:18 +0300
commitf125f2c0683d82b24b9bd49de73f379d609d906a (patch)
tree14dd575f16871623ff844b77fa1955fce9a84b53
parent112fc78512fcf54918cf0805b8a61fa9391cd84d (diff)
downloadydb-f125f2c0683d82b24b9bd49de73f379d609d906a.tar.gz
Implement Sentinel nodes updating with request
implement nodes updating by request
-rw-r--r--ydb/core/cms/cms.cpp11
-rw-r--r--ydb/core/cms/cms_impl.h3
-rw-r--r--ydb/core/cms/sentinel.cpp132
-rw-r--r--ydb/core/cms/sentinel_impl.h34
-rw-r--r--ydb/core/cms/sentinel_ut.cpp37
-rw-r--r--ydb/core/cms/ui/sentinel_state.js3
-rw-r--r--ydb/core/protos/cms.proto21
7 files changed, 178 insertions, 63 deletions
diff --git a/ydb/core/cms/cms.cpp b/ydb/core/cms/cms.cpp
index 050bb3e417..2f2318cfc9 100644
--- a/ydb/core/cms/cms.cpp
+++ b/ydb/core/cms/cms.cpp
@@ -1075,7 +1075,7 @@ void TCms::Die(const TActorContext& ctx)
TActorBase::Die(ctx);
}
-void TCms::AddHostState(const TNodeInfo &node, TClusterStateResponse &resp, TInstant timestamp)
+void TCms::AddHostState(const TClusterInfoPtr &clusterInfo, const TNodeInfo &node, TClusterStateResponse &resp, TInstant timestamp)
{
auto *host = resp.MutableState()->AddHosts();
host->SetName(node.Host);
@@ -1083,6 +1083,7 @@ void TCms::AddHostState(const TNodeInfo &node, TClusterStateResponse &resp, TIns
host->SetNodeId(node.NodeId);
host->SetInterconnectPort(node.IcPort);
host->SetTimestamp(timestamp.GetValue());
+ node.Location.Serialize(host->MutableLocation(), false);
if (node.State == UP || node.VDisks || node.PDisks) {
for (const auto flag : GetEnumAllValues<EService>()) {
if (!(node.Services & flag)) {
@@ -1099,7 +1100,7 @@ void TCms::AddHostState(const TNodeInfo &node, TClusterStateResponse &resp, TIns
}
for (const auto &vdId : node.VDisks) {
- const auto &vdisk = ClusterInfo->VDisk(vdId);
+ const auto &vdisk = clusterInfo->VDisk(vdId);
auto *device = host->AddDevices();
device->SetName(vdisk.GetDeviceName());
device->SetState(vdisk.State);
@@ -1107,7 +1108,7 @@ void TCms::AddHostState(const TNodeInfo &node, TClusterStateResponse &resp, TIns
}
for (const auto &pdId : node.PDisks) {
- const auto &pdisk = ClusterInfo->PDisk(pdId);
+ const auto &pdisk = clusterInfo->PDisk(pdId);
auto *device = host->AddDevices();
device->SetName(pdisk.GetDeviceName());
device->SetState(pdisk.State);
@@ -1604,7 +1605,7 @@ void TCms::Handle(TEvCms::TEvClusterStateRequest::TPtr &ev,
for (const auto &host : rec.GetHosts()) {
if (ClusterInfo->NodesCount(host) >= 1) {
for (const TNodeInfo *node : ClusterInfo->HostNodes(host)) {
- AddHostState(*node, resp->Record, ClusterInfo->GetTimestamp());
+ AddHostState(ClusterInfo, *node, resp->Record, ClusterInfo->GetTimestamp());
}
} else {
return ReplyWithError<TEvCms::TEvClusterStateResponse>(
@@ -1613,7 +1614,7 @@ void TCms::Handle(TEvCms::TEvClusterStateRequest::TPtr &ev,
}
} else {
for (const auto &entry : ClusterInfo->AllNodes())
- AddHostState(*entry.second, resp->Record, ClusterInfo->GetTimestamp());
+ AddHostState(ClusterInfo, *entry.second, resp->Record, ClusterInfo->GetTimestamp());
}
resp->Record.MutableStatus()->SetCode(TStatus::OK);
diff --git a/ydb/core/cms/cms_impl.h b/ydb/core/cms/cms_impl.h
index 4a6e4d3e9f..ccf626dcf4 100644
--- a/ydb/core/cms/cms_impl.h
+++ b/ydb/core/cms/cms_impl.h
@@ -75,6 +75,8 @@ public:
void PersistNodeTenants(TTransactionContext& txc, const TActorContext& ctx);
+ static void AddHostState(const TClusterInfoPtr &clusterInfo, const TNodeInfo &node, NKikimrCms::TClusterStateResponse &resp, TInstant timestamp);
+
private:
using TActorBase = TActor<TCms>;
using EStatusCode = NKikimrCms::TStatus::ECode;
@@ -354,7 +356,6 @@ private:
void Cleanup(const TActorContext &ctx);
void Die(const TActorContext& ctx) override;
- void AddHostState(const TNodeInfo &node, NKikimrCms::TClusterStateResponse &resp, TInstant timestamp);
void GetPermission(TEvCms::TEvManagePermissionRequest::TPtr &ev, bool all, const TActorContext &ctx);
void RemovePermission(TEvCms::TEvManagePermissionRequest::TPtr &ev, bool done, const TActorContext &ctx);
void GetRequest(TEvCms::TEvManageRequestRequest::TPtr &ev, bool all, const TActorContext &ctx);
diff --git a/ydb/core/cms/sentinel.cpp b/ydb/core/cms/sentinel.cpp
index 530202795a..67ee9f914a 100644
--- a/ydb/core/cms/sentinel.cpp
+++ b/ydb/core/cms/sentinel.cpp
@@ -187,15 +187,14 @@ void TPDiskInfo::AddState(EPDiskState state) {
/// TClusterMap
-TClusterMap::TClusterMap(TCmsStatePtr state)
+TClusterMap::TClusterMap(TSentinelState::TPtr state)
: State(state)
{
}
void TClusterMap::AddPDisk(const TPDiskID& id) {
- Y_VERIFY(State->ClusterInfo->HasNode(id.NodeId));
- Y_VERIFY(State->ClusterInfo->HasPDisk(id));
- const auto& location = State->ClusterInfo->Node(id.NodeId).Location;
+ Y_VERIFY(State->Nodes.contains(id.NodeId));
+ const auto& location = State->Nodes[id.NodeId].Location;
ByDataCenter[location.HasKey(TNodeLocation::TKeys::DataCenter) ? location.GetDataCenterId() : ""].insert(id);
ByRoom[location.HasKey(TNodeLocation::TKeys::Module) ? location.GetModuleId() : ""].insert(id);
@@ -205,7 +204,7 @@ void TClusterMap::AddPDisk(const TPDiskID& id) {
/// TGuardian
-TGuardian::TGuardian(TCmsStatePtr state, ui32 dataCenterRatio, ui32 roomRatio, ui32 rackRatio)
+TGuardian::TGuardian(TSentinelState::TPtr state, ui32 dataCenterRatio, ui32 roomRatio, ui32 rackRatio)
: TClusterMap(state)
, DataCenterRatio(dataCenterRatio)
, RoomRatio(roomRatio)
@@ -271,15 +270,6 @@ TClusterMap::TPDiskIDSet TGuardian::GetAllowedPDisks(const TClusterMap& all, TSt
return result;
}
-/// Main state
-struct TSentinelState: public TSimpleRefCount<TSentinelState> {
- using TPtr = TIntrusivePtr<TSentinelState>;
-
- TMap<TPDiskID, TPDiskInfo::TPtr> PDisks;
- THashSet<ui32> StateUpdaterWaitNodes;
- ui32 ConfigUpdaterAttempt = 0;
-};
-
/// Actors
template <typename TDerived>
@@ -339,14 +329,44 @@ protected:
}; // TUpdaterBase
class TConfigUpdater: public TUpdaterBase<TEvSentinel::TEvConfigUpdated, TConfigUpdater> {
- void Retry() {
- ++SentinelState->ConfigUpdaterAttempt;
- Schedule(Config.RetryUpdateConfig, new TEvSentinel::TEvRetry());
+ enum class RetryCookie {
+ BSC,
+ CMS,
+ };
+
+ void MaybeReply() {
+ if (SentinelState->ConfigUpdaterState.GotBSCResponse && SentinelState->ConfigUpdaterState.GotCMSResponse) {
+ Reply();
+ }
+ }
+
+ void RetryBSC() {
+ ++SentinelState->ConfigUpdaterState.BSCAttempt;
+ Schedule(Config.RetryUpdateConfig, new TEvents::TEvWakeup(static_cast<ui64>(RetryCookie::BSC)));
+ }
+
+ void RetryCMS() {
+ ++SentinelState->ConfigUpdaterState.CMSAttempt;
+ Schedule(Config.RetryUpdateConfig, new TEvents::TEvWakeup(static_cast<ui64>(RetryCookie::CMS)));
+ }
+
+ void OnRetry(TEvents::TEvWakeup::TPtr& ev) {
+ const auto* msg = ev->Get();
+ switch (static_cast<RetryCookie>(msg->Tag)) {
+ case RetryCookie::BSC:
+ RequestBSConfig();
+ break;
+ case RetryCookie::CMS:
+ RequestCMSClusterState();
+ break;
+ default:
+ Y_FAIL("Unexpected case");
+ }
}
void RequestBSConfig() {
LOG_D("Request blobstorage config"
- << ": attempt# " << SentinelState->ConfigUpdaterAttempt);
+ << ": attempt# " << SentinelState->ConfigUpdaterState.BSCAttempt);
if (!CmsState->BSControllerPipe) {
ConnectBSC();
@@ -357,6 +377,46 @@ class TConfigUpdater: public TUpdaterBase<TEvSentinel::TEvConfigUpdated, TConfig
NTabletPipe::SendData(SelfId(), CmsState->BSControllerPipe, request.Release());
}
+ void RequestCMSClusterState() {
+ LOG_D("Request CMS cluster state"
+ << ": attempt# " << SentinelState->ConfigUpdaterState.CMSAttempt);
+ // We aren't tracking delivery due to invariant that CMS always kills sentinel when dies itself
+ Send(CmsState->CmsActorId, new TEvCms::TEvClusterStateRequest());
+ }
+
+ void Handle(TEvCms::TEvClusterStateResponse::TPtr& ev) {
+ const auto& record = ev->Get()->Record;
+
+ LOG_D("Handle TEvCms::TEvClusterStateResponse"
+ << ": response# " << record.ShortDebugString());
+
+ if (!record.HasStatus() || !record.GetStatus().HasCode() || record.GetStatus().GetCode() != NKikimrCms::TStatus::OK) {
+ TString error = "<no description>";
+ if (record.HasStatus() && record.GetStatus().HasCode() && record.GetStatus().HasReason()) {
+ error = NKikimrCms::TStatus::ECode_Name(record.GetStatus().GetCode()) + " " + record.GetStatus().GetReason();
+ }
+
+ LOG_E("Unsuccesful response from CMS"
+ << ", error# " << error);
+
+ RetryCMS();
+
+ return;
+ }
+
+ if (record.HasState()) {
+ SentinelState->Nodes.clear();
+ for (ui32 i = 0; i < record.GetState().HostsSize(); ++i) {
+ const auto& host = record.GetState().GetHosts(i);
+ if (host.HasNodeId() && host.HasLocation() && host.HasName()) {
+ SentinelState->Nodes.emplace(host.GetNodeId(), TNodeInfo{host.GetName(), NActors::TNodeLocation(host.GetLocation())});
+ }
+ }
+ }
+ SentinelState->ConfigUpdaterState.GotCMSResponse = true;
+ MaybeReply();
+ }
+
void Handle(TEvBlobStorage::TEvControllerConfigResponse::TPtr& ev) {
const auto& response = ev->Get()->Record.GetResponse();
@@ -372,7 +432,7 @@ class TConfigUpdater: public TUpdaterBase<TEvSentinel::TEvConfigUpdated, TConfig
LOG_E("Unsuccesful response from BSC"
<< ", size# " << response.StatusSize()
<< ", error# " << error);
- Retry();
+ RetryBSC();
} else {
auto& pdisks = SentinelState->PDisks;
@@ -387,13 +447,15 @@ class TConfigUpdater: public TUpdaterBase<TEvSentinel::TEvConfigUpdated, TConfig
pdisks.emplace(id, new TPDiskInfo(pdisk.GetDriveStatus(), Config.DefaultStateLimit, Config.StateLimits));
}
- Reply();
+ SentinelState->ConfigUpdaterState.GotBSCResponse = true;
+
+ MaybeReply();
}
}
void OnPipeDisconnected() {
LOG_E("Pipe to BSC disconnected");
- Retry();
+ RetryBSC();
}
public:
@@ -409,19 +471,22 @@ public:
void Bootstrap() {
RequestBSConfig();
+ RequestCMSClusterState();
Become(&TThis::StateWork);
}
void PassAway() override {
- SentinelState->ConfigUpdaterAttempt = 0;
+ SentinelState->ConfigUpdaterState.Clear();
TActor::PassAway();
}
STATEFN(StateWork) {
switch (ev->GetTypeRewrite()) {
- sFunc(TEvSentinel::TEvRetry, RequestBSConfig);
+ hFunc(TEvents::TEvWakeup, OnRetry);
sFunc(TEvSentinel::TEvBSCPipeDisconnected, OnPipeDisconnected);
+ hFunc(TEvCms::TEvClusterStateResponse, Handle);
+
hFunc(TEvBlobStorage::TEvControllerConfigResponse, Handle);
sFunc(TEvents::TEvPoisonPill, PassAway);
@@ -833,8 +898,8 @@ class TSentinel: public TActorBootstrapped<TSentinel> {
action.SetCurrentStatus(status);
action.SetRequiredStatus(requiredStatus);
- Y_VERIFY(CmsState->ClusterInfo->HasNode(id.NodeId));
- action.SetHost(CmsState->ClusterInfo->Node(id.NodeId).Host);
+ Y_VERIFY(SentinelState->Nodes.contains(id.NodeId));
+ action.SetHost(SentinelState->Nodes[id.NodeId].Host);
if (reason) {
action.SetReason(reason);
@@ -869,7 +934,7 @@ class TSentinel: public TActorBootstrapped<TSentinel> {
EnsureAllTouched();
- if (!CmsState->ClusterInfo) {
+ if (SentinelState->Nodes.empty()) {
LOG_C("Missing cluster info");
ScheduleUpdate<TEvSentinel::TEvUpdateState, TConfigUpdater>(
StateUpdater, Config.UpdateStateInterval, ConfigUpdater
@@ -878,26 +943,20 @@ class TSentinel: public TActorBootstrapped<TSentinel> {
return;
}
- TClusterMap all(CmsState);
- TGuardian changed(CmsState, Config.DataCenterRatio, Config.RoomRatio, Config.RackRatio);
+ TClusterMap all(SentinelState);
+ TGuardian changed(SentinelState, Config.DataCenterRatio, Config.RoomRatio, Config.RackRatio);
TClusterMap::TPDiskIDSet alwaysAllowed;
for (auto& pdisk : SentinelState->PDisks) {
const TPDiskID& id = pdisk.first;
TPDiskInfo& info = *(pdisk.second);
- if (!CmsState->ClusterInfo->HasNode(id.NodeId)) {
+ if (!SentinelState->Nodes.contains(id.NodeId)) {
LOG_E("Missing node info"
<< ": pdiskId# " << id);
continue;
}
- if (!CmsState->ClusterInfo->HasPDisk(id)) {
- LOG_E("Missing pdisk info"
- << ": pdiskId# " << id);
- continue;
- }
-
all.AddPDisk(id);
if (info.IsChanged()) {
if (info.IsNewStatusGood()) {
@@ -981,7 +1040,8 @@ class TSentinel: public TActorBootstrapped<TSentinel> {
configUpdater.MutableUpdaterInfo()->SetActorId(ConfigUpdater.Id.ToString());
configUpdater.MutableUpdaterInfo()->SetStartedAt(ConfigUpdater.StartedAt.ToString());
configUpdater.MutableUpdaterInfo()->SetDelayed(ConfigUpdater.Delayed);
- configUpdater.SetAttempt(SentinelState->ConfigUpdaterAttempt);
+ configUpdater.SetBSCAttempt(SentinelState->ConfigUpdaterState.BSCAttempt);
+ configUpdater.SetCMSAttempt(SentinelState->ConfigUpdaterState.CMSAttempt);
for (const auto& [id, info] : SentinelState->PDisks) {
auto& entry = *record.AddPDisks();
diff --git a/ydb/core/cms/sentinel_impl.h b/ydb/core/cms/sentinel_impl.h
index c634865943..05423e3ab0 100644
--- a/ydb/core/cms/sentinel_impl.h
+++ b/ydb/core/cms/sentinel_impl.h
@@ -102,19 +102,47 @@ private:
bool Touched;
}; // TPDiskInfo
+struct TNodeInfo {
+ TString Host;
+ NActors::TNodeLocation Location;
+};
+
+struct TConfigUpdaterState {
+ ui32 BSCAttempt = 0;
+ ui32 CMSAttempt = 0;
+ bool GotBSCResponse = false;
+ bool GotCMSResponse = false;
+
+ void Clear() {
+ *this = TConfigUpdaterState{};
+ }
+};
+
+/// Main state
+struct TSentinelState: public TSimpleRefCount<TSentinelState> {
+ using TPtr = TIntrusivePtr<TSentinelState>;
+
+ using TNodeId = ui32;
+
+ TMap<TPDiskID, TPDiskInfo::TPtr> PDisks;
+ TMap<TNodeId, TNodeInfo> Nodes;
+ THashSet<ui32> StateUpdaterWaitNodes;
+ TConfigUpdaterState ConfigUpdaterState;
+};
+
class TClusterMap {
public:
using TPDiskIDSet = THashSet<TPDiskID, TPDiskIDHash>;
using TDistribution = THashMap<TString, TPDiskIDSet>;
using TNodeIDSet = THashSet<ui32>;
- TCmsStatePtr State;
+ TSentinelState::TPtr State;
TDistribution ByDataCenter;
TDistribution ByRoom;
TDistribution ByRack;
THashMap<TString, TNodeIDSet> NodeByRack;
- TClusterMap(TCmsStatePtr state);
+ TClusterMap(TSentinelState::TPtr state);
void AddPDisk(const TPDiskID& id);
}; // TClusterMap
@@ -129,7 +157,7 @@ class TGuardian : public TClusterMap {
}
public:
- explicit TGuardian(TCmsStatePtr state, ui32 dataCenterRatio = 100, ui32 roomRatio = 100, ui32 rackRatio = 100);
+ explicit TGuardian(TSentinelState::TPtr state, ui32 dataCenterRatio = 100, ui32 roomRatio = 100, ui32 rackRatio = 100);
TPDiskIDSet GetAllowedPDisks(const TClusterMap& all, TString& issues, TPDiskIDSet& disallowed) const;
diff --git a/ydb/core/cms/sentinel_ut.cpp b/ydb/core/cms/sentinel_ut.cpp
index 0ad3a65f22..0e1f33ae94 100644
--- a/ydb/core/cms/sentinel_ut.cpp
+++ b/ydb/core/cms/sentinel_ut.cpp
@@ -1,6 +1,7 @@
#include "cms_ut_common.h"
#include "sentinel.h"
#include "sentinel_impl.h"
+#include "cms_impl.h"
#include <library/cpp/testing/unittest/registar.h>
@@ -136,7 +137,8 @@ Y_UNIT_TEST_SUITE(TSentinelBaseTests) {
}
}
- TCmsStatePtr MockCmsState(ui16 numDataCenter, ui16 racksPerDataCenter, ui16 nodesPerRack, ui16 pdisksPerNode, bool anyDC, bool anyRack) {
+ std::pair<TCmsStatePtr, TSentinelState::TPtr> MockCmsState(ui16 numDataCenter, ui16 racksPerDataCenter, ui16 nodesPerRack, ui16 pdisksPerNode, bool anyDC, bool anyRack) {
+ TSentinelState::TPtr sentinelState = new TSentinelState;
TCmsStatePtr state = new TCmsState;
state->ClusterInfo = new TClusterInfo;
@@ -156,6 +158,7 @@ Y_UNIT_TEST_SUITE(TSentinelBaseTests) {
location.SetUnit(ToString(id));
state->ClusterInfo->AddNode(TEvInterconnect::TNodeInfo(id, name, name, name, 10000, TNodeLocation(location)), nullptr);
+ sentinelState->Nodes[id] = NSentinel::TNodeInfo{name, NActors::TNodeLocation(location)};
for (ui64 npdisk : xrange(pdisksPerNode)) {
NKikimrBlobStorage::TBaseConfig::TPDisk pdisk;
@@ -168,16 +171,16 @@ Y_UNIT_TEST_SUITE(TSentinelBaseTests) {
}
}
- return state;
+ return {state, sentinelState};
}
void GuardianDataCenterRatio(ui16 numDataCenter, const TVector<ui16>& nodesPerDataCenterVariants, bool anyDC = false) {
UNIT_ASSERT(!anyDC || numDataCenter == 1);
for (ui16 nodesPerDataCenter : nodesPerDataCenterVariants) {
- TCmsStatePtr state = MockCmsState(numDataCenter, nodesPerDataCenter, 1, 1, anyDC, false);
- TGuardian all(state);
- TGuardian changed(state, 50);
+ auto [state, sentinelState] = MockCmsState(numDataCenter, nodesPerDataCenter, 1, 1, anyDC, false);
+ TGuardian all(sentinelState);
+ TGuardian changed(sentinelState, 50);
THashSet<TPDiskID, TPDiskIDHash> changedSet;
const auto& nodes = state->ClusterInfo->AllNodes();
@@ -233,10 +236,10 @@ Y_UNIT_TEST_SUITE(TSentinelBaseTests) {
void GuardianRackRatio(ui16 numRacks, const TVector<ui16>& nodesPerRackVariants, ui16 numPDisks, bool anyRack) {
for (ui16 nodesPerRack : nodesPerRackVariants) {
- TCmsStatePtr state = MockCmsState(1, numRacks, nodesPerRack, numPDisks, false, anyRack);
+ auto [state, sentinelState] = MockCmsState(1, numRacks, nodesPerRack, numPDisks, false, anyRack);
- TGuardian all(state);
- TGuardian changed(state, 100, 100, 50);
+ TGuardian all(sentinelState);
+ TGuardian changed(sentinelState, 100, 100, 50);
THashSet<TPDiskID, TPDiskIDHash> changedSet;
const auto& nodes = state->ClusterInfo->AllNodes();
@@ -371,9 +374,27 @@ Y_UNIT_TEST_SUITE(TSentinelTests) {
return true;
}
});
+ auto prevObserver = SetObserverFunc(&TTestActorRuntimeBase::DefaultObserverFunc);
+ SetObserverFunc([this, prevObserver](TTestActorRuntimeBase& runtime,
+ TAutoPtr<IEventHandle> &event){
+ if (event->GetTypeRewrite() == TEvCms::TEvClusterStateRequest::EventType) {
+ TAutoPtr<TEvCms::TEvClusterStateResponse> resp = new TEvCms::TEvClusterStateResponse;
+ if (State) {
+ resp->Record.MutableStatus()->SetCode(NKikimrCms::TStatus::OK);
+ for (const auto &entry : State->ClusterInfo->AllNodes()) {
+ NCms::TCms::AddHostState(State->ClusterInfo, *entry.second, resp->Record, State->ClusterInfo->GetTimestamp());
+ }
+ }
+ Send(new IEventHandle(event->Sender, TActorId(), resp.Release()));
+ return TTestActorRuntime::EEventAction::PROCESS;
+ }
+
+ return prevObserver(runtime, event);
+ });
State = new TCmsState;
MockClusterInfo(State->ClusterInfo);
+ State->CmsActorId = GetSender();
Sentinel = Register(CreateSentinel(State));
EnableScheduleForActor(Sentinel, true);
diff --git a/ydb/core/cms/ui/sentinel_state.js b/ydb/core/cms/ui/sentinel_state.js
index a17975f7da..36271a9b99 100644
--- a/ydb/core/cms/ui/sentinel_state.js
+++ b/ydb/core/cms/ui/sentinel_state.js
@@ -249,7 +249,8 @@ function onCmsSentinelStateLoaded(data) {
renderPVEntry(CmsSentinelState.stateUpdater, flattenStateUpdaterResp);
var flattenConfigUpdaterResp = data["ConfigUpdater"]["UpdaterInfo"];
- flattenConfigUpdaterResp["Attempt"] = data["ConfigUpdater"]["Attempt"];
+ flattenConfigUpdaterResp["BSCAttempt"] = data["ConfigUpdater"]["BSCAttempt"];
+ flattenConfigUpdaterResp["CMSAttempt"] = data["ConfigUpdater"]["CMSAttempt"];
renderPVEntry(CmsSentinelState.configUpdater, flattenConfigUpdaterResp);
} else {
$("#sentinel-error").text("Error while updating state");
diff --git a/ydb/core/protos/cms.proto b/ydb/core/protos/cms.proto
index c898aa3dd8..c72fc97f84 100644
--- a/ydb/core/protos/cms.proto
+++ b/ydb/core/protos/cms.proto
@@ -1,5 +1,6 @@
import "ydb/core/protos/blobstorage_config.proto";
import "ydb/core/protos/blobstorage_disk.proto";
+import "library/cpp/actors/protos/interconnect.proto";
package NKikimrCms;
option java_package = "ru.yandex.kikimr.proto";
@@ -67,14 +68,15 @@ message TDeviceState {
}
message THostState {
- optional string Name = 1;
- optional EState State = 2;
- repeated TServiceState Services = 3;
- repeated TDeviceState Devices = 4;
- optional uint64 Timestamp = 5;
- optional uint32 NodeId = 6;
- optional uint32 InterconnectPort = 7;
- repeated EMarker Markers = 8;
+ optional string Name = 1;
+ optional EState State = 2;
+ repeated TServiceState Services = 3;
+ repeated TDeviceState Devices = 4;
+ optional uint64 Timestamp = 5;
+ optional uint32 NodeId = 6;
+ optional uint32 InterconnectPort = 7;
+ repeated EMarker Markers = 8;
+ optional NActorsInterconnect.TNodeLocation Location = 9;
}
message TClusterState {
@@ -621,7 +623,8 @@ message TStateUpdaterState {
message TConfigUpdaterState {
optional TUpdaterInfo UpdaterInfo = 1;
- optional uint32 Attempt = 2;
+ optional uint32 BSCAttempt = 2;
+ optional uint32 CMSAttempt = 3;
}
message TGetSentinelStateResponse {