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