Skip to content
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -628,6 +628,10 @@ public final class ScmConfigKeys {
public static final String OZONE_SCM_HA_RATIS_SERVER_RPC_FIRST_ELECTION_TIMEOUT
= "ozone.scm.ha.raft.server.rpc.first-election.timeout";

public static final String OZONE_SCM_RATIS_EVENTS_MAX_LIMIT =
"ozone.scm.ratis.events.max.limit";
public static final int OZONE_SCM_RATIS_EVENTS_MAX_LIMIT_DEFAULT = 100;

/**
* Never constructed.
*/
Expand Down
12 changes: 12 additions & 0 deletions hadoop-hdds/common/src/main/resources/ozone-default.xml
Original file line number Diff line number Diff line change
Expand Up @@ -5044,4 +5044,16 @@
<value>5m</value>
<description>Interval for cleaning up orphan snapshot local data versions corresponding to snapshots</description>
</property>
<property>
<name>ozone.scm.ratis.events.max.limit</name>
<value>100</value>
<tag>OZONE, RATIS, SCM</tag>
<description>The maximum number of events that can be pending in SCM Ratis.</description>
</property>
<property>
<name>ozone.om.ratis.events.max.limit</name>
<value>100</value>
<tag>OZONE, RATIS, OM</tag>
<description>The maximum number of events that can be pending in OM Ratis.</description>
</property>
</configuration>
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
</ul>
</li>
<li><a href="#!/config">Configuration</a></li>
<li><a href="#!/ratis_events">Ratis event timeline</a></li>
<li ng-show="$ctrl.docs"><a href="/docs">Documentation</a></li>
<li>
<a class="dropdown-toggle"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,17 @@

package org.apache.hadoop.hdds.scm.container.placement.metrics;

import java.util.LinkedList;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
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.
Expand All @@ -33,6 +37,9 @@ public class SCMMetrics {
public static final String SOURCE_NAME =
SCMMetrics.class.getSimpleName();

private final LinkedList<String> ratisEvents = new LinkedList<>();
private final int maxRatisEvents;

/**
* Container stat metrics, the meaning of following metrics
* can be found in {@link ContainerStat}.
Expand All @@ -59,14 +66,23 @@ public DBCheckpointMetrics getDBCheckpointMetrics() {
return dbCheckpointMetrics;
}

public SCMMetrics() {
public SCMMetrics(int maxRatisEvents) {
dbCheckpointMetrics = DBCheckpointMetrics.create("SCM Metrics");
this.maxRatisEvents = maxRatisEvents;
}

public static SCMMetrics create() {
return create(null);
}

public static SCMMetrics create(ConfigurationSource conf) {
MetricsSystem ms = DefaultMetricsSystem.instance();
int maxRatisEvents = conf == null
? ScmConfigKeys.OZONE_SCM_RATIS_EVENTS_MAX_LIMIT_DEFAULT
: conf.getInt(ScmConfigKeys.OZONE_SCM_RATIS_EVENTS_MAX_LIMIT,
ScmConfigKeys.OZONE_SCM_RATIS_EVENTS_MAX_LIMIT_DEFAULT);
return ms.register(SOURCE_NAME, "Storage Container Manager Metrics",
new SCMMetrics());
new SCMMetrics(maxRatisEvents));
}

public void setLastContainerReportSize(long size) {
Expand Down Expand Up @@ -155,6 +171,22 @@ public void decrContainerStat(ContainerStat deltaStat) {
this.containerReportWriteCount.incr(-1 * deltaStat.getWriteCount().get());
}

public void addRatisEvent(String event) {
synchronized (ratisEvents) {
if (ratisEvents.size() >= maxRatisEvents) {
ratisEvents.removeFirst();
}
ratisEvents.add(Time.formatTime(Time.now()) + "|" + event);
}
}

@Metric("Ratis state machine events")
public String getRatisEvents() {
synchronized (ratisEvents) {
return String.join("\n", ratisEvents);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Should the metrics publish only the new events?
  • How often this method is involved by the metrics system? It could be inefficient if it keep creating the same, long string again and again.

}
}

public void unRegister() {
MetricsSystem ms = DefaultMetricsSystem.instance();
ms.unregisterSource(SOURCE_NAME);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,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;
Expand Down Expand Up @@ -78,6 +79,7 @@ public class SCMStateMachine extends BaseStateMachine {
private StorageContainerManager scm;
private Map<RequestType, Object> handlers;
private SCMHADBTransactionBuffer transactionBuffer;
private final SCMMetrics metrics;
private final SimpleStateMachineStorage storage =
new SimpleStateMachineStorage();
private final boolean isInitialized;
Expand All @@ -94,6 +96,7 @@ public SCMStateMachine(final StorageContainerManager scm,
this.scm = scm;
this.handlers = new EnumMap<>(RequestType.class);
this.transactionBuffer = buffer;
this.metrics = scm.getMetrics();
TransactionInfo latestTrxInfo = this.transactionBuffer.getLatestTrxInfo();
if (!latestTrxInfo.isDefault()) {
updateLastAppliedTermIndex(latestTrxInfo.getTerm(),
Expand All @@ -111,12 +114,19 @@ public SCMStateMachine(final StorageContainerManager scm,

public SCMStateMachine() {
isInitialized = false;
this.metrics = null;
}

public void registerHandler(RequestType type, Object handler) {
handlers.put(type, handler);
}

private void addRatisEvent(String message) {
if (metrics != null) {
metrics.addRatisEvent(message);
}
}

@Override
public SnapshotInfo getLatestSnapshot() {
// Transaction buffer will be null during scm initlialization phase
Expand Down Expand Up @@ -212,7 +222,9 @@ public void notifyNotLeader(Collection<TransactionContext> pendingEntries) {
if (!isInitialized) {
return;
}
LOG.info("current leader SCM steps down.");
String message = "SCM " + scm.getScmId() + " steps down from being leader.";
LOG.info(message);
addRatisEvent(message);

scm.getScmContext().updateLeaderAndTerm(false, 0);
scm.getSCMServiceManager().notifyStatusChanged();
Expand Down Expand Up @@ -243,6 +255,8 @@ public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader(
final String leaderNodeId = leaderDetails.get().getNodeId();
LOG.info("Received install snapshot notification from SCM leader: {} with "
+ "term index: {}", leaderAddress, firstTermIndexInLog);
addRatisEvent("Installing snapshot from SCM leader " + leaderNodeId +
", term index: " + firstTermIndexInLog);

CompletableFuture<TermIndex> future = CompletableFuture.supplyAsync(
() -> {
Expand Down Expand Up @@ -296,11 +310,17 @@ public void notifyLeaderChanged(RaftGroupMemberId groupMemberId,
}

if (!groupMemberId.getPeerId().equals(newLeaderId)) {
LOG.info("leader changed, yet current SCM is still follower.");
String message = "Leader changed to " + newLeaderId +
", current SCM " + scm.getScmId() + " is still follower.";
LOG.info(message);
addRatisEvent(message);
return;
}

LOG.info("current SCM becomes leader of term {}.", currentLeaderTerm);
String message = "current SCM " + scm.getScmId() +
" becomes leader of term " + currentLeaderTerm;
LOG.info(message);
addRatisEvent(message);

scm.getScmContext().updateLeaderAndTerm(true,
currentLeaderTerm.get());
Expand Down Expand Up @@ -394,11 +414,15 @@ public void notifyLeaderReady() {
scm.getScmContext().setLeaderReady();
scm.getSCMServiceManager().notifyStatusChanged();
scm.getFinalizationManager().onLeaderReady();
addRatisEvent("SCM " + scm.getScmId() +
" is ready to serve requests as the leader");
}

@Override
public void notifyConfigurationChanged(long term, long index,
RaftProtos.RaftConfigurationProto newRaftConfiguration) {
addRatisEvent("Configuration changed at term index (" + term + ", " + index +
") to " + newRaftConfiguration.toString());
}

@Override
Expand Down Expand Up @@ -431,6 +455,7 @@ public void reinitialize() throws IOException {
}

LOG.info("{}: SCMStateMachine is reinitializing. newTermIndex = {}", getId(), termIndex);
addRatisEvent("reinitialize: " + termIndex);

// re-initialize the DBTransactionBuffer and update the lastAppliedIndex.
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.IO_EXCEPTION;
import static org.apache.hadoop.hdds.scm.net.NetConstants.NODE_COST_DEFAULT;
import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT;
import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.getPerfMetrics;
import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer;
import static org.apache.hadoop.hdds.server.ServerUtils.getRemoteUserName;
import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
Expand Down Expand Up @@ -108,7 +107,7 @@ public class SCMBlockProtocolServer implements
public SCMBlockProtocolServer(OzoneConfiguration conf,
StorageContainerManager scm) throws IOException {
this.scm = scm;
this.perfMetrics = getPerfMetrics();
this.perfMetrics = scm.getPerfMetrics();
final int handlerCount = conf.getInt(OZONE_SCM_BLOCK_HANDLER_COUNT_KEY,
OZONE_SCM_HANDLER_COUNT_KEY, OZONE_SCM_HANDLER_COUNT_DEFAULT,
LOG::info);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -226,8 +226,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
/**
* SCM metrics.
*/
private static SCMMetrics metrics;
private static SCMPerformanceMetrics perfMetrics;
private SCMMetrics metrics;
private SCMPerformanceMetrics perfMetrics;
private SCMHAMetrics scmHAMetrics;
private final NettyMetrics nettyMetrics;

Expand Down Expand Up @@ -1427,28 +1427,28 @@ private static InetSocketAddress getScmAddress(SCMHANodeDetails haDetails,
/**
* Initialize SCM metrics.
*/
public static void initMetrics() {
metrics = SCMMetrics.create();
public void initMetrics() {
metrics = SCMMetrics.create(configuration);
}

/**
* Return SCM metrics instance.
*/
public static SCMMetrics getMetrics() {
return metrics == null ? SCMMetrics.create() : metrics;
public SCMMetrics getMetrics() {
return metrics;
}

/**
* Initialize SCMPerformance metrics.
*/
public static void initPerfMetrics() {
public void initPerfMetrics() {
perfMetrics = SCMPerformanceMetrics.create();
}

/**
* Return SCMPerformance metrics instance.
*/
public static SCMPerformanceMetrics getPerfMetrics() {
public SCMPerformanceMetrics getPerfMetrics() {
return perfMetrics == null ? SCMPerformanceMetrics.create() : perfMetrics;
}

Expand Down
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>
26 changes: 26 additions & 0 deletions hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm.js
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,32 @@
"use strict";
angular.module('scm', ['ozone', 'nvd3']);

angular.module('scm').config(function ($routeProvider) {
$routeProvider
.when("/ratis_events", {
template: "<ratis-events></ratis-events>"
});
});

angular.module('scm').component('ratisEvents', {
templateUrl: 'ratis-events.html',
controller: function ($http) {
var ctrl = this;
$http.get("jmx?qry=Hadoop:service=StorageContainerManager,name=SCMMetrics")
.then(function (result) {
var metrics = result.data.beans[0];
var rawEvents = metrics['tag.RatisEvents'] ? metrics['tag.RatisEvents'].split('\n') : [];
ctrl.events = rawEvents.map(function(e) {
var parts = e.split('|');
return {
timestamp: parts[0],
description: parts[1]
};
});
});
}
});

angular.module('scm').component('scmOverview', {
templateUrl: 'scm-overview.html',
require: {
Expand Down
Loading