-
Notifications
You must be signed in to change notification settings - Fork 596
HDDS-13133. Display Ratis state machine event timeline in OM web UI #10034
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
2593d9a
b3beabc
5b2de86
6cce694
1baf887
b2b5b76
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,13 +17,16 @@ | |
|
|
||
| package org.apache.hadoop.hdds.scm.container.placement.metrics; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
| import org.apache.hadoop.hdds.utils.DBCheckpointMetrics; | ||
| import org.apache.hadoop.metrics2.MetricsSystem; | ||
| import org.apache.hadoop.metrics2.annotation.Metric; | ||
| import org.apache.hadoop.metrics2.annotation.Metrics; | ||
| import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; | ||
| import org.apache.hadoop.metrics2.lib.MutableCounterLong; | ||
| import org.apache.hadoop.metrics2.lib.MutableGaugeLong; | ||
| import org.apache.hadoop.util.Time; | ||
|
|
||
| /** | ||
| * This class is for maintaining StorageContainerManager statistics. | ||
|
|
@@ -33,6 +36,9 @@ public class SCMMetrics { | |
| public static final String SOURCE_NAME = | ||
| SCMMetrics.class.getSimpleName(); | ||
|
|
||
| private final List<String> ratisEvents = new ArrayList<>(); | ||
| private static final int MAX_RATIS_EVENTS = 100; | ||
|
|
||
| /** | ||
| * Container stat metrics, the meaning of following metrics | ||
| * can be found in {@link ContainerStat}. | ||
|
|
@@ -155,6 +161,22 @@ public void decrContainerStat(ContainerStat deltaStat) { | |
| this.containerReportWriteCount.incr(-1 * deltaStat.getWriteCount().get()); | ||
| } | ||
|
|
||
| public void addRatisEvent(String event) { | ||
| synchronized (ratisEvents) { | ||
| if (ratisEvents.size() >= MAX_RATIS_EVENTS) { | ||
| ratisEvents.remove(0); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use LinkedList and removeFirst() to avoid array copying. |
||
| } | ||
| ratisEvents.add(Time.formatTime(Time.now()) + "|" + event); | ||
| } | ||
| } | ||
|
|
||
| @Metric("Ratis state machine events") | ||
| public String getRatisEvents() { | ||
| synchronized (ratisEvents) { | ||
| return String.join("\n", ratisEvents); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| } | ||
| } | ||
|
|
||
| public void unRegister() { | ||
| MetricsSystem ms = DefaultMetricsSystem.instance(); | ||
| ms.unregisterSource(SOURCE_NAME); | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,6 +26,7 @@ | |
| import java.io.IOException; | ||
| import java.lang.reflect.InvocationTargetException; | ||
| import java.lang.reflect.Method; | ||
| import java.util.ArrayList; | ||
| import java.util.Collection; | ||
| import java.util.EnumMap; | ||
| import java.util.List; | ||
|
|
@@ -39,6 +40,7 @@ | |
| import org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.RequestType; | ||
| import org.apache.hadoop.hdds.scm.block.DeletedBlockLog; | ||
| import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl; | ||
| import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMMetrics; | ||
| import org.apache.hadoop.hdds.scm.exceptions.SCMException; | ||
| import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes; | ||
| import org.apache.hadoop.hdds.scm.server.StorageContainerManager; | ||
|
|
@@ -213,6 +215,10 @@ public void notifyNotLeader(Collection<TransactionContext> pendingEntries) { | |
| return; | ||
| } | ||
| LOG.info("current leader SCM steps down."); | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
| if (metrics != null) { | ||
|
Comment on lines
+218
to
+219
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems that the returned metrics is never null. Add a field and a helper method. private final SCMMetrics metrics = StorageContainerManager.getMetrics(); private void addRatisEvent(String eventMessage) {
LOG.info(eventMessage);
metrics.addRatisEvent(eventMessage);
} |
||
| metrics.addRatisEvent("current leader SCM steps down."); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's include the id. addRatisEvent("This server " + getId() + " has stepped down from the Leader."); |
||
| } | ||
|
|
||
| scm.getScmContext().updateLeaderAndTerm(false, 0); | ||
| scm.getSCMServiceManager().notifyStatusChanged(); | ||
|
|
@@ -243,6 +249,12 @@ public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader( | |
| final String leaderNodeId = leaderDetails.get().getNodeId(); | ||
| LOG.info("Received install snapshot notification from SCM leader: {} with " | ||
| + "term index: {}", leaderAddress, firstTermIndexInLog); | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
| if (metrics != null) { | ||
| metrics.addRatisEvent( | ||
| "Installing snapshot from SCM leader " + leaderNodeId + | ||
| ", term index: " + firstTermIndexInLog); | ||
| } | ||
|
|
||
| CompletableFuture<TermIndex> future = CompletableFuture.supplyAsync( | ||
| () -> { | ||
|
|
@@ -283,6 +295,7 @@ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId, | |
| if (!isInitialized) { | ||
| return; | ||
| } | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
|
|
||
| currentLeaderTerm.set(scm.getScmHAManager().getRatisServer().getDivision() | ||
| .getInfo().getCurrentTerm()); | ||
|
|
@@ -297,10 +310,16 @@ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId, | |
|
|
||
| if (!groupMemberId.getPeerId().equals(newLeaderId)) { | ||
| LOG.info("leader changed, yet current SCM is still follower."); | ||
| if (metrics != null) { | ||
| metrics.addRatisEvent("Leader changed to " + newLeaderId + ", yet current SCM is still follower."); | ||
|
Comment on lines
312
to
+314
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's include the id of this server. The log message and the addRatisEvent("Leader changed to " + newLeaderId
+ ", this server " + groupMemberId.getPeerId() + " remains a Follower."); |
||
| } | ||
| return; | ||
| } | ||
|
|
||
| LOG.info("current SCM becomes leader of term {}.", currentLeaderTerm); | ||
| if (metrics != null) { | ||
| metrics.addRatisEvent("current SCM becomes leader of term " + currentLeaderTerm); | ||
| } | ||
|
|
||
| scm.getScmContext().updateLeaderAndTerm(true, | ||
| currentLeaderTerm.get()); | ||
|
|
@@ -394,11 +413,35 @@ public void notifyLeaderReady() { | |
| scm.getScmContext().setLeaderReady(); | ||
| scm.getSCMServiceManager().notifyStatusChanged(); | ||
| scm.getFinalizationManager().onLeaderReady(); | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
| if (metrics != null) { | ||
| metrics.addRatisEvent("Ready to serve requests as the leader"); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void notifyConfigurationChanged(long term, long index, | ||
| RaftProtos.RaftConfigurationProto newRaftConfiguration) { | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
| if (metrics != null) { | ||
| List<RaftProtos.RaftPeerProto> newPeers = | ||
| newRaftConfiguration.getPeersList(); | ||
| List<RaftProtos.RaftPeerProto> newListeners = | ||
| newRaftConfiguration.getListenersList(); | ||
| List<String> newPeerIds = new ArrayList<>(); | ||
| List<String> newListenersIds = new ArrayList<>(); | ||
| for (RaftProtos.RaftPeerProto raftPeerProto : newPeers) { | ||
| newPeerIds.add(RaftPeerId.valueOf(raftPeerProto.getId()).toString()); | ||
| } | ||
| for (RaftProtos.RaftPeerProto raftListenerProto : newListeners) { | ||
| newListenersIds.add(RaftPeerId.valueOf(raftListenerProto.getId()).toString()); | ||
| } | ||
| metrics.addRatisEvent( | ||
| "New peers " + newPeerIds + | ||
| (newListenersIds.isEmpty() ? "" : ", new listeners " + newListenersIds) + | ||
| " added at term index (" + | ||
|
Comment on lines
+440
to
+442
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. getPeersList() returns the current peers in the conf. It include both the newly added peers and the existing peers (unless they are removed). addRatisEvent("notifyConfigurationChanged at " + TermIndex.valueOf(term, index)
+ ": " + TextFormat.shortDebugString(newRaftConfiguration)); |
||
| term + ", " + index + ")"); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -431,6 +474,10 @@ public void reinitialize() throws IOException { | |
| } | ||
|
|
||
| LOG.info("{}: SCMStateMachine is reinitializing. newTermIndex = {}", getId(), termIndex); | ||
| SCMMetrics metrics = StorageContainerManager.getMetrics(); | ||
| if (metrics != null) { | ||
| metrics.addRatisEvent("reinitialize: " + termIndex); | ||
| } | ||
|
|
||
| // re-initialize the DBTransactionBuffer and update the lastAppliedIndex. | ||
| try { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,32 @@ | ||
| <!-- | ||
| Licensed to the Apache Software Foundation (ASF) under one or more | ||
| contributor license agreements. See the NOTICE file distributed with | ||
| this work for additional information regarding copyright ownership. | ||
| The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| (the "License"); you may not use this file except in compliance with | ||
| the License. You may obtain a copy of the License at | ||
|
|
||
| http://www.apache.org/licenses/LICENSE-2.0 | ||
|
|
||
| Unless required by applicable law or agreed to in writing, software | ||
| distributed under the License is distributed on an "AS IS" BASIS, | ||
| WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| See the License for the specific language governing permissions and | ||
| limitations under the License. | ||
| --> | ||
| <h1>Ratis event timeline</h1> | ||
|
|
||
| <table class="table table-striped"> | ||
| <thead> | ||
| <tr> | ||
| <th class="col-md-3">Timestamp</th> | ||
| <th class="col-md-9">Event description</th> | ||
| </tr> | ||
| </thead> | ||
| <tbody> | ||
| <tr ng-repeat="event in $ctrl.events track by $index"> | ||
| <td>{{event.timestamp}}</td> | ||
| <td>{{event.description}}</td> | ||
| </tr> | ||
| </tbody> | ||
| </table> |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,56 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hadoop.hdds.scm.ha; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.mockStatic; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMMetrics; | ||
| import org.apache.hadoop.hdds.scm.server.StorageContainerManager; | ||
| import org.apache.hadoop.hdds.utils.TransactionInfo; | ||
| import org.apache.ratis.proto.RaftProtos; | ||
| import org.apache.ratis.server.protocol.TermIndex; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.mockito.MockedStatic; | ||
|
|
||
| /** | ||
| * Test SCMStateMachine events recording. | ||
| */ | ||
| public class TestSCMStateMachine { | ||
|
|
||
| @Test | ||
| public void testRatisEventsRecording() throws Exception { | ||
| StorageContainerManager scm = mock(StorageContainerManager.class); | ||
| SCMMetrics metrics = SCMMetrics.create(); | ||
| SCMHADBTransactionBuffer buffer = mock(SCMHADBTransactionBuffer.class); | ||
| when(buffer.getLatestTrxInfo()).thenReturn(TransactionInfo.valueOf(TermIndex.valueOf(0, 0))); | ||
|
|
||
| SCMStateMachine stateMachine = new SCMStateMachine(scm, buffer); | ||
|
|
||
| try (MockedStatic<StorageContainerManager> scmStaticMock = mockStatic(StorageContainerManager.class)) { | ||
| scmStaticMock.when(StorageContainerManager::getMetrics).thenReturn(metrics); | ||
|
|
||
| stateMachine.notifyConfigurationChanged(1, 1, RaftProtos.RaftConfigurationProto.getDefaultInstance()); | ||
| assertTrue(metrics.getRatisEvents().contains("New peers [] added at term index")); | ||
| } | ||
|
|
||
| metrics.unRegister(); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's make it configurable?