Skip to content

RATIS-1743. (superseded) #782

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

Closed
wants to merge 6 commits into from
Closed
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 @@ -209,9 +209,9 @@ public long[] getFollowerNextIndices() {

this.jmxAdapter = new RaftServerJmxAdapter();
this.leaderElectionMetrics = LeaderElectionMetrics.getLeaderElectionMetrics(
getMemberId(), state::getLastLeaderElapsedTimeMs);
getMemberId(), state.getLastNoLeaderTime());
this.raftServerMetrics = RaftServerMetricsImpl.computeIfAbsentRaftServerMetrics(
getMemberId(), () -> commitInfoCache::get, retryCache::getStatistics);
getMemberId(), commitInfoCache::get, retryCache::getStatistics);

this.startComplete = new AtomicBoolean(false);
this.threadGroup = new ThreadGroup(proxy.getThreadGroup(), getMemberId().toString());
Expand Down Expand Up @@ -594,8 +594,9 @@ RoleInfoProto getRoleInfoProto() {
.setRoleElapsedTimeMs(role.getRoleElapsedTimeMs());
switch (currentRole) {
case CANDIDATE:
final Timestamp lastNoLeaderTime = state.getLastNoLeaderTime().get();
CandidateInfoProto.Builder candidate = CandidateInfoProto.newBuilder()
.setLastLeaderElapsedTimeMs(state.getLastLeaderElapsedTimeMs());
.setLastLeaderElapsedTimeMs(lastNoLeaderTime != null? lastNoLeaderTime.elapsedTimeMs(): 0L);
roleInfo.setCandidateInfo(candidate);
break;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,8 +75,8 @@ class ServerState implements Closeable {
/** local storage for log and snapshot */
private RaftStorageImpl storage;
private final SnapshotManager snapshotManager;
private volatile Timestamp lastNoLeaderTime;
private final TimeDuration noLeaderTimeout;
private final AtomicReference<Timestamp> lastNoLeaderTime = new AtomicReference<>(Timestamp.currentTime());

/**
* Latest term server has seen.
Expand Down Expand Up @@ -150,7 +150,6 @@ class ServerState implements Closeable {

// On start the leader is null, start the clock now
this.leaderId = null;
this.lastNoLeaderTime = Timestamp.currentTime();
this.noLeaderTimeout = RaftServerConfigKeys.Notification.noLeaderTimeout(prop);

final LongSupplier getSnapshotIndexFromStateMachine = () -> Optional.ofNullable(stateMachine.getLatestSnapshot())
Expand Down Expand Up @@ -302,11 +301,10 @@ void setLeader(RaftPeerId newLeaderId, Object op) {
String suffix;
if (newLeaderId == null) {
// reset the time stamp when a null leader is assigned
lastNoLeaderTime = Timestamp.currentTime();
lastNoLeaderTime.set(Timestamp.currentTime());
suffix = "";
} else {
Timestamp previous = lastNoLeaderTime;
lastNoLeaderTime = null;
final Timestamp previous = lastNoLeaderTime.getAndSet(null);
suffix = ", leader elected after " + previous.elapsedTimeMs() + "ms";
server.getStateMachine().event().notifyLeaderChanged(getMemberId(), newLeaderId);
}
Expand All @@ -321,14 +319,14 @@ void setLeader(RaftPeerId newLeaderId, Object op) {
}

boolean shouldNotifyExtendedNoLeader() {
return Optional.ofNullable(lastNoLeaderTime)
return Optional.ofNullable(lastNoLeaderTime.get())
.map(Timestamp::elapsedTime)
.filter(t -> t.compareTo(noLeaderTimeout) > 0)
.isPresent();
}

long getLastLeaderElapsedTimeMs() {
return Optional.ofNullable(lastNoLeaderTime).map(Timestamp::elapsedTimeMs).orElse(0L);
AtomicReference<Timestamp> getLastNoLeaderTime() {
return lastNoLeaderTime;
}

void becomeLeader() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import com.codahale.metrics.Timer;

import java.util.Optional;
import java.util.function.LongSupplier;
import java.util.concurrent.atomic.AtomicReference;

/**
* Class to update the metrics related to Leader Election.
Expand All @@ -46,9 +46,10 @@ public final class LeaderElectionMetrics extends RatisMetrics {
public static final String LAST_LEADER_ELECTION_ELAPSED_TIME = "lastLeaderElectionElapsedTime";
private volatile Timestamp lastElectionTime;

private LeaderElectionMetrics(RaftGroupMemberId serverId, LongSupplier getLastLeaderElapsedTimeMs) {
private LeaderElectionMetrics(RaftGroupMemberId serverId, AtomicReference<Timestamp> lastNoLeaderTime) {
this.registry = getMetricRegistryForLeaderElection(serverId);
registry.gauge(LAST_LEADER_ELAPSED_TIME, () -> getLastLeaderElapsedTimeMs::getAsLong);
registry.gauge(LAST_LEADER_ELAPSED_TIME,
() -> () -> Optional.ofNullable(lastNoLeaderTime.get()).map(Timestamp::elapsedTimeMs).orElse(0L));
registry.gauge(LAST_LEADER_ELECTION_ELAPSED_TIME,
() -> () -> Optional.ofNullable(lastElectionTime).map(Timestamp::elapsedTimeMs).orElse(-1L));
}
Expand All @@ -60,8 +61,8 @@ public static RatisMetricRegistry getMetricRegistryForLeaderElection(RaftGroupMe
}

public static LeaderElectionMetrics getLeaderElectionMetrics(
RaftGroupMemberId serverId, LongSupplier getLastLeaderElapsedTimeMs) {
return new LeaderElectionMetrics(serverId, getLastLeaderElapsedTimeMs);
RaftGroupMemberId serverId, AtomicReference<Timestamp> lastNoLeaderTime) {
return new LeaderElectionMetrics(serverId, lastNoLeaderTime);
}

public void onNewLeaderElectionCompletion() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ public final class RaftServerMetricsImpl extends RatisMetrics implements RaftSer

/** Follower Id -> heartbeat elapsed */
private final Map<RaftPeerId, Long> followerLastHeartbeatElapsedTimeMap = new HashMap<>();
private final Supplier<Function<RaftPeerId, CommitInfoProto>> commitInfoCache;
private final Function<RaftPeerId, CommitInfoProto> commitInfoCache;

/** id -> metric */
private static final Map<RaftGroupMemberId, RaftServerMetricsImpl> METRICS = new ConcurrentHashMap<>();
Expand All @@ -94,7 +94,7 @@ private static String getPeerCommitIndexGaugeKey(RaftPeerId serverId) {
}

public static RaftServerMetricsImpl computeIfAbsentRaftServerMetrics(RaftGroupMemberId serverId,
Supplier<Function<RaftPeerId, CommitInfoProto>> commitInfoCache,
Function<RaftPeerId, CommitInfoProto> commitInfoCache,
Supplier<RetryCache.Statistics> retryCacheStatistics) {
return METRICS.computeIfAbsent(serverId,
key -> new RaftServerMetricsImpl(serverId, commitInfoCache, retryCacheStatistics));
Expand All @@ -105,7 +105,7 @@ public static void removeRaftServerMetrics(RaftGroupMemberId serverId) {
}

public RaftServerMetricsImpl(RaftGroupMemberId serverId,
Supplier<Function<RaftPeerId, CommitInfoProto>> commitInfoCache,
Function<RaftPeerId, CommitInfoProto> commitInfoCache,
Supplier<RetryCache.Statistics> retryCacheStatistics) {
this.registry = getMetricRegistryForRaftServer(serverId.toString());
this.commitInfoCache = commitInfoCache;
Expand Down Expand Up @@ -146,8 +146,7 @@ public void addFollower(RaftPeerId followerName) {
* Register a commit index tracker for the peer in cluster.
*/
public void addPeerCommitIndexGauge(RaftPeerId peerId) {
registry.gauge(getPeerCommitIndexGaugeKey(peerId), () -> () -> Optional.ofNullable(commitInfoCache.get())
.map(cache -> cache.apply(peerId))
registry.gauge(getPeerCommitIndexGaugeKey(peerId), () -> () -> Optional.ofNullable(commitInfoCache.apply(peerId))
.map(CommitInfoProto::getCommitIndex)
.orElse(0L));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,13 @@

package org.apache.ratis.server.metrics;

import com.codahale.metrics.Gauge;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Timer;
import org.apache.ratis.protocol.RaftGroupMemberId;
import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogCache;
import org.apache.ratis.util.DataQueue;

import java.util.Queue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.ToIntFunction;

public class SegmentedRaftLogMetrics extends RaftLogMetricsBase {
//////////////////////////////
Expand Down Expand Up @@ -84,8 +83,15 @@ public SegmentedRaftLogMetrics(RaftGroupMemberId serverId) {
super(serverId);
}

public void addDataQueueSizeGauge(DataQueue queue) {
registry.gauge(RAFT_LOG_DATA_QUEUE_SIZE, () -> queue::getNumElements);
public void addQueueSizeGauges(AtomicReference<? extends ToIntFunction<String>> queues) {
registry.gauge(RAFT_LOG_DATA_QUEUE_SIZE, () -> () -> {
final ToIntFunction<String> f = queues.get();
return f != null? f.applyAsInt(RAFT_LOG_DATA_QUEUE_SIZE): -1;
});
registry.gauge(RAFT_LOG_WORKER_QUEUE_SIZE, () -> () -> {
final ToIntFunction<String> f = queues.get();
return f != null? f.applyAsInt(RAFT_LOG_WORKER_QUEUE_SIZE): -1;
});
}

public void addClosedSegmentsNum(SegmentedRaftLogCache cache) {
Expand All @@ -106,12 +112,8 @@ public void addOpenSegmentSizeInBytes(SegmentedRaftLogCache cache) {
});
}

public void addLogWorkerQueueSizeGauge(Queue queue) {
registry.gauge(RAFT_LOG_WORKER_QUEUE_SIZE, () -> () -> queue.size());
}

public void addFlushBatchSizeGauge(MetricRegistry.MetricSupplier<Gauge> supplier) {
registry.gauge(RAFT_LOG_SYNC_BATCH_SIZE, supplier);
public void addFlushBatchSizeGauge(AtomicInteger syncBatchSize) {
registry.gauge(RAFT_LOG_SYNC_BATCH_SIZE, () -> syncBatchSize::get);
}

private Timer getTimer(String timerName) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,10 @@ CompletableFuture<Long> getFuture() {
return future;
}

void completeExceptionally(Exception e) {
future.completeExceptionally(e);
}

void done() {
completeFuture();
}
Expand Down
Loading