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 conf.setInt("hbase.regionserver.slowlog.systable.queue.size", 5000); 084 HBASE_TESTING_UTILITY.startMiniCluster(); 085 } 086 087 @AfterClass 088 public static void teardown() throws Exception { 089 HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster(); 090 } 091 092 @Before 093 public void setUp() throws Exception { 094 HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0); 095 Field slowLogRecorder = HRegionServer.class.getDeclaredField("namedQueueRecorder"); 096 slowLogRecorder.setAccessible(true); 097 this.namedQueueRecorder = (NamedQueueRecorder) slowLogRecorder.get(hRegionServer); 098 } 099 100 private List<TooSlowLog.SlowLogPayload> 101 getSlowLogPayloads(AdminProtos.SlowLogResponseRequest request) { 102 NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest(); 103 namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT); 104 namedQueueGetRequest.setSlowLogResponseRequest(request); 105 NamedQueueGetResponse namedQueueGetResponse = 106 namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest); 107 return namedQueueGetResponse.getSlowLogPayloads(); 108 } 109 110 @Test 111 public void testSlowLogRecords() throws Exception { 112 113 AdminProtos.SlowLogResponseRequest request = 114 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build(); 115 116 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG); 117 Assert.assertEquals(getSlowLogPayloads(request).size(), 0); 118 119 int i = 0; 120 121 Connection connection = waitForSlowLogTableCreation(); 122 // add 5 records initially 123 for (; i < 5; i++) { 124 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 125 "client_" + (i + 1), "class_" + (i + 1)); 126 namedQueueRecorder.addRecord(rpcLogDetails); 127 } 128 129 // add 2 more records 130 for (; i < 7; i++) { 131 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 132 "client_" + (i + 1), "class_" + (i + 1)); 133 namedQueueRecorder.addRecord(rpcLogDetails); 134 } 135 136 // add 3 more records 137 for (; i < 10; i++) { 138 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 139 "client_" + (i + 1), "class_" + (i + 1)); 140 namedQueueRecorder.addRecord(rpcLogDetails); 141 } 142 143 // add 4 more records 144 for (; i < 14; i++) { 145 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder.getRpcLogDetails("userName_" + (i + 1), 146 "client_" + (i + 1), "class_" + (i + 1)); 147 namedQueueRecorder.addRecord(rpcLogDetails); 148 } 149 150 Assert.assertNotEquals(-1, 151 HBASE_TESTING_UTILITY.waitFor(3000, () -> getSlowLogPayloads(request).size() == 14)); 152 153 Assert.assertNotEquals(-1, 154 HBASE_TESTING_UTILITY.waitFor(3000, () -> getTableCount(connection) == 14)); 155 } 156 157 private int getTableCount(Connection connection) { 158 try (Table table = connection.getTable(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) { 159 ResultScanner resultScanner = table.getScanner(new Scan().setReadType(Scan.ReadType.STREAM)); 160 int count = 0; 161 for (Result result : resultScanner) { 162 ++count; 163 } 164 return count; 165 } catch (Exception e) { 166 return 0; 167 } 168 } 169 170 private Connection waitForSlowLogTableCreation() throws IOException { 171 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000, () -> { 172 try { 173 return HBASE_TESTING_UTILITY.getAdmin() 174 .tableExists(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME); 175 } catch (IOException e) { 176 return false; 177 } 178 })); 179 return HBASE_TESTING_UTILITY.getConnection(); 180 } 181 182 @Test 183 public void testHigherSlowLogs() throws Exception { 184 Connection connection = waitForSlowLogTableCreation(); 185 186 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG); 187 AdminProtos.SlowLogResponseRequest request = 188 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build(); 189 Assert.assertEquals(getSlowLogPayloads(request).size(), 0); 190 191 for (int j = 0; j < 100; j++) { 192 CompletableFuture.runAsync(() -> { 193 for (int i = 0; i < 350; i++) { 194 if (i == 300) { 195 Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS); 196 } 197 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder 198 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1)); 199 namedQueueRecorder.addRecord(rpcLogDetails); 200 } 201 }); 202 } 203 204 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> { 205 int count = getSlowLogPayloads(request).size(); 206 LOG.debug("RingBuffer records count: {}", count); 207 return count > 2000; 208 })); 209 210 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> { 211 int count = getTableCount(connection); 212 LOG.debug("SlowLog Table records count: {}", count); 213 return count > 2000; 214 })); 215 } 216 217}