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.namequeues; 019 020import java.io.IOException; 021import java.lang.reflect.Field; 022import java.util.List; 023import java.util.concurrent.CompletableFuture; 024import java.util.concurrent.TimeUnit; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.HBaseClassTestRule; 027import org.apache.hadoop.hbase.HBaseTestingUtility; 028import org.apache.hadoop.hbase.HConstants; 029import org.apache.hadoop.hbase.client.Connection; 030import org.apache.hadoop.hbase.client.Result; 031import org.apache.hadoop.hbase.client.ResultScanner; 032import org.apache.hadoop.hbase.client.Scan; 033import org.apache.hadoop.hbase.client.Table; 034import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest; 035import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse; 036import org.apache.hadoop.hbase.regionserver.HRegionServer; 037import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor; 038import org.apache.hadoop.hbase.testclassification.MasterTests; 039import org.apache.hadoop.hbase.testclassification.MediumTests; 040import org.junit.AfterClass; 041import org.junit.Assert; 042import org.junit.Before; 043import org.junit.BeforeClass; 044import org.junit.ClassRule; 045import org.junit.Test; 046import org.junit.experimental.categories.Category; 047import org.slf4j.Logger; 048import org.slf4j.LoggerFactory; 049 050import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles; 051 052import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; 053import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog; 054 055/** 056 * Tests for SlowLog System Table 057 */ 058@Category({ MasterTests.class, MediumTests.class }) 059public class TestSlowLogAccessor { 060 061 @ClassRule 062 public static final HBaseClassTestRule CLASS_RULE = 063 HBaseClassTestRule.forClass(TestSlowLogAccessor.class); 064 065 private static final Logger LOG = LoggerFactory.getLogger(TestNamedQueueRecorder.class); 066 067 private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility(); 068 069 private NamedQueueRecorder namedQueueRecorder; 070 071 @BeforeClass 072 public static void setup() throws Exception { 073 try { 074 HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster(); 075 } catch (IOException e) { 076 LOG.debug("No worries."); 077 } 078 Configuration conf = HBASE_TESTING_UTILITY.getConfiguration(); 079 conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true); 080 conf.setBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY, true); 081 conf.setInt("hbase.slowlog.systable.chore.duration", 900); 082 conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", 50000); 083 HBASE_TESTING_UTILITY.startMiniCluster(); 084 } 085 086 @AfterClass 087 public static void teardown() throws Exception { 088 HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster(); 089 } 090 091 @Before 092 public void setUp() throws Exception { 093 HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0); 094 Field slowLogRecorder = HRegionServer.class.getDeclaredField("namedQueueRecorder"); 095 slowLogRecorder.setAccessible(true); 096 this.namedQueueRecorder = (NamedQueueRecorder) slowLogRecorder.get(hRegionServer); 097 } 098 099 private List<TooSlowLog.SlowLogPayload> 100 getSlowLogPayloads(AdminProtos.SlowLogResponseRequest request) { 101 NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest(); 102 namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT); 103 namedQueueGetRequest.setSlowLogResponseRequest(request); 104 NamedQueueGetResponse namedQueueGetResponse = 105 namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest); 106 return namedQueueGetResponse.getSlowLogPayloads(); 107 } 108 109 @Test 110 public void testSlowLogRecords() throws Exception { 111 112 AdminProtos.SlowLogResponseRequest request = 113 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build(); 114 115 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG); 116 Assert.assertEquals(getSlowLogPayloads(request).size(), 0); 117 118 int i = 0; 119 120 Connection connection = waitForSlowLogTableCreation(); 121 // add 5 records initially 122 for (; i < 5; i++) { 123 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 124 "client_" + (i + 1), "class_" + (i + 1)); 125 namedQueueRecorder.addRecord(rpcLogDetails); 126 } 127 128 // add 2 more records 129 for (; i < 7; i++) { 130 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 131 "client_" + (i + 1), "class_" + (i + 1)); 132 namedQueueRecorder.addRecord(rpcLogDetails); 133 } 134 135 // add 3 more records 136 for (; i < 10; i++) { 137 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 138 "client_" + (i + 1), "class_" + (i + 1)); 139 namedQueueRecorder.addRecord(rpcLogDetails); 140 } 141 142 // add 4 more records 143 for (; i < 14; i++) { 144 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 145 "client_" + (i + 1), "class_" + (i + 1)); 146 namedQueueRecorder.addRecord(rpcLogDetails); 147 } 148 149 Assert.assertNotEquals(-1, 150 HBASE_TESTING_UTILITY.waitFor(3000, () -> getSlowLogPayloads(request).size() == 14)); 151 152 Assert.assertNotEquals(-1, 153 HBASE_TESTING_UTILITY.waitFor(3000, () -> getTableCount(connection) == 14)); 154 } 155 156 private int getTableCount(Connection connection) { 157 try (Table table = connection.getTable(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) { 158 ResultScanner resultScanner = table.getScanner(new Scan().setReadType(Scan.ReadType.STREAM)); 159 int count = 0; 160 for (Result result : resultScanner) { 161 ++count; 162 } 163 return count; 164 } catch (Exception e) { 165 return 0; 166 } 167 } 168 169 private Connection waitForSlowLogTableCreation() throws IOException { 170 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000, () -> { 171 try { 172 return HBASE_TESTING_UTILITY.getAdmin() 173 .tableExists(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME); 174 } catch (IOException e) { 175 return false; 176 } 177 })); 178 return HBASE_TESTING_UTILITY.getConnection(); 179 } 180 181 @Test 182 public void testHigherSlowLogs() throws Exception { 183 Connection connection = waitForSlowLogTableCreation(); 184 185 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG); 186 AdminProtos.SlowLogResponseRequest request = 187 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build(); 188 Assert.assertEquals(getSlowLogPayloads(request).size(), 0); 189 190 for (int j = 0; j < 100; j++) { 191 CompletableFuture.runAsync(() -> { 192 for (int i = 0; i < 350; i++) { 193 if (i == 300) { 194 Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS); 195 } 196 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder 197 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1)); 198 namedQueueRecorder.addRecord(rpcLogDetails); 199 } 200 }); 201 } 202 203 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> { 204 int count = getSlowLogPayloads(request).size(); 205 LOG.debug("RingBuffer records count: {}", count); 206 return count > 2000; 207 })); 208 209 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> { 210 int count = getTableCount(connection); 211 LOG.debug("SlowLog Table records count: {}", count); 212 return count > 2000; 213 })); 214 } 215 216}