001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.namequeues;
021
022import java.io.IOException;
023import java.lang.reflect.Field;
024import java.util.List;
025import java.util.concurrent.CompletableFuture;
026import java.util.concurrent.TimeUnit;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.HBaseClassTestRule;
029import org.apache.hadoop.hbase.HBaseTestingUtility;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.client.Connection;
032import org.apache.hadoop.hbase.client.Result;
033import org.apache.hadoop.hbase.client.ResultScanner;
034import org.apache.hadoop.hbase.client.Scan;
035import org.apache.hadoop.hbase.client.Table;
036import org.apache.hadoop.hbase.namequeues.request.NamedQueueGetRequest;
037import org.apache.hadoop.hbase.namequeues.response.NamedQueueGetResponse;
038import org.apache.hadoop.hbase.regionserver.HRegionServer;
039import org.apache.hadoop.hbase.slowlog.SlowLogTableAccessor;
040import org.apache.hadoop.hbase.testclassification.MasterTests;
041import org.apache.hadoop.hbase.testclassification.MediumTests;
042import org.junit.AfterClass;
043import org.junit.Assert;
044import org.junit.Before;
045import org.junit.BeforeClass;
046import org.junit.ClassRule;
047import org.junit.Test;
048import org.junit.experimental.categories.Category;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052import org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
053
054import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
055import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog;
056
057/**
058 * Tests for SlowLog System Table
059 */
060@Category({ MasterTests.class, MediumTests.class })
061public class TestSlowLogAccessor {
062
063  @ClassRule
064  public static final HBaseClassTestRule CLASS_RULE =
065    HBaseClassTestRule.forClass(TestSlowLogAccessor.class);
066
067  private static final Logger LOG = LoggerFactory.getLogger(TestNamedQueueRecorder.class);
068
069  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new HBaseTestingUtility();
070
071  private NamedQueueRecorder namedQueueRecorder;
072
073  @BeforeClass
074  public static void setup() throws Exception {
075    try {
076      HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
077    } catch (IOException e) {
078      LOG.debug("No worries.");
079    }
080    Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
081    conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
082    conf.setBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY, true);
083    conf.setInt("hbase.slowlog.systable.chore.duration", 900);
084    conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", 50000);
085    HBASE_TESTING_UTILITY.startMiniCluster();
086  }
087
088  @AfterClass
089  public static void teardown() throws Exception {
090    HBASE_TESTING_UTILITY.shutdownMiniHBaseCluster();
091  }
092
093  @Before
094  public void setUp() throws Exception {
095    HRegionServer hRegionServer = HBASE_TESTING_UTILITY.getMiniHBaseCluster().getRegionServer(0);
096    Field slowLogRecorder = HRegionServer.class.getDeclaredField("namedQueueRecorder");
097    slowLogRecorder.setAccessible(true);
098    this.namedQueueRecorder = (NamedQueueRecorder) slowLogRecorder.get(hRegionServer);
099  }
100
101  private List<TooSlowLog.SlowLogPayload> getSlowLogPayloads(
102      AdminProtos.SlowLogResponseRequest request) {
103    NamedQueueGetRequest namedQueueGetRequest = new NamedQueueGetRequest();
104    namedQueueGetRequest.setNamedQueueEvent(RpcLogDetails.SLOW_LOG_EVENT);
105    namedQueueGetRequest.setSlowLogResponseRequest(request);
106    NamedQueueGetResponse namedQueueGetResponse =
107      namedQueueRecorder.getNamedQueueRecords(namedQueueGetRequest);
108    return namedQueueGetResponse.getSlowLogPayloads();
109  }
110
111  @Test
112  public void testSlowLogRecords() throws Exception {
113
114    AdminProtos.SlowLogResponseRequest request =
115      AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
116
117    namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
118    Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
119
120    int i = 0;
121
122    Connection connection = waitForSlowLogTableCreation();
123    // add 5 records initially
124    for (; i < 5; i++) {
125      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
126        .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
127      namedQueueRecorder.addRecord(rpcLogDetails);
128    }
129
130    // add 2 more records
131    for (; i < 7; i++) {
132      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
133        .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
134      namedQueueRecorder.addRecord(rpcLogDetails);
135    }
136
137    // add 3 more records
138    for (; i < 10; i++) {
139      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
140        .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
141      namedQueueRecorder.addRecord(rpcLogDetails);
142    }
143
144    // add 4 more records
145    for (; i < 14; i++) {
146      RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
147        .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
148      namedQueueRecorder.addRecord(rpcLogDetails);
149    }
150
151    Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY
152      .waitFor(3000, () -> getSlowLogPayloads(request).size() == 14));
153
154    Assert.assertNotEquals(-1,
155      HBASE_TESTING_UTILITY.waitFor(3000, () -> getTableCount(connection) == 14));
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 IOException {
172    Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(2000, () -> {
173      try {
174        return HBASE_TESTING_UTILITY.getAdmin()
175          .tableExists(SlowLogTableAccessor.SLOW_LOG_TABLE_NAME);
176      } catch (IOException e) {
177        return false;
178      }
179    }));
180    return HBASE_TESTING_UTILITY.getConnection();
181  }
182
183  @Test
184  public void testHigherSlowLogs() throws Exception {
185    Connection connection = waitForSlowLogTableCreation();
186
187    namedQueueRecorder.clearNamedQueue(NamedQueuePayload.NamedQueueEvent.SLOW_LOG);
188    AdminProtos.SlowLogResponseRequest request =
189      AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(500000).build();
190    Assert.assertEquals(getSlowLogPayloads(request).size(), 0);
191
192    for (int j = 0; j < 100; j++) {
193      CompletableFuture.runAsync(() -> {
194        for (int i = 0; i < 350; i++) {
195          if (i == 300) {
196            Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
197          }
198          RpcLogDetails rpcLogDetails = TestNamedQueueRecorder
199            .getRpcLogDetails("userName_" + (i + 1), "client_" + (i + 1), "class_" + (i + 1));
200          namedQueueRecorder.addRecord(rpcLogDetails);
201        }
202      });
203    }
204
205    Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> {
206      int count = getSlowLogPayloads(request).size();
207      LOG.debug("RingBuffer records count: {}", count);
208      return count > 2000;
209    }));
210
211    Assert.assertNotEquals(-1, HBASE_TESTING_UTILITY.waitFor(7000, () -> {
212      int count = getTableCount(connection);
213      LOG.debug("SlowLog Table records count: {}", count);
214      return count > 2000;
215    }));
216  }
217
218}