1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.namequeues;
21
22 import java.io.IOException;
23 import java.lang.reflect.Field;
24 import java.util.List;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.HBaseTestingUtility;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.MetaTableAccessor;
29 import org.apache.hadoop.hbase.Waiter;
30 import org.apache.hadoop.hbase.client.Connection;
31 import org.apache.hadoop.hbase.client.Result;
32 import org.apache.hadoop.hbase.client.ResultScanner;
33 import org.apache.hadoop.hbase.client.Scan;
34 import org.apache.hadoop.hbase.client.Table;
35 import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
36 import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
37 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
38 import org.apache.hadoop.hbase.protobuf.generated.TooSlowLog;
39 import org.apache.hadoop.hbase.regionserver.HRegionServer;
40 import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
41 import org.apache.hadoop.hbase.testclassification.MasterTests;
42 import org.apache.hadoop.hbase.testclassification.MediumTests;
43 import org.junit.AfterClass;
44 import org.junit.Assert;
45 import org.junit.Before;
46 import org.junit.BeforeClass;
47 import org.junit.Test;
48 import org.junit.experimental.categories.Category;
49 import org.slf4j.Logger;
50 import org.slf4j.LoggerFactory;
51
52
53
54
55 @Category({ MasterTests.class, MediumTests.class })
56 public class TestSlowLogAccessor {
57
58 private static final Logger LOG = LoggerFactory.getLogger(TestNamedQueueRecorder.class);
59
60 private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility();
61
62 private NamedQueueRecorder namedQueueRecorder;
63
64 @BeforeClass
65 public static void setup() throws Exception {
66 try {
67 HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
68 } catch (IOException e) {
69 LOG.debug("No worries.");
70 }
71 Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
72 conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
73 conf.setBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY, true);
74 conf.setInt("hbase.slowlog.systable.chore.duration", 900);
75 conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", 50000);
76 HBASE_TESTING_UTILITY.startMiniCluster();
77 }
78
79 @AfterClass
80 public static void teardown() throws Exception {
81 HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
82 }
83
84 @Before
85 public void setUp() throws Exception {
86 HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0);
87 Field slowLogRecorder = HRegionServer.class.getDeclaredField("namedQueueRecorder");
88 slowLogRecorder.setAccessible(true);
89 this.namedQueueRecorder = (NamedQueueRecorder) slowLogRecorder.get(hRegionServer);
90 }
91
92 private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
93 AdminProtos.SlowLogResponseRequest request) {
94 NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
95 namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
96 namedQueueGetRequest.setSlowLogResponseRequest(request);
97 NamedQueueGetResponse namedQueueGetResponse =
98 namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
99 return namedQueueGetResponse.getSlowLogPayloads();
100 }
101
102 @Test
103 public void testSlowLogRecords() throws Exception {
104
105 final AdminProtos.SlowLogResponseRequest request =
106 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
107 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
108 Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
109
110 int i = 0;
111
112 final Connection connection = waitForSlowLogTableCreation();
113
114 for (; i < 5; i++) {
115 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
116 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
117 namedQueueRecorder.addRecord(rpcLogDetails);
118 }
119
120
121 for (; i < 7; i++) {
122 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
123 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
124 namedQueueRecorder.addRecord(rpcLogDetails);
125 }
126
127
128 for (; i < 10; i++) {
129 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
130 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
131 namedQueueRecorder.addRecord(rpcLogDetails);
132 }
133
134
135 for (; i < 14; i++) {
136 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
137 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
138 namedQueueRecorder.addRecord(rpcLogDetails);
139 }
140
141 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY
142 .waitFor(3000, new Waiter.Predicate<Exception>() {
143 @Override
144 public boolean evaluate() throws Exception {
145 return TestSlowLogAccessor.this.getSlowLogPayloads(request).size() == 14;
146 }
147 }));
148
149 Assert.assertNotEquals(-1,
150 HBASE_TESTING_UTILITY.waitFor(3000, new Waiter.Predicate<Exception>() {
151 @Override
152 public boolean evaluate() throws Exception {
153 return TestSlowLogAccessor.this.getTableCount(connection) == 14;
154 }
155 }));
156 }
157
158 private int getTableCount(Connection connection) {
159 try (Table table = connection.getTable(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME)) {
160 ResultScanner resultScanner = table.getScanner(new Scan().setReadType(Scan.ReadType.STREAM));
161 int count = 0;
162 for (Result result : resultScanner) {
163 ++count;
164 }
165 return count;
166 } catch (Exception e) {
167 return 0;
168 }
169 }
170
171 private Connection waitForSlowLogTableCreation() throws Exception {
172 final Connection connection =
173 HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0).getConnection();
174 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000,
175 new Waiter.Predicate<Exception>() {
176 @Override public boolean evaluate() throws Exception {
177 try {
178 return MetaTableAccessor
179 .tableExists(connection, SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
180 } catch (IOException e) {
181 return false;
182 }
183 }
184 }));
185 return connection;
186 }
187
188 @Test
189 public void testHigherSlowLogs() throws Exception {
190 final Connection connection = waitForSlowLogTableCreation();
191
192 namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
193 final AdminProtos.SlowLogResponseRequest request =
194 AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build();
195 Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
196
197 for (int j = 0; j < 100; j++) {
198 new Thread(new Runnable() {
199 @Override
200 public void run() {
201 for (int i = 0; i < 350; i++) {
202 if (i == 300) {
203 try {
204 Thread.sleep(500);
205 } catch (InterruptedException e) {
206 LOG.warn("Interrupted.");
207 }
208 }
209 RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
210 .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
211 namedQueueRecorder.addRecord(rpcLogDetails);
212 }
213 }
214 }).start();
215 }
216
217 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(8000,
218 new Waiter.Predicate<Exception>() {
219 @Override
220 public boolean evaluate() throws Exception {
221 int count = TestSlowLogAccessor.this.getSlowLogPayloads(request).size();
222 LOG.debug("RingBuffer records count: {}", count);
223 return count > 1500;
224 }
225 }));
226
227 Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(11000,
228 new Waiter.Predicate<Exception>() {
229 @Override public boolean evaluate() throws Exception {
230 int count = TestSlowLogAccessor.this.getTableCount(connection);
231 LOG.debug("SlowLog Table records count: {}", count);
232 return count > 1500;
233 }
234 }));
235 }
236
237 }