| /** |
| * 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 |
| * <p> |
| * http://www.apache.org/licenses/LICENSE-2.0 |
| * <p> |
| * 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.ratis.grpc.server; |
| |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_INCONSISTENCY; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_LATENCY; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_NOT_LEADER; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_PENDING_COUNT; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_SUCCESS; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_LOG_APPENDER_TIMEOUT; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_REQUESTS_COUNT; |
| import static org.apache.ratis.grpc.metrics.GrpcServerMetrics.RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT; |
| import static org.mockito.Mockito.when; |
| |
| import java.util.function.Consumer; |
| |
| import org.apache.ratis.metrics.impl.DefaultTimekeeperImpl; |
| import org.apache.ratis.server.metrics.ServerMetricsTestUtils; |
| import org.apache.ratis.grpc.metrics.GrpcServerMetrics; |
| import org.apache.ratis.metrics.RatisMetricRegistry; |
| import org.apache.ratis.proto.RaftProtos; |
| import org.apache.ratis.protocol.RaftGroupId; |
| import org.apache.ratis.protocol.RaftGroupMemberId; |
| import org.apache.ratis.protocol.RaftPeerId; |
| import org.apache.ratis.thirdparty.com.codahale.metrics.Gauge; |
| import org.junit.jupiter.api.Assertions; |
| import org.junit.jupiter.api.BeforeAll; |
| import org.junit.jupiter.api.Test; |
| import org.mockito.Mockito; |
| |
| public class TestGrpcServerMetrics { |
| private static GrpcServerMetrics grpcServerMetrics; |
| private static RatisMetricRegistry ratisMetricRegistry; |
| private static RaftGroupId raftGroupId; |
| private static RaftPeerId raftPeerId; |
| private static RaftPeerId followerId; |
| |
| @BeforeAll |
| public static void setUp() throws Exception { |
| raftGroupId = RaftGroupId.randomId(); |
| raftPeerId = RaftPeerId.valueOf("TestId"); |
| followerId = RaftPeerId.valueOf("FollowerId"); |
| RaftGroupMemberId raftGroupMemberId = RaftGroupMemberId.valueOf(raftPeerId, raftGroupId); |
| grpcServerMetrics = new GrpcServerMetrics(raftGroupMemberId.toString()); |
| ratisMetricRegistry = grpcServerMetrics.getRegistry(); |
| } |
| |
| @Test |
| public void testGrpcLogAppenderLatencyTimer() throws Exception { |
| for (boolean heartbeat : new boolean[] { true, false }) { |
| RaftProtos.AppendEntriesRequestProto.Builder proto = |
| RaftProtos.AppendEntriesRequestProto.newBuilder(); |
| if (!heartbeat) { |
| proto.addEntries(RaftProtos.LogEntryProto.newBuilder().build()); |
| } |
| GrpcLogAppender.AppendEntriesRequest req = |
| new GrpcLogAppender.AppendEntriesRequest(proto.build(), followerId, |
| grpcServerMetrics); |
| final String format = RATIS_GRPC_METRICS_LOG_APPENDER_LATENCY + GrpcServerMetrics.getHeartbeatSuffix(heartbeat); |
| final String name = String.format(format, followerId); |
| final DefaultTimekeeperImpl t = (DefaultTimekeeperImpl) ratisMetricRegistry.timer(name); |
| Assertions.assertEquals(0L, t.getTimer().getSnapshot().getMax()); |
| req.startRequestTimer(); |
| Thread.sleep(1000L); |
| req.stopRequestTimer(); |
| Assertions.assertTrue(t.getTimer().getSnapshot().getMax() > 1000L); |
| } |
| } |
| |
| @Test |
| public void testGrpcLogRequestTotal() { |
| for (boolean heartbeat : new boolean[] { true, false }) { |
| long reqTotal = ratisMetricRegistry.counter( |
| RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics |
| .getHeartbeatSuffix(heartbeat)).getCount(); |
| grpcServerMetrics.onRequestCreate(heartbeat); |
| Assertions.assertEquals(reqTotal + 1, ratisMetricRegistry.counter( |
| RATIS_GRPC_METRICS_REQUESTS_COUNT + GrpcServerMetrics |
| .getHeartbeatSuffix(heartbeat)).getCount()); |
| } |
| } |
| |
| @Test |
| public void testGrpcLogRequestRetry() { |
| Assertions.assertEquals(0L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); |
| grpcServerMetrics.onRequestRetry(); |
| Assertions.assertEquals(1L, ratisMetricRegistry.counter(RATIS_GRPC_METRICS_REQUEST_RETRY_COUNT).getCount()); |
| } |
| |
| @Test |
| public void testGrpcLogPendingRequestCount() { |
| GrpcLogAppender.RequestMap pendingRequest = Mockito.mock(GrpcLogAppender.RequestMap.class); |
| when(pendingRequest.logRequestsSize()).thenReturn(0); |
| grpcServerMetrics.addPendingRequestsCount(raftPeerId.toString(), |
| pendingRequest::logRequestsSize); |
| final String name = String.format(RATIS_GRPC_METRICS_LOG_APPENDER_PENDING_COUNT, raftPeerId); |
| final Gauge gauge = ServerMetricsTestUtils.getGaugeWithName(name, grpcServerMetrics::getRegistry); |
| Assertions.assertEquals(0, gauge.getValue()); |
| when(pendingRequest.logRequestsSize()).thenReturn(10); |
| Assertions.assertEquals(10, gauge.getValue()); |
| } |
| |
| @Test |
| public void testGrpcLogAppenderRequestCounters() { |
| assertCounterIncremented(RATIS_GRPC_METRICS_LOG_APPENDER_NOT_LEADER, grpcServerMetrics::onRequestNotLeader); |
| assertCounterIncremented(RATIS_GRPC_METRICS_LOG_APPENDER_INCONSISTENCY, grpcServerMetrics::onRequestInconsistency); |
| |
| for (boolean heartbeat : new boolean[] { true, false }) { |
| assertCounterIncremented(RATIS_GRPC_METRICS_LOG_APPENDER_SUCCESS+ GrpcServerMetrics |
| .getHeartbeatSuffix(heartbeat), |
| follower -> grpcServerMetrics |
| .onRequestSuccess(follower, heartbeat)); |
| assertCounterIncremented(RATIS_GRPC_METRICS_LOG_APPENDER_TIMEOUT+ GrpcServerMetrics |
| .getHeartbeatSuffix(heartbeat), |
| follower -> grpcServerMetrics.onRequestTimeout(follower, heartbeat)); |
| } |
| } |
| |
| private void assertCounterIncremented(String counterVar, Consumer<String> incFunction) { |
| String counter = String.format(counterVar, raftPeerId.toString()); |
| Assertions.assertEquals(0L, ratisMetricRegistry.counter(counter).getCount()); |
| incFunction.accept(raftPeerId.toString()); |
| Assertions.assertEquals(1L, ratisMetricRegistry.counter(counter).getCount()); |
| } |
| } |