001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.client; 019 020import static org.junit.Assert.assertEquals; 021 022import com.codahale.metrics.RatioGauge; 023import com.codahale.metrics.RatioGauge.Ratio; 024import java.io.IOException; 025import java.util.concurrent.ExecutorService; 026import java.util.concurrent.Executors; 027import org.apache.hadoop.hbase.HBaseClassTestRule; 028import org.apache.hadoop.hbase.testclassification.ClientTests; 029import org.apache.hadoop.hbase.testclassification.MetricsTests; 030import org.apache.hadoop.hbase.testclassification.SmallTests; 031import org.apache.hadoop.hbase.util.Bytes; 032import org.junit.AfterClass; 033import org.junit.BeforeClass; 034import org.junit.ClassRule; 035import org.junit.Test; 036import org.junit.experimental.categories.Category; 037import org.mockito.Mockito; 038 039import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; 040 041import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 042import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; 043import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest; 044import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest; 045import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; 047import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest; 048import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier; 049import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; 050 051@Category({ClientTests.class, MetricsTests.class, SmallTests.class}) 052public class TestMetricsConnection { 053 @ClassRule 054 public static final HBaseClassTestRule CLASS_RULE = 055 HBaseClassTestRule.forClass(TestMetricsConnection.class); 056 057 private static MetricsConnection METRICS; 058 private static final ExecutorService BATCH_POOL = Executors.newFixedThreadPool(2); 059 @BeforeClass 060 public static void beforeClass() { 061 ConnectionImplementation mocked = Mockito.mock(ConnectionImplementation.class); 062 Mockito.when(mocked.toString()).thenReturn("mocked-connection"); 063 Mockito.when(mocked.getCurrentBatchPool()).thenReturn(BATCH_POOL); 064 METRICS = new MetricsConnection(mocked); 065 } 066 067 @AfterClass 068 public static void afterClass() { 069 METRICS.shutdown(); 070 } 071 072 @Test 073 public void testStaticMetrics() throws IOException { 074 final byte[] foo = Bytes.toBytes("foo"); 075 final RegionSpecifier region = RegionSpecifier.newBuilder() 076 .setValue(ByteString.EMPTY) 077 .setType(RegionSpecifierType.REGION_NAME) 078 .build(); 079 final int loop = 5; 080 081 for (int i = 0; i < loop; i++) { 082 METRICS.updateRpc( 083 ClientService.getDescriptor().findMethodByName("Get"), 084 GetRequest.getDefaultInstance(), 085 MetricsConnection.newCallStats()); 086 METRICS.updateRpc( 087 ClientService.getDescriptor().findMethodByName("Scan"), 088 ScanRequest.getDefaultInstance(), 089 MetricsConnection.newCallStats()); 090 METRICS.updateRpc( 091 ClientService.getDescriptor().findMethodByName("Multi"), 092 MultiRequest.getDefaultInstance(), 093 MetricsConnection.newCallStats()); 094 METRICS.updateRpc( 095 ClientService.getDescriptor().findMethodByName("Mutate"), 096 MutateRequest.newBuilder() 097 .setMutation(ProtobufUtil.toMutation(MutationType.APPEND, new Append(foo))) 098 .setRegion(region) 099 .build(), 100 MetricsConnection.newCallStats()); 101 METRICS.updateRpc( 102 ClientService.getDescriptor().findMethodByName("Mutate"), 103 MutateRequest.newBuilder() 104 .setMutation(ProtobufUtil.toMutation(MutationType.DELETE, new Delete(foo))) 105 .setRegion(region) 106 .build(), 107 MetricsConnection.newCallStats()); 108 METRICS.updateRpc( 109 ClientService.getDescriptor().findMethodByName("Mutate"), 110 MutateRequest.newBuilder() 111 .setMutation(ProtobufUtil.toMutation(MutationType.INCREMENT, new Increment(foo))) 112 .setRegion(region) 113 .build(), 114 MetricsConnection.newCallStats()); 115 METRICS.updateRpc( 116 ClientService.getDescriptor().findMethodByName("Mutate"), 117 MutateRequest.newBuilder() 118 .setMutation(ProtobufUtil.toMutation(MutationType.PUT, new Put(foo))) 119 .setRegion(region) 120 .build(), 121 MetricsConnection.newCallStats()); 122 } 123 for (MetricsConnection.CallTracker t : new MetricsConnection.CallTracker[] { 124 METRICS.getTracker, METRICS.scanTracker, METRICS.multiTracker, METRICS.appendTracker, 125 METRICS.deleteTracker, METRICS.incrementTracker, METRICS.putTracker 126 }) { 127 assertEquals("Failed to invoke callTimer on " + t, loop, t.callTimer.getCount()); 128 assertEquals("Failed to invoke reqHist on " + t, loop, t.reqHist.getCount()); 129 assertEquals("Failed to invoke respHist on " + t, loop, t.respHist.getCount()); 130 } 131 RatioGauge executorMetrics = (RatioGauge) METRICS.getMetricRegistry() 132 .getMetrics().get(METRICS.getExecutorPoolName()); 133 RatioGauge metaMetrics = (RatioGauge) METRICS.getMetricRegistry() 134 .getMetrics().get(METRICS.getMetaPoolName()); 135 assertEquals(Ratio.of(0, 3).getValue(), executorMetrics.getValue(), 0); 136 assertEquals(Double.NaN, metaMetrics.getValue(), 0); 137 } 138}