1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.client;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.junit.Assert.assertTrue;
22
23 import com.google.protobuf.ByteString;
24 import com.yammer.metrics.util.RatioGauge;
25 import java.io.IOException;
26 import java.util.concurrent.ExecutorService;
27 import java.util.concurrent.Executors;
28 import java.util.concurrent.TimeUnit;
29 import java.util.concurrent.atomic.AtomicBoolean;
30 import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
31 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
32 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
33 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
34 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
35 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
36 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
37 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
38 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
39 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
40 import org.apache.hadoop.hbase.testclassification.MetricsTests;
41 import org.apache.hadoop.hbase.testclassification.SmallTests;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.junit.AfterClass;
44 import org.junit.Assert;
45 import org.junit.BeforeClass;
46 import org.junit.Test;
47 import org.junit.experimental.categories.Category;
48 import org.mockito.Mockito;
49
50 @Category({MetricsTests.class, SmallTests.class})
51 public class TestMetricsConnection {
52 private static final ExecutorService BATCH_POOL = Executors.newFixedThreadPool(2);
53 private static MetricsConnection METRICS;
54 private static final AtomicBoolean closed = new AtomicBoolean(false);
55 private static final Runnable RUNNER = new Runnable() {
56 @Override
57 public void run() {
58 try {
59 while (!closed.get() && !Thread.interrupted()) {
60 TimeUnit.MILLISECONDS.sleep(10);
61 }
62 } catch (InterruptedException e) {
63 }
64 }
65 };
66 @BeforeClass
67 public static void beforeClass() {
68 HConnectionImplementation mocked = Mockito.mock(HConnectionImplementation.class);
69 Mockito.when(mocked.toString()).thenReturn("mocked-connection");
70 Mockito.when(mocked.getCurrentBatchPool()).thenReturn(BATCH_POOL);
71 BATCH_POOL.submit(RUNNER);
72 METRICS = new MetricsConnection(mocked);
73 }
74
75 @AfterClass
76 public static void afterClass() throws InterruptedException {
77 METRICS.shutdown();
78 BATCH_POOL.shutdownNow();
79 BATCH_POOL.awaitTermination(Long.MAX_VALUE, TimeUnit.DAYS);
80 }
81
82 @Test
83 public void testStaticMetrics() throws IOException {
84 final byte[] foo = Bytes.toBytes("foo");
85 final RegionSpecifier region = RegionSpecifier.newBuilder()
86 .setValue(ByteString.EMPTY)
87 .setType(RegionSpecifierType.REGION_NAME)
88 .build();
89 final int loop = 5;
90
91 for (int i = 0; i < loop; i++) {
92 METRICS.updateRpc(
93 ClientService.getDescriptor().findMethodByName("Get"),
94 GetRequest.getDefaultInstance(),
95 MetricsConnection.newCallStats());
96 METRICS.updateRpc(
97 ClientService.getDescriptor().findMethodByName("Scan"),
98 ScanRequest.getDefaultInstance(),
99 MetricsConnection.newCallStats());
100 METRICS.updateRpc(
101 ClientService.getDescriptor().findMethodByName("Multi"),
102 MultiRequest.getDefaultInstance(),
103 MetricsConnection.newCallStats());
104 METRICS.updateRpc(
105 ClientService.getDescriptor().findMethodByName("Mutate"),
106 MutateRequest.newBuilder()
107 .setMutation(ProtobufUtil.toMutation(MutationType.APPEND, new Append(foo)))
108 .setRegion(region)
109 .build(),
110 MetricsConnection.newCallStats());
111 METRICS.updateRpc(
112 ClientService.getDescriptor().findMethodByName("Mutate"),
113 MutateRequest.newBuilder()
114 .setMutation(ProtobufUtil.toMutation(MutationType.DELETE, new Delete(foo)))
115 .setRegion(region)
116 .build(),
117 MetricsConnection.newCallStats());
118 METRICS.updateRpc(
119 ClientService.getDescriptor().findMethodByName("Mutate"),
120 MutateRequest.newBuilder()
121 .setMutation(ProtobufUtil.toMutation(MutationType.INCREMENT, new Increment(foo)))
122 .setRegion(region)
123 .build(),
124 MetricsConnection.newCallStats());
125 METRICS.updateRpc(
126 ClientService.getDescriptor().findMethodByName("Mutate"),
127 MutateRequest.newBuilder()
128 .setMutation(ProtobufUtil.toMutation(MutationType.PUT, new Put(foo)))
129 .setRegion(region)
130 .build(),
131 MetricsConnection.newCallStats());
132 }
133 for (String method: new String[]{"Get", "Scan", "Mutate"}) {
134 final String metricKey = "rpcCount_" + ClientService.getDescriptor().getName() + "_" + method;
135 final long metricVal = METRICS.rpcCounters.get(metricKey).count();
136 assertTrue("metric: " + metricKey + " val: " + metricVal, metricVal >= loop);
137 }
138 for (MetricsConnection.CallTracker t : new MetricsConnection.CallTracker[] {
139 METRICS.getTracker, METRICS.scanTracker, METRICS.multiTracker, METRICS.appendTracker,
140 METRICS.deleteTracker, METRICS.incrementTracker, METRICS.putTracker
141 }) {
142 Assert.assertEquals("Failed to invoke callTimer on " + t, loop, t.callTimer.count());
143 Assert.assertEquals("Failed to invoke reqHist on " + t, loop, t.reqHist.count());
144 Assert.assertEquals("Failed to invoke respHist on " + t, loop, t.respHist.count());
145 }
146 RatioGauge executorMetrics = (RatioGauge) METRICS.getMetricsRegistry()
147 .allMetrics().get(METRICS.getExecutorPoolName());
148 RatioGauge metaMetrics = (RatioGauge) METRICS.getMetricsRegistry()
149 .allMetrics().get(METRICS.getMetaPoolName());
150 assertEquals((double) 0.5, executorMetrics.value(), 0);
151 assertEquals(Double.NaN, metaMetrics.value(), 0);
152 }
153 }