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